Do not bump follower term while it is isolated
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / behaviors / Follower.java
1 /*
2  * Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
3  *
4  * This program and the accompanying materials are made available under the
5  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
6  * and is available at http://www.eclipse.org/legal/epl-v10.html
7  */
8 package org.opendaylight.controller.cluster.raft.behaviors;
9
10 import akka.actor.ActorRef;
11 import akka.actor.ActorSelection;
12 import akka.actor.Address;
13 import akka.cluster.Cluster;
14 import akka.cluster.ClusterEvent.CurrentClusterState;
15 import akka.cluster.Member;
16 import akka.cluster.MemberStatus;
17 import com.google.common.annotations.VisibleForTesting;
18 import com.google.common.base.Stopwatch;
19 import java.io.IOException;
20 import java.util.ArrayList;
21 import java.util.HashSet;
22 import java.util.List;
23 import java.util.Optional;
24 import java.util.Set;
25 import java.util.concurrent.TimeUnit;
26 import java.util.concurrent.atomic.AtomicBoolean;
27 import java.util.function.Consumer;
28 import org.eclipse.jdt.annotation.Nullable;
29 import org.opendaylight.controller.cluster.messaging.MessageAssembler;
30 import org.opendaylight.controller.cluster.raft.RaftActorContext;
31 import org.opendaylight.controller.cluster.raft.RaftState;
32 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
33 import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
34 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
35 import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
36 import org.opendaylight.controller.cluster.raft.base.messages.TimeoutNow;
37 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
38 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
39 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot;
40 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshotReply;
41 import org.opendaylight.controller.cluster.raft.messages.RaftRPC;
42 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
43 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
44 import org.opendaylight.controller.cluster.raft.persisted.ServerConfigurationPayload;
45 import org.opendaylight.controller.cluster.raft.persisted.Snapshot;
46
47 /**
48  * The behavior of a RaftActor in the Follower raft state.
49  * <ul>
50  * <li> Respond to RPCs from candidates and leaders
51  * <li> If election timeout elapses without receiving AppendEntries
52  * RPC from current leader or granting vote to candidate:
53  * convert to candidate
54  * </ul>
55  */
56 public class Follower extends AbstractRaftActorBehavior {
57     private static final long MAX_ELECTION_TIMEOUT_FACTOR = 18;
58
59     private final SyncStatusTracker initialSyncStatusTracker;
60
61     private final MessageAssembler appendEntriesMessageAssembler;
62
63     private final Stopwatch lastLeaderMessageTimer = Stopwatch.createStarted();
64     private SnapshotTracker snapshotTracker = null;
65     private String leaderId;
66     private short leaderPayloadVersion;
67
68     public Follower(final RaftActorContext context) {
69         this(context, null, (short)-1);
70     }
71
72     public Follower(final RaftActorContext context, final String initialLeaderId,
73             final short initialLeaderPayloadVersion) {
74         super(context, RaftState.Follower);
75         this.leaderId = initialLeaderId;
76         this.leaderPayloadVersion = initialLeaderPayloadVersion;
77
78         initialSyncStatusTracker = new SyncStatusTracker(context.getActor(), getId(), context.getConfigParams()
79             .getSyncIndexThreshold());
80
81         appendEntriesMessageAssembler = MessageAssembler.builder().logContext(logName())
82                 .fileBackedStreamFactory(context.getFileBackedOutputStreamFactory())
83                 .assembledMessageCallback((message, sender) -> handleMessage(sender, message)).build();
84
85         if (context.getPeerIds().isEmpty() && getLeaderId() == null) {
86             actor().tell(TimeoutNow.INSTANCE, actor());
87         } else {
88             scheduleElection(electionDuration());
89         }
90     }
91
92     @Override
93     public final String getLeaderId() {
94         return leaderId;
95     }
96
97     @VisibleForTesting
98     protected final void setLeaderId(final @Nullable String leaderId) {
99         this.leaderId = leaderId;
100     }
101
102     @Override
103     public short getLeaderPayloadVersion() {
104         return leaderPayloadVersion;
105     }
106
107     @VisibleForTesting
108     protected final void setLeaderPayloadVersion(final short leaderPayloadVersion) {
109         this.leaderPayloadVersion = leaderPayloadVersion;
110     }
111
112     private void restartLastLeaderMessageTimer() {
113         if (lastLeaderMessageTimer.isRunning()) {
114             lastLeaderMessageTimer.reset();
115         }
116
117         lastLeaderMessageTimer.start();
118     }
119
120     private boolean isLogEntryPresent(final long index) {
121         if (context.getReplicatedLog().isInSnapshot(index)) {
122             return true;
123         }
124
125         ReplicatedLogEntry entry = context.getReplicatedLog().get(index);
126         return entry != null;
127
128     }
129
130     private void updateInitialSyncStatus(final long currentLeaderCommit, final String newLeaderId) {
131         initialSyncStatusTracker.update(newLeaderId, currentLeaderCommit, context.getCommitIndex());
132     }
133
134     @Override
135     protected RaftActorBehavior handleAppendEntries(final ActorRef sender, final AppendEntries appendEntries) {
136         int numLogEntries = appendEntries.getEntries().size();
137         if (log.isTraceEnabled()) {
138             log.trace("{}: handleAppendEntries: {}", logName(), appendEntries);
139         } else if (log.isDebugEnabled() && numLogEntries > 0) {
140             log.debug("{}: handleAppendEntries: {}", logName(), appendEntries);
141         }
142
143         if (snapshotTracker != null && !snapshotTracker.getLeaderId().equals(appendEntries.getLeaderId())) {
144             log.debug("{}: snapshot install is in progress but the prior snapshot leaderId {} does not match the "
145                 + "AppendEntries leaderId {}", logName(), snapshotTracker.getLeaderId(), appendEntries.getLeaderId());
146             closeSnapshotTracker();
147         }
148
149         if (snapshotTracker != null || context.getSnapshotManager().isApplying()) {
150             // if snapshot install is in progress, follower should just acknowledge append entries with a reply.
151             AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true,
152                     lastIndex(), lastTerm(), context.getPayloadVersion(), false, needsLeaderAddress(),
153                     appendEntries.getLeaderRaftVersion());
154
155             log.debug("{}: snapshot install is in progress, replying immediately with {}", logName(), reply);
156             sender.tell(reply, actor());
157
158             return this;
159         }
160
161         // If we got here then we do appear to be talking to the leader
162         leaderId = appendEntries.getLeaderId();
163         leaderPayloadVersion = appendEntries.getPayloadVersion();
164
165         if (appendEntries.getLeaderAddress().isPresent()) {
166             final String address = appendEntries.getLeaderAddress().get();
167             log.debug("New leader address: {}", address);
168
169             context.setPeerAddress(leaderId, address);
170             context.getConfigParams().getPeerAddressResolver().setResolved(leaderId, address);
171         }
172
173         // First check if the logs are in sync or not
174         if (isOutOfSync(appendEntries, sender)) {
175             updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId());
176             return this;
177         }
178
179         if (!processNewEntries(appendEntries, sender)) {
180             updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId());
181             return this;
182         }
183
184         long lastIndex = lastIndex();
185         long prevCommitIndex = context.getCommitIndex();
186
187         // If leaderCommit > commitIndex, set commitIndex = min(leaderCommit, index of last new entry)
188         if (appendEntries.getLeaderCommit() > prevCommitIndex) {
189             context.setCommitIndex(Math.min(appendEntries.getLeaderCommit(), lastIndex));
190         }
191
192         if (prevCommitIndex != context.getCommitIndex()) {
193             log.debug("{}: Commit index set to {}", logName(), context.getCommitIndex());
194         }
195
196         AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true,
197                 lastIndex, lastTerm(), context.getPayloadVersion(), false, needsLeaderAddress(),
198                 appendEntries.getLeaderRaftVersion());
199
200         if (log.isTraceEnabled()) {
201             log.trace("{}: handleAppendEntries returning : {}", logName(), reply);
202         } else if (log.isDebugEnabled() && numLogEntries > 0) {
203             log.debug("{}: handleAppendEntries returning : {}", logName(), reply);
204         }
205
206         // Reply to the leader before applying any previous state so as not to hold up leader consensus.
207         sender.tell(reply, actor());
208
209         updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId());
210
211         // If leaderCommit > lastApplied, increment lastApplied and apply log[lastApplied] to state machine (§5.3).
212         // lastApplied can be equal to lastIndex.
213         if (appendEntries.getLeaderCommit() > context.getLastApplied() && context.getLastApplied() < lastIndex) {
214             if (log.isDebugEnabled()) {
215                 log.debug("{}: applyLogToStateMachine, appendEntries.getLeaderCommit(): {}, "
216                         + "context.getLastApplied(): {}, lastIndex(): {}", logName(),
217                     appendEntries.getLeaderCommit(), context.getLastApplied(), lastIndex);
218             }
219
220             applyLogToStateMachine(appendEntries.getLeaderCommit());
221         }
222
223         if (!context.getSnapshotManager().isCapturing()) {
224             super.performSnapshotWithoutCapture(appendEntries.getReplicatedToAllIndex());
225         }
226
227         appendEntriesMessageAssembler.checkExpiredAssembledMessageState();
228
229         return this;
230     }
231
232     private boolean processNewEntries(final AppendEntries appendEntries, final ActorRef sender) {
233         int numLogEntries = appendEntries.getEntries().size();
234         if (numLogEntries == 0) {
235             return true;
236         }
237
238         log.debug("{}: Number of entries to be appended = {}", logName(), numLogEntries);
239
240         long lastIndex = lastIndex();
241         int addEntriesFrom = 0;
242
243         // First check for conflicting entries. If an existing entry conflicts with a new one (same index but different
244         // term), delete the existing entry and all that follow it (§5.3)
245         if (context.getReplicatedLog().size() > 0) {
246             // Find the entry up until the one that is not in the follower's log
247             for (int i = 0;i < numLogEntries; i++, addEntriesFrom++) {
248                 ReplicatedLogEntry matchEntry = appendEntries.getEntries().get(i);
249
250                 if (!isLogEntryPresent(matchEntry.getIndex())) {
251                     // newEntry not found in the log
252                     break;
253                 }
254
255                 long existingEntryTerm = getLogEntryTerm(matchEntry.getIndex());
256
257                 log.debug("{}: matchEntry {} is present: existingEntryTerm: {}", logName(), matchEntry,
258                         existingEntryTerm);
259
260                 // existingEntryTerm == -1 means it's in the snapshot and not in the log. We don't know
261                 // what the term was so we'll assume it matches.
262                 if (existingEntryTerm == -1 || existingEntryTerm == matchEntry.getTerm()) {
263                     continue;
264                 }
265
266                 if (!context.getRaftPolicy().applyModificationToStateBeforeConsensus()) {
267                     log.info("{}: Removing entries from log starting at {}, commitIndex: {}, lastApplied: {}",
268                             logName(), matchEntry.getIndex(), context.getCommitIndex(), context.getLastApplied());
269
270                     // Entries do not match so remove all subsequent entries but only if the existing entries haven't
271                     // been applied to the state yet.
272                     if (matchEntry.getIndex() <= context.getLastApplied()
273                             || !context.getReplicatedLog().removeFromAndPersist(matchEntry.getIndex())) {
274                         // Could not remove the entries - this means the matchEntry index must be in the
275                         // snapshot and not the log. In this case the prior entries are part of the state
276                         // so we must send back a reply to force a snapshot to completely re-sync the
277                         // follower's log and state.
278
279                         log.info("{}: Could not remove entries - sending reply to force snapshot", logName());
280                         sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex,
281                                 lastTerm(), context.getPayloadVersion(), true, needsLeaderAddress(),
282                                 appendEntries.getLeaderRaftVersion()), actor());
283                         return false;
284                     }
285
286                     break;
287                 } else {
288                     sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex,
289                             lastTerm(), context.getPayloadVersion(), true, needsLeaderAddress(),
290                             appendEntries.getLeaderRaftVersion()), actor());
291                     return false;
292                 }
293             }
294         }
295
296         lastIndex = lastIndex();
297         log.debug("{}: After cleanup, lastIndex: {}, entries to be added from: {}", logName(), lastIndex,
298                 addEntriesFrom);
299
300         // When persistence successfully completes for each new log entry appended, we need to determine if we
301         // should capture a snapshot to compact the persisted log. shouldCaptureSnapshot tracks whether or not
302         // one of the log entries has exceeded the log size threshold whereby a snapshot should be taken. However
303         // we don't initiate the snapshot at that log entry but rather after the last log entry has been persisted.
304         // This is done because subsequent log entries after the one that tripped the threshold may have been
305         // applied to the state already, as the persistence callback occurs async, and we want those entries
306         // purged from the persisted log as well.
307         final AtomicBoolean shouldCaptureSnapshot = new AtomicBoolean(false);
308         final Consumer<ReplicatedLogEntry> appendAndPersistCallback = logEntry -> {
309             final List<ReplicatedLogEntry> entries = appendEntries.getEntries();
310             final ReplicatedLogEntry lastEntryToAppend = entries.get(entries.size() - 1);
311             if (shouldCaptureSnapshot.get() && logEntry == lastEntryToAppend) {
312                 context.getSnapshotManager().capture(context.getReplicatedLog().last(), getReplicatedToAllIndex());
313             }
314         };
315
316         // Append any new entries not already in the log
317         for (int i = addEntriesFrom; i < numLogEntries; i++) {
318             ReplicatedLogEntry entry = appendEntries.getEntries().get(i);
319
320             log.debug("{}: Append entry to log {}", logName(), entry.getData());
321
322             context.getReplicatedLog().appendAndPersist(entry, appendAndPersistCallback, false);
323
324             shouldCaptureSnapshot.compareAndSet(false,
325                     context.getReplicatedLog().shouldCaptureSnapshot(entry.getIndex()));
326
327             if (entry.getData() instanceof ServerConfigurationPayload) {
328                 context.updatePeerIds((ServerConfigurationPayload)entry.getData());
329             }
330         }
331
332         log.debug("{}: Log size is now {}", logName(), context.getReplicatedLog().size());
333
334         return true;
335     }
336
337     private boolean isOutOfSync(final AppendEntries appendEntries, final ActorRef sender) {
338
339         final long lastIndex = lastIndex();
340         if (lastIndex == -1 && appendEntries.getPrevLogIndex() != -1) {
341
342             // The follower's log is out of sync because the leader does have an entry at prevLogIndex and this
343             // follower has no entries in it's log.
344
345             log.info("{}: The followers log is empty and the senders prevLogIndex is {}", logName(),
346                 appendEntries.getPrevLogIndex());
347
348             sendOutOfSyncAppendEntriesReply(sender, false, appendEntries.getLeaderRaftVersion());
349             return true;
350         }
351
352         if (lastIndex > -1) {
353             if (isLogEntryPresent(appendEntries.getPrevLogIndex())) {
354                 final long leadersPrevLogTermInFollowersLogOrSnapshot =
355                         getLogEntryOrSnapshotTerm(appendEntries.getPrevLogIndex());
356                 if (leadersPrevLogTermInFollowersLogOrSnapshot != appendEntries.getPrevLogTerm()) {
357
358                     // The follower's log is out of sync because the Leader's prevLogIndex entry does exist
359                     // in the follower's log or snapshot but it has a different term.
360
361                     log.info("{}: The prevLogIndex {} was found in the log but the term {} is not equal to the append "
362                         + "entries prevLogTerm {} - lastIndex: {}, snapshotIndex: {}, snapshotTerm: {}", logName(),
363                         appendEntries.getPrevLogIndex(), leadersPrevLogTermInFollowersLogOrSnapshot,
364                         appendEntries.getPrevLogTerm(), lastIndex, context.getReplicatedLog().getSnapshotIndex(),
365                         context.getReplicatedLog().getSnapshotTerm());
366
367                     sendOutOfSyncAppendEntriesReply(sender, false, appendEntries.getLeaderRaftVersion());
368                     return true;
369                 }
370             } else if (appendEntries.getPrevLogIndex() != -1) {
371
372                 // The follower's log is out of sync because the Leader's prevLogIndex entry was not found in it's log
373
374                 log.info("{}: The log is not empty but the prevLogIndex {} was not found in it - lastIndex: {}, "
375                         + "snapshotIndex: {}, snapshotTerm: {}", logName(), appendEntries.getPrevLogIndex(), lastIndex,
376                         context.getReplicatedLog().getSnapshotIndex(), context.getReplicatedLog().getSnapshotTerm());
377
378                 sendOutOfSyncAppendEntriesReply(sender, false, appendEntries.getLeaderRaftVersion());
379                 return true;
380             }
381         }
382
383         if (appendEntries.getPrevLogIndex() == -1 && appendEntries.getPrevLogTerm() == -1
384                 && appendEntries.getReplicatedToAllIndex() != -1) {
385             if (!isLogEntryPresent(appendEntries.getReplicatedToAllIndex())) {
386                 // This append entry comes from a leader who has it's log aggressively trimmed and so does not have
387                 // the previous entry in it's in-memory journal
388
389                 log.info("{}: Cannot append entries because the replicatedToAllIndex {} does not appear to be in the "
390                         + "in-memory journal - lastIndex: {}, snapshotIndex: {}, snapshotTerm: {}", logName(),
391                         appendEntries.getReplicatedToAllIndex(), lastIndex,
392                         context.getReplicatedLog().getSnapshotIndex(), context.getReplicatedLog().getSnapshotTerm());
393
394                 sendOutOfSyncAppendEntriesReply(sender, false, appendEntries.getLeaderRaftVersion());
395                 return true;
396             }
397
398             final List<ReplicatedLogEntry> entries = appendEntries.getEntries();
399             if (entries.size() > 0 && !isLogEntryPresent(entries.get(0).getIndex() - 1)) {
400                 log.info("{}: Cannot append entries because the calculated previousIndex {} was not found in the "
401                         + "in-memory journal - lastIndex: {}, snapshotIndex: {}, snapshotTerm: {}", logName(),
402                         entries.get(0).getIndex() - 1, lastIndex, context.getReplicatedLog().getSnapshotIndex(),
403                         context.getReplicatedLog().getSnapshotTerm());
404
405                 sendOutOfSyncAppendEntriesReply(sender, false, appendEntries.getLeaderRaftVersion());
406                 return true;
407             }
408         }
409
410         return false;
411     }
412
413     private void sendOutOfSyncAppendEntriesReply(final ActorRef sender, final boolean forceInstallSnapshot,
414             final short leaderRaftVersion) {
415         // We found that the log was out of sync so just send a negative reply.
416         final AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex(),
417                 lastTerm(), context.getPayloadVersion(), forceInstallSnapshot, needsLeaderAddress(),
418                 leaderRaftVersion);
419
420         log.info("{}: Follower is out-of-sync so sending negative reply: {}", logName(), reply);
421         sender.tell(reply, actor());
422     }
423
424     private boolean needsLeaderAddress() {
425         return context.getPeerAddress(leaderId) == null;
426     }
427
428     @Override
429     protected RaftActorBehavior handleAppendEntriesReply(final ActorRef sender,
430         final AppendEntriesReply appendEntriesReply) {
431         return this;
432     }
433
434     @Override
435     protected RaftActorBehavior handleRequestVoteReply(final ActorRef sender,
436         final RequestVoteReply requestVoteReply) {
437         return this;
438     }
439
440     @Override
441     final ApplyState getApplyStateFor(final ReplicatedLogEntry entry) {
442         return new ApplyState(null, null, entry);
443     }
444
445     @Override
446     public RaftActorBehavior handleMessage(final ActorRef sender, final Object message) {
447         if (message instanceof ElectionTimeout || message instanceof TimeoutNow) {
448             return handleElectionTimeout(message);
449         }
450
451         if (appendEntriesMessageAssembler.handleMessage(message, actor())) {
452             return this;
453         }
454
455         if (!(message instanceof RaftRPC)) {
456             // The rest of the processing requires the message to be a RaftRPC
457             return null;
458         }
459
460         final RaftRPC rpc = (RaftRPC) message;
461         // If RPC request or response contains term T > currentTerm:
462         // set currentTerm = T, convert to follower (§5.1)
463         // This applies to all RPC messages and responses
464         if (rpc.getTerm() > context.getTermInformation().getCurrentTerm() && shouldUpdateTerm(rpc)) {
465             log.info("{}: Term {} in \"{}\" message is greater than follower's term {} - updating term",
466                 logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm());
467
468             context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
469         }
470
471         if (rpc instanceof InstallSnapshot) {
472             handleInstallSnapshot(sender, (InstallSnapshot) rpc);
473             restartLastLeaderMessageTimer();
474             scheduleElection(electionDuration());
475             return this;
476         }
477
478         if (!(rpc instanceof RequestVote) || canGrantVote((RequestVote) rpc)) {
479             restartLastLeaderMessageTimer();
480             scheduleElection(electionDuration());
481         }
482
483         return super.handleMessage(sender, rpc);
484     }
485
486     private RaftActorBehavior handleElectionTimeout(final Object message) {
487         // If the message is ElectionTimeout, verify we haven't actually seen a message from the leader
488         // during the election timeout interval. It may that the election timer expired b/c this actor
489         // was busy and messages got delayed, in which case leader messages would be backed up in the
490         // queue but would be processed before the ElectionTimeout message and thus would restart the
491         // lastLeaderMessageTimer.
492         long lastLeaderMessageInterval = lastLeaderMessageTimer.elapsed(TimeUnit.MILLISECONDS);
493         long electionTimeoutInMillis = context.getConfigParams().getElectionTimeOutInterval().toMillis();
494         boolean noLeaderMessageReceived = !lastLeaderMessageTimer.isRunning()
495                 || lastLeaderMessageInterval >= electionTimeoutInMillis;
496
497         if (canStartElection()) {
498             if (message instanceof TimeoutNow) {
499                 log.debug("{}: Received TimeoutNow - switching to Candidate", logName());
500                 return internalSwitchBehavior(RaftState.Candidate);
501             } else if (noLeaderMessageReceived) {
502                 // Check the cluster state to see if the leader is known to be up before we go to Candidate.
503                 // However if we haven't heard from the leader in a long time even though the cluster state
504                 // indicates it's up then something is wrong - leader might be stuck indefinitely - so switch
505                 // to Candidate,
506                 long maxElectionTimeout = electionTimeoutInMillis * MAX_ELECTION_TIMEOUT_FACTOR;
507                 if (isLeaderAvailabilityKnown() && lastLeaderMessageInterval < maxElectionTimeout) {
508                     log.debug("{}: Received ElectionTimeout but leader appears to be available", logName());
509                     scheduleElection(electionDuration());
510                 } else if (isThisFollowerIsolated()) {
511                     log.debug("{}: this follower is isolated. Do not switch to Candidate for now.", logName());
512                     setLeaderId(null);
513                     scheduleElection(electionDuration());
514                 } else {
515                     log.debug("{}: Received ElectionTimeout - switching to Candidate", logName());
516                     return internalSwitchBehavior(RaftState.Candidate);
517                 }
518             } else {
519                 log.debug("{}: Received ElectionTimeout but lastLeaderMessageInterval {} < election timeout {}",
520                         logName(), lastLeaderMessageInterval, context.getConfigParams().getElectionTimeOutInterval());
521                 scheduleElection(electionDuration());
522             }
523         } else if (message instanceof ElectionTimeout) {
524             if (noLeaderMessageReceived) {
525                 setLeaderId(null);
526             }
527
528             scheduleElection(electionDuration());
529         }
530
531         return this;
532     }
533
534     private boolean isLeaderAvailabilityKnown() {
535         if (leaderId == null) {
536             return false;
537         }
538
539         Optional<Cluster> cluster = context.getCluster();
540         if (!cluster.isPresent()) {
541             return false;
542         }
543
544         ActorSelection leaderActor = context.getPeerActorSelection(leaderId);
545         if (leaderActor == null) {
546             return false;
547         }
548
549         Address leaderAddress = leaderActor.anchorPath().address();
550
551         CurrentClusterState state = cluster.get().state();
552         Set<Member> unreachable = state.getUnreachable();
553
554         log.debug("{}: Checking for leader {} in the cluster unreachable set {}", logName(), leaderAddress,
555                 unreachable);
556
557         for (Member m: unreachable) {
558             if (leaderAddress.equals(m.address())) {
559                 log.info("{}: Leader {} is unreachable", logName(), leaderAddress);
560                 return false;
561             }
562         }
563
564         for (Member m: state.getMembers()) {
565             if (leaderAddress.equals(m.address())) {
566                 if (m.status() == MemberStatus.up() || m.status() == MemberStatus.weaklyUp()) {
567                     log.debug("{}: Leader {} cluster status is {} - leader is available", logName(),
568                             leaderAddress, m.status());
569                     return true;
570                 } else {
571                     log.debug("{}: Leader {} cluster status is {} - leader is unavailable", logName(),
572                             leaderAddress, m.status());
573                     return false;
574                 }
575             }
576         }
577
578         log.debug("{}: Leader {} not found in the cluster member set", logName(), leaderAddress);
579
580         return false;
581     }
582
583     private boolean isThisFollowerIsolated() {
584         final Optional<Cluster> maybeCluster = context.getCluster();
585         if (!maybeCluster.isPresent()) {
586             return false;
587         }
588
589         final Cluster cluster = maybeCluster.get();
590         final Member selfMember = cluster.selfMember();
591
592         final CurrentClusterState state = cluster.state();
593         final Set<Member> unreachable = state.getUnreachable();
594         final Iterable<Member> members = state.getMembers();
595
596         log.debug("{}: Checking if this node is isolated in the cluster unreachable set {},"
597                         + "all members {} self member: {}", logName(), unreachable, members, selfMember);
598
599         // no unreachable peers means we cannot be isolated
600         if (unreachable.size() == 0) {
601             return false;
602         }
603
604         final Set<Member> membersToCheck = new HashSet<>();
605         members.forEach(membersToCheck::add);
606
607         membersToCheck.removeAll(unreachable);
608
609         // check if the only member not unreachable is us
610         if (membersToCheck.size() == 1 && membersToCheck.iterator().next().equals(selfMember)) {
611             return true;
612         }
613
614         return false;
615     }
616
617     private void handleInstallSnapshot(final ActorRef sender, final InstallSnapshot installSnapshot) {
618
619         log.debug("{}: handleInstallSnapshot: {}", logName(), installSnapshot);
620
621         leaderId = installSnapshot.getLeaderId();
622
623         if (snapshotTracker == null) {
624             snapshotTracker = new SnapshotTracker(log, installSnapshot.getTotalChunks(), installSnapshot.getLeaderId(),
625                     context);
626         }
627
628         updateInitialSyncStatus(installSnapshot.getLastIncludedIndex(), installSnapshot.getLeaderId());
629
630         try {
631             final InstallSnapshotReply reply = new InstallSnapshotReply(
632                     currentTerm(), context.getId(), installSnapshot.getChunkIndex(), true);
633
634             if (snapshotTracker.addChunk(installSnapshot.getChunkIndex(), installSnapshot.getData(),
635                     installSnapshot.getLastChunkHashCode())) {
636
637                 log.info("{}: Snapshot installed from leader: {}", logName(), installSnapshot.getLeaderId());
638
639                 Snapshot snapshot = Snapshot.create(
640                         context.getSnapshotManager().convertSnapshot(snapshotTracker.getSnapshotBytes()),
641                         new ArrayList<>(),
642                         installSnapshot.getLastIncludedIndex(),
643                         installSnapshot.getLastIncludedTerm(),
644                         installSnapshot.getLastIncludedIndex(),
645                         installSnapshot.getLastIncludedTerm(),
646                         context.getTermInformation().getCurrentTerm(),
647                         context.getTermInformation().getVotedFor(),
648                         installSnapshot.getServerConfig().orElse(null));
649
650                 ApplySnapshot.Callback applySnapshotCallback = new ApplySnapshot.Callback() {
651                     @Override
652                     public void onSuccess() {
653                         log.debug("{}: handleInstallSnapshot returning: {}", logName(), reply);
654
655                         sender.tell(reply, actor());
656                     }
657
658                     @Override
659                     public void onFailure() {
660                         sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(), -1, false), actor());
661                     }
662                 };
663
664                 actor().tell(new ApplySnapshot(snapshot, applySnapshotCallback), actor());
665
666                 closeSnapshotTracker();
667             } else {
668                 log.debug("{}: handleInstallSnapshot returning: {}", logName(), reply);
669
670                 sender.tell(reply, actor());
671             }
672         } catch (IOException e) {
673             log.debug("{}: Exception in InstallSnapshot of follower", logName(), e);
674
675             sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(),
676                     -1, false), actor());
677
678             closeSnapshotTracker();
679         }
680     }
681
682     private void closeSnapshotTracker() {
683         if (snapshotTracker != null) {
684             snapshotTracker.close();
685             snapshotTracker = null;
686         }
687     }
688
689     @Override
690     public void close() {
691         closeSnapshotTracker();
692         stopElection();
693         appendEntriesMessageAssembler.close();
694     }
695
696     @VisibleForTesting
697     SnapshotTracker getSnapshotTracker() {
698         return snapshotTracker;
699     }
700 }