Send commitIndex updates to followers as soon as possible
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / behaviors / AbstractLeader.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.Cancellable;
14 import com.google.common.annotations.VisibleForTesting;
15 import com.google.common.base.Optional;
16 import com.google.common.base.Preconditions;
17 import com.google.common.io.ByteSource;
18 import java.io.IOException;
19 import java.io.ObjectOutputStream;
20 import java.util.Collection;
21 import java.util.Collections;
22 import java.util.HashMap;
23 import java.util.Iterator;
24 import java.util.LinkedList;
25 import java.util.List;
26 import java.util.Map;
27 import java.util.Map.Entry;
28 import java.util.Queue;
29 import java.util.concurrent.TimeUnit;
30 import javax.annotation.Nullable;
31 import org.opendaylight.controller.cluster.io.SharedFileBackedOutputStream;
32 import org.opendaylight.controller.cluster.messaging.MessageSlicer;
33 import org.opendaylight.controller.cluster.messaging.SliceOptions;
34 import org.opendaylight.controller.cluster.raft.ClientRequestTracker;
35 import org.opendaylight.controller.cluster.raft.ClientRequestTrackerImpl;
36 import org.opendaylight.controller.cluster.raft.FollowerLogInformation;
37 import org.opendaylight.controller.cluster.raft.PeerInfo;
38 import org.opendaylight.controller.cluster.raft.RaftActorContext;
39 import org.opendaylight.controller.cluster.raft.RaftState;
40 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
41 import org.opendaylight.controller.cluster.raft.VotingState;
42 import org.opendaylight.controller.cluster.raft.base.messages.CheckConsensusReached;
43 import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
44 import org.opendaylight.controller.cluster.raft.base.messages.SendHeartBeat;
45 import org.opendaylight.controller.cluster.raft.base.messages.SendInstallSnapshot;
46 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
47 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
48 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot;
49 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshotReply;
50 import org.opendaylight.controller.cluster.raft.messages.RaftRPC;
51 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
52 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
53 import org.opendaylight.controller.cluster.raft.messages.UnInitializedFollowerSnapshotReply;
54 import org.opendaylight.controller.cluster.raft.persisted.ServerConfigurationPayload;
55 import org.opendaylight.controller.cluster.raft.persisted.Snapshot;
56 import scala.concurrent.duration.FiniteDuration;
57
58 /**
59  * The behavior of a RaftActor when it is in the Leader state.
60  *
61  * <p>
62  * Leaders:
63  * <ul>
64  * <li> Upon election: send initial empty AppendEntries RPCs
65  * (heartbeat) to each server; repeat during idle periods to
66  * prevent election timeouts (§5.2)
67  * <li> If command received from client: append entry to local log,
68  * respond after entry applied to state machine (§5.3)
69  * <li> If last log index ≥ nextIndex for a follower: send
70  * AppendEntries RPC with log entries starting at nextIndex
71  * <li> If successful: update nextIndex and matchIndex for
72  * follower (§5.3)
73  * <li> If AppendEntries fails because of log inconsistency:
74  * decrement nextIndex and retry (§5.3)
75  * <li> If there exists an N such that N &gt; commitIndex, a majority
76  * of matchIndex[i] ≥ N, and log[N].term == currentTerm:
77  * set commitIndex = N (§5.3, §5.4).
78  * </ul>
79  */
80 public abstract class AbstractLeader extends AbstractRaftActorBehavior {
81     private final Map<String, FollowerLogInformation> followerToLog = new HashMap<>();
82
83     /**
84      * Lookup table for request contexts based on journal index. We could use a {@link Map} here, but we really
85      * expect the entries to be modified in sequence, hence we open-code the lookup.
86      * TODO: Evaluate the use of ArrayDeque(), as that has lower memory overhead. Non-head removals are more costly,
87      *       but we already expect those to be far from frequent.
88      */
89     private final Queue<ClientRequestTracker> trackers = new LinkedList<>();
90
91     /**
92      * Map of serialized AppendEntries output streams keyed by log index. This is used in conjunction with the
93      * appendEntriesMessageSlicer for slicing single ReplicatedLogEntry payloads that exceed the message size threshold.
94      * This Map allows the SharedFileBackedOutputStreams to be reused for multiple followers.
95      */
96     private final Map<Long, SharedFileBackedOutputStream> sharedSerializedAppendEntriesStreams = new HashMap<>();
97     private final MessageSlicer appendEntriesMessageSlicer;
98
99     private Cancellable heartbeatSchedule = null;
100     private Optional<SnapshotHolder> snapshotHolder = Optional.absent();
101     private int minReplicationCount;
102
103     protected AbstractLeader(final RaftActorContext context, final RaftState state,
104             @Nullable final AbstractLeader initializeFromLeader) {
105         super(context, state);
106
107         appendEntriesMessageSlicer = MessageSlicer.builder().logContext(logName())
108             .messageSliceSize(context.getConfigParams().getSnapshotChunkSize())
109             .expireStateAfterInactivity(context.getConfigParams().getElectionTimeOutInterval().toMillis() * 3,
110                     TimeUnit.MILLISECONDS).build();
111
112         if (initializeFromLeader != null) {
113             followerToLog.putAll(initializeFromLeader.followerToLog);
114             snapshotHolder = initializeFromLeader.snapshotHolder;
115             trackers.addAll(initializeFromLeader.trackers);
116         } else {
117             for (PeerInfo peerInfo: context.getPeers()) {
118                 FollowerLogInformation followerLogInformation = new FollowerLogInformation(peerInfo, context);
119                 followerToLog.put(peerInfo.getId(), followerLogInformation);
120             }
121         }
122
123         log.debug("{}: Election: Leader has following peers: {}", logName(), getFollowerIds());
124
125         updateMinReplicaCount();
126
127         // Immediately schedule a heartbeat
128         // Upon election: send initial empty AppendEntries RPCs
129         // (heartbeat) to each server; repeat during idle periods to
130         // prevent election timeouts (§5.2)
131         sendAppendEntries(0, false);
132
133         // It is important to schedule this heartbeat here
134         scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
135     }
136
137     protected AbstractLeader(final RaftActorContext context, final RaftState state) {
138         this(context, state, null);
139     }
140
141     /**
142      * Return an immutable collection of follower identifiers.
143      *
144      * @return Collection of follower IDs
145      */
146     public final Collection<String> getFollowerIds() {
147         return followerToLog.keySet();
148     }
149
150     public void addFollower(final String followerId) {
151         FollowerLogInformation followerLogInformation = new FollowerLogInformation(context.getPeerInfo(followerId),
152             context);
153         followerToLog.put(followerId, followerLogInformation);
154
155         if (heartbeatSchedule == null) {
156             scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
157         }
158     }
159
160     public void removeFollower(final String followerId) {
161         followerToLog.remove(followerId);
162     }
163
164     public void updateMinReplicaCount() {
165         int numVoting = 0;
166         for (PeerInfo peer: context.getPeers()) {
167             if (peer.isVoting()) {
168                 numVoting++;
169             }
170         }
171
172         minReplicationCount = getMajorityVoteCount(numVoting);
173     }
174
175     protected int getMinIsolatedLeaderPeerCount() {
176       //the isolated Leader peer count will be 1 less than the majority vote count.
177         //this is because the vote count has the self vote counted in it
178         //for e.g
179         //0 peers = 1 votesRequired , minIsolatedLeaderPeerCount = 0
180         //2 peers = 2 votesRequired , minIsolatedLeaderPeerCount = 1
181         //4 peers = 3 votesRequired, minIsolatedLeaderPeerCount = 2
182
183         return minReplicationCount > 0 ? minReplicationCount - 1 : 0;
184     }
185
186     @VisibleForTesting
187     void setSnapshotHolder(@Nullable final SnapshotHolder snapshotHolder) {
188         this.snapshotHolder = Optional.fromNullable(snapshotHolder);
189     }
190
191     @VisibleForTesting
192     boolean hasSnapshot() {
193         return snapshotHolder.isPresent();
194     }
195
196     @Override
197     protected RaftActorBehavior handleAppendEntries(final ActorRef sender,
198         final AppendEntries appendEntries) {
199
200         log.debug("{}: handleAppendEntries: {}", logName(), appendEntries);
201
202         return this;
203     }
204
205     @Override
206     protected RaftActorBehavior handleAppendEntriesReply(final ActorRef sender,
207             final AppendEntriesReply appendEntriesReply) {
208         log.trace("{}: handleAppendEntriesReply: {}", logName(), appendEntriesReply);
209
210         // Update the FollowerLogInformation
211         String followerId = appendEntriesReply.getFollowerId();
212         FollowerLogInformation followerLogInformation = followerToLog.get(followerId);
213
214         if (followerLogInformation == null) {
215             log.error("{}: handleAppendEntriesReply - unknown follower {}", logName(), followerId);
216             return this;
217         }
218
219         final long lastActivityNanos = followerLogInformation.nanosSinceLastActivity();
220         if (lastActivityNanos > context.getConfigParams().getElectionTimeOutInterval().toNanos()) {
221             log.warn("{} : handleAppendEntriesReply delayed beyond election timeout, "
222                     + "appendEntriesReply : {}, timeSinceLastActivity : {}, lastApplied : {}, commitIndex : {}",
223                     logName(), appendEntriesReply, TimeUnit.NANOSECONDS.toMillis(lastActivityNanos),
224                     context.getLastApplied(), context.getCommitIndex());
225         }
226
227         followerLogInformation.markFollowerActive();
228         followerLogInformation.setPayloadVersion(appendEntriesReply.getPayloadVersion());
229         followerLogInformation.setRaftVersion(appendEntriesReply.getRaftVersion());
230         followerLogInformation.setNeedsLeaderAddress(appendEntriesReply.isNeedsLeaderAddress());
231
232         long followerLastLogIndex = appendEntriesReply.getLogLastIndex();
233         boolean updated = false;
234         if (appendEntriesReply.getLogLastIndex() > context.getReplicatedLog().lastIndex()) {
235             // The follower's log is actually ahead of the leader's log. Normally this doesn't happen
236             // in raft as a node cannot become leader if it's log is behind another's. However, the
237             // non-voting semantics deviate a bit from raft. Only voting members participate in
238             // elections and can become leader so it's possible for a non-voting follower to be ahead
239             // of the leader. This can happen if persistence is disabled and all voting members are
240             // restarted. In this case, the voting leader will start out with an empty log however
241             // the non-voting followers still retain the previous data in memory. On the first
242             // AppendEntries, the non-voting follower returns a successful reply b/c the prevLogIndex
243             // sent by the leader is -1 and thus the integrity checks pass. However the follower's returned
244             // lastLogIndex may be higher in which case we want to reset the follower by installing a
245             // snapshot. It's also possible that the follower's last log index is behind the leader's.
246             // However in this case the log terms won't match and the logs will conflict - this is handled
247             // elsewhere.
248             log.info("{}: handleAppendEntriesReply: follower {} lastIndex {} is ahead of our lastIndex {} "
249                     + "(snapshotIndex {}, snapshotTerm {}) - forcing install snaphot", logName(),
250                     followerLogInformation.getId(), appendEntriesReply.getLogLastIndex(),
251                     context.getReplicatedLog().lastIndex(), context.getReplicatedLog().getSnapshotIndex(),
252                     context.getReplicatedLog().getSnapshotTerm());
253
254             followerLogInformation.setMatchIndex(-1);
255             followerLogInformation.setNextIndex(-1);
256
257             initiateCaptureSnapshot(followerId);
258
259             updated = true;
260         } else if (appendEntriesReply.isSuccess()) {
261             long followersLastLogTermInLeadersLog = getLogEntryTerm(followerLastLogIndex);
262             if (followerLastLogIndex >= 0 && followersLastLogTermInLeadersLog >= 0
263                     && followersLastLogTermInLeadersLog != appendEntriesReply.getLogLastTerm()) {
264                 // The follower's last entry is present in the leader's journal but the terms don't match so the
265                 // follower has a conflicting entry. Since the follower didn't report that it's out of sync, this means
266                 // either the previous leader entry sent didn't conflict or the previous leader entry is in the snapshot
267                 // and no longer in the journal. Either way, we set the follower's next index to 1 less than the last
268                 // index reported by the follower. For the former case, the leader will send all entries starting with
269                 // the previous follower's index and the follower will remove and replace the conflicting entries as
270                 // needed. For the latter, the leader will initiate an install snapshot.
271
272                 followerLogInformation.setNextIndex(followerLastLogIndex - 1);
273                 updated = true;
274
275                 log.info("{}: handleAppendEntriesReply: follower {} last log term {} for index {} conflicts with the "
276                         + "leader's {} - set the follower's next index to {}", logName(),
277                         followerId, appendEntriesReply.getLogLastTerm(), appendEntriesReply.getLogLastIndex(),
278                         followersLastLogTermInLeadersLog, followerLogInformation.getNextIndex());
279             } else {
280                 updated = updateFollowerLogInformation(followerLogInformation, appendEntriesReply);
281             }
282         } else {
283             log.info("{}: handleAppendEntriesReply - received unsuccessful reply: {}, leader snapshotIndex: {}, "
284                     + "snapshotTerm: {}, replicatedToAllIndex: {}", logName(), appendEntriesReply,
285                     context.getReplicatedLog().getSnapshotIndex(), context.getReplicatedLog().getSnapshotTerm(),
286                     getReplicatedToAllIndex());
287
288             long followersLastLogTermInLeadersLogOrSnapshot = getLogEntryOrSnapshotTerm(followerLastLogIndex);
289             if (appendEntriesReply.isForceInstallSnapshot()) {
290                 // Reset the followers match and next index. This is to signal that this follower has nothing
291                 // in common with this Leader and so would require a snapshot to be installed
292                 followerLogInformation.setMatchIndex(-1);
293                 followerLogInformation.setNextIndex(-1);
294
295                 // Force initiate a snapshot capture
296                 initiateCaptureSnapshot(followerId);
297             } else if (followerLastLogIndex < 0 || followersLastLogTermInLeadersLogOrSnapshot >= 0
298                     && followersLastLogTermInLeadersLogOrSnapshot == appendEntriesReply.getLogLastTerm()) {
299                 // The follower's log is empty or the follower's last entry is present in the leader's journal or
300                 // snapshot and the terms match so the follower is just behind the leader's journal from the last
301                 // snapshot, if any. We'll catch up the follower quickly by starting at the follower's last log index.
302
303                 updated = updateFollowerLogInformation(followerLogInformation, appendEntriesReply);
304
305                 log.info("{}: follower {} appears to be behind the leader from the last snapshot - "
306                     + "updated: matchIndex: {}, nextIndex: {}", logName(), followerId,
307                     followerLogInformation.getMatchIndex(), followerLogInformation.getNextIndex());
308             } else {
309                 // The follower's log conflicts with leader's log so decrement follower's next index
310                 // in an attempt to find where the logs match.
311                 if (followerLogInformation.decrNextIndex(appendEntriesReply.getLogLastIndex())) {
312                     updated = true;
313
314                     log.info("{}: follower {} last log term {} conflicts with the leader's {} - dec next index to {}",
315                             logName(), followerId, appendEntriesReply.getLogLastTerm(),
316                             followersLastLogTermInLeadersLogOrSnapshot, followerLogInformation.getNextIndex());
317                 }
318             }
319         }
320
321         if (log.isTraceEnabled()) {
322             log.trace("{}: handleAppendEntriesReply from {}: commitIndex: {}, lastAppliedIndex: {}, currentTerm: {}",
323                     logName(), followerId, context.getCommitIndex(), context.getLastApplied(), currentTerm());
324         }
325
326         possiblyUpdateCommitIndex();
327
328         //Send the next log entry immediately, if possible, no need to wait for heartbeat to trigger that event
329         sendUpdatesToFollower(followerId, followerLogInformation, false, !updated);
330
331         return this;
332     }
333
334     private void possiblyUpdateCommitIndex() {
335         // Figure out if we can update the the commitIndex as follows:
336         //   If there exists an index N such that N > commitIndex, a majority of matchIndex[i] ≥ N,
337         //     and log[N].term == currentTerm:
338         //   set commitIndex = N (§5.3, §5.4).
339         for (long index = context.getCommitIndex() + 1; ; index++) {
340             ReplicatedLogEntry replicatedLogEntry = context.getReplicatedLog().get(index);
341             if (replicatedLogEntry == null) {
342                 log.trace("{}: ReplicatedLogEntry not found for index {} - snapshotIndex: {}, journal size: {}",
343                         logName(), index, context.getReplicatedLog().getSnapshotIndex(),
344                         context.getReplicatedLog().size());
345                 break;
346             }
347
348             // Count our entry if it has been persisted.
349             int replicatedCount = replicatedLogEntry.isPersistencePending() ? 0 : 1;
350
351             if (replicatedCount == 0) {
352                 // We don't commit and apply a log entry until we've gotten the ack from our local persistence,
353                 // even though there *shouldn't* be any issue with updating the commit index if we get a consensus
354                 // amongst the followers w/o the local persistence ack.
355                 break;
356             }
357
358             log.trace("{}: checking Nth index {}", logName(), index);
359             for (FollowerLogInformation info : followerToLog.values()) {
360                 final PeerInfo peerInfo = context.getPeerInfo(info.getId());
361                 if (info.getMatchIndex() >= index && peerInfo != null && peerInfo.isVoting()) {
362                     replicatedCount++;
363                 } else if (log.isTraceEnabled()) {
364                     log.trace("{}: Not counting follower {} - matchIndex: {}, {}", logName(), info.getId(),
365                             info.getMatchIndex(), peerInfo);
366                 }
367             }
368
369             if (log.isTraceEnabled()) {
370                 log.trace("{}: replicatedCount {}, minReplicationCount: {}", logName(), replicatedCount,
371                         minReplicationCount);
372             }
373
374             if (replicatedCount >= minReplicationCount) {
375                 // Don't update the commit index if the log entry is from a previous term, as per §5.4.1:
376                 // "Raft never commits log entries from previous terms by counting replicas".
377                 // However we keep looping so we can make progress when new entries in the current term
378                 // reach consensus, as per §5.4.1: "once an entry from the current term is committed by
379                 // counting replicas, then all prior entries are committed indirectly".
380                 if (replicatedLogEntry.getTerm() == currentTerm()) {
381                     log.trace("{}: Setting commit index to {}", logName(), index);
382                     context.setCommitIndex(index);
383                 } else {
384                     log.debug("{}: Not updating commit index to {} - retrieved log entry with index {}, "
385                             + "term {} does not match the current term {}", logName(), index,
386                             replicatedLogEntry.getIndex(), replicatedLogEntry.getTerm(), currentTerm());
387                 }
388             } else {
389                 log.trace("{}: minReplicationCount not reached, actual {} - breaking", logName(), replicatedCount);
390                 break;
391             }
392         }
393
394         // Apply the change to the state machine
395         if (context.getCommitIndex() > context.getLastApplied()) {
396             log.debug("{}: Applying to log - commitIndex: {}, lastAppliedIndex: {}", logName(),
397                     context.getCommitIndex(), context.getLastApplied());
398
399             applyLogToStateMachine(context.getCommitIndex());
400         }
401
402         if (!context.getSnapshotManager().isCapturing()) {
403             purgeInMemoryLog();
404         }
405     }
406
407     private boolean updateFollowerLogInformation(final FollowerLogInformation followerLogInformation,
408             final AppendEntriesReply appendEntriesReply) {
409         boolean updated = followerLogInformation.setMatchIndex(appendEntriesReply.getLogLastIndex());
410         updated = followerLogInformation.setNextIndex(appendEntriesReply.getLogLastIndex() + 1) || updated;
411
412         if (updated && log.isDebugEnabled()) {
413             log.debug(
414                 "{}: handleAppendEntriesReply - FollowerLogInformation for {} updated: matchIndex: {}, nextIndex: {}",
415                 logName(), followerLogInformation.getId(), followerLogInformation.getMatchIndex(),
416                 followerLogInformation.getNextIndex());
417         }
418         return updated;
419     }
420
421     private void purgeInMemoryLog() {
422         //find the lowest index across followers which has been replicated to all.
423         // lastApplied if there are no followers, so that we keep clearing the log for single-node
424         // we would delete the in-mem log from that index on, in-order to minimize mem usage
425         // we would also share this info thru AE with the followers so that they can delete their log entries as well.
426         long minReplicatedToAllIndex = followerToLog.isEmpty() ? context.getLastApplied() : Long.MAX_VALUE;
427         for (FollowerLogInformation info : followerToLog.values()) {
428             minReplicatedToAllIndex = Math.min(minReplicatedToAllIndex, info.getMatchIndex());
429         }
430
431         super.performSnapshotWithoutCapture(minReplicatedToAllIndex);
432     }
433
434     @Override
435     protected ClientRequestTracker removeClientRequestTracker(final long logIndex) {
436         final Iterator<ClientRequestTracker> it = trackers.iterator();
437         while (it.hasNext()) {
438             final ClientRequestTracker t = it.next();
439             if (t.getIndex() == logIndex) {
440                 it.remove();
441                 return t;
442             }
443         }
444
445         return null;
446     }
447
448     @Override
449     protected RaftActorBehavior handleRequestVoteReply(final ActorRef sender,
450         final RequestVoteReply requestVoteReply) {
451         return this;
452     }
453
454     protected void beforeSendHeartbeat(){}
455
456     @Override
457     public RaftActorBehavior handleMessage(final ActorRef sender, final Object message) {
458         Preconditions.checkNotNull(sender, "sender should not be null");
459
460         if (appendEntriesMessageSlicer.handleMessage(message)) {
461             return this;
462         }
463
464         if (message instanceof RaftRPC) {
465             RaftRPC rpc = (RaftRPC) message;
466             // If RPC request or response contains term T > currentTerm:
467             // set currentTerm = T, convert to follower (§5.1)
468             // This applies to all RPC messages and responses
469             if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
470                 log.info("{}: Term {} in \"{}\" message is greater than leader's term {} - switching to Follower",
471                         logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm());
472
473                 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
474
475                 // This is a special case. Normally when stepping down as leader we don't process and reply to the
476                 // RaftRPC as per raft. But if we're in the process of transferring leadership and we get a
477                 // RequestVote, process the RequestVote before switching to Follower. This enables the requesting
478                 // candidate node to be elected the leader faster and avoids us possibly timing out in the Follower
479                 // state and starting a new election and grabbing leadership back before the other candidate node can
480                 // start a new election due to lack of responses. This case would only occur if there isn't a majority
481                 // of other nodes available that can elect the requesting candidate. Since we're transferring
482                 // leadership, we should make every effort to get the requesting node elected.
483                 if (message instanceof RequestVote && context.getRaftActorLeadershipTransferCohort() != null) {
484                     log.debug("{}: Leadership transfer in progress - processing RequestVote", logName());
485                     super.handleMessage(sender, message);
486                 }
487
488                 return internalSwitchBehavior(RaftState.Follower);
489             }
490         }
491
492         if (message instanceof SendHeartBeat) {
493             beforeSendHeartbeat();
494             sendHeartBeat();
495             scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
496         } else if (message instanceof SendInstallSnapshot) {
497             SendInstallSnapshot sendInstallSnapshot = (SendInstallSnapshot) message;
498             setSnapshotHolder(new SnapshotHolder(sendInstallSnapshot.getSnapshot(),
499                 sendInstallSnapshot.getSnapshotBytes()));
500             sendInstallSnapshot();
501         } else if (message instanceof Replicate) {
502             replicate((Replicate) message);
503         } else if (message instanceof InstallSnapshotReply) {
504             handleInstallSnapshotReply((InstallSnapshotReply) message);
505         } else if (message instanceof CheckConsensusReached) {
506             possiblyUpdateCommitIndex();
507         } else {
508             return super.handleMessage(sender, message);
509         }
510
511         return this;
512     }
513
514     private void handleInstallSnapshotReply(final InstallSnapshotReply reply) {
515         log.debug("{}: handleInstallSnapshotReply: {}", logName(), reply);
516
517         String followerId = reply.getFollowerId();
518         FollowerLogInformation followerLogInformation = followerToLog.get(followerId);
519         if (followerLogInformation == null) {
520             // This can happen during AddServer if it times out.
521             log.error("{}: FollowerLogInformation not found for follower {} in InstallSnapshotReply",
522                     logName(), followerId);
523             return;
524         }
525
526         LeaderInstallSnapshotState installSnapshotState = followerLogInformation.getInstallSnapshotState();
527         if (installSnapshotState == null) {
528             log.error("{}: LeaderInstallSnapshotState not found for follower {} in InstallSnapshotReply",
529                     logName(), followerId);
530             return;
531         }
532
533         installSnapshotState.resetChunkTimer();
534         followerLogInformation.markFollowerActive();
535
536         if (installSnapshotState.getChunkIndex() == reply.getChunkIndex()) {
537             boolean wasLastChunk = false;
538             if (reply.isSuccess()) {
539                 if (installSnapshotState.isLastChunk(reply.getChunkIndex())) {
540                     //this was the last chunk reply
541
542                     long followerMatchIndex = snapshotHolder.get().getLastIncludedIndex();
543                     followerLogInformation.setMatchIndex(followerMatchIndex);
544                     followerLogInformation.setNextIndex(followerMatchIndex + 1);
545                     followerLogInformation.clearLeaderInstallSnapshotState();
546
547                     log.info("{}: Snapshot successfully installed on follower {} (last chunk {}) - "
548                         + "matchIndex set to {}, nextIndex set to {}", logName(), followerId, reply.getChunkIndex(),
549                         followerLogInformation.getMatchIndex(), followerLogInformation.getNextIndex());
550
551                     if (!anyFollowersInstallingSnapshot()) {
552                         // once there are no pending followers receiving snapshots
553                         // we can remove snapshot from the memory
554                         setSnapshotHolder(null);
555                     }
556
557                     wasLastChunk = true;
558                     if (context.getPeerInfo(followerId).getVotingState() == VotingState.VOTING_NOT_INITIALIZED) {
559                         UnInitializedFollowerSnapshotReply unInitFollowerSnapshotSuccess =
560                                              new UnInitializedFollowerSnapshotReply(followerId);
561                         context.getActor().tell(unInitFollowerSnapshotSuccess, context.getActor());
562                         log.debug("Sent message UnInitializedFollowerSnapshotReply to self");
563                     }
564                 } else {
565                     installSnapshotState.markSendStatus(true);
566                 }
567             } else {
568                 log.warn("{}: Received failed InstallSnapshotReply - will retry: {}", logName(), reply);
569
570                 installSnapshotState.markSendStatus(false);
571             }
572
573             if (wasLastChunk) {
574                 if (!context.getSnapshotManager().isCapturing()) {
575                     // Since the follower is now caught up try to purge the log.
576                     purgeInMemoryLog();
577                 }
578             } else {
579                 ActorSelection followerActor = context.getPeerActorSelection(followerId);
580                 if (followerActor != null) {
581                     sendSnapshotChunk(followerActor, followerLogInformation);
582                 }
583             }
584
585         } else {
586             log.error("{}: Chunk index {} in InstallSnapshotReply from follower {} does not match expected index {}",
587                     logName(), reply.getChunkIndex(), followerId,
588                     installSnapshotState.getChunkIndex());
589
590             if (reply.getChunkIndex() == LeaderInstallSnapshotState.INVALID_CHUNK_INDEX) {
591                 // Since the Follower did not find this index to be valid we should reset the follower snapshot
592                 // so that Installing the snapshot can resume from the beginning
593                 installSnapshotState.reset();
594             }
595         }
596     }
597
598     private boolean anyFollowersInstallingSnapshot() {
599         for (FollowerLogInformation info: followerToLog.values()) {
600             if (info.getInstallSnapshotState() != null) {
601                 return true;
602             }
603
604         }
605
606         return false;
607     }
608
609     private void replicate(final Replicate replicate) {
610         long logIndex = replicate.getReplicatedLogEntry().getIndex();
611
612         log.debug("{}: Replicate message: identifier: {}, logIndex: {}, payload: {}, isSendImmediate: {}", logName(),
613                 replicate.getIdentifier(), logIndex, replicate.getReplicatedLogEntry().getData().getClass(),
614                 replicate.isSendImmediate());
615
616         // Create a tracker entry we will use this later to notify the
617         // client actor
618         if (replicate.getClientActor() != null) {
619             trackers.add(new ClientRequestTrackerImpl(replicate.getClientActor(), replicate.getIdentifier(),
620                     logIndex));
621         }
622
623         boolean applyModificationToState = !context.anyVotingPeers()
624                 || context.getRaftPolicy().applyModificationToStateBeforeConsensus();
625
626         if (applyModificationToState) {
627             context.setCommitIndex(logIndex);
628             applyLogToStateMachine(logIndex);
629         }
630
631         if (replicate.isSendImmediate() && !followerToLog.isEmpty()) {
632             sendAppendEntries(0, false);
633         }
634     }
635
636     protected void sendAppendEntries(final long timeSinceLastActivityIntervalNanos, final boolean isHeartbeat) {
637         // Send an AppendEntries to all followers
638         for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
639             final String followerId = e.getKey();
640             final FollowerLogInformation followerLogInformation = e.getValue();
641             // This checks helps not to send a repeat message to the follower
642             if (!followerLogInformation.isFollowerActive()
643                     || followerLogInformation.nanosSinceLastActivity() >= timeSinceLastActivityIntervalNanos) {
644                 sendUpdatesToFollower(followerId, followerLogInformation, true, isHeartbeat);
645             }
646         }
647     }
648
649     /**
650      * This method checks if any update needs to be sent to the given follower. This includes append log entries,
651      * sending next snapshot chunk, and initiating a snapshot.
652      */
653     private void sendUpdatesToFollower(final String followerId, final FollowerLogInformation followerLogInformation,
654                                        final boolean sendHeartbeat, final boolean isHeartbeat) {
655
656         ActorSelection followerActor = context.getPeerActorSelection(followerId);
657         if (followerActor != null) {
658             long followerNextIndex = followerLogInformation.getNextIndex();
659             boolean isFollowerActive = followerLogInformation.isFollowerActive();
660             boolean sendAppendEntries = false;
661             List<ReplicatedLogEntry> entries = Collections.emptyList();
662
663             LeaderInstallSnapshotState installSnapshotState = followerLogInformation.getInstallSnapshotState();
664             if (installSnapshotState != null) {
665
666                 // if install snapshot is in process , then sent next chunk if possible
667                 if (isFollowerActive) {
668                     // 30 seconds with default settings, can be modified via heartbeat or election timeout factor
669                     FiniteDuration snapshotReplyTimeout = context.getConfigParams().getHeartBeatInterval()
670                             .$times(context.getConfigParams().getElectionTimeoutFactor() * 3);
671
672                     if (installSnapshotState.isChunkTimedOut(snapshotReplyTimeout)) {
673                         sendAppendEntries = !resendSnapshotChunk(followerActor, followerLogInformation);
674                     } else if (installSnapshotState.canSendNextChunk()) {
675                         sendSnapshotChunk(followerActor, followerLogInformation);
676                     }
677                 } else if (sendHeartbeat || followerLogInformation.hasStaleCommitIndex(context.getCommitIndex())) {
678                     // we send a heartbeat even if we have not received a reply for the last chunk
679                     sendAppendEntries = true;
680                 }
681             } else if (followerLogInformation.isLogEntrySlicingInProgress()) {
682                 sendAppendEntries = sendHeartbeat;
683             } else {
684                 long leaderLastIndex = context.getReplicatedLog().lastIndex();
685                 long leaderSnapShotIndex = context.getReplicatedLog().getSnapshotIndex();
686
687                 if (!isHeartbeat && log.isDebugEnabled() || log.isTraceEnabled()) {
688                     log.debug("{}: Checking sendAppendEntries for follower {}: active: {}, followerNextIndex: {}, "
689                             + "leaderLastIndex: {}, leaderSnapShotIndex: {}", logName(), followerId, isFollowerActive,
690                             followerNextIndex, leaderLastIndex, leaderSnapShotIndex);
691                 }
692
693                 if (isFollowerActive && context.getReplicatedLog().isPresent(followerNextIndex)) {
694
695                     log.debug("{}: sendAppendEntries: {} is present for follower {}", logName(),
696                             followerNextIndex, followerId);
697
698                     if (followerLogInformation.okToReplicate(context.getCommitIndex())) {
699                         entries = getEntriesToSend(followerLogInformation, followerActor);
700                         sendAppendEntries = true;
701                     }
702                 } else if (isFollowerActive && followerNextIndex >= 0
703                         && leaderLastIndex > followerNextIndex && !context.getSnapshotManager().isCapturing()) {
704                     // if the followers next index is not present in the leaders log, and
705                     // if the follower is just not starting and if leader's index is more than followers index
706                     // then snapshot should be sent
707
708                     // Send heartbeat to follower whenever install snapshot is initiated.
709                     sendAppendEntries = true;
710                     if (canInstallSnapshot(followerNextIndex)) {
711                         log.info("{}: Initiating install snapshot to follower {}: follower nextIndex: {}, leader "
712                                 + "snapshotIndex: {}, leader lastIndex: {}, leader log size: {}", logName(), followerId,
713                                 followerNextIndex, leaderSnapShotIndex, leaderLastIndex,
714                                 context.getReplicatedLog().size());
715
716                         initiateCaptureSnapshot(followerId);
717                     } else {
718                         // It doesn't seem like we should ever reach here - most likely indicates sonething is
719                         // wrong.
720                         log.info("{}: Follower {} is behind but cannot install snapshot: follower nextIndex: {}, "
721                                 + "leader snapshotIndex: {}, leader lastIndex: {}, leader log size: {}", logName(),
722                                 followerId, followerNextIndex, leaderSnapShotIndex, leaderLastIndex,
723                                 context.getReplicatedLog().size());
724                     }
725
726                 } else if (sendHeartbeat || followerLogInformation.hasStaleCommitIndex(context.getCommitIndex())) {
727                     // we send an AppendEntries, even if the follower is inactive
728                     // in-order to update the followers timestamp, in case it becomes active again
729                     sendAppendEntries = true;
730                 }
731
732             }
733
734             if (sendAppendEntries) {
735                 sendAppendEntriesToFollower(followerActor, entries, followerLogInformation);
736             }
737         }
738     }
739
740     private List<ReplicatedLogEntry> getEntriesToSend(final FollowerLogInformation followerLogInfo,
741             final ActorSelection followerActor) {
742         // Try to get all the entries in the journal but not exceeding the max data size for a single AppendEntries
743         // message.
744         int maxEntries = (int) context.getReplicatedLog().size();
745         final int maxDataSize = context.getConfigParams().getSnapshotChunkSize();
746         final long followerNextIndex = followerLogInfo.getNextIndex();
747         List<ReplicatedLogEntry> entries = context.getReplicatedLog().getFrom(followerNextIndex,
748                 maxEntries, maxDataSize);
749
750         // If the first entry's size exceeds the max data size threshold, it will be returned from the call above. If
751         // that is the case, then we need to slice it into smaller chunks.
752         if (!(entries.size() == 1 && entries.get(0).getData().size() > maxDataSize)) {
753             // Don't need to slice.
754             return entries;
755         }
756
757         log.debug("{}: Log entry size {} exceeds max payload size {}", logName(), entries.get(0).getData().size(),
758                 maxDataSize);
759
760         // If an AppendEntries has already been serialized for the log index then reuse the
761         // SharedFileBackedOutputStream.
762         final Long logIndex = entries.get(0).getIndex();
763         SharedFileBackedOutputStream fileBackedStream = sharedSerializedAppendEntriesStreams.get(logIndex);
764         if (fileBackedStream == null) {
765             fileBackedStream = context.getFileBackedOutputStreamFactory().newSharedInstance();
766
767             final AppendEntries appendEntries = new AppendEntries(currentTerm(), context.getId(),
768                     getLogEntryIndex(followerNextIndex - 1), getLogEntryTerm(followerNextIndex - 1), entries,
769                     context.getCommitIndex(), getReplicatedToAllIndex(), context.getPayloadVersion());
770
771             log.debug("{}: Serializing {} for slicing for follower {}", logName(), appendEntries,
772                     followerLogInfo.getId());
773
774             try (ObjectOutputStream out = new ObjectOutputStream(fileBackedStream)) {
775                 out.writeObject(appendEntries);
776             } catch (IOException e) {
777                 log.error("{}: Error serializing {}", logName(), appendEntries, e);
778                 fileBackedStream.cleanup();
779                 return Collections.emptyList();
780             }
781
782             sharedSerializedAppendEntriesStreams.put(logIndex, fileBackedStream);
783
784             fileBackedStream.setOnCleanupCallback(index -> {
785                 log.debug("{}: On SharedFileBackedOutputStream cleanup for index {}", logName(), index);
786                 sharedSerializedAppendEntriesStreams.remove(index);
787             }, logIndex);
788         } else {
789             log.debug("{}: Reusing SharedFileBackedOutputStream for follower {}", logName(), followerLogInfo.getId());
790             fileBackedStream.incrementUsageCount();
791         }
792
793         log.debug("{}: Slicing stream for index {}, follower {}", logName(), logIndex, followerLogInfo.getId());
794
795         // Record that slicing is in progress for the follower.
796         followerLogInfo.setSlicedLogEntryIndex(logIndex);
797
798         final FollowerIdentifier identifier = new FollowerIdentifier(followerLogInfo.getId());
799         appendEntriesMessageSlicer.slice(SliceOptions.builder().identifier(identifier)
800                 .fileBackedOutputStream(fileBackedStream).sendTo(followerActor).replyTo(actor())
801                 .onFailureCallback(failure -> {
802                     log.error("{}: Error slicing AppendEntries for follower {}", logName(),
803                             followerLogInfo.getId(), failure);
804                     followerLogInfo.setSlicedLogEntryIndex(FollowerLogInformation.NO_INDEX);
805                 }).build());
806
807         return Collections.emptyList();
808     }
809
810     private void sendAppendEntriesToFollower(final ActorSelection followerActor, final List<ReplicatedLogEntry> entries,
811             final FollowerLogInformation followerLogInformation) {
812         // In certain cases outlined below we don't want to send the actual commit index to prevent the follower from
813         // possibly committing and applying conflicting entries (those with same index, different term) from a prior
814         // term that weren't replicated to a majority, which would be a violation of raft.
815         //     - if the follower isn't active. In this case we don't know the state of the follower and we send an
816         //       empty AppendEntries as a heart beat to prevent election.
817         //     - if we're in the process of installing a snapshot. In this case we don't send any new entries but still
818         //       need to send AppendEntries to prevent election.
819         //     - if we're in the process of slicing an AppendEntries with a large log entry payload. In this case we
820         //       need to send an empty AppendEntries to prevent election.
821         boolean isInstallingSnaphot = followerLogInformation.getInstallSnapshotState() != null;
822         long leaderCommitIndex = isInstallingSnaphot || followerLogInformation.isLogEntrySlicingInProgress()
823                 || !followerLogInformation.isFollowerActive() ? -1 : context.getCommitIndex();
824
825         long followerNextIndex = followerLogInformation.getNextIndex();
826         AppendEntries appendEntries = new AppendEntries(currentTerm(), context.getId(),
827             getLogEntryIndex(followerNextIndex - 1),
828             getLogEntryTerm(followerNextIndex - 1), entries,
829             leaderCommitIndex, super.getReplicatedToAllIndex(), context.getPayloadVersion(),
830             followerLogInformation.getRaftVersion(), followerLogInformation.needsLeaderAddress(getId()));
831
832         if (!entries.isEmpty() || log.isTraceEnabled()) {
833             log.debug("{}: Sending AppendEntries to follower {}: {}", logName(), followerLogInformation.getId(),
834                     appendEntries);
835         }
836
837         followerLogInformation.setSentCommitIndex(leaderCommitIndex);
838         followerActor.tell(appendEntries, actor());
839     }
840
841     /**
842      * Initiates a snapshot capture to install on a follower.
843      *
844      * <p>
845      * Install Snapshot works as follows
846      *   1. Leader initiates the capture snapshot by calling createSnapshot on the RaftActor.
847      *   2. On receipt of the CaptureSnapshotReply message, the RaftActor persists the snapshot and makes a call to
848      *      the Leader's handleMessage with a SendInstallSnapshot message.
849      *   3. The Leader obtains and stores the Snapshot from the SendInstallSnapshot message and sends it in chunks to
850      *      the Follower via InstallSnapshot messages.
851      *   4. For each chunk, the Follower sends back an InstallSnapshotReply.
852      *   5. On receipt of the InstallSnapshotReply for the last chunk, the Leader marks the install complete for that
853      *      follower.
854      *   6. If another follower requires a snapshot and a snapshot has been collected (via SendInstallSnapshot)
855      *      then send the existing snapshot in chunks to the follower.
856      *
857      * @param followerId the id of the follower.
858      * @return true if capture was initiated, false otherwise.
859      */
860     public boolean initiateCaptureSnapshot(final String followerId) {
861         FollowerLogInformation followerLogInfo = followerToLog.get(followerId);
862         if (snapshotHolder.isPresent()) {
863             // If a snapshot is present in the memory, most likely another install is in progress no need to capture
864             // snapshot. This could happen if another follower needs an install when one is going on.
865             final ActorSelection followerActor = context.getPeerActorSelection(followerId);
866
867             // Note: sendSnapshotChunk will set the LeaderInstallSnapshotState.
868             sendSnapshotChunk(followerActor, followerLogInfo);
869             return true;
870         }
871
872         boolean captureInitiated = context.getSnapshotManager().captureToInstall(context.getReplicatedLog().last(),
873             this.getReplicatedToAllIndex(), followerId);
874         if (captureInitiated) {
875             followerLogInfo.setLeaderInstallSnapshotState(new LeaderInstallSnapshotState(
876                 context.getConfigParams().getSnapshotChunkSize(), logName()));
877         }
878
879         return captureInitiated;
880     }
881
882     private boolean canInstallSnapshot(final long nextIndex) {
883         // If the follower's nextIndex is -1 then we might as well send it a snapshot
884         // Otherwise send it a snapshot only if the nextIndex is not present in the log but is present
885         // in the snapshot
886         return nextIndex == -1 || !context.getReplicatedLog().isPresent(nextIndex)
887                 && context.getReplicatedLog().isInSnapshot(nextIndex);
888
889     }
890
891
892     private void sendInstallSnapshot() {
893         log.debug("{}: sendInstallSnapshot", logName());
894         for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
895             String followerId = e.getKey();
896             ActorSelection followerActor = context.getPeerActorSelection(followerId);
897             FollowerLogInformation followerLogInfo = e.getValue();
898
899             if (followerActor != null) {
900                 long nextIndex = followerLogInfo.getNextIndex();
901                 if (followerLogInfo.getInstallSnapshotState() != null
902                         || context.getPeerInfo(followerId).getVotingState() == VotingState.VOTING_NOT_INITIALIZED
903                         || canInstallSnapshot(nextIndex)) {
904                     sendSnapshotChunk(followerActor, followerLogInfo);
905                 }
906             }
907         }
908     }
909
910     /**
911      *  Sends a snapshot chunk to a given follower
912      *  InstallSnapshot should qualify as a heartbeat too.
913      */
914     private void sendSnapshotChunk(final ActorSelection followerActor, final FollowerLogInformation followerLogInfo) {
915         if (snapshotHolder.isPresent()) {
916             LeaderInstallSnapshotState installSnapshotState = followerLogInfo.getInstallSnapshotState();
917             if (installSnapshotState == null) {
918                 installSnapshotState = new LeaderInstallSnapshotState(context.getConfigParams().getSnapshotChunkSize(),
919                         logName());
920                 followerLogInfo.setLeaderInstallSnapshotState(installSnapshotState);
921             }
922
923             try {
924                 // Ensure the snapshot bytes are set - this is a no-op.
925                 installSnapshotState.setSnapshotBytes(snapshotHolder.get().getSnapshotBytes());
926
927                 if (!installSnapshotState.canSendNextChunk()) {
928                     return;
929                 }
930
931                 byte[] nextSnapshotChunk = installSnapshotState.getNextChunk();
932
933                 log.debug("{}: next snapshot chunk size for follower {}: {}", logName(), followerLogInfo.getId(),
934                         nextSnapshotChunk.length);
935
936                 int nextChunkIndex = installSnapshotState.incrementChunkIndex();
937                 Optional<ServerConfigurationPayload> serverConfig = Optional.absent();
938                 if (installSnapshotState.isLastChunk(nextChunkIndex)) {
939                     serverConfig = Optional.fromNullable(context.getPeerServerInfo(true));
940                 }
941
942                 sendSnapshotChunk(followerActor, followerLogInfo, nextSnapshotChunk, nextChunkIndex, serverConfig);
943
944             } catch (IOException e) {
945                 throw new RuntimeException(e);
946             }
947
948             log.debug("{}: InstallSnapshot sent to follower {}, Chunk: {}/{}", logName(), followerActor.path(),
949                 installSnapshotState.getChunkIndex(), installSnapshotState.getTotalChunks());
950         }
951     }
952
953     private void sendSnapshotChunk(final ActorSelection followerActor, final FollowerLogInformation followerLogInfo,
954                                    final byte[] snapshotChunk, final int chunkIndex,
955                                    final Optional<ServerConfigurationPayload> serverConfig) {
956         LeaderInstallSnapshotState installSnapshotState = followerLogInfo.getInstallSnapshotState();
957
958         installSnapshotState.startChunkTimer();
959         followerActor.tell(
960                 new InstallSnapshot(currentTerm(), context.getId(),
961                         snapshotHolder.get().getLastIncludedIndex(),
962                         snapshotHolder.get().getLastIncludedTerm(),
963                         snapshotChunk,
964                         chunkIndex,
965                         installSnapshotState.getTotalChunks(),
966                         Optional.of(installSnapshotState.getLastChunkHashCode()),
967                         serverConfig
968                 ).toSerializable(followerLogInfo.getRaftVersion()),
969                 actor()
970         );
971     }
972
973     private boolean resendSnapshotChunk(final ActorSelection followerActor,
974                                         final FollowerLogInformation followerLogInfo) {
975         if (!snapshotHolder.isPresent()) {
976             // Seems like we should never hit this case, but just in case we do, reset the snapshot progress so that it
977             // can restart from the next AppendEntries.
978             log.warn("{}: Attempting to resend snapshot with no snapshot holder present.", logName());
979             followerLogInfo.clearLeaderInstallSnapshotState();
980             return false;
981         }
982
983         LeaderInstallSnapshotState installSnapshotState = followerLogInfo.getInstallSnapshotState();
984         // we are resending, timer needs to be reset
985         installSnapshotState.resetChunkTimer();
986         installSnapshotState.markSendStatus(false);
987
988         sendSnapshotChunk(followerActor, followerLogInfo);
989
990         return true;
991     }
992
993     private void sendHeartBeat() {
994         if (!followerToLog.isEmpty()) {
995             log.trace("{}: Sending heartbeat", logName());
996             sendAppendEntries(context.getConfigParams().getHeartBeatInterval().toNanos(), true);
997
998             appendEntriesMessageSlicer.checkExpiredSlicedMessageState();
999         }
1000     }
1001
1002     private void stopHeartBeat() {
1003         if (heartbeatSchedule != null && !heartbeatSchedule.isCancelled()) {
1004             heartbeatSchedule.cancel();
1005         }
1006     }
1007
1008     private void scheduleHeartBeat(final FiniteDuration interval) {
1009         if (followerToLog.isEmpty()) {
1010             // Optimization - do not bother scheduling a heartbeat as there are
1011             // no followers
1012             return;
1013         }
1014
1015         stopHeartBeat();
1016
1017         // Schedule a heartbeat. When the scheduler triggers a SendHeartbeat
1018         // message is sent to itself.
1019         // Scheduling the heartbeat only once here because heartbeats do not
1020         // need to be sent if there are other messages being sent to the remote
1021         // actor.
1022         heartbeatSchedule = context.getActorSystem().scheduler().scheduleOnce(
1023             interval, context.getActor(), SendHeartBeat.INSTANCE,
1024             context.getActorSystem().dispatcher(), context.getActor());
1025     }
1026
1027     @Override
1028     public void close() {
1029         stopHeartBeat();
1030         appendEntriesMessageSlicer.close();
1031     }
1032
1033     @Override
1034     public final String getLeaderId() {
1035         return context.getId();
1036     }
1037
1038     @Override
1039     public final short getLeaderPayloadVersion() {
1040         return context.getPayloadVersion();
1041     }
1042
1043     protected boolean isLeaderIsolated() {
1044         int minPresent = getMinIsolatedLeaderPeerCount();
1045         for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
1046             final PeerInfo peerInfo = context.getPeerInfo(followerLogInformation.getId());
1047             if (peerInfo != null && peerInfo.isVoting() && followerLogInformation.isFollowerActive()) {
1048                 --minPresent;
1049                 if (minPresent == 0) {
1050                     return false;
1051                 }
1052             }
1053         }
1054         return minPresent != 0;
1055     }
1056
1057     // called from example-actor for printing the follower-states
1058     public String printFollowerStates() {
1059         final StringBuilder sb = new StringBuilder();
1060
1061         sb.append('[');
1062         for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
1063             sb.append('{');
1064             sb.append(followerLogInformation.getId());
1065             sb.append(" state:");
1066             sb.append(followerLogInformation.isFollowerActive());
1067             sb.append("},");
1068         }
1069         sb.append(']');
1070
1071         return sb.toString();
1072     }
1073
1074     @VisibleForTesting
1075     public FollowerLogInformation getFollower(final String followerId) {
1076         return followerToLog.get(followerId);
1077     }
1078
1079     @VisibleForTesting
1080     public int followerLogSize() {
1081         return followerToLog.size();
1082     }
1083
1084     static class SnapshotHolder {
1085         private final long lastIncludedTerm;
1086         private final long lastIncludedIndex;
1087         private final ByteSource snapshotBytes;
1088
1089         SnapshotHolder(final Snapshot snapshot, final ByteSource snapshotBytes) {
1090             this.lastIncludedTerm = snapshot.getLastAppliedTerm();
1091             this.lastIncludedIndex = snapshot.getLastAppliedIndex();
1092             this.snapshotBytes = snapshotBytes;
1093         }
1094
1095         long getLastIncludedTerm() {
1096             return lastIncludedTerm;
1097         }
1098
1099         long getLastIncludedIndex() {
1100             return lastIncludedIndex;
1101         }
1102
1103         ByteSource getSnapshotBytes() {
1104             return snapshotBytes;
1105         }
1106     }
1107 }