Bug-2397:Provide a mechanism for stakeholders to get notifications on Raft state...
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / RaftActor.java
index 988789b4011e4f3ba9e9e9abe6b0c3009704c810..d647475e4d53319e1638c0e4c03fe4dfcd58a891 100644 (file)
@@ -18,24 +18,31 @@ import akka.persistence.SaveSnapshotFailure;
 import akka.persistence.SaveSnapshotSuccess;
 import akka.persistence.SnapshotOffer;
 import akka.persistence.SnapshotSelectionCriteria;
-import akka.persistence.UntypedPersistentActor;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import com.google.common.base.Stopwatch;
+import com.google.protobuf.ByteString;
+import java.io.Serializable;
+import java.util.Map;
+import org.opendaylight.controller.cluster.DataPersistenceProvider;
+import org.opendaylight.controller.cluster.common.actor.AbstractUntypedPersistentActor;
+import org.opendaylight.controller.cluster.notifications.RoleChanged;
+import org.opendaylight.controller.cluster.raft.base.messages.ApplyLogEntries;
 import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
+import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshot;
+import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshotReply;
 import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
-import com.google.common.base.Optional;
-import org.opendaylight.controller.cluster.raft.behaviors.Candidate;
+import org.opendaylight.controller.cluster.raft.base.messages.SendHeartBeat;
+import org.opendaylight.controller.cluster.raft.base.messages.SendInstallSnapshot;
+import org.opendaylight.controller.cluster.raft.behaviors.AbstractRaftActorBehavior;
 import org.opendaylight.controller.cluster.raft.behaviors.Follower;
-import org.opendaylight.controller.cluster.raft.behaviors.Leader;
 import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior;
-import org.opendaylight.controller.cluster.raft.client.messages.AddRaftPeer;
 import org.opendaylight.controller.cluster.raft.client.messages.FindLeader;
 import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply;
-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 java.io.Serializable;
-import java.util.List;
-import java.util.Map;
+import org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages;
 
 /**
  * RaftActor encapsulates a state machine that needs to be kept synchronized
@@ -77,7 +84,7 @@ import java.util.Map;
  * <li> when a snapshot should be saved </li>
  * </ul>
  */
-public abstract class RaftActor extends UntypedPersistentActor {
+public abstract class RaftActor extends AbstractUntypedPersistentActor {
     protected final LoggingAdapter LOG =
         Logging.getLogger(getContext().system(), this);
 
@@ -91,13 +98,20 @@ 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
      */
-    private RaftActorContext context;
+    private final RaftActorContext context;
 
     /**
      * The in-memory journal
      */
     private ReplicatedLogImpl replicatedLog = new ReplicatedLogImpl();
 
+    private CaptureSnapshot captureSnapshot = null;
+
+    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());
@@ -113,58 +127,194 @@ public abstract class RaftActor extends UntypedPersistentActor {
             LOG);
     }
 
