Optimize Follower.isOutOfSync()
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / behaviors / Follower.java
index bcc2480f459318f35ce3aa30e2d818404626d99b..8aee8c1af88c31da507bede83065d3d457b76428 100644 (file)
@@ -9,15 +9,29 @@
 package org.opendaylight.controller.cluster.raft.behaviors;
 
 import akka.actor.ActorRef;
+import akka.actor.ActorSelection;
+import akka.actor.Address;
+import akka.cluster.Cluster;
+import akka.cluster.ClusterEvent.CurrentClusterState;
+import akka.cluster.Member;
+import akka.cluster.MemberStatus;
+import akka.japi.Procedure;
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Stopwatch;
+import java.io.IOException;
 import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import javax.annotation.Nullable;
 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.ServerConfigurationPayload;
-import org.opendaylight.controller.cluster.raft.Snapshot;
 import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
 import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
+import org.opendaylight.controller.cluster.raft.base.messages.TimeoutNow;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot;
@@ -25,10 +39,11 @@ 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.persisted.ServerConfigurationPayload;
+import org.opendaylight.controller.cluster.raft.persisted.Snapshot;
 
 /**
- * The behavior of a RaftActor in the Follower state
- * <p/>
+ * The behavior of a RaftActor in the Follower raft state.
  * <ul>
  * <li> Respond to RPCs from candidates and leaders
  * <li> If election timeout elapses without receiving AppendEntries
@@ -37,86 +52,103 @@ import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
  * </ul>
  */
 public class Follower extends AbstractRaftActorBehavior {
-
-    private SnapshotTracker snapshotTracker = null;
+    private static final long MAX_ELECTION_TIMEOUT_FACTOR = 18;
 
     private final SyncStatusTracker initialSyncStatusTracker;
 
-    private static final int SYNC_THRESHOLD = 10;
+    private final Stopwatch lastLeaderMessageTimer = Stopwatch.createStarted();
+    private SnapshotTracker snapshotTracker = null;
+    private String leaderId;
+    private short leaderPayloadVersion;
 
-    public Follower(RaftActorContext context) {
-        this(context, null);
+    public Follower(final RaftActorContext context) {
+        this(context, null, (short)-1);
     }
 
-    public Follower(RaftActorContext context, String initialLeaderId) {
+    public Follower(final RaftActorContext context, final String initialLeaderId,
+            final short initialLeaderPayloadVersion) {
         super(context, RaftState.Follower);
-        leaderId = initialLeaderId;
+        this.leaderId = initialLeaderId;
+        this.leaderPayloadVersion = initialLeaderPayloadVersion;
 
-        initialSyncStatusTracker = new SyncStatusTracker(context.getActor(), getId(), SYNC_THRESHOLD);
+        initialSyncStatusTracker = new SyncStatusTracker(context.getActor(), getId(), context.getConfigParams()
+            .getSyncIndexThreshold());
 
-        if(canStartElection()) {
-            if (context.getPeerIds().isEmpty() && getLeaderId() == null) {
-                actor().tell(ELECTION_TIMEOUT, actor());
-            } else {
-                scheduleElection(electionDuration());
-            }
+        if (context.getPeerIds().isEmpty() && getLeaderId() == null) {
+            actor().tell(TimeoutNow.INSTANCE, actor());
+        } else {
+            scheduleElection(electionDuration());
         }
-
     }
 
-    private boolean isLogEntryPresent(long index){
-        if(index == context.getReplicatedLog().getSnapshotIndex()){
-            return true;
-        }
+    @Override
+    public final String getLeaderId() {
+        return leaderId;
+    }
 
-        ReplicatedLogEntry previousEntry = context.getReplicatedLog()
-                .get(index);
+    @VisibleForTesting
+    protected final void setLeaderId(@Nullable final String leaderId) {
+        this.leaderId = leaderId;
+    }
 
-        return previousEntry != null;
+    @Override
+    public short getLeaderPayloadVersion() {
+        return leaderPayloadVersion;
+    }
 
+    @VisibleForTesting
+    protected final void setLeaderPayloadVersion(final short leaderPayloadVersion) {
+        this.leaderPayloadVersion = leaderPayloadVersion;
     }
 
-    private long getLogEntryTerm(long index){
-        if(index == context.getReplicatedLog().getSnapshotIndex()){
-            return context.getReplicatedLog().getSnapshotTerm();
+    private void restartLastLeaderMessageTimer() {
+        if (lastLeaderMessageTimer.isRunning()) {
+            lastLeaderMessageTimer.reset();
         }
 
-        ReplicatedLogEntry previousEntry = context.getReplicatedLog()
-                .get(index);
+        lastLeaderMessageTimer.start();
+    }
 
-        if(previousEntry != null){
-            return previousEntry.getTerm();
+    private boolean isLogEntryPresent(final long index) {
+        if (context.getReplicatedLog().isInSnapshot(index)) {
+            return true;
         }
 
-        return -1;
+        ReplicatedLogEntry entry = context.getReplicatedLog().get(index);
+        return entry != null;
+
     }
 
-    private void updateInitialSyncStatus(long currentLeaderCommit, String leaderId){
-        initialSyncStatusTracker.update(leaderId, currentLeaderCommit, context.getCommitIndex());
+    private void updateInitialSyncStatus(final long currentLeaderCommit, final String newLeaderId) {
+        initialSyncStatusTracker.update(newLeaderId, currentLeaderCommit, context.getCommitIndex());
     }
 
-    @Override protected RaftActorBehavior handleAppendEntries(ActorRef sender,
-                                                              AppendEntries appendEntries) {
+    @Override
+    protected RaftActorBehavior handleAppendEntries(final ActorRef sender, final AppendEntries appendEntries) {
 
         int numLogEntries = appendEntries.getEntries() != null ? appendEntries.getEntries().size() : 0;
-        if(LOG.isTraceEnabled()) {
-            LOG.trace("{}: handleAppendEntries: {}", logName(), appendEntries);
-        } else if(LOG.isDebugEnabled() && numLogEntries > 0) {
-            LOG.debug("{}: handleAppendEntries: {}", logName(), appendEntries);
+        if (log.isTraceEnabled()) {
+            log.trace("{}: handleAppendEntries: {}", logName(), appendEntries);
+        } else if (log.isDebugEnabled() && numLogEntries > 0) {
+            log.debug("{}: handleAppendEntries: {}", logName(), appendEntries);
         }
 
         // TODO : Refactor this method into a bunch of smaller methods
         // to make it easier to read. Before refactoring ensure tests
         // cover the code properly
 
+        if (snapshotTracker != null && !snapshotTracker.getLeaderId().equals(appendEntries.getLeaderId())) {
+            log.debug("{}: snapshot install is in progress but the prior snapshot leaderId {} does not match the "
+                + "AppendEntries leaderId {}", logName(), snapshotTracker.getLeaderId(), appendEntries.getLeaderId());
+            closeSnapshotTracker();
+        }
+
         if (snapshotTracker != null || context.getSnapshotManager().isApplying()) {
             // if snapshot install is in progress, follower should just acknowledge append entries with a reply.
             AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true,
                     lastIndex(), lastTerm(), context.getPayloadVersion());
 
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("{}: snapshot install is in progress, replying immediately with {}", logName(), reply);
-            }
+            log.debug("{}: snapshot install is in progress, replying immediately with {}", logName(), reply);
             sender.tell(reply, actor());
 
             return this;
@@ -124,10 +156,8 @@ public class Follower extends AbstractRaftActorBehavior {
 
         // If we got here then we do appear to be talking to the leader
         leaderId = appendEntries.getLeaderId();
+        leaderPayloadVersion = appendEntries.getPayloadVersion();
 
-        setLeaderPayloadVersion(appendEntries.getPayloadVersion());
-
-        updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId());
         // First check if the logs are in sync or not
         long lastIndex = lastIndex();
 
@@ -135,17 +165,18 @@ public class Follower extends AbstractRaftActorBehavior {
             // We found that the log was out of sync so just send a negative
             // reply and return
 
-            LOG.debug("{}: Follower is out-of-sync, so sending negative reply, lastIndex: {}, lastTerm: {}",
-                        logName(), lastIndex, lastTerm());
+            final AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex,
+                    lastTerm(), context.getPayloadVersion());
 
-            sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex,
-                    lastTerm(), context.getPayloadVersion()), actor());
+            log.info("{}: Follower is out-of-sync so sending negative reply: {}", logName(), reply);
+            updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId());
+            sender.tell(reply, actor());
             return this;
         }
 
         if (appendEntries.getEntries() != null && appendEntries.getEntries().size() > 0) {
 
-            LOG.debug("{}: Number of entries to be appended = {}", logName(),
+            log.debug("{}: Number of entries to be appended = {}", logName(),
                         appendEntries.getEntries().size());
 
             // 3. If an existing entry conflicts with a new one (same index
@@ -157,26 +188,44 @@ public class Follower extends AbstractRaftActorBehavior {
                 // Find the entry up until the one that is not in the follower's log
                 for (int i = 0;i < appendEntries.getEntries().size(); i++, addEntriesFrom++) {
                     ReplicatedLogEntry matchEntry = appendEntries.getEntries().get(i);
-                    ReplicatedLogEntry newEntry = context.getReplicatedLog().get(matchEntry.getIndex());
 
-                    if (newEntry == null) {
-                        //newEntry not found in the log
+                    if (!isLogEntryPresent(matchEntry.getIndex())) {
+                        // newEntry not found in the log
                         break;
                     }
 
-                    if (newEntry.getTerm() == matchEntry.getTerm()) {
+                    long existingEntryTerm = getLogEntryTerm(matchEntry.getIndex());
+
+                    log.debug("{}: matchEntry {} is present: existingEntryTerm: {}", logName(), matchEntry,
+                            existingEntryTerm);
+
+                    // existingEntryTerm == -1 means it's in the snapshot and not in the log. We don't know
+                    // what the term was so we'll assume it matches.
+                    if (existingEntryTerm == -1 || existingEntryTerm == matchEntry.getTerm()) {
                         continue;
                     }
 
-                    if(!context.getRaftPolicy().applyModificationToStateBeforeConsensus()) {
+                    if (!context.getRaftPolicy().applyModificationToStateBeforeConsensus()) {
 
-                        LOG.debug("{}: Removing entries from log starting at {}", logName(),
-                                matchEntry.getIndex());
+                        log.info("{}: Removing entries from log starting at {}", logName(), matchEntry.getIndex());
 
                         // Entries do not match so remove all subsequent entries
-                        context.getReplicatedLog().removeFromAndPersist(matchEntry.getIndex());
+                        if (!context.getReplicatedLog().removeFromAndPersist(matchEntry.getIndex())) {
+                            // Could not remove the entries - this means the matchEntry index must be in the
+                            // snapshot and not the log. In this case the prior entries are part of the state
+                            // so we must send back a reply to force a snapshot to completely re-sync the
+                            // follower's log and state.
+
+                            log.info("{}: Could not remove entries - sending reply to force snapshot", logName());
+                            updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId());
+                            sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex,
+                                    lastTerm(), context.getPayloadVersion(), true), actor());
+                            return this;
+                        }
+
                         break;
                     } else {
+                        updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId());
                         sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex,
                                 lastTerm(), context.getPayloadVersion(), true), actor());
                         return this;
@@ -185,23 +234,42 @@ public class Follower extends AbstractRaftActorBehavior {
             }
 
             lastIndex = lastIndex();
-            LOG.debug("{}: After cleanup entries to be added from = {}", logName(),
-                        (addEntriesFrom + lastIndex));
+            log.debug("{}: After cleanup, lastIndex: {}, entries to be added from: {}", logName(),
+                    lastIndex, addEntriesFrom);
+
+            // When persistence successfully completes for each new log entry appended, we need to determine if we
+            // should capture a snapshot to compact the persisted log. shouldCaptureSnapshot tracks whether or not
+            // one of the log entries has exceeded the log size threshold whereby a snapshot should be taken. However
+            // we don't initiate the snapshot at that log entry but rather after the last log entry has been persisted.
+            // This is done because subsequent log entries after the one that tripped the threshold may have been
+            // applied to the state already, as the persistence callback occurs async, and we want those entries
+            // purged from the persisted log as well.
+            final AtomicBoolean shouldCaptureSnapshot = new AtomicBoolean(false);
+            final Procedure<ReplicatedLogEntry> appendAndPersistCallback = logEntry -> {
+                final ReplicatedLogEntry lastEntryToAppend = appendEntries.getEntries().get(
+                        appendEntries.getEntries().size() - 1);
+                if (shouldCaptureSnapshot.get() && logEntry == lastEntryToAppend) {
+                    context.getSnapshotManager().capture(context.getReplicatedLog().last(), getReplicatedToAllIndex());
+                }
+            };
 
             // 4. Append any new entries not already in the log
             for (int i = addEntriesFrom; i < appendEntries.getEntries().size(); i++) {
                 ReplicatedLogEntry entry = appendEntries.getEntries().get(i);
 
-                LOG.debug("{}: Append entry to log {}", logName(), entry.getData());
+                log.debug("{}: Append entry to log {}", logName(), entry.getData());
+
+                context.getReplicatedLog().appendAndPersist(entry, appendAndPersistCallback, false);
 
-                context.getReplicatedLog().appendAndPersist(entry);
+                shouldCaptureSnapshot.compareAndSet(false,
+                        context.getReplicatedLog().shouldCaptureSnapshot(entry.getIndex()));
 
-                if(entry.getData() instanceof ServerConfigurationPayload) {
+                if (entry.getData() instanceof ServerConfigurationPayload) {
                     context.updatePeerIds((ServerConfigurationPayload)entry.getData());
                 }
             }
 
-            LOG.debug("{}: Log size is now {}", logName(), context.getReplicatedLog().size());
+            log.debug("{}: Log size is now {}", logName(), context.getReplicatedLog().size());
         }
 
         // 5. If leaderCommit > commitIndex, set commitIndex =
@@ -210,38 +278,41 @@ public class Follower extends AbstractRaftActorBehavior {
         lastIndex = lastIndex();
         long prevCommitIndex = context.getCommitIndex();
 
-        context.setCommitIndex(Math.min(appendEntries.getLeaderCommit(), lastIndex));
+        if (appendEntries.getLeaderCommit() > prevCommitIndex) {
+            context.setCommitIndex(Math.min(appendEntries.getLeaderCommit(), lastIndex));
+        }
 
         if (prevCommitIndex != context.getCommitIndex()) {
-            LOG.debug("{}: Commit index set to {}", logName(), context.getCommitIndex());
+            log.debug("{}: Commit index set to {}", logName(), context.getCommitIndex());
+        }
+
+        AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true,
+                lastIndex, lastTerm(), context.getPayloadVersion());
+
+        if (log.isTraceEnabled()) {
+            log.trace("{}: handleAppendEntries returning : {}", logName(), reply);
+        } else if (log.isDebugEnabled() && numLogEntries > 0) {
+            log.debug("{}: handleAppendEntries returning : {}", logName(), reply);
         }
 
+        // Reply to the leader before applying any previous state so as not to hold up leader consensus.
+        updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId());
+        sender.tell(reply, actor());
+
         // If commitIndex > lastApplied: increment lastApplied, apply
         // log[lastApplied] to state machine (§5.3)
         // check if there are any entries to be applied. last-applied can be equal to last-index
-        if (appendEntries.getLeaderCommit() > context.getLastApplied() &&
-            context.getLastApplied() < lastIndex) {
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("{}: applyLogToStateMachine, " +
-                        "appendEntries.getLeaderCommit(): {}," +
-                        "context.getLastApplied(): {}, lastIndex(): {}", logName(),
+        if (appendEntries.getLeaderCommit() > context.getLastApplied()
+                && context.getLastApplied() < lastIndex) {
+            if (log.isDebugEnabled()) {
+                log.debug("{}: applyLogToStateMachine, appendEntries.getLeaderCommit(): {}, "
+                        + "context.getLastApplied(): {}, lastIndex(): {}", logName(),
                     appendEntries.getLeaderCommit(), context.getLastApplied(), lastIndex);
             }
 
             applyLogToStateMachine(appendEntries.getLeaderCommit());
         }
 
-        AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true,
-            lastIndex, lastTerm(), context.getPayloadVersion());
-
-        if(LOG.isTraceEnabled()) {
-            LOG.trace("{}: handleAppendEntries returning : {}", logName(), reply);
-        } else if(LOG.isDebugEnabled() && numLogEntries > 0) {
-            LOG.debug("{}: handleAppendEntries returning : {}", logName(), reply);
-        }
-
-        sender.tell(reply, actor());
-
         if (!context.getSnapshotManager().isCapturing()) {
             super.performSnapshotWithoutCapture(appendEntries.getReplicatedToAllIndex());
         }
@@ -249,116 +320,217 @@ public class Follower extends AbstractRaftActorBehavior {
         return this;
     }
 
-    private boolean isOutOfSync(AppendEntries appendEntries) {
-
-        long prevLogTerm = getLogEntryTerm(appendEntries.getPrevLogIndex());
-        boolean prevEntryPresent = isLogEntryPresent(appendEntries.getPrevLogIndex());
-        long lastIndex = lastIndex();
-        int numLogEntries = appendEntries.getEntries() != null ? appendEntries.getEntries().size() : 0;
-        boolean outOfSync = true;
+    private boolean isOutOfSync(final AppendEntries appendEntries) {
 
+        final long lastIndex = lastIndex();
         if (lastIndex == -1 && appendEntries.getPrevLogIndex() != -1) {
 
-            // The follower's log is out of sync because the leader does have
-            // an entry at prevLogIndex and this follower has no entries in
-            // it's log.
-
-            LOG.debug("{}: The followers log is empty and the senders prevLogIndex is {}",
-                        logName(), appendEntries.getPrevLogIndex());
-        } else if (lastIndex > -1 && appendEntries.getPrevLogIndex() != -1 && !prevEntryPresent) {
-
-            // The follower's log is out of sync because the Leader's
-            // prevLogIndex entry was not found in it's log
-
-            LOG.debug("{}: The log is not empty but the prevLogIndex {} was not found in it",
-                        logName(), appendEntries.getPrevLogIndex());
-        } else if (lastIndex > -1 && prevEntryPresent && prevLogTerm != appendEntries.getPrevLogTerm()) {
-
-            // The follower's log is out of sync because the Leader's
-            // prevLogIndex entry does exist in the follower's log but it has
-            // a different term in it
-
-            LOG.debug(
-                    "{}: Cannot append entries because previous entry term {}  is not equal to append entries prevLogTerm {}",
-                    logName(), prevLogTerm, appendEntries.getPrevLogTerm());
-        } else if(appendEntries.getPrevLogIndex() == -1 && appendEntries.getPrevLogTerm() == -1
-                && appendEntries.getReplicatedToAllIndex() != -1
-                && !isLogEntryPresent(appendEntries.getReplicatedToAllIndex())
-                && !context.getReplicatedLog().isInSnapshot(appendEntries.getReplicatedToAllIndex())) {
-            // This append entry comes from a leader who has it's log aggressively trimmed and so does not have
-            // the previous entry in it's in-memory journal
-
-            LOG.debug(
-                    "{}: Cannot append entries because the replicatedToAllIndex {} does not appear to be in the in-memory journal",
-                    logName(), appendEntries.getReplicatedToAllIndex());
-        } else if(appendEntries.getPrevLogIndex() == -1 && appendEntries.getPrevLogTerm() == -1
-                && appendEntries.getReplicatedToAllIndex() != -1 && numLogEntries > 0
-                && !isLogEntryPresent(appendEntries.getEntries().get(0).getIndex() - 1)
-                && !context.getReplicatedLog().isInSnapshot(appendEntries.getEntries().get(0).getIndex() - 1)) {
-            LOG.debug(
-                    "{}: Cannot append entries because the calculated previousIndex {} was not found in the in-memory journal",
-                    logName(), appendEntries.getEntries().get(0).getIndex() - 1);
-        } else {
-            outOfSync = false;
+            // The follower's log is out of sync because the leader does have an entry at prevLogIndex and this
+            // follower has no entries in it's log.
+
+            log.info("{}: The followers log is empty and the senders prevLogIndex is {}", logName(),
+                appendEntries.getPrevLogIndex());
+            return true;
+        }
+
+        if (lastIndex > -1) {
+            if (isLogEntryPresent(appendEntries.getPrevLogIndex())) {
+                final long prevLogTerm = getLogEntryTerm(appendEntries.getPrevLogIndex());
+                if (prevLogTerm != appendEntries.getPrevLogTerm()) {
+
+                    // The follower's log is out of sync because the Leader's prevLogIndex entry does exist
+                    // in the follower's log but it has a different term in it
+
+                    log.info("{}: The prevLogIndex {} was found in the log but the term {} is not equal to the append "
+                            + "entries prevLogTerm {} - lastIndex: {}, snapshotIndex: {}", logName(),
+                            appendEntries.getPrevLogIndex(), prevLogTerm, appendEntries.getPrevLogTerm(), lastIndex,
+                            context.getReplicatedLog().getSnapshotIndex());
+                    return true;
+                }
+            } else if (appendEntries.getPrevLogIndex() != -1) {
+
+                // The follower's log is out of sync because the Leader's prevLogIndex entry was not found in it's log
+
+                log.info("{}: The log is not empty but the prevLogIndex {} was not found in it - lastIndex: {}, "
+                        + "snapshotIndex: {}", logName(), appendEntries.getPrevLogIndex(), lastIndex,
+                        context.getReplicatedLog().getSnapshotIndex());
+                return true;
+            }
         }
-        return outOfSync;
+
+        if (appendEntries.getPrevLogIndex() == -1 && appendEntries.getPrevLogTerm() == -1
+                && appendEntries.getReplicatedToAllIndex() != -1) {
+            if (!isLogEntryPresent(appendEntries.getReplicatedToAllIndex())) {
+                // This append entry comes from a leader who has it's log aggressively trimmed and so does not have
+                // the previous entry in it's in-memory journal
+
+                log.info("{}: Cannot append entries because the replicatedToAllIndex {} does not appear to be in the "
+                        + "in-memory journal", logName(), appendEntries.getReplicatedToAllIndex());
+                return true;
+            }
+
+            final List<ReplicatedLogEntry> entries = appendEntries.getEntries();
+            if (entries != null && entries.size() > 0 && !isLogEntryPresent(entries.get(0).getIndex() - 1)) {
+                log.info("{}: Cannot append entries because the calculated previousIndex {} was not found in the "
+                        + "in-memory journal", logName(), entries.get(0).getIndex() - 1);
+                return true;
+            }
+        }
+
+        return false;
     }
 
-    @Override protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
-        AppendEntriesReply appendEntriesReply) {
+    @Override
+    protected RaftActorBehavior handleAppendEntriesReply(final ActorRef sender,
+        final AppendEntriesReply appendEntriesReply) {
         return this;
     }
 
-    @Override protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
-        RequestVoteReply requestVoteReply) {
+    @Override
+    protected RaftActorBehavior handleRequestVoteReply(final ActorRef sender,
+        final RequestVoteReply requestVoteReply) {
         return this;
     }
 
-    @Override public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) {
+    @Override
+    public RaftActorBehavior handleMessage(final ActorRef sender, final Object message) {
+        if (message instanceof ElectionTimeout || message instanceof TimeoutNow) {
+            return handleElectionTimeout(message);
+        }
 
-        Object message = fromSerializableMessage(originalMessage);
+        if (!(message instanceof RaftRPC)) {
+            // The rest of the processing requires the message to be a RaftRPC
+            return null;
+        }
 
-        if (message instanceof RaftRPC) {
-            RaftRPC rpc = (RaftRPC) message;
-            // If RPC request or response contains term T > currentTerm:
-            // 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 follower's term {} - updating term",
-                        logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm());
+        final RaftRPC rpc = (RaftRPC) message;
+        // If RPC request or response contains term T > currentTerm:
+        // set currentTerm = T, convert to follower (§5.1)
+        // This applies to all RPC messages and responses
+        if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
+            log.info("{}: Term {} in \"{}\" message is greater than follower's term {} - updating term",
+                logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm());
 
-                context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
-            }
+            context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
+        }
+
+        if (rpc instanceof InstallSnapshot) {
+            handleInstallSnapshot(sender, (InstallSnapshot) rpc);
+            restartLastLeaderMessageTimer();
+            scheduleElection(electionDuration());
+            return this;
+        }
+
+        if (!(rpc instanceof RequestVote) || canGrantVote((RequestVote) rpc)) {
+            restartLastLeaderMessageTimer();
+            scheduleElection(electionDuration());
         }
 
-        if (message instanceof ElectionTimeout) {
-            if(canStartElection()) {
-                LOG.debug("{}: Received ElectionTimeout - switching to Candidate", logName());
+        return super.handleMessage(sender, rpc);
+    }
+
+    private RaftActorBehavior handleElectionTimeout(final Object message) {
+        // If the message is ElectionTimeout, verify we haven't actually seen a message from the leader
+        // during the election timeout interval. It may that the election timer expired b/c this actor
+        // was busy and messages got delayed, in which case leader messages would be backed up in the
+        // queue but would be processed before the ElectionTimeout message and thus would restart the
+        // lastLeaderMessageTimer.
+        long lastLeaderMessageInterval = lastLeaderMessageTimer.elapsed(TimeUnit.MILLISECONDS);
+        long electionTimeoutInMillis = context.getConfigParams().getElectionTimeOutInterval().toMillis();
+        boolean noLeaderMessageReceived = !lastLeaderMessageTimer.isRunning()
+                || lastLeaderMessageInterval >= electionTimeoutInMillis;
+
+        if (canStartElection()) {
+            if (message instanceof TimeoutNow) {
+                log.debug("{}: Received TimeoutNow - switching to Candidate", logName());
                 return internalSwitchBehavior(RaftState.Candidate);
+            } else if (noLeaderMessageReceived) {
+                // Check the cluster state to see if the leader is known to be up before we go to Candidate.
+                // However if we haven't heard from the leader in a long time even though the cluster state
+                // indicates it's up then something is wrong - leader might be stuck indefinitely - so switch
+                // to Candidate,
+                long maxElectionTimeout = electionTimeoutInMillis * MAX_ELECTION_TIMEOUT_FACTOR;
+                if (isLeaderAvailabilityKnown() && lastLeaderMessageInterval < maxElectionTimeout) {
+                    log.debug("{}: Received ElectionTimeout but leader appears to be available", logName());
+                    scheduleElection(electionDuration());
+                } else {
+                    log.debug("{}: Received ElectionTimeout - switching to Candidate", logName());
+                    return internalSwitchBehavior(RaftState.Candidate);
+                }
             } else {
-                return this;
+                log.debug("{}: Received ElectionTimeout but lastLeaderMessageInterval {} < election timeout {}",
+                        logName(), lastLeaderMessageInterval, context.getConfigParams().getElectionTimeOutInterval());
+                scheduleElection(electionDuration());
+            }
+        } else if (message instanceof ElectionTimeout) {
+            if (noLeaderMessageReceived) {
+                setLeaderId(null);
             }
 
-        } else if (message instanceof InstallSnapshot) {
-            InstallSnapshot installSnapshot = (InstallSnapshot) message;
-            handleInstallSnapshot(sender, installSnapshot);
+            scheduleElection(electionDuration());
         }
 
-        if(message instanceof RaftRPC && (!(message instanceof RequestVote) || (canGrantVote((RequestVote) message)))){
-            scheduleElection(electionDuration());
+        return this;
+    }
+
+    private boolean isLeaderAvailabilityKnown() {
+        if (leaderId == null) {
+            return false;
+        }
+
+        Optional<Cluster> cluster = context.getCluster();
+        if (!cluster.isPresent()) {
+            return false;
+        }
+
+        ActorSelection leaderActor = context.getPeerActorSelection(leaderId);
+        if (leaderActor == null) {
+            return false;
+        }
+
+        Address leaderAddress = leaderActor.anchorPath().address();
+
+        CurrentClusterState state = cluster.get().state();
+        Set<Member> unreachable = state.getUnreachable();
+
+        log.debug("{}: Checking for leader {} in the cluster unreachable set {}", logName(), leaderAddress,
+                unreachable);
+
+        for (Member m: unreachable) {
+            if (leaderAddress.equals(m.address())) {
+                log.info("{}: Leader {} is unreachable", logName(), leaderAddress);
+                return false;
+            }
         }
 
-        return super.handleMessage(sender, message);
+        for (Member m: state.getMembers()) {
+            if (leaderAddress.equals(m.address())) {
+                if (m.status() == MemberStatus.up() || m.status() == MemberStatus.weaklyUp()) {
+                    log.debug("{}: Leader {} cluster status is {} - leader is available", logName(),
+                            leaderAddress, m.status());
+                    return true;
+                } else {
+                    log.debug("{}: Leader {} cluster status is {} - leader is unavailable", logName(),
+                            leaderAddress, m.status());
+                    return false;
+                }
+            }
+        }
+
+        log.debug("{}: Leader {} not found in the cluster member set", logName(), leaderAddress);
+
+        return false;
     }
 
-    private void handleInstallSnapshot(final ActorRef sender, InstallSnapshot installSnapshot) {
+    private void handleInstallSnapshot(final ActorRef sender, final InstallSnapshot installSnapshot) {
 
-        LOG.debug("{}: handleInstallSnapshot: {}", logName(), installSnapshot);
+        log.debug("{}: handleInstallSnapshot: {}", logName(), installSnapshot);
 
         leaderId = installSnapshot.getLeaderId();
 
-        if(snapshotTracker == null){
-            snapshotTracker = new SnapshotTracker(LOG, installSnapshot.getTotalChunks());
+        if (snapshotTracker == null) {
+            snapshotTracker = new SnapshotTracker(log, installSnapshot.getTotalChunks(), installSnapshot.getLeaderId(),
+                    context);
         }
 
         updateInitialSyncStatus(installSnapshot.getLastIncludedIndex(), installSnapshot.getLeaderId());
@@ -367,22 +539,26 @@ public class Follower extends AbstractRaftActorBehavior {
             final InstallSnapshotReply reply = new InstallSnapshotReply(
                     currentTerm(), context.getId(), installSnapshot.getChunkIndex(), true);
 
-            if(snapshotTracker.addChunk(installSnapshot.getChunkIndex(), installSnapshot.getData(),
-                    installSnapshot.getLastChunkHashCode())){
-                Snapshot snapshot = Snapshot.create(snapshotTracker.getSnapshot(),
-                        new ArrayList<ReplicatedLogEntry>(),
+            if (snapshotTracker.addChunk(installSnapshot.getChunkIndex(), installSnapshot.getData(),
+                    installSnapshot.getLastChunkHashCode())) {
+
+                log.info("{}: Snapshot installed from leader: {}", logName(), installSnapshot.getLeaderId());
+
+                Snapshot snapshot = Snapshot.create(
+                        context.getSnapshotManager().convertSnapshot(snapshotTracker.getSnapshotBytes()),
+                        new ArrayList<>(),
                         installSnapshot.getLastIncludedIndex(),
                         installSnapshot.getLastIncludedTerm(),
                         installSnapshot.getLastIncludedIndex(),
                         installSnapshot.getLastIncludedTerm(),
                         context.getTermInformation().getCurrentTerm(),
                         context.getTermInformation().getVotedFor(),
-                        context.getPeerServerInfo(true));
+                        installSnapshot.getServerConfig().orNull());
 
                 ApplySnapshot.Callback applySnapshotCallback = new ApplySnapshot.Callback() {
                     @Override
                     public void onSuccess() {
-                        LOG.debug("{}: handleInstallSnapshot returning: {}", logName(), reply);
+                        log.debug("{}: handleInstallSnapshot returning: {}", logName(), reply);
 
                         sender.tell(reply, actor());
                     }
@@ -395,36 +571,37 @@ public class Follower extends AbstractRaftActorBehavior {
 
                 actor().tell(new ApplySnapshot(snapshot, applySnapshotCallback), actor());
 
-                snapshotTracker = null;
+                closeSnapshotTracker();
             } else {
-                LOG.debug("{}: handleInstallSnapshot returning: {}", logName(), reply);
+                log.debug("{}: handleInstallSnapshot returning: {}", logName(), reply);
 
                 sender.tell(reply, actor());
             }
-        } catch (SnapshotTracker.InvalidChunkException e) {
-            LOG.debug("{}: Exception in InstallSnapshot of follower", logName(), e);
+        } catch (IOException e) {
+            log.debug("{}: Exception in InstallSnapshot of follower", logName(), e);
 
             sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(),
                     -1, false), actor());
-            snapshotTracker = null;
 
-        } catch (Exception e){
-            LOG.error("{}: Exception in InstallSnapshot of follower", logName(), e);
-
-            //send reply with success as false. The chunk will be sent again on failure
-            sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(),
-                    installSnapshot.getChunkIndex(), false), actor());
+            closeSnapshotTracker();
+        }
+    }
 
+    private void closeSnapshotTracker() {
+        if (snapshotTracker != null) {
+            snapshotTracker.close();
+            snapshotTracker = null;
         }
     }
 
     @Override
-    public void close() throws Exception {
+    public void close() {
+        closeSnapshotTracker();
         stopElection();
     }
 
     @VisibleForTesting
-    SnapshotTracker getSnapshotTracker(){
+    SnapshotTracker getSnapshotTracker() {
         return snapshotTracker;
     }
 }