Merge "Add more info to ShardStats JXM bean"
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / behaviors / AbstractLeader.java
index 9b6c08857a50d50e81065c52c39c6c5469687ad9..a63c62fa30740b5830676ab6f15f3de9c1988e7b 100644 (file)
@@ -34,7 +34,6 @@ import org.opendaylight.controller.cluster.raft.RaftActorContext;
 import org.opendaylight.controller.cluster.raft.RaftState;
 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshot;
-import org.opendaylight.controller.cluster.raft.base.messages.InitiateInstallSnapshot;
 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;
@@ -92,17 +91,13 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
 
     private Optional<ByteString> snapshot;
 
-    private long replicatedToAllIndex = -1;
-
     public AbstractLeader(RaftActorContext context) {
-        super(context);
+        super(context, RaftState.Leader);
 
         final Builder<String, FollowerLogInformation> ftlBuilder = ImmutableMap.builder();
         for (String followerId : context.getPeerAddresses().keySet()) {
             FollowerLogInformation followerLogInformation =
-                new FollowerLogInformationImpl(followerId,
-                    context.getCommitIndex(), -1,
-                    context.getConfigParams().getElectionTimeOutInterval());
+                new FollowerLogInformationImpl(followerId, -1, context);
 
             ftlBuilder.put(followerId, followerLogInformation);
         }
@@ -110,7 +105,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
 
         leaderId = context.getId();
 
-        LOG.debug("{}: Election: Leader has following peers: {}", context.getId(), getFollowerIds());
+        LOG.debug("{}: Election: Leader has following peers: {}", logName(), getFollowerIds());
 
         minReplicationCount = getMajorityVoteCount(getFollowerIds().size());
 
@@ -128,7 +123,10 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         // Upon election: send initial empty AppendEntries RPCs
         // (heartbeat) to each server; repeat during idle periods to
         // prevent election timeouts (§5.2)
-        sendAppendEntries(0);
+        sendAppendEntries(0, false);
+
+        // It is important to schedule this heartbeat here
+        scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
     }
 
     /**
@@ -136,14 +134,10 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
      *
      * @return Collection of follower IDs
      */
