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