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