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