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