Measure follower activity in nanoseconds
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / behaviors / AbstractLeader.java
index 74e8b2049f1506d148c4c4782c17091266209778..15dbd74d01d9b2d3b3aed4b98fdb414b3943c5ae 100644 (file)
@@ -14,9 +14,9 @@ import akka.actor.Cancellable;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
-import com.google.common.base.Throwables;
 import com.google.common.io.ByteSource;
 import java.io.IOException;
+import java.io.ObjectOutputStream;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
@@ -26,11 +26,14 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Queue;
+import java.util.concurrent.TimeUnit;
 import javax.annotation.Nullable;
+import org.opendaylight.controller.cluster.io.SharedFileBackedOutputStream;
+import org.opendaylight.controller.cluster.messaging.MessageSlicer;
+import org.opendaylight.controller.cluster.messaging.SliceOptions;
 import org.opendaylight.controller.cluster.raft.ClientRequestTracker;
 import org.opendaylight.controller.cluster.raft.ClientRequestTrackerImpl;
 import org.opendaylight.controller.cluster.raft.FollowerLogInformation;
-import org.opendaylight.controller.cluster.raft.FollowerLogInformationImpl;
 import org.opendaylight.controller.cluster.raft.PeerInfo;
 import org.opendaylight.controller.cluster.raft.RaftActorContext;
 import org.opendaylight.controller.cluster.raft.RaftState;
@@ -45,6 +48,7 @@ import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot;
 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshotReply;
 import org.opendaylight.controller.cluster.raft.messages.RaftRPC;
+import org.opendaylight.controller.cluster.raft.messages.RequestVote;
 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
 import org.opendaylight.controller.cluster.raft.messages.UnInitializedFollowerSnapshotReply;
 import org.opendaylight.controller.cluster.raft.persisted.ServerConfigurationPayload;
@@ -52,7 +56,7 @@ import org.opendaylight.controller.cluster.raft.persisted.Snapshot;
 import scala.concurrent.duration.FiniteDuration;
 
 /**
- * The behavior of a RaftActor when it is in the Leader state
+ * The behavior of a RaftActor when it is in the Leader state.
  *
  * <p>
  * Leaders:
@@ -84,21 +88,34 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
      */
     private final Queue<ClientRequestTracker> trackers = new LinkedList<>();
 
+    /**
+     * Map of serialized AppendEntries output streams keyed by log index. This is used in conjunction with the
+     * appendEntriesMessageSlicer for slicing single ReplicatedLogEntry payloads that exceed the message size threshold.
+     * This Map allows the SharedFileBackedOutputStreams to be reused for multiple followers.
+     */
+    private final Map<Long, SharedFileBackedOutputStream> sharedSerializedAppendEntriesStreams = new HashMap<>();
+    private final MessageSlicer appendEntriesMessageSlicer;
+
     private Cancellable heartbeatSchedule = null;
     private Optional<SnapshotHolder> snapshotHolder = Optional.absent();
     private int minReplicationCount;
 
