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