BUG-5626: use lambdas, Runnable and Consumer instead of Procedure
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / SnapshotManager.java
index 432d678491e96a5509502265014ea755bc7e8eee..b195686d2092ff4f54353a6d18295c62ebfe9bec 100644 (file)
@@ -8,20 +8,20 @@
 
 package org.opendaylight.controller.cluster.raft;
 
-import akka.japi.Procedure;
 import akka.persistence.SnapshotSelectionCriteria;
-import com.google.protobuf.ByteString;
-import org.opendaylight.controller.cluster.DataPersistenceProvider;
+import com.google.common.annotations.VisibleForTesting;
+import java.util.List;
+import java.util.function.Consumer;
+import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshot;
 import org.opendaylight.controller.cluster.raft.base.messages.SendInstallSnapshot;
+import org.opendaylight.controller.cluster.raft.base.messages.SnapshotComplete;
 import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior;
 import org.slf4j.Logger;
 
 public class SnapshotManager implements SnapshotState {
 
-
     private final SnapshotState IDLE = new Idle();
-    private final SnapshotState CAPTURING = new Capturing();
     private final SnapshotState PERSISTING = new Persisting();
     private final SnapshotState CREATING = new Creating();
 
@@ -35,12 +35,22 @@ public class SnapshotManager implements SnapshotState {
 
     private SnapshotState currentState = IDLE;
     private CaptureSnapshot captureSnapshot;
+    private long lastSequenceNumber = -1;
+
+    private Runnable createSnapshotProcedure;
+
+    private ApplySnapshot applySnapshot;
+    private Consumer<byte[]> applySnapshotProcedure;
 
     public SnapshotManager(RaftActorContext context, Logger logger) {
         this.context = context;
         this.LOG = logger;
     }
 
+    public boolean isApplying() {
+        return applySnapshot != null;
+    }
+
     @Override
     public boolean isCapturing() {
         return currentState.isCapturing();
@@ -57,19 +67,18 @@ public class SnapshotManager implements SnapshotState {
     }
 
     @Override
-    public void create(Procedure<Void> callback) {
-        currentState.create(callback);
+    public void apply(ApplySnapshot snapshot) {
+        currentState.apply(snapshot);
     }
 
     @Override
-    public void persist(DataPersistenceProvider persistenceProvider, byte[] snapshotBytes,
-                        RaftActorBehavior currentBehavior, long totalMemory) {
-        currentState.persist(persistenceProvider, snapshotBytes, currentBehavior, totalMemory);
+    public void persist(final byte[] snapshotBytes, final long totalMemory) {
+        currentState.persist(snapshotBytes, totalMemory);
     }
 
     @Override
-    public void commit(DataPersistenceProvider persistenceProvider, long sequenceNumber) {
-        currentState.commit(persistenceProvider, sequenceNumber);
+    public void commit(final long sequenceNumber) {
+        currentState.commit(sequenceNumber);
     }
 
     @Override
@@ -78,23 +87,71 @@ public class SnapshotManager implements SnapshotState {
     }
 
     @Override
-    public long trimLog(long desiredTrimIndex, RaftActorBehavior currentBehavior) {
-        return currentState.trimLog(desiredTrimIndex, currentBehavior);
+    public long trimLog(final long desiredTrimIndex) {
+        return currentState.trimLog(desiredTrimIndex);
+    }
+
+    public void setCreateSnapshotRunnable(Runnable createSnapshotProcedure) {
+        this.createSnapshotProcedure = createSnapshotProcedure;
+    }
+
+    public void setApplySnapshotConsumer(Consumer<byte[]> applySnapshotProcedure) {
+        this.applySnapshotProcedure = applySnapshotProcedure;
+    }
+
+    public long getLastSequenceNumber() {
+        return lastSequenceNumber;
+    }
+
+    @VisibleForTesting
+    public CaptureSnapshot getCaptureSnapshot() {
+        return captureSnapshot;
     }
 
     private boolean hasFollowers(){
-        return context.getPeerAddresses().keySet().size() > 0;
+        return context.hasFollowers();
     }
 
     private String persistenceId(){
         return context.getId();
     }
 
+    public CaptureSnapshot newCaptureSnapshot(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex,
+            boolean installSnapshotInitiated) {
+        TermInformationReader lastAppliedTermInfoReader =
+                lastAppliedTermInformationReader.init(context.getReplicatedLog(), context.getLastApplied(),
+                        lastLogEntry, hasFollowers());
+
+        long lastAppliedIndex = lastAppliedTermInfoReader.getIndex();
+        long lastAppliedTerm = lastAppliedTermInfoReader.getTerm();
+
+        TermInformationReader replicatedToAllTermInfoReader =
+                replicatedToAllTermInformationReader.init(context.getReplicatedLog(), replicatedToAllIndex);
+
+        long newReplicatedToAllIndex = replicatedToAllTermInfoReader.getIndex();
+        long newReplicatedToAllTerm = replicatedToAllTermInfoReader.getTerm();
+
+        List<ReplicatedLogEntry> unAppliedEntries = context.getReplicatedLog().getFrom(lastAppliedIndex + 1);
+
+        long lastLogEntryIndex = lastAppliedIndex;
+        long lastLogEntryTerm = lastAppliedTerm;
+        if(lastLogEntry != null) {
+            lastLogEntryIndex = lastLogEntry.getIndex();
+            lastLogEntryTerm = lastLogEntry.getTerm();
+        } else {
+            LOG.debug("{}: Capturing Snapshot : lastLogEntry is null. Using lastAppliedIndex {} and lastAppliedTerm {} instead.",
+                    persistenceId(), lastAppliedIndex, lastAppliedTerm);
+        }
+
+        return new CaptureSnapshot(lastLogEntryIndex, lastLogEntryTerm, lastAppliedIndex, lastAppliedTerm,
+                newReplicatedToAllIndex, newReplicatedToAllTerm, unAppliedEntries, installSnapshotInitiated);
+    }
+
     private class AbstractSnapshotState implements SnapshotState {
 
         @Override
         public boolean isCapturing() {
-            return false;
+            return true;
         }
 
         @Override
@@ -110,18 +167,17 @@ public class SnapshotManager implements SnapshotState {
         }
 
         @Override
-        public void create(Procedure<Void> callback) {
-            LOG.debug("create should not be called in state {}", this);
+        public void apply(ApplySnapshot snapshot) {
+            LOG.debug("apply should not be called in state {}", this);
         }
 
         @Override
-        public void persist(DataPersistenceProvider persistenceProvider, byte[] snapshotBytes,
-                            RaftActorBehavior currentBehavior, long totalMemory) {
+        public void persist(final byte[] snapshotBytes, final long totalMemory) {
             LOG.debug("persist should not be called in state {}", this);
         }
 
         @Override
-        public void commit(DataPersistenceProvider persistenceProvider, long sequenceNumber) {
+        public void commit(final long sequenceNumber) {
             LOG.debug("commit should not be called in state {}", this);
         }
 
@@ -131,12 +187,12 @@ public class SnapshotManager implements SnapshotState {
         }
 
         @Override
-        public long trimLog(long desiredTrimIndex, RaftActorBehavior currentBehavior) {
+        public long trimLog(final long desiredTrimIndex) {
             LOG.debug("trimLog should not be called in state {}", this);
             return -1;
         }
 
-        protected long doTrimLog(long desiredTrimIndex, RaftActorBehavior currentBehavior){
+        protected long doTrimLog(final long desiredTrimIndex) {
             //  we would want to keep the lastApplied as its used while capturing snapshots
             long lastApplied = context.getLastApplied();
             long tempMin = Math.min(desiredTrimIndex, (lastApplied > -1 ? lastApplied - 1 : -1));
@@ -155,7 +211,10 @@ public class SnapshotManager implements SnapshotState {
                 context.getReplicatedLog().snapshotPreCommit(tempMin, entry.getTerm());
                 context.getReplicatedLog().snapshotCommit();
                 return tempMin;
-            } else if(tempMin > currentBehavior.getReplicatedToAllIndex()) {
+            }
+
+            final RaftActorBehavior currentBehavior = context.getCurrentBehavior();
+            if(tempMin > currentBehavior.getReplicatedToAllIndex()) {
                 // It's possible a follower was lagging and an install snapshot advanced its match index past
                 // the current replicatedToAllIndex. Since the follower is now caught up we should advance the
                 // replicatedToAllIndex (to tempMin). The fact that tempMin wasn't found in the log is likely
@@ -169,36 +228,35 @@ public class SnapshotManager implements SnapshotState {
 
     private class Idle extends AbstractSnapshotState {
 
-        private boolean capture(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex, String targetFollower) {
-            TermInformationReader lastAppliedTermInfoReader =
-                    lastAppliedTermInformationReader.init(context.getReplicatedLog(), context.getLastApplied(),
-                            lastLogEntry, hasFollowers());
+        @Override
+        public boolean isCapturing() {
+            return false;
+        }
 
-            long lastAppliedIndex = lastAppliedTermInfoReader.getIndex();
-            long lastAppliedTerm = lastAppliedTermInfoReader.getTerm();
+        private boolean capture(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex, String targetFollower) {
+            captureSnapshot = newCaptureSnapshot(lastLogEntry, replicatedToAllIndex, targetFollower != null);
 
-            TermInformationReader replicatedToAllTermInfoReader =
-                    replicatedToAllTermInformationReader.init(context.getReplicatedLog(), replicatedToAllIndex);
+            if(captureSnapshot.isInstallSnapshotInitiated()) {
+                LOG.info("{}: Initiating snapshot capture {} to install on {}",
+                        persistenceId(), captureSnapshot, targetFollower);
+            } else {
+                LOG.info("{}: Initiating snapshot capture {}", persistenceId(), captureSnapshot);
+            }
 
-            long newReplicatedToAllIndex = replicatedToAllTermInfoReader.getIndex();
-            long newReplicatedToAllTerm = replicatedToAllTermInfoReader.getTerm();
+            lastSequenceNumber = context.getPersistenceProvider().getLastSequenceNumber();
 
-            // send a CaptureSnapshot to self to make the expensive operation async.
-            captureSnapshot = new CaptureSnapshot(lastLogEntry.getIndex(),
-                    lastLogEntry.getTerm(), lastAppliedIndex, lastAppliedTerm,
-                    newReplicatedToAllIndex, newReplicatedToAllTerm, targetFollower!=null);
+            LOG.debug("{}: lastSequenceNumber prior to capture: {}", persistenceId(), lastSequenceNumber);
 
-            SnapshotManager.this.currentState = CAPTURING;
+            SnapshotManager.this.currentState = CREATING;
 
-            if(targetFollower != null){
-                LOG.info("{}: Initiating snapshot capture {}", persistenceId(), captureSnapshot);
-            } else {
-                LOG.info("{}: Initiating snapshot capture {} to install on {}",
-                        persistenceId(), captureSnapshot, targetFollower);
+            try {
+                createSnapshotProcedure.run();
+            } catch (Exception e) {
+                SnapshotManager.this.currentState = IDLE;
+                LOG.error("Error creating snapshot", e);
+                return false;
             }
 
-            context.getActor().tell(captureSnapshot, context.getActor());
-
             return true;
         }
 
@@ -213,75 +271,72 @@ public class SnapshotManager implements SnapshotState {
         }
 
         @Override
-        public String toString() {
-            return "Idle";
-        }
+        public void apply(ApplySnapshot applySnapshot) {
+            SnapshotManager.this.applySnapshot = applySnapshot;
 
-        @Override
-        public long trimLog(long desiredTrimIndex, RaftActorBehavior currentBehavior) {
-            return doTrimLog(desiredTrimIndex, currentBehavior);
-        }
-    }
+            lastSequenceNumber = context.getPersistenceProvider().getLastSequenceNumber();
 
-    private class Capturing extends AbstractSnapshotState {
+            LOG.debug("lastSequenceNumber prior to persisting applied snapshot: {}", lastSequenceNumber);
 
-        @Override
-        public boolean isCapturing() {
-            return true;
-        }
+            context.getPersistenceProvider().saveSnapshot(applySnapshot.getSnapshot());
 
-        @Override
-        public void create(Procedure<Void> callback) {
-            try {
-                callback.apply(null);
-                SnapshotManager.this.currentState = CREATING;
-            } catch (Exception e) {
-                LOG.error("Unexpected error occurred", e);
-            }
+            SnapshotManager.this.currentState = PERSISTING;
         }
 
         @Override
         public String toString() {
-            return "Capturing";
+            return "Idle";
         }
 
+        @Override
+        public long trimLog(final long desiredTrimIndex) {
+            return doTrimLog(desiredTrimIndex);
+        }
     }
 
     private class Creating extends AbstractSnapshotState {
 
         @Override
-        public boolean isCapturing() {
-            return true;
-        }
-
-        @Override
-        public void persist(DataPersistenceProvider persistenceProvider, byte[] snapshotBytes,
-                            RaftActorBehavior currentBehavior, long totalMemory) {
+        public void persist(final byte[] snapshotBytes, final long totalMemory) {
             // create a snapshot object from the state provided and save it
             // when snapshot is saved async, SaveSnapshotSuccess is raised.
 
-            Snapshot sn = Snapshot.create(snapshotBytes,
-                    context.getReplicatedLog().getFrom(captureSnapshot.getLastAppliedIndex() + 1),
+            Snapshot snapshot = Snapshot.create(snapshotBytes,
+                    captureSnapshot.getUnAppliedEntries(),
                     captureSnapshot.getLastIndex(), captureSnapshot.getLastTerm(),
-                    captureSnapshot.getLastAppliedIndex(), captureSnapshot.getLastAppliedTerm());
+                    captureSnapshot.getLastAppliedIndex(), captureSnapshot.getLastAppliedTerm(),
+                    context.getTermInformation().getCurrentTerm(),
+                    context.getTermInformation().getVotedFor(), context.getPeerServerInfo(true));
 
-            persistenceProvider.saveSnapshot(sn);
+            context.getPersistenceProvider().saveSnapshot(snapshot);
 
-            LOG.info("{}: Persisting of snapshot done:{}", persistenceId(), sn.getLogMessage());
+            LOG.info("{}: Persisting of snapshot done: {}", persistenceId(), snapshot);
 
             long dataThreshold = totalMemory *
                     context.getConfigParams().getSnapshotDataThresholdPercentage() / 100;
-            if (context.getReplicatedLog().dataSize() > dataThreshold) {
+            boolean dataSizeThresholdExceeded = context.getReplicatedLog().dataSize() > dataThreshold;
 
+            boolean logSizeExceededSnapshotBatchCount =
+                    context.getReplicatedLog().size() >= context.getConfigParams().getSnapshotBatchCount();
+
+            final RaftActorBehavior currentBehavior = context.getCurrentBehavior();
+            if (dataSizeThresholdExceeded || logSizeExceededSnapshotBatchCount) {
                 if(LOG.isDebugEnabled()) {
-                    LOG.debug("{}: dataSize {} exceeds dataThreshold {} - doing snapshotPreCommit with index {}",
-                            persistenceId(), context.getReplicatedLog().dataSize(), dataThreshold,
-                            captureSnapshot.getLastAppliedIndex());
+                    if(dataSizeThresholdExceeded) {
+                        LOG.debug("{}: log data size {} exceeds the memory threshold {} - doing snapshotPreCommit with index {}",
+                                context.getId(), context.getReplicatedLog().dataSize(), dataThreshold,
+                                captureSnapshot.getLastAppliedIndex());
+                    } else {
+                        LOG.debug("{}: log size {} exceeds the snapshot batch count {} - doing snapshotPreCommit with index {}",
+                                context.getId(), context.getReplicatedLog().size(),
+                                context.getConfigParams().getSnapshotBatchCount(), captureSnapshot.getLastAppliedIndex());
+                    }
                 }
 
-                // if memory is less, clear the log based on lastApplied.
-                // this could/should only happen if one of the followers is down
-                // as normally we keep removing from the log when its replicated to all.
+                // We either exceeded the memory threshold or the log size exceeded the snapshot batch
+                // count so, to keep the log memory footprint in check, clear the log based on lastApplied.
+                // This could/should only happen if one of the followers is down as normally we keep
+                // removing from the log as entries are replicated to all.
                 context.getReplicatedLog().snapshotPreCommit(captureSnapshot.getLastAppliedIndex(),
                         captureSnapshot.getLastAppliedTerm());
 
@@ -306,15 +361,14 @@ public class SnapshotManager implements SnapshotState {
                         context.getReplicatedLog().getSnapshotTerm());
             }
 
-            LOG.info("{}: Removed in-memory snapshotted entries, adjusted snaphsotIndex:{} " +
-                            "and term:{}", persistenceId(), captureSnapshot.getLastAppliedIndex(),
-                    captureSnapshot.getLastAppliedTerm());
+            LOG.info("{}: Removed in-memory snapshotted entries, adjusted snaphsotIndex: {} " +
+                    "and term: {}", context.getId(), context.getReplicatedLog().getSnapshotIndex(),
+                    context.getReplicatedLog().getSnapshotTerm());
 
             if (context.getId().equals(currentBehavior.getLeaderId())
                     && captureSnapshot.isInstallSnapshotInitiated()) {
                 // this would be call straight to the leader and won't initiate in serialization
-                currentBehavior.handleMessage(context.getActor(), new SendInstallSnapshot(
-                        ByteString.copyFrom(snapshotBytes)));
+                currentBehavior.handleMessage(context.getActor(), new SendInstallSnapshot(snapshot));
             }
 
             captureSnapshot = null;
@@ -331,27 +385,63 @@ public class SnapshotManager implements SnapshotState {
     private class Persisting extends AbstractSnapshotState {
 
         @Override
-        public void commit(DataPersistenceProvider persistenceProvider, long sequenceNumber) {
-            context.getReplicatedLog().snapshotCommit();
-            persistenceProvider.deleteSnapshots(new SnapshotSelectionCriteria(
-                    sequenceNumber - context.getConfigParams().getSnapshotBatchCount(), 43200000));
+        public void commit(final long sequenceNumber) {
+            LOG.debug("{}: Snapshot success -  sequence number: {}", persistenceId(), sequenceNumber);
+
+            if(applySnapshot != null) {
+                try {
+                    Snapshot snapshot = applySnapshot.getSnapshot();
+
+                    //clears the followers log, sets the snapshot index to ensure adjusted-index works
+                    context.setReplicatedLog(ReplicatedLogImpl.newInstance(snapshot, context));
+                    context.setLastApplied(snapshot.getLastAppliedIndex());
+                    context.setCommitIndex(snapshot.getLastAppliedIndex());
+                    context.getTermInformation().update(snapshot.getElectionTerm(), snapshot.getElectionVotedFor());
+
+                    if(snapshot.getState().length > 0 ) {
+                        applySnapshotProcedure.accept(snapshot.getState());
+                    }
+
+                    applySnapshot.getCallback().onSuccess();
+                } catch (Exception e) {
+                    LOG.error("{}: Error applying snapshot", context.getId(), e);
+                }
+            } else {
+                context.getReplicatedLog().snapshotCommit();
+            }
 
-            persistenceProvider.deleteMessages(sequenceNumber);
+            context.getPersistenceProvider().deleteSnapshots(new SnapshotSelectionCriteria(
+                    sequenceNumber - context.getConfigParams().getSnapshotBatchCount(), Long.MAX_VALUE, 0L, 0L));
 
-            SnapshotManager.this.currentState = IDLE;
+            context.getPersistenceProvider().deleteMessages(lastSequenceNumber);
+
+            snapshotComplete();
         }
 
         @Override
         public void rollback() {
-            context.getReplicatedLog().snapshotRollback();
+            // Nothing to rollback if we're applying a snapshot from the leader.
+            if(applySnapshot == null) {
+                context.getReplicatedLog().snapshotRollback();
+
+                LOG.info("{}: Replicated Log rolled back. Snapshot will be attempted in the next cycle." +
+                        "snapshotIndex:{}, snapshotTerm:{}, log-size:{}", persistenceId(),
+                        context.getReplicatedLog().getSnapshotIndex(),
+                        context.getReplicatedLog().getSnapshotTerm(),
+                        context.getReplicatedLog().size());
+            } else {
+                applySnapshot.getCallback().onFailure();
+            }
 
-            LOG.info("{}: Replicated Log rolled back. Snapshot will be attempted in the next cycle." +
-                            "snapshotIndex:{}, snapshotTerm:{}, log-size:{}", persistenceId(),
-                    context.getReplicatedLog().getSnapshotIndex(),
-                    context.getReplicatedLog().getSnapshotTerm(),
-                    context.getReplicatedLog().size());
+            snapshotComplete();
+        }
 
+        private void snapshotComplete() {
+            lastSequenceNumber = -1;
+            applySnapshot = null;
             SnapshotManager.this.currentState = IDLE;
+
+            context.getActor().tell(SnapshotComplete.INSTANCE, context.getActor());
         }
 
         @Override
@@ -366,7 +456,7 @@ public class SnapshotManager implements SnapshotState {
         long getTerm();
     }
 
-    private static class LastAppliedTermInformationReader implements TermInformationReader{
+    static class LastAppliedTermInformationReader implements TermInformationReader{
         private long index;
         private long term;
 
@@ -377,6 +467,8 @@ public class SnapshotManager implements SnapshotState {
             this.term = -1L;
             if (!hasFollowers) {
                 if(lastLogEntry != null) {
+                    // since we have persisted the last-log-entry to persistent journal before the capture,
+                    // we would want to snapshot from this entry.
                     index = lastLogEntry.getIndex();
                     term = lastLogEntry.getTerm();
                 }