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