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