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