2 * Copyright (c) 2014 Cisco Systems, Inc. and others. All rights reserved.
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
8 package org.opendaylight.controller.cluster.raft.behaviors;
10 import static java.util.Objects.requireNonNull;
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;
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;
60 * The behavior of a RaftActor when it is in the Leader state.
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
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 > commitIndex, a majority
76 * of matchIndex[i] ≥ N, and log[N].term == currentTerm:
77 * set commitIndex = N (§5.3, §5.4).
80 public abstract class AbstractLeader extends AbstractRaftActorBehavior {
81 private final Map<String, FollowerLogInformation> followerToLog = new HashMap<>();
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.
89 private final Queue<ClientRequestTracker> trackers = new LinkedList<>();
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.
96 private final Map<Long, SharedFileBackedOutputStream> sharedSerializedAppendEntriesStreams = new HashMap<>();
97 private final MessageSlicer appendEntriesMessageSlicer;
99 private Cancellable heartbeatSchedule = null;
100 private Optional<SnapshotHolder> snapshotHolder = Optional.empty();
101 private int minReplicationCount;
103 protected AbstractLeader(final RaftActorContext context, final RaftState state,
104 final @Nullable AbstractLeader initializeFromLeader) {
105 super(context, state);
107 appendEntriesMessageSlicer = MessageSlicer.builder().logContext(logName())
108 .messageSliceSize(context.getConfigParams().getMaximumMessageSliceSize())
109 .expireStateAfterInactivity(context.getConfigParams().getElectionTimeOutInterval().toMillis() * 3,
110 TimeUnit.MILLISECONDS).build();
112 if (initializeFromLeader != null) {
113 followerToLog.putAll(initializeFromLeader.followerToLog);
114 snapshotHolder = initializeFromLeader.snapshotHolder;
115 trackers.addAll(initializeFromLeader.trackers);
117 for (PeerInfo peerInfo: context.getPeers()) {
118 FollowerLogInformation followerLogInformation = new FollowerLogInformation(peerInfo, context);
119 followerToLog.put(peerInfo.getId(), followerLogInformation);
123 log.debug("{}: Election: Leader has following peers: {}", logName(), getFollowerIds());
125 updateMinReplicaCount();
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);
133 // It is important to schedule this heartbeat here
134 scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
137 protected AbstractLeader(final RaftActorContext context, final RaftState state) {
138 this(context, state, null);
142 * Return an immutable collection of follower identifiers.
144 * @return Collection of follower IDs
146 public final Collection<String> getFollowerIds() {
147 return followerToLog.keySet();
150 public void addFollower(final String followerId) {
151 FollowerLogInformation followerLogInformation = new FollowerLogInformation(context.getPeerInfo(followerId),
153 followerToLog.put(followerId, followerLogInformation);
155 if (heartbeatSchedule == null) {
156 scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
160 public void removeFollower(final String followerId) {
161 followerToLog.remove(followerId);
164 public final void updateMinReplicaCount() {
166 for (PeerInfo peer: context.getPeers()) {
167 if (peer.isVoting()) {
172 minReplicationCount = getMajorityVoteCount(numVoting);
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
179 //0 peers = 1 votesRequired , minIsolatedLeaderPeerCount = 0
180 //2 peers = 2 votesRequired , minIsolatedLeaderPeerCount = 1
181 //4 peers = 3 votesRequired, minIsolatedLeaderPeerCount = 2
183 return minReplicationCount > 0 ? minReplicationCount - 1 : 0;
187 void setSnapshotHolder(final @Nullable SnapshotHolder snapshotHolder) {
188 this.snapshotHolder = Optional.ofNullable(snapshotHolder);
192 boolean hasSnapshot() {
193 return snapshotHolder.isPresent();
197 protected RaftActorBehavior handleAppendEntries(final ActorRef sender,
198 final AppendEntries appendEntries) {
200 log.debug("{}: handleAppendEntries: {}", logName(), appendEntries);
206 protected RaftActorBehavior handleAppendEntriesReply(final ActorRef sender,
207 final AppendEntriesReply appendEntriesReply) {
208 log.trace("{}: handleAppendEntriesReply: {}", logName(), appendEntriesReply);
210 // Update the FollowerLogInformation
211 String followerId = appendEntriesReply.getFollowerId();
212 FollowerLogInformation followerLogInformation = followerToLog.get(followerId);
214 if (followerLogInformation == null) {
215 log.error("{}: handleAppendEntriesReply - unknown follower {}", logName(), followerId);
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);
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());
234 followerLogInformation.markFollowerActive();
235 followerLogInformation.setPayloadVersion(appendEntriesReply.getPayloadVersion());
236 followerLogInformation.setRaftVersion(followerRaftVersion);
237 followerLogInformation.setNeedsLeaderAddress(appendEntriesReply.isNeedsLeaderAddress());
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
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());
261 followerLogInformation.setMatchIndex(-1);
262 followerLogInformation.setNextIndex(-1);
264 initiateCaptureSnapshot(followerId);
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.
279 followerLogInformation.setNextIndex(followerLastLogIndex - 1);
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());
287 updated = updateFollowerLogInformation(followerLogInformation, appendEntriesReply);
290 log.info("{}: handleAppendEntriesReply - received unsuccessful reply: {}, leader snapshotIndex: {}, "
291 + "snapshotTerm: {}, replicatedToAllIndex: {}", logName(), appendEntriesReply,
292 context.getReplicatedLog().getSnapshotIndex(), context.getReplicatedLog().getSnapshotTerm(),
293 getReplicatedToAllIndex());
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);
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.
310 updated = updateFollowerLogInformation(followerLogInformation, appendEntriesReply);
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());
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())) {
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());
328 if (log.isTraceEnabled()) {
329 log.trace("{}: handleAppendEntriesReply from {}: commitIndex: {}, lastAppliedIndex: {}, currentTerm: {}",
330 logName(), followerId, context.getCommitIndex(), context.getLastApplied(), currentTerm());
333 possiblyUpdateCommitIndex();
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);
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());
355 // Count our entry if it has been persisted.
356 int replicatedCount = replicatedLogEntry.isPersistencePending() ? 0 : 1;
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.
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()) {
370 } else if (log.isTraceEnabled()) {
371 log.trace("{}: Not counting follower {} - matchIndex: {}, {}", logName(), info.getId(),
372 info.getMatchIndex(), peerInfo);
376 if (log.isTraceEnabled()) {
377 log.trace("{}: replicatedCount {}, minReplicationCount: {}", logName(), replicatedCount,
378 minReplicationCount);
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);
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());
396 log.trace("{}: minReplicationCount not reached, actual {} - breaking", logName(), replicatedCount);
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());
406 applyLogToStateMachine(context.getCommitIndex());
409 if (!context.getSnapshotManager().isCapturing()) {
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;
419 if (updated && log.isDebugEnabled()) {
421 "{}: handleAppendEntriesReply - FollowerLogInformation for {} updated: matchIndex: {}, nextIndex: {}",
422 logName(), followerLogInformation.getId(), followerLogInformation.getMatchIndex(),
423 followerLogInformation.getNextIndex());
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());
438 super.performSnapshotWithoutCapture(minReplicatedToAllIndex);
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
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) {
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);
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);
476 return new ApplyState(null, null, entry);
480 protected RaftActorBehavior handleRequestVoteReply(final ActorRef sender, final RequestVoteReply requestVoteReply) {
484 protected void beforeSendHeartbeat() {
489 public RaftActorBehavior handleMessage(final ActorRef sender, final Object message) {
490 requireNonNull(sender, "sender should not be null");
492 if (appendEntriesMessageSlicer.handleMessage(message)) {
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)) {
502 log.info("{}: Term {} in \"{}\" message is greater than leader's term {} - switching to Follower",
503 logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm());
505 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
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);
520 return internalSwitchBehavior(RaftState.Follower);
523 if (message instanceof SendHeartBeat) {
524 beforeSendHeartbeat();
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();
538 return super.handleMessage(sender, message);
544 private void handleInstallSnapshotReply(final InstallSnapshotReply reply) {
545 log.debug("{}: handleInstallSnapshotReply: {}", logName(), reply);
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(),
556 final var installSnapshotState = followerLogInfo.getInstallSnapshotState();
557 if (installSnapshotState == null) {
558 log.error("{}: LeaderInstallSnapshotState not found for follower {} in InstallSnapshotReply", logName(),
563 installSnapshotState.resetChunkTimer();
564 followerLogInfo.markFollowerActive();
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);
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();
580 if (!reply.isSuccess()) {
581 log.warn("{}: Received failed InstallSnapshotReply - will retry: {}", logName(), reply);
582 installSnapshotState.markSendStatus(false);
583 sendNextSnapshotChunk(followerId, followerLogInfo);
587 if (!installSnapshotState.isLastChunk(replyChunkIndex)) {
588 log.debug("{}: Success InstallSnapshotReply from {}, sending next chunk", logName(), followerId);
589 installSnapshotState.markSendStatus(true);
590 sendNextSnapshotChunk(followerId, followerLogInfo);
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();
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());
604 if (!anyFollowersInstallingSnapshot()) {
605 // once there are no pending followers receiving snapshots we can remove snapshot from the memory
606 setSnapshotHolder(null);
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");
614 if (!context.getSnapshotManager().isCapturing()) {
615 // Since the follower is now caught up try to purge the log.
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);
627 private boolean anyFollowersInstallingSnapshot() {
628 for (var info : followerToLog.values()) {
629 if (info.getInstallSnapshotState() != null) {
636 private void replicate(final Replicate replicate) {
637 final long logIndex = replicate.logIndex();
639 log.debug("{}: Replicate message: identifier: {}, logIndex: {}, isSendImmediate: {}", logName(),
640 replicate.identifier(), logIndex, replicate.sendImmediate());
642 // Create a tracker entry we will use this later to notify the
644 final var clientActor = replicate.clientActor();
645 if (clientActor != null) {
646 trackers.add(new ClientRequestTracker(logIndex, clientActor, replicate.identifier()));
649 boolean applyModificationToState = !context.anyVotingPeers()
650 || context.getRaftPolicy().applyModificationToStateBeforeConsensus();
652 if (applyModificationToState) {
653 context.setCommitIndex(logIndex);
654 applyLogToStateMachine(logIndex);
657 if (replicate.sendImmediate() && !followerToLog.isEmpty()) {
658 sendAppendEntries(0, false);
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);
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.
679 private void sendUpdatesToFollower(final String followerId, final FollowerLogInformation followerLogInformation,
680 final boolean sendHeartbeat, final boolean isHeartbeat) {
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();
689 LeaderInstallSnapshotState installSnapshotState = followerLogInformation.getInstallSnapshotState();
690 if (installSnapshotState != null) {
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);
698 if (installSnapshotState.isChunkTimedOut(snapshotReplyTimeout)) {
699 sendAppendEntries = !resendSnapshotChunk(followerActor, followerLogInformation);
700 } else if (installSnapshotState.canSendNextChunk()) {
701 sendSnapshotChunk(followerActor, followerLogInformation);
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;
707 } else if (followerLogInformation.isLogEntrySlicingInProgress()) {
708 sendAppendEntries = sendHeartbeat;
710 long leaderLastIndex = context.getReplicatedLog().lastIndex();
711 long leaderSnapShotIndex = context.getReplicatedLog().getSnapshotIndex();
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);
719 if (isFollowerActive && context.getReplicatedLog().isPresent(followerNextIndex)) {
721 log.debug("{}: sendAppendEntries: {} is present for follower {}", logName(),
722 followerNextIndex, followerId);
724 if (followerLogInformation.okToReplicate(context.getCommitIndex())) {
725 entries = getEntriesToSend(followerLogInformation, followerActor);
726 sendAppendEntries = true;
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
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());
742 initiateCaptureSnapshot(followerId);
744 // It doesn't seem like we should ever reach here - most likely indicates sonething is
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());
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;
760 if (sendAppendEntries) {
761 sendAppendEntriesToFollower(followerActor, entries, followerLogInformation);
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
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);
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.
782 final var firstEntry = entries.get(0);
783 log.debug("{}: Log entry size {} exceeds max payload size {}", logName(), firstEntry.getData().size(),
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();
793 final AppendEntries appendEntries = new AppendEntries(currentTerm(), context.getId(),
794 getLogEntryIndex(followerNextIndex - 1), getLogEntryTerm(followerNextIndex - 1), entries,
795 context.getCommitIndex(), getReplicatedToAllIndex(), context.getPayloadVersion());
797 log.debug("{}: Serializing {} for slicing for follower {}", logName(), appendEntries,
798 followerLogInfo.getId());
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();
808 sharedSerializedAppendEntriesStreams.put(logIndex, fileBackedStream);
810 fileBackedStream.setOnCleanupCallback(index -> {
811 log.debug("{}: On SharedFileBackedOutputStream cleanup for index {}", logName(), index);
812 sharedSerializedAppendEntriesStreams.remove(index);
815 log.debug("{}: Reusing SharedFileBackedOutputStream for follower {}", logName(), followerLogInfo.getId());
816 fileBackedStream.incrementUsageCount();
819 log.debug("{}: Slicing stream for index {}, follower {}", logName(), logIndex, followerLogInfo.getId());
821 // Record that slicing is in progress for the follower.
822 followerLogInfo.setSlicedLogEntryIndex(logIndex);
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);
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();
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()));
858 if (!entries.isEmpty() || log.isTraceEnabled()) {
859 log.debug("{}: Sending AppendEntries to follower {}: {}", logName(), followerLogInformation.getId(),
863 followerLogInformation.setSentCommitIndex(leaderCommitIndex);
864 followerActor.tell(appendEntries, actor());
868 * Initiates a snapshot capture to install on a follower. Install Snapshot works as follows:
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
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>
882 * @param followerId the id of the follower.
883 * @return true if capture was initiated, false otherwise.
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);
892 // Note: sendSnapshotChunk will set the LeaderInstallSnapshotState.
893 sendSnapshotChunk(followerActor, followerLogInfo);
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()));
904 return captureInitiated;
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
911 return nextIndex == -1 || !context.getReplicatedLog().isPresent(nextIndex)
912 && context.getReplicatedLog().isInSnapshot(nextIndex);
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();
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);
936 * Sends a snapshot chunk to a given follower
937 * InstallSnapshot should qualify as a heartbeat too.
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);
949 // Ensure the snapshot bytes are set - this is a no-op.
950 installSnapshotState.setSnapshotBytes(snapshotHolder.orElseThrow().getSnapshotBytes());
952 if (!installSnapshotState.canSendNextChunk()) {
956 byte[] nextSnapshotChunk = installSnapshotState.getNextChunk();
958 log.debug("{}: next snapshot chunk size for follower {}: {}", logName(), followerLogInfo.getId(),
959 nextSnapshotChunk.length);
961 int nextChunkIndex = installSnapshotState.incrementChunkIndex();
962 Optional<ServerConfigurationPayload> serverConfig = Optional.empty();
963 if (installSnapshotState.isLastChunk(nextChunkIndex)) {
964 serverConfig = Optional.ofNullable(context.getPeerServerInfo(true));
967 sendSnapshotChunk(followerActor, followerLogInfo, nextSnapshotChunk, nextChunkIndex, serverConfig);
969 log.debug("{}: InstallSnapshot sent to follower {}, Chunk: {}/{}", logName(), followerActor.path(),
970 installSnapshotState.getChunkIndex(), installSnapshotState.getTotalChunks());
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();
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();
986 installSnapshotState.startChunkTimer();
988 new InstallSnapshot(currentTerm(), context.getId(),
989 snapshotHolder.orElseThrow().getLastIncludedIndex(),
990 snapshotHolder.orElseThrow().getLastIncludedTerm(),
993 installSnapshotState.getTotalChunks(),
994 OptionalInt.of(installSnapshotState.getLastChunkHashCode()),
996 followerLogInfo.getRaftVersion()),
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();
1011 LeaderInstallSnapshotState installSnapshotState = followerLogInfo.getInstallSnapshotState();
1012 // we are resending, timer needs to be reset
1013 installSnapshotState.resetChunkTimer();
1014 installSnapshotState.markSendStatus(false);
1016 sendSnapshotChunk(followerActor, followerLogInfo);
1021 private void sendHeartBeat() {
1022 if (!followerToLog.isEmpty()) {
1023 log.trace("{}: Sending heartbeat", logName());
1024 sendAppendEntries(context.getConfigParams().getHeartBeatInterval().toNanos(), true);
1026 appendEntriesMessageSlicer.checkExpiredSlicedMessageState();
1030 private void stopHeartBeat() {
1031 if (heartbeatSchedule != null && !heartbeatSchedule.isCancelled()) {
1032 heartbeatSchedule.cancel();
1036 private void scheduleHeartBeat(final FiniteDuration interval) {
1037 if (followerToLog.isEmpty()) {
1038 // Optimization - do not bother scheduling a heartbeat as there are
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
1050 heartbeatSchedule = context.getActorSystem().scheduler().scheduleOnce(
1051 interval, context.getActor(), SendHeartBeat.INSTANCE,
1052 context.getActorSystem().dispatcher(), context.getActor());
1056 public void close() {
1058 appendEntriesMessageSlicer.close();
1062 public final String getLeaderId() {
1063 return context.getId();
1067 public final short getLeaderPayloadVersion() {
1068 return context.getPayloadVersion();
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()) {
1077 if (minPresent == 0) {
1082 return minPresent != 0;
1085 // called from example-actor for printing the follower-states
1086 public String printFollowerStates() {
1087 final StringBuilder sb = new StringBuilder();
1090 for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
1092 sb.append(followerLogInformation.getId());
1093 sb.append(" state:");
1094 sb.append(followerLogInformation.isFollowerActive());
1099 return sb.toString();
1103 public FollowerLogInformation getFollower(final String followerId) {
1104 return followerToLog.get(followerId);
1108 public int followerLogSize() {
1109 return followerToLog.size();
1112 static class SnapshotHolder {
1113 private final long lastIncludedTerm;
1114 private final long lastIncludedIndex;
1115 private final ByteSource snapshotBytes;
1117 SnapshotHolder(final Snapshot snapshot, final ByteSource snapshotBytes) {
1118 lastIncludedTerm = snapshot.getLastAppliedTerm();
1119 lastIncludedIndex = snapshot.getLastAppliedIndex();
1120 this.snapshotBytes = snapshotBytes;
1123 long getLastIncludedTerm() {
1124 return lastIncludedTerm;
1127 long getLastIncludedIndex() {
1128 return lastIncludedIndex;
1131 ByteSource getSnapshotBytes() {
1132 return snapshotBytes;