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
9 package org.opendaylight.controller.cluster.raft.behaviors;
11 import akka.actor.ActorRef;
12 import akka.actor.ActorSelection;
13 import akka.actor.Cancellable;
14 import com.google.common.annotations.VisibleForTesting;
15 import com.google.common.base.Optional;
16 import com.google.common.base.Preconditions;
17 import com.google.common.base.Throwables;
18 import com.google.common.io.ByteSource;
19 import java.io.IOException;
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;
27 import java.util.Map.Entry;
28 import java.util.Queue;
29 import javax.annotation.Nullable;
30 import org.opendaylight.controller.cluster.raft.ClientRequestTracker;
31 import org.opendaylight.controller.cluster.raft.ClientRequestTrackerImpl;
32 import org.opendaylight.controller.cluster.raft.FollowerLogInformation;
33 import org.opendaylight.controller.cluster.raft.FollowerLogInformationImpl;
34 import org.opendaylight.controller.cluster.raft.PeerInfo;
35 import org.opendaylight.controller.cluster.raft.RaftActorContext;
36 import org.opendaylight.controller.cluster.raft.RaftState;
37 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
38 import org.opendaylight.controller.cluster.raft.VotingState;
39 import org.opendaylight.controller.cluster.raft.base.messages.CheckConsensusReached;
40 import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
41 import org.opendaylight.controller.cluster.raft.base.messages.SendHeartBeat;
42 import org.opendaylight.controller.cluster.raft.base.messages.SendInstallSnapshot;
43 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
44 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
45 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot;
46 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshotReply;
47 import org.opendaylight.controller.cluster.raft.messages.RaftRPC;
48 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
49 import org.opendaylight.controller.cluster.raft.messages.UnInitializedFollowerSnapshotReply;
50 import org.opendaylight.controller.cluster.raft.persisted.ServerConfigurationPayload;
51 import org.opendaylight.controller.cluster.raft.persisted.Snapshot;
52 import scala.concurrent.duration.FiniteDuration;
55 * The behavior of a RaftActor when it is in the Leader state
60 * <li> Upon election: send initial empty AppendEntries RPCs
61 * (heartbeat) to each server; repeat during idle periods to
62 * prevent election timeouts (§5.2)
63 * <li> If command received from client: append entry to local log,
64 * respond after entry applied to state machine (§5.3)
65 * <li> If last log index ≥ nextIndex for a follower: send
66 * AppendEntries RPC with log entries starting at nextIndex
67 * <li> If successful: update nextIndex and matchIndex for
69 * <li> If AppendEntries fails because of log inconsistency:
70 * decrement nextIndex and retry (§5.3)
71 * <li> If there exists an N such that N > commitIndex, a majority
72 * of matchIndex[i] ≥ N, and log[N].term == currentTerm:
73 * set commitIndex = N (§5.3, §5.4).
76 public abstract class AbstractLeader extends AbstractRaftActorBehavior {
77 private final Map<String, FollowerLogInformation> followerToLog = new HashMap<>();
80 * Lookup table for request contexts based on journal index. We could use a {@link Map} here, but we really
81 * expect the entries to be modified in sequence, hence we open-code the lookup.
82 * TODO: Evaluate the use of ArrayDeque(), as that has lower memory overhead. Non-head removals are more costly,
83 * but we already expect those to be far from frequent.
85 private final Queue<ClientRequestTracker> trackers = new LinkedList<>();
87 private Cancellable heartbeatSchedule = null;
88 private Optional<SnapshotHolder> snapshotHolder = Optional.absent();
89 private int minReplicationCount;
91 protected AbstractLeader(RaftActorContext context, RaftState state,
92 @Nullable AbstractLeader initializeFromLeader) {
93 super(context, state);
95 if (initializeFromLeader != null) {
96 followerToLog.putAll(initializeFromLeader.followerToLog);
97 snapshotHolder = initializeFromLeader.snapshotHolder;
98 trackers.addAll(initializeFromLeader.trackers);
100 for (PeerInfo peerInfo: context.getPeers()) {
101 FollowerLogInformation followerLogInformation = new FollowerLogInformationImpl(peerInfo, -1, context);
102 followerToLog.put(peerInfo.getId(), followerLogInformation);
106 log.debug("{}: Election: Leader has following peers: {}", logName(), getFollowerIds());
108 updateMinReplicaCount();
110 // Immediately schedule a heartbeat
111 // Upon election: send initial empty AppendEntries RPCs
112 // (heartbeat) to each server; repeat during idle periods to
113 // prevent election timeouts (§5.2)
114 sendAppendEntries(0, false);
116 // It is important to schedule this heartbeat here
117 scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
120 protected AbstractLeader(RaftActorContext context, RaftState state) {
121 this(context, state, null);
125 * Return an immutable collection of follower identifiers.
127 * @return Collection of follower IDs
129 public final Collection<String> getFollowerIds() {
130 return followerToLog.keySet();
133 public void addFollower(String followerId) {
134 FollowerLogInformation followerLogInformation = new FollowerLogInformationImpl(
135 context.getPeerInfo(followerId), -1, context);
136 followerToLog.put(followerId, followerLogInformation);
138 if (heartbeatSchedule == null) {
139 scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
143 public void removeFollower(String followerId) {
144 followerToLog.remove(followerId);
147 public void updateMinReplicaCount() {
149 for (PeerInfo peer: context.getPeers()) {
150 if (peer.isVoting()) {
155 minReplicationCount = getMajorityVoteCount(numVoting);
158 protected int getMinIsolatedLeaderPeerCount() {
159 //the isolated Leader peer count will be 1 less than the majority vote count.
160 //this is because the vote count has the self vote counted in it
162 //0 peers = 1 votesRequired , minIsolatedLeaderPeerCount = 0
163 //2 peers = 2 votesRequired , minIsolatedLeaderPeerCount = 1
164 //4 peers = 3 votesRequired, minIsolatedLeaderPeerCount = 2
166 return minReplicationCount > 0 ? minReplicationCount - 1 : 0;
170 void setSnapshot(@Nullable SnapshotHolder snapshotHolder) {
171 this.snapshotHolder = Optional.fromNullable(snapshotHolder);
175 boolean hasSnapshot() {
176 return snapshotHolder.isPresent();
180 protected RaftActorBehavior handleAppendEntries(ActorRef sender,
181 AppendEntries appendEntries) {
183 log.debug("{}: handleAppendEntries: {}", logName(), appendEntries);
189 protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender, AppendEntriesReply appendEntriesReply) {
190 log.trace("{}: handleAppendEntriesReply: {}", logName(), appendEntriesReply);
192 // Update the FollowerLogInformation
193 String followerId = appendEntriesReply.getFollowerId();
194 FollowerLogInformation followerLogInformation = followerToLog.get(followerId);
196 if (followerLogInformation == null) {
197 log.error("{}: handleAppendEntriesReply - unknown follower {}", logName(), followerId);
201 if (followerLogInformation.timeSinceLastActivity()
202 > context.getConfigParams().getElectionTimeOutInterval().toMillis()) {
203 log.warn("{} : handleAppendEntriesReply delayed beyond election timeout, "
204 + "appendEntriesReply : {}, timeSinceLastActivity : {}, lastApplied : {}, commitIndex : {}",
205 logName(), appendEntriesReply, followerLogInformation.timeSinceLastActivity(),
206 context.getLastApplied(), context.getCommitIndex());
209 followerLogInformation.markFollowerActive();
210 followerLogInformation.setPayloadVersion(appendEntriesReply.getPayloadVersion());
211 followerLogInformation.setRaftVersion(appendEntriesReply.getRaftVersion());
213 long followerLastLogIndex = appendEntriesReply.getLogLastIndex();
214 long followersLastLogTermInLeadersLog = getLogEntryTerm(followerLastLogIndex);
215 boolean updated = false;
216 if (appendEntriesReply.getLogLastIndex() > context.getReplicatedLog().lastIndex()) {
217 // The follower's log is actually ahead of the leader's log. Normally this doesn't happen
218 // in raft as a node cannot become leader if it's log is behind another's. However, the
219 // non-voting semantics deviate a bit from raft. Only voting members participate in
220 // elections and can become leader so it's possible for a non-voting follower to be ahead
221 // of the leader. This can happen if persistence is disabled and all voting members are
222 // restarted. In this case, the voting leader will start out with an empty log however
223 // the non-voting followers still retain the previous data in memory. On the first
224 // AppendEntries, the non-voting follower returns a successful reply b/c the prevLogIndex
225 // sent by the leader is -1 and thus the integrity checks pass. However the follower's returned
226 // lastLogIndex may be higher in which case we want to reset the follower by installing a
227 // snapshot. It's also possible that the follower's last log index is behind the leader's.
228 // However in this case the log terms won't match and the logs will conflict - this is handled
230 log.info("{}: handleAppendEntriesReply: follower {} lastIndex {} is ahead of our lastIndex {} "
231 + "(snapshotIndex {}) - forcing install snaphot", logName(), followerLogInformation.getId(),
232 appendEntriesReply.getLogLastIndex(), context.getReplicatedLog().lastIndex(),
233 context.getReplicatedLog().getSnapshotIndex());
235 followerLogInformation.setMatchIndex(-1);
236 followerLogInformation.setNextIndex(-1);
238 initiateCaptureSnapshot(followerId);
241 } else if (appendEntriesReply.isSuccess()) {
242 if (followerLastLogIndex >= 0 && followersLastLogTermInLeadersLog >= 0
243 && followersLastLogTermInLeadersLog != appendEntriesReply.getLogLastTerm()) {
244 // The follower's last entry is present in the leader's journal but the terms don't match so the
245 // follower has a conflicting entry. Since the follower didn't report that it's out of sync, this means
246 // either the previous leader entry sent didn't conflict or the previous leader entry is in the snapshot
247 // and no longer in the journal. Either way, we set the follower's next index to 1 less than the last
248 // index reported by the follower. For the former case, the leader will send all entries starting with
249 // the previous follower's index and the follower will remove and replace the conflicting entries as
250 // needed. For the latter, the leader will initiate an install snapshot.
252 followerLogInformation.setNextIndex(followerLastLogIndex - 1);
255 log.info("{}: handleAppendEntriesReply: follower {} last log term {} for index {} conflicts with the "
256 + "leader's {} - set the follower's next index to {}", logName(),
257 followerId, appendEntriesReply.getLogLastTerm(), appendEntriesReply.getLogLastIndex(),
258 followersLastLogTermInLeadersLog, followerLogInformation.getNextIndex());
260 updated = updateFollowerLogInformation(followerLogInformation, appendEntriesReply);
263 log.info("{}: handleAppendEntriesReply - received unsuccessful reply: {}, leader snapshotIndex: {}",
264 logName(), appendEntriesReply, context.getReplicatedLog().getSnapshotIndex());
266 if (appendEntriesReply.isForceInstallSnapshot()) {
267 // Reset the followers match and next index. This is to signal that this follower has nothing
268 // in common with this Leader and so would require a snapshot to be installed
269 followerLogInformation.setMatchIndex(-1);
270 followerLogInformation.setNextIndex(-1);
272 // Force initiate a snapshot capture
273 initiateCaptureSnapshot(followerId);
274 } else if (followerLastLogIndex < 0 || followersLastLogTermInLeadersLog >= 0
275 && followersLastLogTermInLeadersLog == appendEntriesReply.getLogLastTerm()) {
276 // The follower's log is empty or the last entry is present in the leader's journal
277 // and the terms match so the follower is just behind the leader's journal from
278 // the last snapshot, if any. We'll catch up the follower quickly by starting at the
279 // follower's last log index.
281 updated = updateFollowerLogInformation(followerLogInformation, appendEntriesReply);
283 log.info("{}: follower {} appears to be behind the leader from the last snapshot - "
284 + "updated: matchIndex: {}, nextIndex: {}", logName(), followerId,
285 appendEntriesReply.getLogLastTerm(), followerLogInformation.getMatchIndex(),
286 followerLogInformation.getNextIndex());
288 // The follower's log conflicts with leader's log so decrement follower's next index by 1
289 // in an attempt to find where the logs match.
291 followerLogInformation.decrNextIndex();
294 log.info("{}: follower {} last log term {} conflicts with the leader's {} - dec next index to {}",
295 logName(), followerId, appendEntriesReply.getLogLastTerm(), followersLastLogTermInLeadersLog,
296 followerLogInformation.getNextIndex());
300 if (log.isTraceEnabled()) {
301 log.trace("{}: handleAppendEntriesReply from {}: commitIndex: {}, lastAppliedIndex: {}, currentTerm: {}",
302 logName(), followerId, context.getCommitIndex(), context.getLastApplied(), currentTerm());
305 possiblyUpdateCommitIndex();
307 //Send the next log entry immediately, if possible, no need to wait for heartbeat to trigger that event
308 sendUpdatesToFollower(followerId, followerLogInformation, false, !updated);
313 private void possiblyUpdateCommitIndex() {
314 // Figure out if we can update the the commitIndex as follows:
315 // If there exists an index N such that N > commitIndex, a majority of matchIndex[i] ≥ N,
316 // and log[N].term == currentTerm:
317 // set commitIndex = N (§5.3, §5.4).
318 for (long index = context.getCommitIndex() + 1; ; index++) {
319 ReplicatedLogEntry replicatedLogEntry = context.getReplicatedLog().get(index);
320 if (replicatedLogEntry == null) {
321 log.trace("{}: ReplicatedLogEntry not found for index {} - snapshotIndex: {}, journal size: {}",
322 logName(), index, context.getReplicatedLog().getSnapshotIndex(),
323 context.getReplicatedLog().size());
327 // Count our entry if it has been persisted.
328 int replicatedCount = replicatedLogEntry.isPersistencePending() ? 0 : 1;
330 if (replicatedCount == 0) {
331 // We don't commit and apply a log entry until we've gotten the ack from our local persistence,
332 // even though there *shouldn't* be any issue with updating the commit index if we get a consensus
333 // amongst the followers w/o the local persistence ack.
337 log.trace("{}: checking Nth index {}", logName(), index);
338 for (FollowerLogInformation info : followerToLog.values()) {
339 final PeerInfo peerInfo = context.getPeerInfo(info.getId());
340 if (info.getMatchIndex() >= index && peerInfo != null && peerInfo.isVoting()) {
342 } else if (log.isTraceEnabled()) {
343 log.trace("{}: Not counting follower {} - matchIndex: {}, {}", logName(), info.getId(),
344 info.getMatchIndex(), peerInfo);
348 if (log.isTraceEnabled()) {
349 log.trace("{}: replicatedCount {}, minReplicationCount: {}", logName(), replicatedCount,
350 minReplicationCount);
353 if (replicatedCount >= minReplicationCount) {
354 // Don't update the commit index if the log entry is from a previous term, as per §5.4.1:
355 // "Raft never commits log entries from previous terms by counting replicas".
356 // However we keep looping so we can make progress when new entries in the current term
357 // reach consensus, as per §5.4.1: "once an entry from the current term is committed by
358 // counting replicas, then all prior entries are committed indirectly".
359 if (replicatedLogEntry.getTerm() == currentTerm()) {
360 log.trace("{}: Setting commit index to {}", logName(), index);
361 context.setCommitIndex(index);
363 log.debug("{}: Not updating commit index to {} - retrieved log entry with index {}, "
364 + "term {} does not match the current term {}", logName(), index,
365 replicatedLogEntry.getIndex(), replicatedLogEntry.getTerm(), currentTerm());
368 log.trace("{}: minReplicationCount not reached, actual {} - breaking", logName(), replicatedCount);
373 // Apply the change to the state machine
374 if (context.getCommitIndex() > context.getLastApplied()) {
375 log.debug("{}: Applying to log - commitIndex: {}, lastAppliedIndex: {}", logName(),
376 context.getCommitIndex(), context.getLastApplied());
378 applyLogToStateMachine(context.getCommitIndex());
381 if (!context.getSnapshotManager().isCapturing()) {
386 private boolean updateFollowerLogInformation(FollowerLogInformation followerLogInformation,
387 AppendEntriesReply appendEntriesReply) {
388 boolean updated = followerLogInformation.setMatchIndex(appendEntriesReply.getLogLastIndex());
389 updated = followerLogInformation.setNextIndex(appendEntriesReply.getLogLastIndex() + 1) || updated;
391 if (updated && log.isDebugEnabled()) {
393 "{}: handleAppendEntriesReply - FollowerLogInformation for {} updated: matchIndex: {}, nextIndex: {}",
394 logName(), followerLogInformation.getId(), followerLogInformation.getMatchIndex(),
395 followerLogInformation.getNextIndex());
400 private void purgeInMemoryLog() {
401 //find the lowest index across followers which has been replicated to all.
402 // lastApplied if there are no followers, so that we keep clearing the log for single-node
403 // we would delete the in-mem log from that index on, in-order to minimize mem usage
404 // we would also share this info thru AE with the followers so that they can delete their log entries as well.
405 long minReplicatedToAllIndex = followerToLog.isEmpty() ? context.getLastApplied() : Long.MAX_VALUE;
406 for (FollowerLogInformation info : followerToLog.values()) {
407 minReplicatedToAllIndex = Math.min(minReplicatedToAllIndex, info.getMatchIndex());
410 super.performSnapshotWithoutCapture(minReplicatedToAllIndex);
414 protected ClientRequestTracker removeClientRequestTracker(long logIndex) {
415 final Iterator<ClientRequestTracker> it = trackers.iterator();
416 while (it.hasNext()) {
417 final ClientRequestTracker t = it.next();
418 if (t.getIndex() == logIndex) {
428 protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
429 RequestVoteReply requestVoteReply) {
433 protected void beforeSendHeartbeat(){}
436 public RaftActorBehavior handleMessage(ActorRef sender, Object message) {
437 Preconditions.checkNotNull(sender, "sender should not be null");
439 if (message instanceof RaftRPC) {
440 RaftRPC rpc = (RaftRPC) message;
441 // If RPC request or response contains term T > currentTerm:
442 // set currentTerm = T, convert to follower (§5.1)
443 // This applies to all RPC messages and responses
444 if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
445 log.info("{}: Term {} in \"{}\" message is greater than leader's term {} - switching to Follower",
446 logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm());
448 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
450 return internalSwitchBehavior(RaftState.Follower);
454 if (message instanceof SendHeartBeat) {
455 beforeSendHeartbeat();
457 scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
458 } else if (message instanceof SendInstallSnapshot) {
459 SendInstallSnapshot sendInstallSnapshot = (SendInstallSnapshot) message;
460 setSnapshot(new SnapshotHolder(sendInstallSnapshot.getSnapshot(), sendInstallSnapshot.getSnapshotBytes()));
461 sendInstallSnapshot();
462 } else if (message instanceof Replicate) {
463 replicate((Replicate) message);
464 } else if (message instanceof InstallSnapshotReply) {
465 handleInstallSnapshotReply((InstallSnapshotReply) message);
466 } else if (message instanceof CheckConsensusReached) {
467 possiblyUpdateCommitIndex();
469 return super.handleMessage(sender, message);
475 private void handleInstallSnapshotReply(InstallSnapshotReply reply) {
476 log.debug("{}: handleInstallSnapshotReply: {}", logName(), reply);
478 String followerId = reply.getFollowerId();
479 FollowerLogInformation followerLogInformation = followerToLog.get(followerId);
480 if (followerLogInformation == null) {
481 // This can happen during AddServer if it times out.
482 log.error("{}: FollowerLogInformation not found for follower {} in InstallSnapshotReply",
483 logName(), followerId);
487 LeaderInstallSnapshotState installSnapshotState = followerLogInformation.getInstallSnapshotState();
488 if (installSnapshotState == null) {
489 log.error("{}: LeaderInstallSnapshotState not found for follower {} in InstallSnapshotReply",
490 logName(), followerId);
494 followerLogInformation.markFollowerActive();
496 if (installSnapshotState.getChunkIndex() == reply.getChunkIndex()) {
497 boolean wasLastChunk = false;
498 if (reply.isSuccess()) {
499 if (installSnapshotState.isLastChunk(reply.getChunkIndex())) {
500 //this was the last chunk reply
502 long followerMatchIndex = snapshotHolder.get().getLastIncludedIndex();
503 followerLogInformation.setMatchIndex(followerMatchIndex);
504 followerLogInformation.setNextIndex(followerMatchIndex + 1);
505 followerLogInformation.clearLeaderInstallSnapshotState();
507 log.info("{}: Snapshot successfully installed on follower {} (last chunk {}) - "
508 + "matchIndex set to {}, nextIndex set to {}", logName(), followerId, reply.getChunkIndex(),
509 followerLogInformation.getMatchIndex(), followerLogInformation.getNextIndex());
511 if (!anyFollowersInstallingSnapshot()) {
512 // once there are no pending followers receiving snapshots
513 // we can remove snapshot from the memory
518 if (context.getPeerInfo(followerId).getVotingState() == VotingState.VOTING_NOT_INITIALIZED) {
519 UnInitializedFollowerSnapshotReply unInitFollowerSnapshotSuccess =
520 new UnInitializedFollowerSnapshotReply(followerId);
521 context.getActor().tell(unInitFollowerSnapshotSuccess, context.getActor());
522 log.debug("Sent message UnInitializedFollowerSnapshotReply to self");
525 installSnapshotState.markSendStatus(true);
528 log.warn("{}: Received failed InstallSnapshotReply - will retry: {}", logName(), reply);
530 installSnapshotState.markSendStatus(false);
533 if (wasLastChunk && !context.getSnapshotManager().isCapturing()) {
534 // Since the follower is now caught up try to purge the log.
536 } else if (!wasLastChunk && installSnapshotState.canSendNextChunk()) {
537 ActorSelection followerActor = context.getPeerActorSelection(followerId);
538 if (followerActor != null) {
539 sendSnapshotChunk(followerActor, followerLogInformation);
544 log.error("{}: Chunk index {} in InstallSnapshotReply from follower {} does not match expected index {}",
545 logName(), reply.getChunkIndex(), followerId,
546 installSnapshotState.getChunkIndex());
548 if (reply.getChunkIndex() == LeaderInstallSnapshotState.INVALID_CHUNK_INDEX) {
549 // Since the Follower did not find this index to be valid we should reset the follower snapshot
550 // so that Installing the snapshot can resume from the beginning
551 installSnapshotState.reset();
556 private boolean anyFollowersInstallingSnapshot() {
557 for (FollowerLogInformation info: followerToLog.values()) {
558 if (info.getInstallSnapshotState() != null) {
567 private void replicate(Replicate replicate) {
568 long logIndex = replicate.getReplicatedLogEntry().getIndex();
570 log.debug("{}: Replicate message: identifier: {}, logIndex: {}, payload: {}, isSendImmediate: {}", logName(),
571 replicate.getIdentifier(), logIndex, replicate.getReplicatedLogEntry().getData().getClass(),
572 replicate.isSendImmediate());
574 // Create a tracker entry we will use this later to notify the
576 if (replicate.getClientActor() != null) {
577 trackers.add(new ClientRequestTrackerImpl(replicate.getClientActor(), replicate.getIdentifier(),
581 boolean applyModificationToState = !context.anyVotingPeers()
582 || context.getRaftPolicy().applyModificationToStateBeforeConsensus();
584 if (applyModificationToState) {
585 context.setCommitIndex(logIndex);
586 applyLogToStateMachine(logIndex);
589 if (replicate.isSendImmediate() && !followerToLog.isEmpty()) {
590 sendAppendEntries(0, false);
594 protected void sendAppendEntries(long timeSinceLastActivityInterval, boolean isHeartbeat) {
595 // Send an AppendEntries to all followers
596 for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
597 final String followerId = e.getKey();
598 final FollowerLogInformation followerLogInformation = e.getValue();
599 // This checks helps not to send a repeat message to the follower
600 if (!followerLogInformation.isFollowerActive()
601 || followerLogInformation.timeSinceLastActivity() >= timeSinceLastActivityInterval) {
602 sendUpdatesToFollower(followerId, followerLogInformation, true, isHeartbeat);
608 * This method checks if any update needs to be sent to the given follower. This includes append log entries,
609 * sending next snapshot chunk, and initiating a snapshot.
611 private void sendUpdatesToFollower(String followerId, FollowerLogInformation followerLogInformation,
612 boolean sendHeartbeat, boolean isHeartbeat) {
614 ActorSelection followerActor = context.getPeerActorSelection(followerId);
615 if (followerActor != null) {
616 long followerNextIndex = followerLogInformation.getNextIndex();
617 boolean isFollowerActive = followerLogInformation.isFollowerActive();
618 boolean sendAppendEntries = false;
619 List<ReplicatedLogEntry> entries = Collections.emptyList();
621 LeaderInstallSnapshotState installSnapshotState = followerLogInformation.getInstallSnapshotState();
622 if (installSnapshotState != null) {
623 // if install snapshot is in process , then sent next chunk if possible
624 if (isFollowerActive && installSnapshotState.canSendNextChunk()) {
625 sendSnapshotChunk(followerActor, followerLogInformation);
626 } else if (sendHeartbeat) {
627 // we send a heartbeat even if we have not received a reply for the last chunk
628 sendAppendEntries = true;
631 long leaderLastIndex = context.getReplicatedLog().lastIndex();
632 long leaderSnapShotIndex = context.getReplicatedLog().getSnapshotIndex();
634 if (!isHeartbeat && log.isDebugEnabled() || log.isTraceEnabled()) {
635 log.debug("{}: Checking sendAppendEntries for follower {}: active: {}, followerNextIndex: {}, "
636 + "leaderLastIndex: {}, leaderSnapShotIndex: {}", logName(), followerId, isFollowerActive,
637 followerNextIndex, leaderLastIndex, leaderSnapShotIndex);
640 if (isFollowerActive && context.getReplicatedLog().isPresent(followerNextIndex)) {
642 log.debug("{}: sendAppendEntries: {} is present for follower {}", logName(),
643 followerNextIndex, followerId);
645 if (followerLogInformation.okToReplicate()) {
646 // Try to send all the entries in the journal but not exceeding the max data size
647 // for a single AppendEntries message.
648 int maxEntries = (int) context.getReplicatedLog().size();
649 entries = context.getReplicatedLog().getFrom(followerNextIndex, maxEntries,
650 context.getConfigParams().getSnapshotChunkSize());
651 sendAppendEntries = true;
653 } else if (isFollowerActive && followerNextIndex >= 0
654 && leaderLastIndex > followerNextIndex && !context.getSnapshotManager().isCapturing()) {
655 // if the followers next index is not present in the leaders log, and
656 // if the follower is just not starting and if leader's index is more than followers index
657 // then snapshot should be sent
659 // Send heartbeat to follower whenever install snapshot is initiated.
660 sendAppendEntries = true;
661 if (canInstallSnapshot(followerNextIndex)) {
662 log.info("{}: Initiating install snapshot to follower {}: follower nextIndex: {}, leader "
663 + "snapshotIndex: {}, leader lastIndex: {}, leader log size: {}", logName(), followerId,
664 followerNextIndex, leaderSnapShotIndex, leaderLastIndex,
665 context.getReplicatedLog().size());
667 initiateCaptureSnapshot(followerId);
669 // It doesn't seem like we should ever reach here - most likely indicates sonething is
671 log.info("{}: Follower {} is behind but cannot install snapshot: follower nextIndex: {}, "
672 + "leader snapshotIndex: {}, leader lastIndex: {}, leader log size: {}", logName(),
673 followerId, followerNextIndex, leaderSnapShotIndex, leaderLastIndex,
674 context.getReplicatedLog().size());
677 } else if (sendHeartbeat) {
678 // we send an AppendEntries, even if the follower is inactive
679 // in-order to update the followers timestamp, in case it becomes active again
680 sendAppendEntries = true;
685 if (sendAppendEntries) {
686 sendAppendEntriesToFollower(followerActor, entries, followerLogInformation);
691 private void sendAppendEntriesToFollower(ActorSelection followerActor, List<ReplicatedLogEntry> entries,
692 FollowerLogInformation followerLogInformation) {
693 // In certain cases outlined below we don't want to send the actual commit index to prevent the follower from
694 // possibly committing and applying conflicting entries (those with same index, different term) from a prior
695 // term that weren't replicated to a majority, which would be a violation of raft.
696 // - if the follower isn't active. In this case we don't know the state of the follower and we send an
697 // empty AppendEntries as a heart beat to prevent election.
698 // - if we're in the process of installing a snapshot. In this case we don't send any new entries but still
699 // need to send AppendEntries to prevent election.
700 boolean isInstallingSnaphot = followerLogInformation.getInstallSnapshotState() != null;
701 long leaderCommitIndex = isInstallingSnaphot || !followerLogInformation.isFollowerActive() ? -1 :
702 context.getCommitIndex();
704 long followerNextIndex = followerLogInformation.getNextIndex();
705 AppendEntries appendEntries = new AppendEntries(currentTerm(), context.getId(),
706 getLogEntryIndex(followerNextIndex - 1),
707 getLogEntryTerm(followerNextIndex - 1), entries,
708 leaderCommitIndex, super.getReplicatedToAllIndex(), context.getPayloadVersion());
710 if (!entries.isEmpty() || log.isTraceEnabled()) {
711 log.debug("{}: Sending AppendEntries to follower {}: {}", logName(), followerLogInformation.getId(),
715 followerActor.tell(appendEntries, actor());
719 * Initiates a snapshot capture to install on a follower.
722 * Install Snapshot works as follows
723 * 1. Leader initiates the capture snapshot by calling createSnapshot on the RaftActor.
724 * 2. On receipt of the CaptureSnapshotReply message, the RaftActor persists the snapshot and makes a call to
725 * the Leader's handleMessage with a SendInstallSnapshot message.
726 * 3. The Leader obtains and stores the Snapshot from the SendInstallSnapshot message and sends it in chunks to
727 * the Follower via InstallSnapshot messages.
728 * 4. For each chunk, the Follower sends back an InstallSnapshotReply.
729 * 5. On receipt of the InstallSnapshotReply for the last chunk, the Leader marks the install complete for that
731 * 6. If another follower requires a snapshot and a snapshot has been collected (via SendInstallSnapshot)
732 * then send the existing snapshot in chunks to the follower.
734 * @param followerId the id of the follower.
735 * @return true if capture was initiated, false otherwise.
737 public boolean initiateCaptureSnapshot(String followerId) {
738 FollowerLogInformation followerLogInfo = followerToLog.get(followerId);
739 if (snapshotHolder.isPresent()) {
740 // If a snapshot is present in the memory, most likely another install is in progress no need to capture
741 // snapshot. This could happen if another follower needs an install when one is going on.
742 final ActorSelection followerActor = context.getPeerActorSelection(followerId);
744 // Note: sendSnapshotChunk will set the LeaderInstallSnapshotState.
745 sendSnapshotChunk(followerActor, followerLogInfo);
749 boolean captureInitiated = context.getSnapshotManager().captureToInstall(context.getReplicatedLog().last(),
750 this.getReplicatedToAllIndex(), followerId);
751 if (captureInitiated) {
752 followerLogInfo.setLeaderInstallSnapshotState(new LeaderInstallSnapshotState(
753 context.getConfigParams().getSnapshotChunkSize(), logName()));
756 return captureInitiated;
759 private boolean canInstallSnapshot(long nextIndex) {
760 // If the follower's nextIndex is -1 then we might as well send it a snapshot
761 // Otherwise send it a snapshot only if the nextIndex is not present in the log but is present
763 return nextIndex == -1 || !context.getReplicatedLog().isPresent(nextIndex)
764 && context.getReplicatedLog().isInSnapshot(nextIndex);
769 private void sendInstallSnapshot() {
770 log.debug("{}: sendInstallSnapshot", logName());
771 for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
772 String followerId = e.getKey();
773 ActorSelection followerActor = context.getPeerActorSelection(followerId);
774 FollowerLogInformation followerLogInfo = e.getValue();
776 if (followerActor != null) {
777 long nextIndex = followerLogInfo.getNextIndex();
778 if (followerLogInfo.getInstallSnapshotState() != null
779 || context.getPeerInfo(followerId).getVotingState() == VotingState.VOTING_NOT_INITIALIZED
780 || canInstallSnapshot(nextIndex)) {
781 sendSnapshotChunk(followerActor, followerLogInfo);
788 * Sends a snapshot chunk to a given follower
789 * InstallSnapshot should qualify as a heartbeat too.
791 private void sendSnapshotChunk(ActorSelection followerActor, FollowerLogInformation followerLogInfo) {
792 if (snapshotHolder.isPresent()) {
793 LeaderInstallSnapshotState installSnapshotState = followerLogInfo.getInstallSnapshotState();
794 if (installSnapshotState == null) {
795 installSnapshotState = new LeaderInstallSnapshotState(context.getConfigParams().getSnapshotChunkSize(),
797 followerLogInfo.setLeaderInstallSnapshotState(installSnapshotState);
801 // Ensure the snapshot bytes are set - this is a no-op.
802 installSnapshotState.setSnapshotBytes(snapshotHolder.get().getSnapshotBytes());
804 byte[] nextSnapshotChunk = installSnapshotState.getNextChunk();
806 log.debug("{}: next snapshot chunk size for follower {}: {}", logName(), followerLogInfo.getId(),
807 nextSnapshotChunk.length);
809 int nextChunkIndex = installSnapshotState.incrementChunkIndex();
810 Optional<ServerConfigurationPayload> serverConfig = Optional.absent();
811 if (installSnapshotState.isLastChunk(nextChunkIndex)) {
812 serverConfig = Optional.fromNullable(context.getPeerServerInfo(true));
816 new InstallSnapshot(currentTerm(), context.getId(),
817 snapshotHolder.get().getLastIncludedIndex(),
818 snapshotHolder.get().getLastIncludedTerm(),
821 installSnapshotState.getTotalChunks(),
822 Optional.of(installSnapshotState.getLastChunkHashCode()),
824 ).toSerializable(followerLogInfo.getRaftVersion()),
828 log.debug("{}: InstallSnapshot sent to follower {}, Chunk: {}/{}", logName(), followerActor.path(),
829 installSnapshotState.getChunkIndex(), installSnapshotState.getTotalChunks());
830 } catch (IOException e) {
831 throw Throwables.propagate(e);
836 private void sendHeartBeat() {
837 if (!followerToLog.isEmpty()) {
838 log.trace("{}: Sending heartbeat", logName());
839 sendAppendEntries(context.getConfigParams().getHeartBeatInterval().toMillis(), true);
843 private void stopHeartBeat() {
844 if (heartbeatSchedule != null && !heartbeatSchedule.isCancelled()) {
845 heartbeatSchedule.cancel();
849 private void scheduleHeartBeat(FiniteDuration interval) {
850 if (followerToLog.isEmpty()) {
851 // Optimization - do not bother scheduling a heartbeat as there are
858 // Schedule a heartbeat. When the scheduler triggers a SendHeartbeat
859 // message is sent to itself.
860 // Scheduling the heartbeat only once here because heartbeats do not
861 // need to be sent if there are other messages being sent to the remote
863 heartbeatSchedule = context.getActorSystem().scheduler().scheduleOnce(
864 interval, context.getActor(), SendHeartBeat.INSTANCE,
865 context.getActorSystem().dispatcher(), context.getActor());
869 public void close() {
874 public final String getLeaderId() {
875 return context.getId();
879 public final short getLeaderPayloadVersion() {
880 return context.getPayloadVersion();
883 protected boolean isLeaderIsolated() {
884 int minPresent = getMinIsolatedLeaderPeerCount();
885 for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
886 final PeerInfo peerInfo = context.getPeerInfo(followerLogInformation.getId());
887 if (peerInfo != null && peerInfo.isVoting() && followerLogInformation.isFollowerActive()) {
889 if (minPresent == 0) {
894 return minPresent != 0;
897 // called from example-actor for printing the follower-states
898 public String printFollowerStates() {
899 final StringBuilder sb = new StringBuilder();
902 for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
904 sb.append(followerLogInformation.getId());
905 sb.append(" state:");
906 sb.append(followerLogInformation.isFollowerActive());
911 return sb.toString();
915 public FollowerLogInformation getFollower(String followerId) {
916 return followerToLog.get(followerId);
920 public int followerLogSize() {
921 return followerToLog.size();
924 static class SnapshotHolder {
925 private final long lastIncludedTerm;
926 private final long lastIncludedIndex;
927 private final ByteSource snapshotBytes;
929 SnapshotHolder(Snapshot snapshot, ByteSource snapshotBytes) {
930 this.lastIncludedTerm = snapshot.getLastAppliedTerm();
931 this.lastIncludedIndex = snapshot.getLastAppliedIndex();
932 this.snapshotBytes = snapshotBytes;
935 long getLastIncludedTerm() {
936 return lastIncludedTerm;
939 long getLastIncludedIndex() {
940 return lastIncludedIndex;
943 ByteSource getSnapshotBytes() {
944 return snapshotBytes;