-    protected final Collection<String> getFollowerIds() {
+    public final Collection<String> getFollowerIds() {
         return followerToLog.keySet();
     }
 
-    private Optional<ByteString> getSnapshot() {
-        return snapshot;
-    }
-
     @VisibleForTesting
     void setSnapshot(Optional<ByteString> snapshot) {
         this.snapshot = snapshot;
@@ -153,9 +147,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
     protected RaftActorBehavior handleAppendEntries(ActorRef sender,
         AppendEntries appendEntries) {
 
-        if(LOG.isDebugEnabled()) {
-            LOG.debug("{}: handleAppendEntries: {}", context.getId(), appendEntries);
-        }
+        LOG.debug("{}: handleAppendEntries: {}", logName(), appendEntries);
 
         return this;
     }
@@ -164,10 +156,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
     protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
         AppendEntriesReply appendEntriesReply) {
 
-        if(! appendEntriesReply.isSuccess()) {
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("{}: handleAppendEntriesReply: {}", context.getId(), appendEntriesReply);
-            }
+        if(LOG.isTraceEnabled()) {
+            LOG.trace("{}: handleAppendEntriesReply: {}", logName(), appendEntriesReply);
         }
 
         // Update the FollowerLogInformation
@@ -176,18 +166,31 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
             followerToLog.get(followerId);
 
         if(followerLogInformation == null){
-            LOG.error("{}: handleAppendEntriesReply - unknown follower {}", context.getId(), followerId);
+            LOG.error("{}: handleAppendEntriesReply - unknown follower {}", logName(), followerId);
             return this;
         }
 
+        if(followerLogInformation.timeSinceLastActivity() >
+                context.getConfigParams().getElectionTimeOutInterval().toMillis()) {
+            LOG.warn("{} : handleAppendEntriesReply delayed beyond election timeout, " +
+                            "appendEntriesReply : {}, timeSinceLastActivity : {}, lastApplied : {}, commitIndex : {}",
+                    logName(), appendEntriesReply, followerLogInformation.timeSinceLastActivity(),
+                    context.getLastApplied(), context.getCommitIndex());
+        }
+
         followerLogInformation.markFollowerActive();
 
+        boolean updated = false;
         if (appendEntriesReply.isSuccess()) {
-            followerLogInformation
-                .setMatchIndex(appendEntriesReply.getLogLastIndex());
-            followerLogInformation
-                .setNextIndex(appendEntriesReply.getLogLastIndex() + 1);
+            updated = followerLogInformation.setMatchIndex(appendEntriesReply.getLogLastIndex());
+            updated = followerLogInformation.setNextIndex(appendEntriesReply.getLogLastIndex() + 1) || updated;
+
+            if(updated && LOG.isDebugEnabled()) {
+                LOG.debug("{}: handleAppendEntriesReply - FollowerLogInformation for {} updated: matchIndex: {}, nextIndex: {}", logName(),
+                        followerId, followerLogInformation.getMatchIndex(), followerLogInformation.getNextIndex());
+            }
         } else {
+            LOG.debug("{}: handleAppendEntriesReply: received unsuccessful reply: {}", logName(), appendEntriesReply);
 
             // TODO: When we find that the follower is out of sync with the
             // Leader we simply decrement that followers next index by 1.
@@ -224,6 +227,11 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
 
         // Apply the change to the state machine
         if (context.getCommitIndex() > context.getLastApplied()) {
+            if(LOG.isDebugEnabled()) {
+                LOG.debug("{}: handleAppendEntriesReply from {}: applying to log - commitIndex: {}, lastAppliedIndex: {}",
+                        logName(), followerId, context.getCommitIndex(), context.getLastApplied());
+            }
+
             applyLogToStateMachine(context.getCommitIndex());
         }
 
@@ -232,20 +240,21 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         }
 
         //Send the next log entry immediately, if possible, no need to wait for heartbeat to trigger that event
-        sendUpdatesToFollower(followerId, followerLogInformation, false);
+        sendUpdatesToFollower(followerId, followerLogInformation, false, !updated);
         return this;
     }
 
     private void purgeInMemoryLog() {
-        //find the lowest index across followers which has been replicated to all. -1 if there are no followers.
+        //find the lowest index across followers which has been replicated to all.
+        // lastApplied if there are no followers, so that we keep clearing the log for single-node
         // we would delete the in-mem log from that index on, in-order to minimize mem usage
         // we would also share this info thru AE with the followers so that they can delete their log entries as well.
-        long minReplicatedToAllIndex = followerToLog.isEmpty() ? -1 : Long.MAX_VALUE;
+        long minReplicatedToAllIndex = followerToLog.isEmpty() ? context.getLastApplied() : Long.MAX_VALUE;
         for (FollowerLogInformation info : followerToLog.values()) {
             minReplicatedToAllIndex = Math.min(minReplicatedToAllIndex, info.getMatchIndex());
         }
 
-        replicatedToAllIndex = fakeSnapshot(minReplicatedToAllIndex, replicatedToAllIndex);
+        super.performSnapshotWithoutCapture(minReplicatedToAllIndex);
     }
 
     @Override
@@ -278,10 +287,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         return this;
     }
 
-    @Override
-    public RaftState state() {
-        return RaftState.Leader;
-    }
+    protected void beforeSendHeartbeat(){}
 
     @Override
     public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) {
@@ -295,8 +301,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
             // set currentTerm = T, convert to follower (§5.1)
             // This applies to all RPC messages and responses
             if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
-                LOG.debug("{}: Term {} in \"{}\" message is greater than leader's term {}", context.getId(),
-                        rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm());
+                LOG.debug("{}: Term {} in \"{}\" message is greater than leader's term {} - switching to Follower",
+                        logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm());
 
                 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
 
@@ -304,40 +310,38 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
             }
         }
 
