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