-    protected AbstractLeader(RaftActorContext context, RaftState state,
-            @Nullable AbstractLeader initializeFromLeader) {
+    protected AbstractLeader(final RaftActorContext context, final RaftState state,
+            @Nullable final AbstractLeader initializeFromLeader) {
         super(context, state);
 
+        appendEntriesMessageSlicer = MessageSlicer.builder().logContext(logName())
+            .messageSliceSize(context.getConfigParams().getSnapshotChunkSize())
+            .expireStateAfterInactivity(context.getConfigParams().getElectionTimeOutInterval().toMillis() * 3,
+                    TimeUnit.MILLISECONDS).build();
+
         if (initializeFromLeader != null) {
             followerToLog.putAll(initializeFromLeader.followerToLog);
             snapshotHolder = initializeFromLeader.snapshotHolder;
             trackers.addAll(initializeFromLeader.trackers);
         } else {
             for (PeerInfo peerInfo: context.getPeers()) {
-                FollowerLogInformation followerLogInformation = new FollowerLogInformationImpl(peerInfo, -1, context);
+                FollowerLogInformation followerLogInformation = new FollowerLogInformation(peerInfo, context);
                 followerToLog.put(peerInfo.getId(), followerLogInformation);
             }
         }
@@ -117,7 +134,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
     }
 
-    protected AbstractLeader(RaftActorContext context, RaftState state) {
+    protected AbstractLeader(final RaftActorContext context, final RaftState state) {
         this(context, state, null);
     }
 
@@ -130,9 +147,9 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         return followerToLog.keySet();
     }
 
-    public void addFollower(String followerId) {
-        FollowerLogInformation followerLogInformation = new FollowerLogInformationImpl(
-                context.getPeerInfo(followerId), -1, context);
+    public void addFollower(final String followerId) {
+        FollowerLogInformation followerLogInformation = new FollowerLogInformation(context.getPeerInfo(followerId),
+            context);
         followerToLog.put(followerId, followerLogInformation);
 
         if (heartbeatSchedule == null) {
@@ -140,7 +157,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         }
     }
 
-    public void removeFollower(String followerId) {
+    public void removeFollower(final String followerId) {
         followerToLog.remove(followerId);
     }
 
@@ -167,7 +184,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
     }
 
     @VisibleForTesting
-    void setSnapshot(@Nullable SnapshotHolder snapshotHolder) {
+    void setSnapshotHolder(@Nullable final SnapshotHolder snapshotHolder) {
         this.snapshotHolder = Optional.fromNullable(snapshotHolder);
     }
 
@@ -177,8 +194,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
     }
 
     @Override
-    protected RaftActorBehavior handleAppendEntries(ActorRef sender,
-        AppendEntries appendEntries) {
+    protected RaftActorBehavior handleAppendEntries(final ActorRef sender,
+        final AppendEntries appendEntries) {
 
         log.debug("{}: handleAppendEntries: {}", logName(), appendEntries);
 
@@ -186,7 +203,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
     }
 
     @Override
-    protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender, AppendEntriesReply appendEntriesReply) {
+    protected RaftActorBehavior handleAppendEntriesReply(final ActorRef sender,
+            final AppendEntriesReply appendEntriesReply) {
         log.trace("{}: handleAppendEntriesReply: {}", logName(), appendEntriesReply);
 
         // Update the FollowerLogInformation
@@ -198,11 +216,11 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
             return this;
         }
 
-        if (followerLogInformation.timeSinceLastActivity()
-                > context.getConfigParams().getElectionTimeOutInterval().toMillis()) {
+        final long lastActivityNanos = followerLogInformation.nanosSinceLastActivity();
+        if (lastActivityNanos > context.getConfigParams().getElectionTimeOutInterval().toNanos()) {
             log.warn("{} : handleAppendEntriesReply delayed beyond election timeout, "
                     + "appendEntriesReply : {}, timeSinceLastActivity : {}, lastApplied : {}, commitIndex : {}",
-                    logName(), appendEntriesReply, followerLogInformation.timeSinceLastActivity(),
+                    logName(), appendEntriesReply, TimeUnit.NANOSECONDS.toMillis(lastActivityNanos),
                     context.getLastApplied(), context.getCommitIndex());
         }
 
@@ -282,18 +300,18 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
 
                 log.info("{}: follower {} appears to be behind the leader from the last snapshot - "
                     + "updated: matchIndex: {}, nextIndex: {}", logName(), followerId,
-                    appendEntriesReply.getLogLastTerm(), followerLogInformation.getMatchIndex(),
-                    followerLogInformation.getNextIndex());
+                    followerLogInformation.getMatchIndex(), followerLogInformation.getNextIndex());
             } else {
                 // The follower's log conflicts with leader's log so decrement follower's next index by 1
                 // in an attempt to find where the logs match.
 
-                followerLogInformation.decrNextIndex();
-                updated = true;
+                if (followerLogInformation.decrNextIndex()) {
+                    updated = true;
 
-                log.info("{}: follower {} last log term {} conflicts with the leader's {} - dec next index to {}",
-                        logName(), followerId, appendEntriesReply.getLogLastTerm(), followersLastLogTermInLeadersLog,
-                        followerLogInformation.getNextIndex());
+                    log.info("{}: follower {} last log term {} conflicts with the leader's {} - dec next index to {}",
+                            logName(), followerId, appendEntriesReply.getLogLastTerm(),
+                            followersLastLogTermInLeadersLog, followerLogInformation.getNextIndex());
+                }
             }
         }
 
@@ -383,8 +401,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         }
     }
 