-        try {
-            if (message instanceof SendHeartBeat) {
-                sendHeartBeat();
-                return this;
-
-            } else if(message instanceof InitiateInstallSnapshot) {
-                installSnapshotIfNeeded();
+        if (message instanceof SendHeartBeat) {
+            beforeSendHeartbeat();
+            sendHeartBeat();
+            scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
+            return this;
 
-            } else if(message instanceof SendInstallSnapshot) {
-                // received from RaftActor
-                setSnapshot(Optional.of(((SendInstallSnapshot) message).getSnapshot()));
-                sendInstallSnapshot();
+        } else if(message instanceof SendInstallSnapshot) {
+            // received from RaftActor
+            setSnapshot(Optional.of(((SendInstallSnapshot) message).getSnapshot()));
+            sendInstallSnapshot();
 
-            } else if (message instanceof Replicate) {
-                replicate((Replicate) message);
+        } else if (message instanceof Replicate) {
+            replicate((Replicate) message);
 
-            } else if (message instanceof InstallSnapshotReply){
-                handleInstallSnapshotReply((InstallSnapshotReply) message);
+        } else if (message instanceof InstallSnapshotReply){
+            handleInstallSnapshotReply((InstallSnapshotReply) message);
 
-            }
-        } finally {
-            scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
         }
 
+
         return super.handleMessage(sender, message);
     }
 
     private void handleInstallSnapshotReply(InstallSnapshotReply reply) {
+        LOG.debug("{}: handleInstallSnapshotReply: {}", logName(), reply);
+
         String followerId = reply.getFollowerId();
         FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
 
         if (followerToSnapshot == null) {
             LOG.error("{}: FollowerId {} in InstallSnapshotReply not known to Leader",
-                    context.getId(), followerId);
+                    logName(), followerId);
             return;
         }
 
@@ -351,8 +355,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
                     //this was the last chunk reply
                     if(LOG.isDebugEnabled()) {
                         LOG.debug("{}: InstallSnapshotReply received, " +
-                                "last chunk received, Chunk:{}. Follower:{} Setting nextIndex:{}",
-                                context.getId(), reply.getChunkIndex(), followerId,
+                                "last chunk received, Chunk: {}. Follower: {} Setting nextIndex: {}",
+                                logName(), reply.getChunkIndex(), followerId,
                             context.getReplicatedLog().getSnapshotIndex() + 1
                         );
                     }
@@ -363,10 +367,9 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
                         context.getReplicatedLog().getSnapshotIndex() + 1);
                     mapFollowerToSnapshot.remove(followerId);
 
-                    if(LOG.isDebugEnabled()) {
-                        LOG.debug("{}: followerToLog.get(followerId).getNextIndex()=" +
-                                context.getId(), followerToLog.get(followerId).getNextIndex());
-                    }
+                    LOG.debug("{}: follower: {}, matchIndex set to {}, nextIndex set to {}",
+                                logName(), followerId, followerLogInformation.getMatchIndex(),
+                                followerLogInformation.getNextIndex());
 
                     if (mapFollowerToSnapshot.isEmpty()) {
                         // once there are no pending followers receiving snapshots
@@ -380,12 +383,15 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
                 }
             } else {
                 LOG.info("{}: InstallSnapshotReply received sending snapshot chunk failed, Will retry, Chunk: {}",
-                        context.getId(), reply.getChunkIndex());
+                        logName(), reply.getChunkIndex());
 
                 followerToSnapshot.markSendStatus(false);
             }
 