-    @Override public void onReceiveRecover(Object message) {
-        if (message instanceof SnapshotOffer) {
+    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 handleRecover(Object message) {
+        if(persistence().isRecoveryApplicable()) {
+            if (message instanceof SnapshotOffer) {
+                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();
+            }
+        } else {
+            if (message instanceof RecoveryCompleted) {
+                // Delete all the messages from the akka journal so that we do not end up with consistency issues
+                // Note I am not using the dataPersistenceProvider and directly using the akka api here
+                deleteMessages(lastSequenceNr());
+
+                // Delete all the akka snapshots as they will not be needed
+                deleteSnapshots(new SnapshotSelectionCriteria(scala.Long.MaxValue(), scala.Long.MaxValue()));
+
+                onRecoveryComplete();
+
+                RaftActorBehavior oldBehavior = currentBehavior;
+                currentBehavior = new Follower(context);
+                handleBehaviorChange(oldBehavior, currentBehavior);
+            }
+        }
+    }
+
+    private void onRecoveredSnapshot(SnapshotOffer offer) {
+        if(LOG.isDebugEnabled()) {
             LOG.debug("SnapshotOffer called..");
-            SnapshotOffer offer = (SnapshotOffer) message;
-            Snapshot snapshot = (Snapshot) offer.snapshot();
+        }
 
-            // 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);
+        initRecoveryTimer();
 
-            context.setReplicatedLog(replicatedLog);
+        Snapshot snapshot = (Snapshot) offer.snapshot();
 
-            LOG.debug("Applied snapshot to replicatedLog. " +
-                "snapshotIndex={}, snapshotTerm={}, journal-size={}",
-                replicatedLog.snapshotIndex, replicatedLog.snapshotTerm,
-                replicatedLog.size());
-
-            // Apply the snapshot to the actors state
-            applySnapshot(snapshot.getState());
-
-        } else if (message instanceof ReplicatedLogEntry) {
-            replicatedLog.append((ReplicatedLogEntry) 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) {
-            LOG.debug(
-                "RecoveryCompleted - Switching actor to Follower - " +
-                    "Last index in log:{}, snapshotIndex={}, snapshotTerm={}, " +
-                    "journal-size={}",
-                replicatedLog.lastIndex(), replicatedLog.snapshotIndex,
-                replicatedLog.snapshotTerm, replicatedLog.size());
-            currentBehavior = switchBehavior(RaftState.Follower);
-            onStateChanged();
+        // 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);
+
+        context.setReplicatedLog(replicatedLog);
+        context.setLastApplied(snapshot.getLastAppliedIndex());
+        context.setCommitIndex(snapshot.getLastAppliedIndex());
+
+        Stopwatch timer = new Stopwatch();
+        timer.start();
+
+        // 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++) {
+            batchRecoveredLogEntry(replicatedLog.get(i));
+        }
+
+        context.setLastApplied(ale.getToIndex());
+        context.setCommitIndex(ale.getToIndex());
+    }
+
+    private void batchRecoveredLogEntry(ReplicatedLogEntry logEntry) {
+        initRecoveryTimer();
+
+        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());
+
+        RaftActorBehavior oldBehavior = currentBehavior;
+        currentBehavior = new Follower(context);
+        handleBehaviorChange(oldBehavior, currentBehavior);
     }
 
-    @Override public void onReceiveCommand(Object message) {
+    @Override public void handleCommand(Object message) {
         if (message instanceof ApplyState){
             ApplyState applyState = (ApplyState) message;
 
-            LOG.debug("Applying state for log index {} data {}",
-                applyState.getReplicatedLogEntry().getIndex(),
-                applyState.getReplicatedLogEntry().getData());
+            if(LOG.isDebugEnabled()) {
+                LOG.debug("Applying state for log index {} data {}",
+                    applyState.getReplicatedLogEntry().getIndex(),
+                    applyState.getReplicatedLogEntry().getData());
+            }
 
             applyState(applyState.getClientActor(), applyState.getIdentifier(),
                 applyState.getReplicatedLogEntry().getData());
 
+        } else if (message instanceof ApplyLogEntries){
+            ApplyLogEntries ale = (ApplyLogEntries) message;
+            if(LOG.isDebugEnabled()) {
+                LOG.debug("Persisting ApplyLogEntries with index={}", ale.getToIndex());
+            }
+            persistence().persist(new ApplyLogEntries(ale.getToIndex()), new Procedure<ApplyLogEntries>() {
+                @Override
+                public void apply(ApplyLogEntries param) throws Exception {
+                }
+            });
+
         } else if(message instanceof ApplySnapshot ) {
-            applySnapshot(((ApplySnapshot) message).getSnapshot());
+            Snapshot snapshot = ((ApplySnapshot) message).getSnapshot();
+
+            if(LOG.isDebugEnabled()) {
+                LOG.debug("ApplySnapshot called on Follower Actor " +
+                        "snapshotIndex:{}, snapshotTerm:{}", snapshot.getLastAppliedIndex(),
+                    snapshot.getLastAppliedTerm()
+                );
+            }
+            applySnapshot(ByteString.copyFrom(snapshot.getState()));
+
+            //clears the followers log, sets the snapshot index to ensure adjusted-index works
+            replicatedLog = new ReplicatedLogImpl(snapshot);
+            context.setReplicatedLog(replicatedLog);
+            context.setLastApplied(snapshot.getLastAppliedIndex());
 
         } else if (message instanceof FindLeader) {
             getSender().tell(
@@ -174,51 +324,71 @@ public abstract class RaftActor extends UntypedPersistentActor {
 
         } else if (message instanceof SaveSnapshotSuccess) {
             SaveSnapshotSuccess success = (SaveSnapshotSuccess) message;
+            LOG.info("SaveSnapshotSuccess received for snapshot");
 
-            // TODO: Not sure if we want to be this aggressive with trimming stuff
-            trimPersistentData(success.metadata().sequenceNr());
+            long sequenceNumber = success.metadata().sequenceNr();
+
+            commitSnapshot(sequenceNumber);
 
         } else if (message instanceof SaveSnapshotFailure) {
+            SaveSnapshotFailure saveSnapshotFailure = (SaveSnapshotFailure) message;
+
+            LOG.info("saveSnapshotFailure.metadata():{}", saveSnapshotFailure.metadata().toString());
+            LOG.error(saveSnapshotFailure.cause(), "SaveSnapshotFailure received for snapshot Cause:");
 
-            // TODO: Handle failure in saving the snapshot
+            context.getReplicatedLog().snapshotRollback();
 
-        } else if (message instanceof AddRaftPeer){
+            LOG.info("Replicated Log rollbacked. Snapshot will be attempted in the next cycle." +
+                "snapshotIndex:{}, snapshotTerm:{}, log-size:{}",
+                context.getReplicatedLog().getSnapshotIndex(),
+                context.getReplicatedLog().getSnapshotTerm(),
+                context.getReplicatedLog().size());
 
-            // FIXME : Do not add raft peers like this.
-            // When adding a new Peer we have to ensure that the a majority of
-            // the peers know about the new Peer. Doing it this way may cause
-            // a situation where multiple Leaders may emerge
-            AddRaftPeer arp = (AddRaftPeer)message;
-           context.addToPeers(arp.getName(), arp.getAddress());
+        } else if (message instanceof CaptureSnapshot) {
+            LOG.info("CaptureSnapshot received by actor");
+            CaptureSnapshot cs = (CaptureSnapshot)message;
+            captureSnapshot = cs;
+            createSnapshot();
 
-        } else if (message instanceof RemoveRaftPeer){
+        } else if (message instanceof CaptureSnapshotReply){
+            LOG.info("CaptureSnapshotReply received by actor");
+            CaptureSnapshotReply csr = (CaptureSnapshotReply) message;
 
-            RemoveRaftPeer rrp = (RemoveRaftPeer)message;
-            context.removePeer(rrp.getName());
+            ByteString stateInBytes = csr.getSnapshot();
+            LOG.info("CaptureSnapshotReply stateInBytes size:{}", stateInBytes.size());
+            handleCaptureSnapshotReply(stateInBytes);
 
         } else {
+            if (!(message instanceof AppendEntriesMessages.AppendEntries)
+                && !(message instanceof AppendEntriesReply) && !(message instanceof SendHeartBeat)) {
+                if(LOG.isDebugEnabled()) {
+                    LOG.debug("onReceiveCommand: message: {}", message.getClass());
+                }
+            }
 
-            RaftState state =
-                currentBehavior.handleMessage(getSender(), message);
             RaftActorBehavior oldBehavior = currentBehavior;
-            currentBehavior = switchBehavior(state);
-            if(oldBehavior != currentBehavior){
-                onStateChanged();
-            }
-        }
-    }
+            currentBehavior = currentBehavior.handleMessage(getSender(), message);
 
-    public java.util.Set<String> getPeers() {
-        return context.getPeerAddresses().keySet();
+            handleBehaviorChange(oldBehavior, currentBehavior);
+        }
     }
 
-    protected String getReplicatedLogState() {
-        return "snapshotIndex=" + context.getReplicatedLog().getSnapshotIndex()
-            + ", snapshotTerm=" + context.getReplicatedLog().getSnapshotTerm()
-            + ", im-mem journal size=" + context.getReplicatedLog().size();
+    private void handleBehaviorChange(RaftActorBehavior oldBehavior, RaftActorBehavior currentBehavior) {
+        if (oldBehavior != currentBehavior){
+            onStateChanged();
+        }
+        if (oldBehavior != null) {
+            // it can happen that the state has not changed but the leader has changed.
+            onLeaderChanged(oldBehavior.getLeaderId(), currentBehavior.getLeaderId());
+
+            if (getRoleChangeNotifier().isPresent() && oldBehavior.state() != currentBehavior.state()) {
+                // we do not want to notify when the behavior/role is set for the first time (i.e follower)
+                getRoleChangeNotifier().get().tell(new RoleChanged(getId(), oldBehavior.state().name(),
+                    currentBehavior.state().name()), getSelf());
+            }
+        }
     }
 
-
     /**
      * When a derived RaftActor needs to persist something it must call
      * persistData.
@@ -234,7 +404,9 @@ public abstract class RaftActor extends UntypedPersistentActor {
             context.getReplicatedLog().lastIndex() + 1,
             context.getTermInformation().getCurrentTerm(), data);
 
-        LOG.debug("Persist data {}", replicatedLogEntry);
+        if(LOG.isDebugEnabled()) {
+            LOG.debug("Persist data {}", replicatedLogEntry);
+        }
 
         replicatedLog
             .appendAndPersist(clientActor, identifier, replicatedLogEntry);
@@ -264,6 +436,10 @@ public abstract class RaftActor extends UntypedPersistentActor {
     protected ActorSelection getLeader(){
         String leaderAddress = getLeaderAddress();
 
+        if(leaderAddress == null){
+            return null;
+        }
+
         return context.actorSelection(leaderAddress);
     }
 
@@ -295,6 +471,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>
@@ -313,7 +493,12 @@ public abstract class RaftActor extends UntypedPersistentActor {
         context.setPeerAddress(peerId, peerAddress);
     }
 
+    protected void commitSnapshot(long sequenceNumber) {
+        context.getReplicatedLog().snapshotCommit();
 
+        // TODO: Not sure if we want to be this aggressive with trimming stuff
+        trimPersistentData(sequenceNumber);
+    }
 
     /**
      * The applyState method will be called by the RaftActor when some data
@@ -335,6 +520,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 #startLogRecoveryBatch}.
+     *
+     * @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 #appendRecoveredLogEntry}.
+     */
+    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.
@@ -344,19 +561,16 @@ public abstract class RaftActor extends UntypedPersistentActor {
      *
      * @return The current state of the actor
      */
-    protected abstract Object createSnapshot();
+    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
      *
      * @param snapshot A snapshot of the state of the actor
      */
-    protected abstract void applySnapshot(Object snapshot);
+    protected abstract void applySnapshot(ByteString snapshot);
 
     /**
      * This method will be called by the RaftActor when the state of the
@@ -365,47 +579,25 @@ public abstract class RaftActor extends UntypedPersistentActor {
      */
     protected abstract void onStateChanged();
 
-    private RaftActorBehavior switchBehavior(RaftState state) {
-        if (currentBehavior != null) {
-            if (currentBehavior.state() == state) {
-                return currentBehavior;
-            }
-            LOG.info("Switching from state " + currentBehavior.state() + " to "
-                + state);
-
-            try {
-                currentBehavior.close();
-            } catch (Exception e) {
-                LOG.error(e,
-                    "Failed to close behavior : " + currentBehavior.state());
-            }
-
-        } else {
-            LOG.info("Switching behavior to " + state);
-        }
-        RaftActorBehavior behavior = null;
-        if (state == RaftState.Candidate) {
-            behavior = new Candidate(context);
-        } else if (state == RaftState.Follower) {
-            behavior = new Follower(context);
-        } else {
-            behavior = new Leader(context);
-        }
-
+    protected abstract DataPersistenceProvider persistence();
 
+    /**
+     * Notifier Actor for this RaftActor to notify when a role change happens
+     * @return ActorRef - ActorRef of the notifier or Optional.absent if none.
+     */
+    protected abstract Optional<ActorRef> getRoleChangeNotifier();
 
-        return behavior;
-    }
+    protected void onLeaderChanged(String oldLeader, String newLeader){};
 
     private void trimPersistentData(long sequenceNumber) {
         // Trim akka snapshots
         // FIXME : Not sure how exactly the SnapshotSelectionCriteria is applied
         // For now guessing that it is ANDed.
-        deleteSnapshots(new SnapshotSelectionCriteria(
+        persistence().deleteSnapshots(new SnapshotSelectionCriteria(
             sequenceNumber - context.getConfigParams().getSnapshotBatchCount(), 43200000));
 
         // Trim akka journal
-        deleteMessages(sequenceNumber);
+        persistence().deleteMessages(sequenceNumber);
     }
 
     private String getLeaderAddress(){
@@ -417,18 +609,51 @@ public abstract class RaftActor extends UntypedPersistentActor {
             return null;
         }
         String peerAddress = context.getPeerAddress(leaderId);
-        LOG.debug("getLeaderAddress leaderId = " + leaderId + " peerAddress = "
-            + peerAddress);
+        if(LOG.isDebugEnabled()) {
+            LOG.debug("getLeaderAddress leaderId = {} peerAddress = {}",
+                    leaderId, peerAddress);
+        }
 
         return peerAddress;
     }
 
+    private void handleCaptureSnapshotReply(ByteString stateInBytes) {
+        // create a snapshot object from the state provided and save it
+        // when snapshot is saved async, SaveSnapshotSuccess is raised.
+
+        Snapshot sn = Snapshot.create(stateInBytes.toByteArray(),
+            context.getReplicatedLog().getFrom(captureSnapshot.getLastAppliedIndex() + 1),
+            captureSnapshot.getLastIndex(), captureSnapshot.getLastTerm(),
+            captureSnapshot.getLastAppliedIndex(), captureSnapshot.getLastAppliedTerm());
+
+        persistence().saveSnapshot(sn);
+
+        LOG.info("Persisting of snapshot done:{}", sn.getLogMessage());
+
+        //be greedy and remove entries from in-mem journal which are in the snapshot
+        // and update snapshotIndex and snapshotTerm without waiting for the success,
+
+        context.getReplicatedLog().snapshotPreCommit(
+            captureSnapshot.getLastAppliedIndex(),
+            captureSnapshot.getLastAppliedTerm());
+
+        LOG.info("Removed in-memory snapshotted entries, adjusted snaphsotIndex:{} " +
+            "and term:{}", captureSnapshot.getLastAppliedIndex(),
+            captureSnapshot.getLastAppliedTerm());
+
+        if (isLeader() && captureSnapshot.isInstallSnapshotInitiated()) {
+            // this would be call straight to the leader and won't initiate in serialization
+            currentBehavior.handleMessage(getSelf(), new SendInstallSnapshot(stateInBytes));
+        }
+
+        captureSnapshot = null;
+        hasSnapshotCaptureInitiated = false;
+    }
 
     private class ReplicatedLogImpl extends AbstractReplicatedLogImpl {
 
         public ReplicatedLogImpl(Snapshot snapshot) {
-            super(snapshot.getState(),
-                snapshot.getLastAppliedIndex(), snapshot.getLastAppliedTerm(),
+            super(snapshot.getLastAppliedIndex(), snapshot.getLastAppliedTerm(),
                 snapshot.getUnAppliedEntries());
         }
 
@@ -446,7 +671,7 @@ public abstract class RaftActor extends UntypedPersistentActor {
             // FIXME: Maybe this should be done after the command is saved
             journal.subList(adjustedIndex , journal.size()).clear();
 
-            persist(new DeleteEntries(adjustedIndex), new Procedure<DeleteEntries>(){
+            persistence().persist(new DeleteEntries(adjustedIndex), new Procedure<DeleteEntries>(){
 
                 @Override public void apply(DeleteEntries param)
                     throws Exception {
@@ -463,8 +688,11 @@ public abstract class RaftActor extends UntypedPersistentActor {
         public void appendAndPersist(final ActorRef clientActor,
             final String identifier,
             final ReplicatedLogEntry replicatedLogEntry) {
-            context.getLogger().debug(
-                "Append log entry and persist {} ", replicatedLogEntry);
+
+            if(LOG.isDebugEnabled()) {
+                LOG.debug("Append log entry and persist {} ", replicatedLogEntry);
+            }
+
             // FIXME : By adding the replicated log entry to the in-memory journal we are not truly ensuring durability of the logs
             journal.add(replicatedLogEntry);
 
@@ -473,11 +701,14 @@ public abstract class RaftActor extends UntypedPersistentActor {
             // persist call and the execution(s) of the associated event
             // handler. This also holds for multiple persist calls in context
             // of a single command.
-            persist(replicatedLogEntry,
+            persistence().persist(replicatedLogEntry,
                 new Procedure<ReplicatedLogEntry>() {
+                    @Override
                     public void apply(ReplicatedLogEntry evt) throws Exception {
-                        // FIXME : Tentatively create a snapshot every hundred thousand entries. To be tuned.
-                        if (journal.size() > context.getConfigParams().getSnapshotBatchCount()) {
+                        // when a snaphsot is being taken, captureSnapshot != null
+                        if (hasSnapshotCaptureInitiated == false &&
+                            journal.size() % context.getConfigParams().getSnapshotBatchCount() == 0) {
+
                             LOG.info("Initiating Snapshot Capture..");
                             long lastAppliedIndex = -1;
                             long lastAppliedTerm = -1;
@@ -488,31 +719,19 @@ public abstract class RaftActor extends UntypedPersistentActor {
                                 lastAppliedTerm = lastAppliedEntry.getTerm();
                             }
 
-                            LOG.debug("Snapshot Capture logSize: {}", journal.size());
-                            LOG.debug("Snapshot Capture lastApplied:{} ", context.getLastApplied());
-                            LOG.debug("Snapshot Capture lastAppliedIndex:{}", lastAppliedIndex);
-                            LOG.debug("Snapshot Capture lastAppliedTerm:{}", lastAppliedTerm);
-
-                            // create a snapshot object from the state provided and save it
-                            // when snapshot is saved async, SaveSnapshotSuccess is raised.
-                            Snapshot sn = Snapshot.create(createSnapshot(),
-                                getFrom(context.getLastApplied() + 1),
-                                lastIndex(), lastTerm(), lastAppliedIndex,
-                                lastAppliedTerm);
-                            saveSnapshot(sn);
-
-                            LOG.info("Persisting of snapshot done:{}", sn.getLogMessage());
-
-                            //be greedy and remove entries from in-mem journal which are in the snapshot
-                            // and update snapshotIndex and snapshotTerm without waiting for the success,
-                            // TODO: damage-recovery to be done on failure
-                            journal.subList(0, (int) (lastAppliedIndex - snapshotIndex)).clear();
-                            snapshotIndex = lastAppliedIndex;
-                            snapshotTerm = lastAppliedTerm;
-
-                            LOG.info("Removed in-memory snapshotted entries, " +
-                                "adjusted snaphsotIndex:{}" +
-                                "and term:{}", snapshotIndex, lastAppliedTerm);
+                            if(LOG.isDebugEnabled()) {
+                                LOG.debug("Snapshot Capture logSize: {}", journal.size());
+                                LOG.debug("Snapshot Capture lastApplied:{} ",
+                                    context.getLastApplied());
+                                LOG.debug("Snapshot Capture lastAppliedIndex:{}", lastAppliedIndex);
+                                LOG.debug("Snapshot Capture lastAppliedTerm:{}", lastAppliedTerm);
+                            }
+
+                            // send a CaptureSnapshot to self to make the expensive operation async.
+                            getSelf().tell(new CaptureSnapshot(
+                                lastIndex(), lastTerm(), lastAppliedIndex, lastAppliedTerm),
+                                null);
+                            hasSnapshotCaptureInitiated = true;
                         }
                         // Send message for replication
                         if (clientActor != null) {
@@ -528,10 +747,10 @@ public abstract class RaftActor extends UntypedPersistentActor {
 
     }
 
-    private static class DeleteEntries implements Serializable {
+    static class DeleteEntries implements Serializable {
+        private static final long serialVersionUID = 1L;
         private final int fromIndex;
 
-
         public DeleteEntries(int fromIndex) {
             this.fromIndex = fromIndex;
         }
@@ -542,65 +761,6 @@ public abstract class RaftActor extends UntypedPersistentActor {
     }
 
 
-    private static class Snapshot implements Serializable {
-        private final Object state;
-        private final List<ReplicatedLogEntry> unAppliedEntries;
-        private final long lastIndex;
-        private final long lastTerm;
-        private final long lastAppliedIndex;
-        private final long lastAppliedTerm;
-
-        private Snapshot(Object state,
-            List<ReplicatedLogEntry> unAppliedEntries, long lastIndex,
-            long lastTerm, long lastAppliedIndex, long lastAppliedTerm) {
-            this.state = state;
-            this.unAppliedEntries = unAppliedEntries;
-            this.lastIndex = lastIndex;
-            this.lastTerm = lastTerm;
-            this.lastAppliedIndex = lastAppliedIndex;
-            this.lastAppliedTerm = lastAppliedTerm;
-        }
-
-
-        public static Snapshot create(Object state,
-            List<ReplicatedLogEntry> entries, long lastIndex, long lastTerm,
-            long lastAppliedIndex, long lastAppliedTerm) {
-            return new Snapshot(state, entries, lastIndex, lastTerm,
-                lastAppliedIndex, lastAppliedTerm);
-        }
-
-        public Object getState() {
-            return state;
-        }
-
-        public List<ReplicatedLogEntry> getUnAppliedEntries() {
-            return unAppliedEntries;
-        }
-
-        public long getLastTerm() {
-            return lastTerm;
-        }
-
-        public long getLastAppliedIndex() {
-            return lastAppliedIndex;
-        }
-
-        public long getLastAppliedTerm() {
-            return lastAppliedTerm;
-        }
-
-        public String getLogMessage() {
-            StringBuilder sb = new StringBuilder();
-            return sb.append("Snapshot={")
-                .append("lastTerm:" + this.getLastTerm()  + ", ")
-                .append("LastAppliedIndex:" + this.getLastAppliedIndex()  + ", ")
-                .append("LastAppliedTerm:" + this.getLastAppliedTerm()  + ", ")
-                .append("UnAppliedEntries size:" + this.getUnAppliedEntries().size()  + "}")
-                .toString();
-
-        }
-    }
-
     private class ElectionTermImpl implements ElectionTerm {
         /**
          * Identifier of the actor whose election term information this is
@@ -608,17 +768,20 @@ 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;
         }
 
         @Override public void update(long currentTerm, String votedFor) {
-            LOG.debug("Set currentTerm={}, votedFor={}", currentTerm, votedFor);
-
+            if(LOG.isDebugEnabled()) {
+                LOG.debug("Set currentTerm={}, votedFor={}", currentTerm, votedFor);
+            }
             this.currentTerm = currentTerm;
             this.votedFor = votedFor;
         }
@@ -627,7 +790,7 @@ public abstract class RaftActor extends UntypedPersistentActor {
         public void updateAndPersist(long currentTerm, String votedFor){
             update(currentTerm, votedFor);
             // FIXME : Maybe first persist then update the state
-            persist(new UpdateElectionTerm(this.currentTerm, this.votedFor), new Procedure<UpdateElectionTerm>(){
+            persistence().persist(new UpdateElectionTerm(this.currentTerm, this.votedFor), new Procedure<UpdateElectionTerm>(){
 
                 @Override public void apply(UpdateElectionTerm param)
                     throws Exception {
@@ -637,7 +800,8 @@ public abstract class RaftActor extends UntypedPersistentActor {
         }
     }
 
-    private static class UpdateElectionTerm implements Serializable {
+    static class UpdateElectionTerm implements Serializable {
+        private static final long serialVersionUID = 1L;
         private final long currentTerm;
         private final String votedFor;
 
@@ -655,4 +819,38 @@ public abstract class RaftActor extends UntypedPersistentActor {
         }
     }
 
+    protected class NonPersistentRaftDataProvider extends NonPersistentDataProvider {
+
+        public NonPersistentRaftDataProvider(){
+
+        }
+
+        /**
+         * The way snapshotting works is,
+         * <ol>
+         * <li> RaftActor calls createSnapshot on the Shard
+         * <li> Shard sends a CaptureSnapshotReply and RaftActor then calls saveSnapshot
+         * <li> When saveSnapshot is invoked on the akka-persistence API it uses the SnapshotStore to save the snapshot.
+         * The SnapshotStore sends SaveSnapshotSuccess or SaveSnapshotFailure. When the RaftActor gets SaveSnapshot
+         * success it commits the snapshot to the in-memory journal. This commitSnapshot is mimicking what is done
+         * in SaveSnapshotSuccess.
+         * </ol>
+         * @param o
+         */
+        @Override
+        public void saveSnapshot(Object o) {
+            // Make saving Snapshot successful
+            commitSnapshot(-1L);
+        }
+    }
+
+    @VisibleForTesting
+    void setCurrentBehavior(AbstractRaftActorBehavior behavior) {
+        currentBehavior = behavior;
+    }
+
+    protected RaftActorBehavior getCurrentBehavior() {
+        return currentBehavior;
+    }
+
 }