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