-            if (!wasLastChunk && followerToSnapshot.canSendNextChunk()) {
+            if (wasLastChunk && !context.isSnapshotCaptureInitiated()) {
+                // Since the follower is now caught up try to purge the log.
+                purgeInMemoryLog();
+            } else if (!wasLastChunk && followerToSnapshot.canSendNextChunk()) {
                 ActorSelection followerActor = context.getPeerActorSelection(followerId);
                 if(followerActor != null) {
                     sendSnapshotChunk(followerActor, followerId);
@@ -394,7 +400,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
 
         } else {
             LOG.error("{}: Chunk index {} in InstallSnapshotReply from follower {} does not match expected index {}",
-                    context.getId(), reply.getChunkIndex(), followerId,
+                    logName(), reply.getChunkIndex(), followerId,
                     followerToSnapshot.getChunkIndex());
 
             if(reply.getChunkIndex() == INVALID_CHUNK_INDEX){
@@ -408,9 +414,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
     private void replicate(Replicate replicate) {
         long logIndex = replicate.getReplicatedLogEntry().getIndex();
 
-        if(LOG.isDebugEnabled()) {
-            LOG.debug("{}: Replicate message {}", context.getId(), logIndex);
-        }
+        LOG.debug("{}: Replicate message: identifier: {}, logIndex: {}", logName(),
+                replicate.getIdentifier(), logIndex);
 
         // Create a tracker entry we will use this later to notify the
         // client actor
@@ -424,11 +429,11 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
             context.setCommitIndex(logIndex);
             applyLogToStateMachine(logIndex);
         } else {
-            sendAppendEntries(0);
+            sendAppendEntries(0, false);
         }
     }
 
-    private void sendAppendEntries(long timeSinceLastActivityInterval) {
+    private void sendAppendEntries(long timeSinceLastActivityInterval, boolean isHeartbeat) {
         // Send an AppendEntries to all followers
         for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
             final String followerId = e.getKey();
@@ -436,7 +441,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
             // This checks helps not to send a repeat message to the follower
             if(!followerLogInformation.isFollowerActive() ||
                     followerLogInformation.timeSinceLastActivity() >= timeSinceLastActivityInterval) {
-                sendUpdatesToFollower(followerId, followerLogInformation, true);
+                sendUpdatesToFollower(followerId, followerLogInformation, true, isHeartbeat);
             }
         }
     }
@@ -449,12 +454,14 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
      */
 
     private void sendUpdatesToFollower(String followerId, FollowerLogInformation followerLogInformation,
-                                          boolean sendHeartbeat) {
+                                       boolean sendHeartbeat, boolean isHeartbeat) {
 
         ActorSelection followerActor = context.getPeerActorSelection(followerId);
         if (followerActor != null) {
             long followerNextIndex = followerLogInformation.getNextIndex();
             boolean isFollowerActive = followerLogInformation.isFollowerActive();
+            boolean sendAppendEntries = false;
+            List<ReplicatedLogEntry> entries = Collections.EMPTY_LIST;
 
             if (mapFollowerToSnapshot.get(followerId) != null) {
                 // if install snapshot is in process , then sent next chunk if possible
@@ -462,46 +469,56 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
                     sendSnapshotChunk(followerActor, followerId);
                 } else if(sendHeartbeat) {
                     // we send a heartbeat even if we have not received a reply for the last chunk
-                    sendAppendEntriesToFollower(followerActor, followerLogInformation.getNextIndex(),
-                        Collections.<ReplicatedLogEntry>emptyList(), followerId);
+                    sendAppendEntries = true;
                 }
             } else {
                 long leaderLastIndex = context.getReplicatedLog().lastIndex();
                 long leaderSnapShotIndex = context.getReplicatedLog().getSnapshotIndex();
-                if (isFollowerActive &&
-                    context.getReplicatedLog().isPresent(followerNextIndex)) {
-                    // FIXME : Sending one entry at a time
-                    final List<ReplicatedLogEntry> entries = context.getReplicatedLog().getFrom(followerNextIndex, 1);
 
-                    sendAppendEntriesToFollower(followerActor, followerNextIndex, entries, followerId);
+                if((!isHeartbeat && LOG.isDebugEnabled()) || LOG.isTraceEnabled()) {
+                    LOG.debug("{}: Checking sendAppendEntries for follower {}, followerNextIndex {}, leaderLastIndex: {}, leaderSnapShotIndex: {}",
+                            logName(), followerId, followerNextIndex, leaderLastIndex, leaderSnapShotIndex);
+                }
+
+                if (isFollowerActive && context.getReplicatedLog().isPresent(followerNextIndex)) {
+
+                    LOG.debug("{}: sendAppendEntries: {} is present for follower {}", logName(),
+                            followerNextIndex, followerId);
 
+                    // FIXME : Sending one entry at a time
+                    if(followerLogInformation.okToReplicate()) {
+                        entries = context.getReplicatedLog().getFrom(followerNextIndex, 1);
+                        sendAppendEntries = true;
+                    }
                 } else if (isFollowerActive && followerNextIndex >= 0 &&
-                    leaderLastIndex >= followerNextIndex) {
+                    leaderLastIndex > followerNextIndex && !context.isSnapshotCaptureInitiated()) {
                     // if the followers next index is not present in the leaders log, and
                     // if the follower is just not starting and if leader's index is more than followers index
                     // then snapshot should be sent
 
                     if (LOG.isDebugEnabled()) {
-                        LOG.debug("InitiateInstallSnapshot to follower:{}," +
-                                "follower-nextIndex:{}, leader-snapshot-index:{},  " +
-                                "leader-last-index:{}", followerId,
-                            followerNextIndex, leaderSnapShotIndex, leaderLastIndex
-                        );
+                        LOG.debug(String.format("%s: InitiateInstallSnapshot to follower: %s," +
+                                    "follower-nextIndex: %d, leader-snapshot-index: %d,  " +
+                                    "leader-last-index: %d", logName(), followerId,
+                                    followerNextIndex, leaderSnapShotIndex, leaderLastIndex));
                     }
-                    actor().tell(new InitiateInstallSnapshot(), actor());
 
                     // Send heartbeat to follower whenever install snapshot is initiated.
-                    sendAppendEntriesToFollower(followerActor, followerLogInformation.getNextIndex(),
-                            Collections.<ReplicatedLogEntry>emptyList(), followerId);
+                    sendAppendEntries = true;
+                    initiateCaptureSnapshot(followerId, followerNextIndex);
 
                 } else if(sendHeartbeat) {
-                    //we send an AppendEntries, even if the follower is inactive
+                    // we send an AppendEntries, even if the follower is inactive
                     // in-order to update the followers timestamp, in case it becomes active again
-                    sendAppendEntriesToFollower(followerActor, followerLogInformation.getNextIndex(),
-                        Collections.<ReplicatedLogEntry>emptyList(), followerId);
+                    sendAppendEntries = true;
                 }
 
             }
+
+            if(sendAppendEntries) {
+                sendAppendEntriesToFollower(followerActor, followerNextIndex,
+                        entries, followerId);
+            }
         }
     }
 
@@ -510,10 +527,10 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         AppendEntries appendEntries = new AppendEntries(currentTerm(), context.getId(),
             prevLogIndex(followerNextIndex),
             prevLogTerm(followerNextIndex), entries,
-            context.getCommitIndex(), replicatedToAllIndex);
+            context.getCommitIndex(), super.getReplicatedToAllIndex());
 
-        if(!entries.isEmpty()) {
-            LOG.debug("{}: Sending AppendEntries to follower {}: {}", context.getId(), followerId,
+        if(!entries.isEmpty() || LOG.isTraceEnabled()) {
+            LOG.debug("{}: Sending AppendEntries to follower {}: {}", logName(), followerId,
                     appendEntries);
         }
 
@@ -521,78 +538,68 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
     }
 
     /**
-     * An installSnapshot is scheduled at a interval that is a multiple of
-     * a HEARTBEAT_INTERVAL. This is to avoid the need to check for installing
-     * snapshots at every heartbeat.
-     *
      * Install Snapshot works as follows
-     * 1. Leader sends a InitiateInstallSnapshot message to self
-     * 2. Leader then initiates the capture snapshot by sending a CaptureSnapshot message to actor
-     * 3. RaftActor on receipt of the CaptureSnapshotReply (from Shard), stores the received snapshot in the replicated log
+     * 1. Leader initiates the capture snapshot by sending a CaptureSnapshot message to actor
+     * 2. RaftActor on receipt of the CaptureSnapshotReply (from Shard), stores the received snapshot in the replicated log
      * and makes a call to Leader's handleMessage , with SendInstallSnapshot message.
-     * 4. Leader , picks the snapshot from im-mem ReplicatedLog and sends it in chunks to the Follower
-     * 5. On complete, Follower sends back a InstallSnapshotReply.
-     * 6. On receipt of the InstallSnapshotReply for the last chunk, Leader marks the install complete for that follower
+     * 3. Leader , picks the snapshot from im-mem ReplicatedLog and sends it in chunks to the Follower
+     * 4. On complete, Follower sends back a InstallSnapshotReply.
+     * 5. On receipt of the InstallSnapshotReply for the last chunk, Leader marks the install complete for that follower
      * and replenishes the memory by deleting the snapshot in Replicated log.
-     *
+     * 6. If another follower requires a snapshot and a snapshot has been collected (via CaptureSnapshotReply)
+     * then send the existing snapshot in chunks to the follower.
+     * @param followerId
+     * @param followerNextIndex
      */
-    private void installSnapshotIfNeeded() {
-        if(LOG.isDebugEnabled()) {
-            LOG.debug("{}: installSnapshotIfNeeded, followers {}", context.getId(), followerToLog.keySet());
-        }
+    private void initiateCaptureSnapshot(String followerId, long followerNextIndex) {
+        if (!context.getReplicatedLog().isPresent(followerNextIndex) &&
+                context.getReplicatedLog().isInSnapshot(followerNextIndex)) {
 
-        for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
-            final ActorSelection followerActor = context.getPeerActorSelection(e.getKey());
+            if (snapshot.isPresent()) {
+                // if a snapshot is present in the memory, most likely another install is in progress
+                // no need to capture snapshot.
+                // This could happen if another follower needs an install when one is going on.
+                final ActorSelection followerActor = context.getPeerActorSelection(followerId);
+                sendSnapshotChunk(followerActor, followerId);
+
+            } else if (!context.isSnapshotCaptureInitiated()) {
+
+                ReplicatedLogEntry lastAppliedEntry = context.getReplicatedLog().get(context.getLastApplied());
+                long lastAppliedIndex = -1;
+                long lastAppliedTerm = -1;
+
+                if (lastAppliedEntry != null) {
+                    lastAppliedIndex = lastAppliedEntry.getIndex();
+                    lastAppliedTerm = lastAppliedEntry.getTerm();
+                } else if (context.getReplicatedLog().getSnapshotIndex() > -1) {
+                    lastAppliedIndex = context.getReplicatedLog().getSnapshotIndex();
+                    lastAppliedTerm = context.getReplicatedLog().getSnapshotTerm();
+                }
 
-            if (followerActor != null) {
-                long nextIndex = e.getValue().getNextIndex();
+                boolean isInstallSnapshotInitiated = true;
+                long replicatedToAllIndex = super.getReplicatedToAllIndex();
+                ReplicatedLogEntry replicatedToAllEntry = context.getReplicatedLog().get(replicatedToAllIndex);
 
-                if (!context.getReplicatedLog().isPresent(nextIndex) &&
-                        context.getReplicatedLog().isInSnapshot(nextIndex)) {
-                    LOG.info("{}: {} follower needs a snapshot install", context.getId(), e.getKey());
-                    if (snapshot.isPresent()) {
-                        // if a snapshot is present in the memory, most likely another install is in progress
-                        // no need to capture snapshot
-                        sendSnapshotChunk(followerActor, e.getKey());
-
-                    } else if (!context.isSnapshotCaptureInitiated()) {
-                        initiateCaptureSnapshot();
-                        //we just need 1 follower who would need snapshot to be installed.
-                        // when we have the snapshot captured, we would again check (in SendInstallSnapshot)
-                        // who needs an install and send to all who need
-                        break;
-                    }
+                CaptureSnapshot captureSnapshot = new CaptureSnapshot(
+                        lastIndex(), lastTerm(), lastAppliedIndex, lastAppliedTerm,
+                        (replicatedToAllEntry != null ? replicatedToAllEntry.getIndex() : -1),
+                        (replicatedToAllEntry != null ? replicatedToAllEntry.getTerm() : -1),
+                        isInstallSnapshotInitiated);
 
+                if(LOG.isDebugEnabled()) {
+                    LOG.debug("{}: Initiating install snapshot to follower {}: {}", logName(), followerId,
+                            captureSnapshot);
                 }
+
+                actor().tell(captureSnapshot, actor());
+                context.setSnapshotCaptureInitiated(true);
             }
         }
     }
 
-    // on every install snapshot, we try to capture the snapshot.
-    // Once a capture is going on, another one issued will get ignored by RaftActor.
-    private void initiateCaptureSnapshot() {
-        LOG.info("{}: Initiating Snapshot Capture to Install Snapshot, Leader:{}", context.getId(), getLeaderId());
-        ReplicatedLogEntry lastAppliedEntry = context.getReplicatedLog().get(context.getLastApplied());
-        long lastAppliedIndex = -1;
-        long lastAppliedTerm = -1;
-
-        if (lastAppliedEntry != null) {
-            lastAppliedIndex = lastAppliedEntry.getIndex();
-            lastAppliedTerm = lastAppliedEntry.getTerm();
-        } else if (context.getReplicatedLog().getSnapshotIndex() > -1)  {
-            lastAppliedIndex = context.getReplicatedLog().getSnapshotIndex();
-            lastAppliedTerm = context.getReplicatedLog().getSnapshotTerm();
-        }
-
-        boolean isInstallSnapshotInitiated = true;
-        actor().tell(new CaptureSnapshot(lastIndex(), lastTerm(),
-                lastAppliedIndex, lastAppliedTerm, isInstallSnapshotInitiated),
-            actor());
-        context.setSnapshotCaptureInitiated(true);
-    }
-
 
     private void sendInstallSnapshot() {
+        LOG.debug("{}: sendInstallSnapshot", logName());
         for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
             ActorSelection followerActor = context.getPeerActorSelection(e.getKey());
 
@@ -625,19 +632,21 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
                         context.getReplicatedLog().getSnapshotIndex(),
                         context.getReplicatedLog().getSnapshotTerm(),
                         nextSnapshotChunk,
-                            followerToSnapshot.incrementChunkIndex(),
-                            followerToSnapshot.getTotalChunks(),
+                        followerToSnapshot.incrementChunkIndex(),
+                        followerToSnapshot.getTotalChunks(),
                         Optional.of(followerToSnapshot.getLastChunkHashCode())
                     ).toSerializable(),
                     actor()
                 );
-                LOG.info("{}: InstallSnapshot sent to follower {}, Chunk: {}/{}",
-                        context.getId(), followerActor.path(),
-                        followerToSnapshot.getChunkIndex(),
-                        followerToSnapshot.getTotalChunks());
+
+                if(LOG.isDebugEnabled()) {
+                    LOG.debug("{}: InstallSnapshot sent to follower {}, Chunk: {}/{}",
+                            logName(), followerActor.path(), followerToSnapshot.getChunkIndex(),
+                            followerToSnapshot.getTotalChunks());
+                }
             }
         } catch (IOException e) {
-            LOG.error("{}: InstallSnapshot failed for Leader.", context.getId(), e);
+            LOG.error("{}: InstallSnapshot failed for Leader.", logName(), e);
         }
     }
 
@@ -652,15 +661,16 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
             mapFollowerToSnapshot.put(followerId, followerToSnapshot);
         }
         ByteString nextChunk = followerToSnapshot.getNextChunk();
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("{}: Leader's snapshot nextChunk size:{}", context.getId(), nextChunk.size());
-        }
+
+        LOG.debug("{}: next snapshot chunk size for follower {}: {}", logName(), followerId, nextChunk.size());
+
         return nextChunk;
     }
 
     private void sendHeartBeat() {
         if (!followerToLog.isEmpty()) {
-            sendAppendEntries(context.getConfigParams().getHeartBeatInterval().toMillis());
+            LOG.trace("{}: Sending heartbeat", logName());
+            sendAppendEntries(context.getConfigParams().getHeartBeatInterval().toMillis(), true);
         }
     }
 
@@ -735,7 +745,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
                 ((size % context.getConfigParams().getSnapshotChunkSize()) > 0 ? 1 : 0);
             if(LOG.isDebugEnabled()) {
                 LOG.debug("{}: Snapshot {} bytes, total chunks to send:{}",
-                        context.getId(), size, totalChunks);
+                        logName(), size, totalChunks);
             }
             replyReceivedForOffset = -1;
             chunkIndex = AbstractLeader.FIRST_CHUNK_INDEX;
@@ -803,10 +813,10 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
                 }
             }
 
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("{}: Next chunk: length={}, offset={},size={}", context.getId(),
+
+            LOG.debug("{}: Next chunk: length={}, offset={},size={}", logName(),
                     snapshotLength, start, size);
-            }
+
             ByteString substring = getSnapshotBytes().substring(start, start + size);
             nextChunkHashCode = substring.hashCode();
             return substring;