-    private boolean updateFollowerLogInformation(FollowerLogInformation followerLogInformation,
-            AppendEntriesReply appendEntriesReply) {
+    private boolean updateFollowerLogInformation(final FollowerLogInformation followerLogInformation,
+            final AppendEntriesReply appendEntriesReply) {
         boolean updated = followerLogInformation.setMatchIndex(appendEntriesReply.getLogLastIndex());
         updated = followerLogInformation.setNextIndex(appendEntriesReply.getLogLastIndex() + 1) || updated;
 
@@ -411,7 +429,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
     }
 
     @Override
-    protected ClientRequestTracker removeClientRequestTracker(long logIndex) {
+    protected ClientRequestTracker removeClientRequestTracker(final long logIndex) {
         final Iterator<ClientRequestTracker> it = trackers.iterator();
         while (it.hasNext()) {
             final ClientRequestTracker t = it.next();
@@ -425,17 +443,21 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
     }
 
     @Override
-    protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
-        RequestVoteReply requestVoteReply) {
+    protected RaftActorBehavior handleRequestVoteReply(final ActorRef sender,
+        final RequestVoteReply requestVoteReply) {
         return this;
     }
 
     protected void beforeSendHeartbeat(){}
 
     @Override
-    public RaftActorBehavior handleMessage(ActorRef sender, Object message) {
+    public RaftActorBehavior handleMessage(final ActorRef sender, final Object message) {
         Preconditions.checkNotNull(sender, "sender should not be null");
 
+        if (appendEntriesMessageSlicer.handleMessage(message)) {
+            return this;
+        }
+
         if (message instanceof RaftRPC) {
             RaftRPC rpc = (RaftRPC) message;
             // If RPC request or response contains term T > currentTerm:
@@ -447,6 +469,19 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
 
                 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
 
+                // This is a special case. Normally when stepping down as leader we don't process and reply to the
+                // RaftRPC as per raft. But if we're in the process of transferring leadership and we get a
+                // RequestVote, process the RequestVote before switching to Follower. This enables the requesting
+                // candidate node to be elected the leader faster and avoids us possibly timing out in the Follower
+                // state and starting a new election and grabbing leadership back before the other candidate node can
+                // start a new election due to lack of responses. This case would only occur if there isn't a majority
+                // of other nodes available that can elect the requesting candidate. Since we're transferring
+                // leadership, we should make every effort to get the requesting node elected.
+                if (message instanceof RequestVote && context.getRaftActorLeadershipTransferCohort() != null) {
+                    log.debug("{}: Leadership transfer in progress - processing RequestVote", logName());
+                    super.handleMessage(sender, message);
+                }
+
                 return internalSwitchBehavior(RaftState.Follower);
             }
         }
@@ -457,7 +492,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
             scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
         } else if (message instanceof SendInstallSnapshot) {
             SendInstallSnapshot sendInstallSnapshot = (SendInstallSnapshot) message;
-            setSnapshot(new SnapshotHolder(sendInstallSnapshot.getSnapshot(), sendInstallSnapshot.getSnapshotBytes()));
+            setSnapshotHolder(new SnapshotHolder(sendInstallSnapshot.getSnapshot(),
+                sendInstallSnapshot.getSnapshotBytes()));
             sendInstallSnapshot();
         } else if (message instanceof Replicate) {
             replicate((Replicate) message);
@@ -472,7 +508,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         return this;
     }
 
-    private void handleInstallSnapshotReply(InstallSnapshotReply reply) {
+    private void handleInstallSnapshotReply(final InstallSnapshotReply reply) {
         log.debug("{}: handleInstallSnapshotReply: {}", logName(), reply);
 
         String followerId = reply.getFollowerId();
@@ -511,7 +547,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
                     if (!anyFollowersInstallingSnapshot()) {
                         // once there are no pending followers receiving snapshots
                         // we can remove snapshot from the memory
-                        setSnapshot(null);
+                        setSnapshotHolder(null);
                     }
 
                     wasLastChunk = true;
@@ -530,10 +566,12 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
                 installSnapshotState.markSendStatus(false);
             }
 
-            if (wasLastChunk && !context.getSnapshotManager().isCapturing()) {
-                // Since the follower is now caught up try to purge the log.
-                purgeInMemoryLog();
-            } else if (!wasLastChunk && installSnapshotState.canSendNextChunk()) {
+            if (wasLastChunk) {
+                if (!context.getSnapshotManager().isCapturing()) {
+                    // Since the follower is now caught up try to purge the log.
+                    purgeInMemoryLog();
+                }
+            } else {
                 ActorSelection followerActor = context.getPeerActorSelection(followerId);
                 if (followerActor != null) {
                     sendSnapshotChunk(followerActor, followerLogInformation);
@@ -564,7 +602,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         return false;
     }
 
-    private void replicate(Replicate replicate) {
+    private void replicate(final Replicate replicate) {
         long logIndex = replicate.getReplicatedLogEntry().getIndex();
 
         log.debug("{}: Replicate message: identifier: {}, logIndex: {}, payload: {}, isSendImmediate: {}", logName(),
@@ -591,14 +629,14 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         }
     }
 
-    protected void sendAppendEntries(long timeSinceLastActivityInterval, boolean isHeartbeat) {
+    protected void sendAppendEntries(final long timeSinceLastActivityIntervalNanos, final boolean isHeartbeat) {
         // Send an AppendEntries to all followers
         for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
             final String followerId = e.getKey();
             final FollowerLogInformation followerLogInformation = e.getValue();
             // This checks helps not to send a repeat message to the follower
             if (!followerLogInformation.isFollowerActive()
-                    || followerLogInformation.timeSinceLastActivity() >= timeSinceLastActivityInterval) {
+                    || followerLogInformation.nanosSinceLastActivity() >= timeSinceLastActivityIntervalNanos) {
                 sendUpdatesToFollower(followerId, followerLogInformation, true, isHeartbeat);
             }
         }
@@ -608,8 +646,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
      * This method checks if any update needs to be sent to the given follower. This includes append log entries,
      * sending next snapshot chunk, and initiating a snapshot.
      */
-    private void sendUpdatesToFollower(String followerId, FollowerLogInformation followerLogInformation,
-                                       boolean sendHeartbeat, boolean isHeartbeat) {
+    private void sendUpdatesToFollower(final String followerId, final FollowerLogInformation followerLogInformation,
+                                       final boolean sendHeartbeat, final boolean isHeartbeat) {
 
         ActorSelection followerActor = context.getPeerActorSelection(followerId);
         if (followerActor != null) {
@@ -627,6 +665,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
                     // we send a heartbeat even if we have not received a reply for the last chunk
                     sendAppendEntries = true;
                 }
+            } else if (followerLogInformation.isLogEntrySlicingInProgress()) {
+                sendAppendEntries = sendHeartbeat;
             } else {
                 long leaderLastIndex = context.getReplicatedLog().lastIndex();
                 long leaderSnapShotIndex = context.getReplicatedLog().getSnapshotIndex();
@@ -643,11 +683,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
                             followerNextIndex, followerId);
 
                     if (followerLogInformation.okToReplicate()) {
-                        // Try to send all the entries in the journal but not exceeding the max data size
-                        // for a single AppendEntries message.
-                        int maxEntries = (int) context.getReplicatedLog().size();
-                        entries = context.getReplicatedLog().getFrom(followerNextIndex, maxEntries,
-                                context.getConfigParams().getSnapshotChunkSize());
+                        entries = getEntriesToSend(followerLogInformation, followerActor);
                         sendAppendEntries = true;
                     }
                 } else if (isFollowerActive && followerNextIndex >= 0
@@ -688,8 +724,78 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         }
     }
 
-    private void sendAppendEntriesToFollower(ActorSelection followerActor, List<ReplicatedLogEntry> entries,
-            FollowerLogInformation followerLogInformation) {
+    private List<ReplicatedLogEntry> getEntriesToSend(final FollowerLogInformation followerLogInfo,
+            final ActorSelection followerActor) {
+        // Try to get all the entries in the journal but not exceeding the max data size for a single AppendEntries
+        // message.
+        int maxEntries = (int) context.getReplicatedLog().size();
+        final int maxDataSize = context.getConfigParams().getSnapshotChunkSize();
+        final long followerNextIndex = followerLogInfo.getNextIndex();
+        List<ReplicatedLogEntry> entries = context.getReplicatedLog().getFrom(followerNextIndex,
+                maxEntries, maxDataSize);
+
+        // If the first entry's size exceeds the max data size threshold, it will be returned from the call above. If
+        // that is the case, then we need to slice it into smaller chunks.
+        if (!(entries.size() == 1 && entries.get(0).getData().size() > maxDataSize)) {
+            // Don't need to slice.
+            return entries;
+        }
+
+        log.debug("{}: Log entry size {} exceeds max payload size {}", logName(), entries.get(0).getData().size(),
+                maxDataSize);
+
+        // If an AppendEntries has already been serialized for the log index then reuse the
+        // SharedFileBackedOutputStream.
+        final Long logIndex = entries.get(0).getIndex();
+        SharedFileBackedOutputStream fileBackedStream = sharedSerializedAppendEntriesStreams.get(logIndex);
+        if (fileBackedStream == null) {
+            fileBackedStream = context.getFileBackedOutputStreamFactory().newSharedInstance();
+
+            final AppendEntries appendEntries = new AppendEntries(currentTerm(), context.getId(),
+                    getLogEntryIndex(followerNextIndex - 1), getLogEntryTerm(followerNextIndex - 1), entries,
+                    context.getCommitIndex(), getReplicatedToAllIndex(), context.getPayloadVersion());
+
+            log.debug("{}: Serializing {} for slicing for follower {}", logName(), appendEntries,
+                    followerLogInfo.getId());
+
+            try (ObjectOutputStream out = new ObjectOutputStream(fileBackedStream)) {
+                out.writeObject(appendEntries);
+            } catch (IOException e) {
+                log.error("{}: Error serializing {}", logName(), appendEntries, e);
+                fileBackedStream.cleanup();
+                return Collections.emptyList();
+            }
+
+            sharedSerializedAppendEntriesStreams.put(logIndex, fileBackedStream);
+
+            fileBackedStream.setOnCleanupCallback(index -> {
+                log.debug("{}: On SharedFileBackedOutputStream cleanup for index {}", logName(), index);
+                sharedSerializedAppendEntriesStreams.remove(index);
+            }, logIndex);
+        } else {
+            log.debug("{}: Reusing SharedFileBackedOutputStream for follower {}", logName(), followerLogInfo.getId());
+            fileBackedStream.incrementUsageCount();
+        }
+
+        log.debug("{}: Slicing stream for index {}, follower {}", logName(), logIndex, followerLogInfo.getId());
+
+        // Record that slicing is in progress for the follower.
+        followerLogInfo.setSlicedLogEntryIndex(logIndex);
+
+        final FollowerIdentifier identifier = new FollowerIdentifier(followerLogInfo.getId());
+        appendEntriesMessageSlicer.slice(SliceOptions.builder().identifier(identifier)
+                .fileBackedOutputStream(fileBackedStream).sendTo(followerActor).replyTo(actor())
+                .onFailureCallback(failure -> {
+                    log.error("{}: Error slicing AppendEntries for follower {}", logName(),
+                            followerLogInfo.getId(), failure);
+                    followerLogInfo.setSlicedLogEntryIndex(FollowerLogInformation.NO_INDEX);
+                }).build());
+
+        return Collections.emptyList();
+    }
+
+    private void sendAppendEntriesToFollower(final ActorSelection followerActor, final List<ReplicatedLogEntry> entries,
+            final FollowerLogInformation followerLogInformation) {
         // In certain cases outlined below we don't want to send the actual commit index to prevent the follower from
         // possibly committing and applying conflicting entries (those with same index, different term) from a prior
         // term that weren't replicated to a majority, which would be a violation of raft.
@@ -697,9 +803,11 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         //       empty AppendEntries as a heart beat to prevent election.
         //     - if we're in the process of installing a snapshot. In this case we don't send any new entries but still
         //       need to send AppendEntries to prevent election.
+        //     - if we're in the process of slicing an AppendEntries with a large log entry payload. In this case we
+        //       need to send an empty AppendEntries to prevent election.
         boolean isInstallingSnaphot = followerLogInformation.getInstallSnapshotState() != null;
-        long leaderCommitIndex = isInstallingSnaphot || !followerLogInformation.isFollowerActive() ? -1 :
-            context.getCommitIndex();
+        long leaderCommitIndex = isInstallingSnaphot || followerLogInformation.isLogEntrySlicingInProgress()
+                || !followerLogInformation.isFollowerActive() ? -1 : context.getCommitIndex();
 
         long followerNextIndex = followerLogInformation.getNextIndex();
         AppendEntries appendEntries = new AppendEntries(currentTerm(), context.getId(),
@@ -734,7 +842,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
      * @param followerId the id of the follower.
      * @return true if capture was initiated, false otherwise.
      */
-    public boolean initiateCaptureSnapshot(String followerId) {
+    public boolean initiateCaptureSnapshot(final String followerId) {
         FollowerLogInformation followerLogInfo = followerToLog.get(followerId);
         if (snapshotHolder.isPresent()) {
             // If a snapshot is present in the memory, most likely another install is in progress no need to capture
@@ -756,7 +864,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         return captureInitiated;
     }
 
-    private boolean canInstallSnapshot(long nextIndex) {
+    private boolean canInstallSnapshot(final long nextIndex) {
         // If the follower's nextIndex is -1 then we might as well send it a snapshot
         // Otherwise send it a snapshot only if the nextIndex is not present in the log but is present
         // in the snapshot
@@ -788,7 +896,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
      *  Sends a snapshot chunk to a given follower
      *  InstallSnapshot should qualify as a heartbeat too.
      */
-    private void sendSnapshotChunk(ActorSelection followerActor, FollowerLogInformation followerLogInfo) {
+    private void sendSnapshotChunk(final ActorSelection followerActor, final FollowerLogInformation followerLogInfo) {
         if (snapshotHolder.isPresent()) {
             LeaderInstallSnapshotState installSnapshotState = followerLogInfo.getInstallSnapshotState();
             if (installSnapshotState == null) {
@@ -801,6 +909,10 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
                 // Ensure the snapshot bytes are set - this is a no-op.
                 installSnapshotState.setSnapshotBytes(snapshotHolder.get().getSnapshotBytes());
 
+                if (!installSnapshotState.canSendNextChunk()) {
+                    return;
+                }
+
                 byte[] nextSnapshotChunk = installSnapshotState.getNextChunk();
 
                 log.debug("{}: next snapshot chunk size for follower {}: {}", logName(), followerLogInfo.getId(),
@@ -825,18 +937,21 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
                     actor()
                 );
 
-                log.debug("{}: InstallSnapshot sent to follower {}, Chunk: {}/{}", logName(), followerActor.path(),
-                        installSnapshotState.getChunkIndex(), installSnapshotState.getTotalChunks());
             } catch (IOException e) {
-                throw Throwables.propagate(e);
+                throw new RuntimeException(e);
             }
+
+            log.debug("{}: InstallSnapshot sent to follower {}, Chunk: {}/{}", logName(), followerActor.path(),
+                installSnapshotState.getChunkIndex(), installSnapshotState.getTotalChunks());
         }
     }
 
     private void sendHeartBeat() {
         if (!followerToLog.isEmpty()) {
             log.trace("{}: Sending heartbeat", logName());
-            sendAppendEntries(context.getConfigParams().getHeartBeatInterval().toMillis(), true);
+            sendAppendEntries(context.getConfigParams().getHeartBeatInterval().toNanos(), true);
+
+            appendEntriesMessageSlicer.checkExpiredSlicedMessageState();
         }
     }
 
@@ -846,7 +961,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         }
     }
 
-    private void scheduleHeartBeat(FiniteDuration interval) {
+    private void scheduleHeartBeat(final FiniteDuration interval) {
         if (followerToLog.isEmpty()) {
             // Optimization - do not bother scheduling a heartbeat as there are
             // no followers
@@ -868,6 +983,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
     @Override
     public void close() {
         stopHeartBeat();
+        appendEntriesMessageSlicer.close();
     }
 
     @Override
@@ -912,7 +1028,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
     }
 
     @VisibleForTesting
-    public FollowerLogInformation getFollower(String followerId) {
+    public FollowerLogInformation getFollower(final String followerId) {
         return followerToLog.get(followerId);
     }
 
@@ -926,7 +1042,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         private final long lastIncludedIndex;
         private final ByteSource snapshotBytes;
 
-        SnapshotHolder(Snapshot snapshot, ByteSource snapshotBytes) {
+        SnapshotHolder(final Snapshot snapshot, final ByteSource snapshotBytes) {
             this.lastIncludedTerm = snapshot.getLastAppliedTerm();
             this.lastIncludedIndex = snapshot.getLastAppliedIndex();
             this.snapshotBytes = snapshotBytes;