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