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