Bug 1831 Batch messages on journal recovery
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / RaftActor.java
index 0a4e2170e527e0dd74cae73802ae7893eba436be..64fa7496042466e58bd51cf0a488c265898866da 100644 (file)
@@ -20,6 +20,7 @@ import akka.persistence.SnapshotOffer;
 import akka.persistence.SnapshotSelectionCriteria;
 import akka.persistence.UntypedPersistentActor;
 import com.google.common.base.Optional;
+import com.google.common.base.Stopwatch;
 import com.google.protobuf.ByteString;
 import org.opendaylight.controller.cluster.raft.base.messages.ApplyLogEntries;
 import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
@@ -39,7 +40,6 @@ import org.opendaylight.controller.cluster.raft.client.messages.RemoveRaftPeer;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
 import org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages;
-
 import java.io.Serializable;
 import java.util.Map;
 
@@ -97,7 +97,7 @@ public abstract class RaftActor extends UntypedPersistentActor {
      * This context should NOT be passed directly to any other actor it is
      * only to be consumed by the RaftActorBehaviors
      */
-    protected RaftActorContext context;
+    private final RaftActorContext context;
 
     /**
      * The in-memory journal
@@ -108,6 +108,10 @@ public abstract class RaftActor extends UntypedPersistentActor {
 
     private volatile boolean hasSnapshotCaptureInitiated = false;
 
+    private Stopwatch recoveryTimer;
+
+    private int currentRecoveryBatchCount;
+
     public RaftActor(String id, Map<String, String> peerAddresses) {
         this(id, peerAddresses, Optional.<ConfigParams>absent());
     }
@@ -122,71 +126,134 @@ public abstract class RaftActor extends UntypedPersistentActor {
             LOG);
     }
 
-    @Override public void onReceiveRecover(Object message) {
+    private void initRecoveryTimer() {
+        if(recoveryTimer == null) {
+            recoveryTimer = new Stopwatch();
+            recoveryTimer.start();
+        }
+    }
+
+    @Override
+    public void preStart() throws Exception {
+        LOG.info("Starting recovery for {} with journal batch size {}", persistenceId(),
+                context.getConfigParams().getJournalRecoveryLogBatchSize());
+        super.preStart();
+    }
+
+    @Override
+    public void onReceiveRecover(Object message) {
         if (message instanceof SnapshotOffer) {
-            LOG.info("SnapshotOffer called..");
-            SnapshotOffer offer = (SnapshotOffer) message;
-            Snapshot snapshot = (Snapshot) offer.snapshot();
+            onRecoveredSnapshot((SnapshotOffer)message);
+        } else if (message instanceof ReplicatedLogEntry) {
+            onRecoveredJournalLogEntry((ReplicatedLogEntry)message);
+        } else if (message instanceof ApplyLogEntries) {
+            onRecoveredApplyLogEntries((ApplyLogEntries)message);
+        } else if (message instanceof DeleteEntries) {
+            replicatedLog.removeFrom(((DeleteEntries) message).getFromIndex());
+        } else if (message instanceof UpdateElectionTerm) {
+            context.getTermInformation().update(((UpdateElectionTerm) message).getCurrentTerm(),
+                    ((UpdateElectionTerm) message).getVotedFor());
+        } else if (message instanceof RecoveryCompleted) {
+            onRecoveryCompletedMessage();
+        }
+    }
 
-            // Create a replicated log with the snapshot information
-            // The replicated log can be used later on to retrieve this snapshot
-            // when we need to install it on a peer
-            replicatedLog = new ReplicatedLogImpl(snapshot);
+    private void onRecoveredSnapshot(SnapshotOffer offer) {
+        LOG.debug("SnapshotOffer called..");
 
-            context.setReplicatedLog(replicatedLog);
-            context.setLastApplied(snapshot.getLastAppliedIndex());
-            context.setCommitIndex(snapshot.getLastAppliedIndex());
+        initRecoveryTimer();
 
-            LOG.info("Applied snapshot to replicatedLog. " +
-                    "snapshotIndex={}, snapshotTerm={}, journal-size={}",
-                replicatedLog.snapshotIndex, replicatedLog.snapshotTerm,
-                replicatedLog.size()
-            );
+        Snapshot snapshot = (Snapshot) offer.snapshot();
 
-            // Apply the snapshot to the actors state
-            applySnapshot(ByteString.copyFrom(snapshot.getState()));
+        // Create a replicated log with the snapshot information
+        // The replicated log can be used later on to retrieve this snapshot
+        // when we need to install it on a peer
+        replicatedLog = new ReplicatedLogImpl(snapshot);
 
-        } else if (message instanceof ReplicatedLogEntry) {
-            ReplicatedLogEntry logEntry = (ReplicatedLogEntry) message;
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Received ReplicatedLogEntry for recovery:{}", logEntry.getIndex());
-            }
-            replicatedLog.append(logEntry);
+        context.setReplicatedLog(replicatedLog);
+        context.setLastApplied(snapshot.getLastAppliedIndex());
+        context.setCommitIndex(snapshot.getLastAppliedIndex());
 
-        } else if (message instanceof ApplyLogEntries) {
-            ApplyLogEntries ale = (ApplyLogEntries) message;
+        Stopwatch timer = new Stopwatch();
+        timer.start();
 
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Received ApplyLogEntries for recovery, applying to state:{} to {}",
+        // Apply the snapshot to the actors state
+        applyRecoverySnapshot(ByteString.copyFrom(snapshot.getState()));
+
+        timer.stop();
+        LOG.info("Recovery snapshot applied for {} in {}: snapshotIndex={}, snapshotTerm={}, journal-size=" +
+                replicatedLog.size(), persistenceId(), timer.toString(),
+                replicatedLog.snapshotIndex, replicatedLog.snapshotTerm);
+    }
+
+    private void onRecoveredJournalLogEntry(ReplicatedLogEntry logEntry) {
+        if(LOG.isDebugEnabled()) {
+            LOG.debug("Received ReplicatedLogEntry for recovery: {}", logEntry.getIndex());
+        }
+
+        replicatedLog.append(logEntry);
+    }
+
+    private void onRecoveredApplyLogEntries(ApplyLogEntries ale) {
+        if(LOG.isDebugEnabled()) {
+            LOG.debug("Received ApplyLogEntries for recovery, applying to state: {} to {}",
                     context.getLastApplied() + 1, ale.getToIndex());
-            }
+        }
 
-            for (long i = context.getLastApplied() + 1; i <= ale.getToIndex(); i++) {
-                applyState(null, "recovery", replicatedLog.get(i).getData());
-            }
-            context.setLastApplied(ale.getToIndex());
-            context.setCommitIndex(ale.getToIndex());
+        for (long i = context.getLastApplied() + 1; i <= ale.getToIndex(); i++) {
+            batchRecoveredLogEntry(replicatedLog.get(i));
+        }
 
-        } else if (message instanceof DeleteEntries) {
-            replicatedLog.removeFrom(((DeleteEntries) message).getFromIndex());
+        context.setLastApplied(ale.getToIndex());
+        context.setCommitIndex(ale.getToIndex());
+    }
 
-        } else if (message instanceof UpdateElectionTerm) {
-            context.getTermInformation().update(((UpdateElectionTerm) message).getCurrentTerm(),
-                ((UpdateElectionTerm) message).getVotedFor());
+    private void batchRecoveredLogEntry(ReplicatedLogEntry logEntry) {
+        initRecoveryTimer();
 
-        } else if (message instanceof RecoveryCompleted) {
-            LOG.info(
-                "RecoveryCompleted - Switching actor to Follower - " +
-                    "Persistence Id =  " + persistenceId() +
-                    " Last index in log:{}, snapshotIndex={}, snapshotTerm={}, " +
-                    "journal-size={}",
-                replicatedLog.lastIndex(), replicatedLog.snapshotIndex,
-                replicatedLog.snapshotTerm, replicatedLog.size());
-            currentBehavior = switchBehavior(RaftState.Follower);
-            onStateChanged();
+        int batchSize = context.getConfigParams().getJournalRecoveryLogBatchSize();
+        if(currentRecoveryBatchCount == 0) {
+            startLogRecoveryBatch(batchSize);
+        }
+
+        appendRecoveredLogEntry(logEntry.getData());
+
+        if(++currentRecoveryBatchCount >= batchSize) {
+            endCurrentLogRecoveryBatch();
         }
     }
 
+    private void endCurrentLogRecoveryBatch() {
+        applyCurrentLogRecoveryBatch();
+        currentRecoveryBatchCount = 0;
+    }
+
+    private void onRecoveryCompletedMessage() {
+        if(currentRecoveryBatchCount > 0) {
+            endCurrentLogRecoveryBatch();
+        }
+
+        onRecoveryComplete();
+
+        String recoveryTime = "";
+        if(recoveryTimer != null) {
+            recoveryTimer.stop();
+            recoveryTime = " in " + recoveryTimer.toString();
+            recoveryTimer = null;
+        }
+
+        LOG.info(
+            "Recovery completed" + recoveryTime + " - Switching actor to Follower - " +
+                "Persistence Id =  " + persistenceId() +
+                " Last index in log={}, snapshotIndex={}, snapshotTerm={}, " +
+                "journal-size={}",
+            replicatedLog.lastIndex(), replicatedLog.snapshotIndex,
+            replicatedLog.snapshotTerm, replicatedLog.size());
+
+        currentBehavior = switchBehavior(RaftState.Follower);
+        onStateChanged();
+    }
+
     @Override public void onReceiveCommand(Object message) {
         if (message instanceof ApplyState){
             ApplyState applyState = (ApplyState) message;
@@ -397,6 +464,10 @@ public abstract class RaftActor extends UntypedPersistentActor {
         return context.getLastApplied();
     }
 
+    protected RaftActorContext getRaftActorContext() {
+        return context;
+    }
+
     /**
      * setPeerAddress sets the address of a known peer at a later time.
      * <p>
@@ -437,6 +508,38 @@ public abstract class RaftActor extends UntypedPersistentActor {
     protected abstract void applyState(ActorRef clientActor, String identifier,
         Object data);
 
+    /**
+     * This method is called during recovery at the start of a batch of state entries. Derived
+     * classes should perform any initialization needed to start a batch.
+     */
+    protected abstract void startLogRecoveryBatch(int maxBatchSize);
+
+    /**
+     * This method is called during recovery to append state data to the current batch. This method
+     * is called 1 or more times after {@link #startRecoveryStateBatch}.
+     *
+     * @param data the state data
+     */
+    protected abstract void appendRecoveredLogEntry(Payload data);
+
+    /**
+     * This method is called during recovery to reconstruct the state of the actor.
+     *
+     * @param snapshot A snapshot of the state of the actor
+     */
+    protected abstract void applyRecoverySnapshot(ByteString snapshot);
+
+    /**
+     * This method is called during recovery at the end of a batch to apply the current batched
+     * log entries. This method is called after {@link #appendRecoveryLogEntry}.
+     */
+    protected abstract void applyCurrentLogRecoveryBatch();
+
+    /**
+     * This method is called when recovery is complete.
+     */
+    protected abstract void onRecoveryComplete();
+
     /**
      * This method will be called by the RaftActor when a snapshot needs to be
      * created. The derived actor should respond with its current state.
@@ -449,10 +552,7 @@ public abstract class RaftActor extends UntypedPersistentActor {
     protected abstract void createSnapshot();
 
     /**
-     * This method will be called by the RaftActor during recovery to
-     * reconstruct the state of the actor.
-     * <p/>
-     * This method may also be called at any other point during normal
+     * This method can be called at any other point during normal
      * operations when the derived actor is out of sync with it's peers
      * and the only way to bring it in sync is by applying a snapshot
      *
@@ -609,6 +709,7 @@ public abstract class RaftActor extends UntypedPersistentActor {
             // of a single command.
             persist(replicatedLogEntry,
                 new Procedure<ReplicatedLogEntry>() {
+                    @Override
                     public void apply(ReplicatedLogEntry evt) throws Exception {
                         // when a snaphsot is being taken, captureSnapshot != null
                         if (hasSnapshotCaptureInitiated == false &&
@@ -673,10 +774,12 @@ public abstract class RaftActor extends UntypedPersistentActor {
         private long currentTerm = 0;
         private String votedFor = null;
 
+        @Override
         public long getCurrentTerm() {
             return currentTerm;
         }
 
+        @Override
         public String getVotedFor() {
             return votedFor;
         }