BUG 2676 : Add more context to the Initiate snapshot capture log message
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / RaftActor.java
index 66a46ef3bde0ca8a8d1fa8fe1056ccb463a594d6..65254f2d6277c34c9773570e5938d8a005ab3015 100644 (file)
@@ -10,36 +10,36 @@ package org.opendaylight.controller.cluster.raft;
 
 import akka.actor.ActorRef;
 import akka.actor.ActorSelection;
-import akka.event.Logging;
-import akka.event.LoggingAdapter;
 import akka.japi.Procedure;
 import akka.persistence.RecoveryCompleted;
 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 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.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 org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages;
-import java.io.Serializable;
-import java.util.Map;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * RaftActor encapsulates a state machine that needs to be kept synchronized
@@ -81,9 +81,8 @@ import java.util.Map;
  * <li> when a snapshot should be saved </li>
  * </ul>
  */
-public abstract class RaftActor extends UntypedPersistentActor {
-    protected final LoggingAdapter LOG =
-        Logging.getLogger(getContext().system(), this);
+public abstract class RaftActor extends AbstractUntypedPersistentActor {
+    protected final Logger LOG = LoggerFactory.getLogger(getClass());
 
     /**
      * The current state determines the current behavior of a RaftActor
@@ -104,8 +103,6 @@ public abstract class RaftActor extends UntypedPersistentActor {
 
     private CaptureSnapshot captureSnapshot = null;
 
-    private volatile boolean hasSnapshotCaptureInitiated = false;
-
     private Stopwatch recoveryTimer;
 
     private int currentRecoveryBatchCount;
@@ -126,8 +123,7 @@ public abstract class RaftActor extends UntypedPersistentActor {
 
     private void initRecoveryTimer() {
         if(recoveryTimer == null) {
-            recoveryTimer = new Stopwatch();
-            recoveryTimer.start();
+            recoveryTimer = Stopwatch.createStarted();
         }
     }
 
@@ -135,30 +131,46 @@ public abstract class RaftActor extends UntypedPersistentActor {
     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) {
-            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();
+    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();
+
+                initializeBehavior();
+            }
         }
     }
 
     private void onRecoveredSnapshot(SnapshotOffer offer) {
         if(LOG.isDebugEnabled()) {
-            LOG.debug("SnapshotOffer called..");
+            LOG.debug("{}: SnapshotOffer called..", persistenceId());
         }
 
         initRecoveryTimer();
@@ -174,21 +186,20 @@ public abstract class RaftActor extends UntypedPersistentActor {
         context.setLastApplied(snapshot.getLastAppliedIndex());
         context.setCommitIndex(snapshot.getLastAppliedIndex());
 
-        Stopwatch timer = new Stopwatch();
-        timer.start();
+        Stopwatch timer = Stopwatch.createStarted();
 
         // Apply the snapshot to the actors state
-        applyRecoverySnapshot(ByteString.copyFrom(snapshot.getState()));
+        applyRecoverySnapshot(snapshot.getState());
 
         timer.stop();
         LOG.info("Recovery snapshot applied for {} in {}: snapshotIndex={}, snapshotTerm={}, journal-size=" +
                 replicatedLog.size(), persistenceId(), timer.toString(),
-                replicatedLog.snapshotIndex, replicatedLog.snapshotTerm);
+                replicatedLog.getSnapshotIndex(), replicatedLog.getSnapshotTerm());
     }
 
     private void onRecoveredJournalLogEntry(ReplicatedLogEntry logEntry) {
         if(LOG.isDebugEnabled()) {
-            LOG.debug("Received ReplicatedLogEntry for recovery: {}", logEntry.getIndex());
+            LOG.debug("{}: Received ReplicatedLogEntry for recovery: {}", persistenceId(), logEntry.getIndex());
         }
 
         replicatedLog.append(logEntry);
@@ -196,8 +207,8 @@ public abstract class RaftActor extends UntypedPersistentActor {
 
     private void onRecoveredApplyLogEntries(ApplyLogEntries ale) {
         if(LOG.isDebugEnabled()) {
-            LOG.debug("Received ApplyLogEntries for recovery, applying to state: {} to {}",
-                    context.getLastApplied() + 1, ale.getToIndex());
+            LOG.debug("{}: Received ApplyLogEntries for recovery, applying to state: {} to {}",
+                    persistenceId(), context.getLastApplied() + 1, ale.getToIndex());
         }
 
         for (long i = context.getLastApplied() + 1; i <= ale.getToIndex(); i++) {
@@ -247,20 +258,29 @@ public abstract class RaftActor extends UntypedPersistentActor {
                 "Persistence Id =  " + persistenceId() +
                 " Last index in log={}, snapshotIndex={}, snapshotTerm={}, " +
                 "journal-size={}",
-            replicatedLog.lastIndex(), replicatedLog.snapshotIndex,
-            replicatedLog.snapshotTerm, replicatedLog.size());
+            replicatedLog.lastIndex(), replicatedLog.getSnapshotIndex(),
+            replicatedLog.getSnapshotTerm(), replicatedLog.size());
+
+        initializeBehavior();
+    }
+
+    protected void initializeBehavior(){
+        changeCurrentBehavior(new Follower(context));
+    }
 
-        currentBehavior = new Follower(context);
-        onStateChanged();
+    protected void changeCurrentBehavior(RaftActorBehavior newBehavior){
+        RaftActorBehavior oldBehavior = currentBehavior;
+        currentBehavior = newBehavior;
+        handleBehaviorChange(oldBehavior, currentBehavior);
     }
 
-    @Override public void onReceiveCommand(Object message) {
+    @Override public void handleCommand(Object message) {
         if (message instanceof ApplyState){
             ApplyState applyState = (ApplyState) message;
 
             if(LOG.isDebugEnabled()) {
-                LOG.debug("Applying state for log index {} data {}",
-                    applyState.getReplicatedLogEntry().getIndex(),
+                LOG.debug("{}: Applying state for log index {} data {}",
+                    persistenceId(), applyState.getReplicatedLogEntry().getIndex(),
                     applyState.getReplicatedLogEntry().getData());
             }
 
@@ -270,9 +290,9 @@ public abstract class RaftActor extends UntypedPersistentActor {
         } else if (message instanceof ApplyLogEntries){
             ApplyLogEntries ale = (ApplyLogEntries) message;
             if(LOG.isDebugEnabled()) {
-                LOG.debug("Persisting ApplyLogEntries with index={}", ale.getToIndex());
+                LOG.debug("{}: Persisting ApplyLogEntries with index={}", persistenceId(), ale.getToIndex());
             }
-            persist(new ApplyLogEntries(ale.getToIndex()), new Procedure<ApplyLogEntries>() {
+            persistence().persist(new ApplyLogEntries(ale.getToIndex()), new Procedure<ApplyLogEntries>() {
                 @Override
                 public void apply(ApplyLogEntries param) throws Exception {
                 }
@@ -282,12 +302,13 @@ public abstract class RaftActor extends UntypedPersistentActor {
             Snapshot snapshot = ((ApplySnapshot) message).getSnapshot();
 
             if(LOG.isDebugEnabled()) {
-                LOG.debug("ApplySnapshot called on Follower Actor " +
-                        "snapshotIndex:{}, snapshotTerm:{}", snapshot.getLastAppliedIndex(),
+                LOG.debug("{}: ApplySnapshot called on Follower Actor " +
+                        "snapshotIndex:{}, snapshotTerm:{}", persistenceId(), snapshot.getLastAppliedIndex(),
                     snapshot.getLastAppliedTerm()
                 );
             }
-            applySnapshot(ByteString.copyFrom(snapshot.getState()));
+
+            applySnapshot(snapshot.getState());
 
             //clears the followers log, sets the snapshot index to ensure adjusted-index works
             replicatedLog = new ReplicatedLogImpl(snapshot);
@@ -302,84 +323,63 @@ public abstract class RaftActor extends UntypedPersistentActor {
 
         } else if (message instanceof SaveSnapshotSuccess) {
             SaveSnapshotSuccess success = (SaveSnapshotSuccess) message;
-            LOG.info("SaveSnapshotSuccess received for snapshot");
+            LOG.info("{}: SaveSnapshotSuccess received for snapshot", persistenceId());
 
-            context.getReplicatedLog().snapshotCommit();
+            long sequenceNumber = success.metadata().sequenceNr();
 
-            // TODO: Not sure if we want to be this aggressive with trimming stuff
-            trimPersistentData(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:");
+            LOG.error("{}: SaveSnapshotFailure received for snapshot Cause:",
+                    persistenceId(), saveSnapshotFailure.cause());
 
             context.getReplicatedLog().snapshotRollback();
 
-            LOG.info("Replicated Log rollbacked. Snapshot will be attempted in the next cycle." +
-                "snapshotIndex:{}, snapshotTerm:{}, log-size:{}",
+            LOG.info("{}: Replicated Log rollbacked. Snapshot will be attempted in the next cycle." +
+                "snapshotIndex:{}, snapshotTerm:{}, log-size:{}", persistenceId(),
                 context.getReplicatedLog().getSnapshotIndex(),
                 context.getReplicatedLog().getSnapshotTerm(),
                 context.getReplicatedLog().size());
 
-        } else if (message instanceof AddRaftPeer){
-
-            // 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 RemoveRaftPeer){
-
-            RemoveRaftPeer rrp = (RemoveRaftPeer)message;
-            context.removePeer(rrp.getName());
-
         } else if (message instanceof CaptureSnapshot) {
-            LOG.info("CaptureSnapshot received by actor");
-            CaptureSnapshot cs = (CaptureSnapshot)message;
-            captureSnapshot = cs;
-            createSnapshot();
+            LOG.info("{}: CaptureSnapshot received by actor", persistenceId());
 
-        } else if (message instanceof CaptureSnapshotReply){
-            LOG.info("CaptureSnapshotReply received by actor");
-            CaptureSnapshotReply csr = (CaptureSnapshotReply) message;
+            if(captureSnapshot == null) {
+                captureSnapshot = (CaptureSnapshot)message;
+                createSnapshot();
+            }
 
-            ByteString stateInBytes = csr.getSnapshot();
-            LOG.info("CaptureSnapshotReply stateInBytes size:{}", stateInBytes.size());
-            handleCaptureSnapshotReply(stateInBytes);
+        } else if (message instanceof CaptureSnapshotReply){
+            handleCaptureSnapshotReply(((CaptureSnapshotReply) message).getSnapshot());
 
         } else {
-            if (!(message instanceof AppendEntriesMessages.AppendEntries)
-                && !(message instanceof AppendEntriesReply) && !(message instanceof SendHeartBeat)) {
-                if(LOG.isDebugEnabled()) {
-                    LOG.debug("onReceiveCommand: message: {}", message.getClass());
-                }
-            }
-
             RaftActorBehavior oldBehavior = currentBehavior;
             currentBehavior = currentBehavior.handleMessage(getSender(), message);
 
-            if(oldBehavior != currentBehavior){
-                onStateChanged();
-            }
-
-            onLeaderChanged(oldBehavior.getLeaderId(), currentBehavior.getLeaderId());
+            handleBehaviorChange(oldBehavior, currentBehavior);
         }
     }
 
-    public java.util.Set<String> getPeers() {
-        return context.getPeerAddresses().keySet();
-    }
+    private void handleBehaviorChange(RaftActorBehavior oldBehavior, RaftActorBehavior currentBehavior) {
+        if (oldBehavior != currentBehavior){
+            onStateChanged();
+        }
 
-    protected String getReplicatedLogState() {
-        return "snapshotIndex=" + context.getReplicatedLog().getSnapshotIndex()
-            + ", snapshotTerm=" + context.getReplicatedLog().getSnapshotTerm()
-            + ", im-mem journal size=" + context.getReplicatedLog().size();
-    }
+        String oldBehaviorLeaderId = oldBehavior == null? null : oldBehavior.getLeaderId();
+        String oldBehaviorState = oldBehavior == null? null : oldBehavior.state().name();
 
+        // it can happen that the state has not changed but the leader has changed.
+        onLeaderChanged(oldBehaviorLeaderId, currentBehavior.getLeaderId());
+
+        if (getRoleChangeNotifier().isPresent() &&
+                (oldBehavior == null || (oldBehavior.state() != currentBehavior.state()))) {
+            getRoleChangeNotifier().get().tell(
+                    new RoleChanged(getId(), oldBehaviorState , currentBehavior.state().name()),
+                    getSelf());
+        }
+    }
 
     /**
      * When a derived RaftActor needs to persist something it must call
@@ -389,20 +389,54 @@ public abstract class RaftActor extends UntypedPersistentActor {
      * @param identifier
      * @param data
      */
-    protected void persistData(ActorRef clientActor, String identifier,
-        Payload data) {
+    protected void persistData(final ActorRef clientActor, final String identifier,
+        final Payload data) {
 
         ReplicatedLogEntry replicatedLogEntry = new ReplicatedLogImplEntry(
             context.getReplicatedLog().lastIndex() + 1,
             context.getTermInformation().getCurrentTerm(), data);
 
         if(LOG.isDebugEnabled()) {
-            LOG.debug("Persist data {}", replicatedLogEntry);
+            LOG.debug("{}: Persist data {}", persistenceId(), replicatedLogEntry);
         }
 
+        final RaftActorContext raftContext = getRaftActorContext();
+
         replicatedLog
-            .appendAndPersist(clientActor, identifier, replicatedLogEntry);
-    }
+                .appendAndPersist(replicatedLogEntry, new Procedure<ReplicatedLogEntry>() {
+                    @Override
+                    public void apply(ReplicatedLogEntry replicatedLogEntry) throws Exception {
+                        if(!hasFollowers()){
+                            // Increment the Commit Index and the Last Applied values
+                            raftContext.setCommitIndex(replicatedLogEntry.getIndex());
+                            raftContext.setLastApplied(replicatedLogEntry.getIndex());
+
+                            // Apply the state immediately
+                            applyState(clientActor, identifier, data);
+
+                            // Send a ApplyLogEntries message so that we write the fact that we applied
+                            // the state to durable storage
+                            self().tell(new ApplyLogEntries((int) replicatedLogEntry.getIndex()), self());
+
+                            // Check if the "real" snapshot capture has been initiated. If no then do the fake snapshot
+                            if(!context.isSnapshotCaptureInitiated()){
+                                raftContext.getReplicatedLog().snapshotPreCommit(raftContext.getLastApplied(),
+                                        raftContext.getTermInformation().getCurrentTerm());
+                                raftContext.getReplicatedLog().snapshotCommit();
+                            } else {
+                                LOG.debug("{}: Skipping fake snapshotting for {} because real snapshotting is in progress",
+                                        persistenceId(), getId());
+                            }
+                        } else if (clientActor != null) {
+                            // Send message for replication
+                            currentBehavior.handleMessage(getSelf(),
+                                    new Replicate(clientActor, identifier,
+                                            replicatedLogEntry)
+                            );
+                        }
+
+                    }
+                });    }
 
     protected String getId() {
         return context.getId();
@@ -485,7 +519,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
@@ -515,7 +554,7 @@ public abstract class RaftActor extends UntypedPersistentActor {
 
     /**
      * 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}.
+     * is called 1 or more times after {@link #startLogRecoveryBatch}.
      *
      * @param data the state data
      */
@@ -524,13 +563,13 @@ public abstract class RaftActor extends UntypedPersistentActor {
     /**
      * This method is called during recovery to reconstruct the state of the actor.
      *
-     * @param snapshot A snapshot of the state of the actor
+     * @param snapshotBytes A snapshot of the state of the actor
      */
-    protected abstract void applyRecoverySnapshot(ByteString snapshot);
+    protected abstract void applyRecoverySnapshot(byte[] snapshotBytes);
 
     /**
      * 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}.
+     * log entries. This method is called after {@link #appendRecoveredLogEntry}.
      */
     protected abstract void applyCurrentLogRecoveryBatch();
 
@@ -555,9 +594,9 @@ public abstract class RaftActor extends UntypedPersistentActor {
      * 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
+     * @param snapshotBytes A snapshot of the state of the actor
      */
-    protected abstract void applySnapshot(ByteString snapshot);
+    protected abstract void applySnapshot(byte[] snapshotBytes);
 
     /**
      * This method will be called by the RaftActor when the state of the
@@ -566,17 +605,25 @@ public abstract class RaftActor extends UntypedPersistentActor {
      */
     protected abstract void onStateChanged();
 
+    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();
+
     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(){
@@ -589,47 +636,69 @@ public abstract class RaftActor extends UntypedPersistentActor {
         }
         String peerAddress = context.getPeerAddress(leaderId);
         if(LOG.isDebugEnabled()) {
-            LOG.debug("getLeaderAddress leaderId = {} peerAddress = {}",
-                    leaderId, peerAddress);
+            LOG.debug("{}: getLeaderAddress leaderId = {} peerAddress = {}",
+                    persistenceId(), leaderId, peerAddress);
         }
 
         return peerAddress;
     }
 
-    private void handleCaptureSnapshotReply(ByteString stateInBytes) {
+    private void handleCaptureSnapshotReply(byte[] snapshotBytes) {
+        LOG.info("{}: CaptureSnapshotReply received by actor: snapshot size {}", persistenceId(), snapshotBytes.length);
+
         // 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(),
+        Snapshot sn = Snapshot.create(snapshotBytes,
             context.getReplicatedLog().getFrom(captureSnapshot.getLastAppliedIndex() + 1),
             captureSnapshot.getLastIndex(), captureSnapshot.getLastTerm(),
             captureSnapshot.getLastAppliedIndex(), captureSnapshot.getLastAppliedTerm());
 
-        saveSnapshot(sn);
+        persistence().saveSnapshot(sn);
 
-        LOG.info("Persisting of snapshot done:{}", sn.getLogMessage());
+        LOG.info("{}: Persisting of snapshot done:{}", persistenceId(), 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,
+        long dataThreshold = Runtime.getRuntime().totalMemory() *
+                getRaftActorContext().getConfigParams().getSnapshotDataThresholdPercentage() / 100;
+        if (context.getReplicatedLog().dataSize() > dataThreshold) {
+            // 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.
+            context.getReplicatedLog().snapshotPreCommit(captureSnapshot.getLastAppliedIndex(),
+                    captureSnapshot.getLastAppliedTerm());
 
-        context.getReplicatedLog().snapshotPreCommit(stateInBytes,
-            captureSnapshot.getLastAppliedIndex(),
-            captureSnapshot.getLastAppliedTerm());
+        } else {
+            // clear the log based on replicatedToAllIndex
+            context.getReplicatedLog().snapshotPreCommit(captureSnapshot.getReplicatedToAllIndex(),
+                    captureSnapshot.getReplicatedToAllTerm());
+        }
+        getCurrentBehavior().setReplicatedToAllIndex(captureSnapshot.getReplicatedToAllIndex());
 
-        LOG.info("Removed in-memory snapshotted entries, adjusted snaphsotIndex:{} " +
-            "and term:{}", captureSnapshot.getLastAppliedIndex(),
+        LOG.info("{}: Removed in-memory snapshotted entries, adjusted snaphsotIndex:{} " +
+            "and term:{}", persistenceId(), 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(
+                    ByteString.copyFrom(snapshotBytes)));
+        }
+
         captureSnapshot = null;
-        hasSnapshotCaptureInitiated = false;
+        context.setSnapshotCaptureInitiated(false);
     }
 
+    protected boolean hasFollowers(){
+        return getRaftActorContext().getPeerAddresses().keySet().size() > 0;
+    }
 
     private class ReplicatedLogImpl extends AbstractReplicatedLogImpl {
 
+        private static final int DATA_SIZE_DIVIDER = 5;
+        private long dataSizeSinceLastSnapshot = 0;
+
         public ReplicatedLogImpl(Snapshot snapshot) {
-            super(ByteString.copyFrom(snapshot.getState()),
-                snapshot.getLastAppliedIndex(), snapshot.getLastAppliedTerm(),
+            super(snapshot.getLastAppliedIndex(), snapshot.getLastAppliedTerm(),
                 snapshot.getUnAppliedEntries());
         }
 
@@ -647,26 +716,31 @@ 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 {
+                @Override
+                public void apply(DeleteEntries param)
+                        throws Exception {
                     //FIXME : Doing nothing for now
+                    dataSize = 0;
+                    for (ReplicatedLogEntry entry : journal) {
+                        dataSize += entry.size();
+                    }
                 }
             });
         }
 
         @Override public void appendAndPersist(
             final ReplicatedLogEntry replicatedLogEntry) {
-            appendAndPersist(null, null, replicatedLogEntry);
+            appendAndPersist(replicatedLogEntry, null);
         }
 
-        public void appendAndPersist(final ActorRef clientActor,
-            final String identifier,
-            final ReplicatedLogEntry replicatedLogEntry) {
+        public void appendAndPersist(
+            final ReplicatedLogEntry replicatedLogEntry,
+            final Procedure<ReplicatedLogEntry> callback)  {
 
             if(LOG.isDebugEnabled()) {
-                LOG.debug("Append log entry and persist {} ", replicatedLogEntry);
+                LOG.debug("{}: Append log entry and persist {} ", persistenceId(), replicatedLogEntry);
             }
 
             // FIXME : By adding the replicated log entry to the in-memory journal we are not truly ensuring durability of the logs
@@ -677,44 +751,79 @@ 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 {
+                        int logEntrySize = replicatedLogEntry.size();
+
+                        dataSize += logEntrySize;
+                        long dataSizeForCheck = dataSize;
+
+                        dataSizeSinceLastSnapshot += logEntrySize;
+                        long journalSize = lastIndex() + 1;
+
+                        if(!hasFollowers()) {
+                            // When we do not have followers we do not maintain an in-memory log
+                            // due to this the journalSize will never become anything close to the
+                            // snapshot batch count. In fact will mostly be 1.
+                            // Similarly since the journal's dataSize depends on the entries in the
+                            // journal the journal's dataSize will never reach a value close to the
+                            // memory threshold.
+                            // By maintaining the dataSize outside the journal we are tracking essentially
+                            // what we have written to the disk however since we no longer are in
+                            // need of doing a snapshot just for the sake of freeing up memory we adjust
+                            // the real size of data by the DATA_SIZE_DIVIDER so that we do not snapshot as often
+                            // as if we were maintaining a real snapshot
+                            dataSizeForCheck = dataSizeSinceLastSnapshot / DATA_SIZE_DIVIDER;
+                        }
+
+                        long dataThreshold = Runtime.getRuntime().totalMemory() *
+                                getRaftActorContext().getConfigParams().getSnapshotDataThresholdPercentage() / 100;
+
                         // when a snaphsot is being taken, captureSnapshot != null
-                        if (hasSnapshotCaptureInitiated == false &&
-                            journal.size() % context.getConfigParams().getSnapshotBatchCount() == 0) {
+                        if (!context.isSnapshotCaptureInitiated() &&
+                                ( journalSize % context.getConfigParams().getSnapshotBatchCount() == 0 ||
+                                        dataSizeForCheck > dataThreshold)) {
+
+                            dataSizeSinceLastSnapshot = 0;
+
+                            LOG.info("{}: Initiating Snapshot Capture, journalSize = {}, dataSizeForCheck = {}," +
+                                " dataThreshold = {}", persistenceId(), journalSize, dataSizeForCheck, dataThreshold);
 
-                            LOG.info("Initiating Snapshot Capture..");
                             long lastAppliedIndex = -1;
                             long lastAppliedTerm = -1;
 
                             ReplicatedLogEntry lastAppliedEntry = get(context.getLastApplied());
-                            if (lastAppliedEntry != null) {
+                            if (!hasFollowers()) {
+                                lastAppliedIndex = replicatedLogEntry.getIndex();
+                                lastAppliedTerm = replicatedLogEntry.getTerm();
+                            } else if (lastAppliedEntry != null) {
                                 lastAppliedIndex = lastAppliedEntry.getIndex();
                                 lastAppliedTerm = lastAppliedEntry.getTerm();
                             }
 
                             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);
+                                LOG.debug("{}: Snapshot Capture logSize: {}", persistenceId(), journal.size());
+                                LOG.debug("{}: Snapshot Capture lastApplied:{} ",
+                                        persistenceId(), context.getLastApplied());
+                                LOG.debug("{}: Snapshot Capture lastAppliedIndex:{}", persistenceId(),
+                                        lastAppliedIndex);
+                                LOG.debug("{}: Snapshot Capture lastAppliedTerm:{}", persistenceId(),
+                                        lastAppliedTerm);
                             }
 
                             // send a CaptureSnapshot to self to make the expensive operation async.
-                            getSelf().tell(new CaptureSnapshot(
-                                lastIndex(), lastTerm(), lastAppliedIndex, lastAppliedTerm),
+                            long replicatedToAllIndex = getCurrentBehavior().getReplicatedToAllIndex();
+                            ReplicatedLogEntry replicatedToAllEntry = context.getReplicatedLog().get(replicatedToAllIndex);
+                            getSelf().tell(new CaptureSnapshot(lastIndex(), lastTerm(), lastAppliedIndex, lastAppliedTerm,
+                                (replicatedToAllEntry != null ? replicatedToAllEntry.getIndex() : -1),
+                                (replicatedToAllEntry != null ? replicatedToAllEntry.getTerm() : -1)),
                                 null);
-                            hasSnapshotCaptureInitiated = true;
+                            context.setSnapshotCaptureInitiated(true);
                         }
-                        // Send message for replication
-                        if (clientActor != null) {
-                            currentBehavior.handleMessage(getSelf(),
-                                new Replicate(clientActor, identifier,
-                                    replicatedLogEntry)
-                            );
+                        if (callback != null){
+                            callback.apply(replicatedLogEntry);
                         }
                     }
                 }
@@ -723,10 +832,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;
         }
@@ -756,7 +865,7 @@ public abstract class RaftActor extends UntypedPersistentActor {
 
         @Override public void update(long currentTerm, String votedFor) {
             if(LOG.isDebugEnabled()) {
-                LOG.debug("Set currentTerm={}, votedFor={}", currentTerm, votedFor);
+                LOG.debug("{}: Set currentTerm={}, votedFor={}", persistenceId(), currentTerm, votedFor);
             }
             this.currentTerm = currentTerm;
             this.votedFor = votedFor;
@@ -766,7 +875,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 {
@@ -776,7 +885,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;
 
@@ -794,4 +904,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;
+    }
+
 }