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.protobuf.ByteString;
18 import java.io.IOException;
19 import java.util.Arrays;
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.Snapshot;
39 import org.opendaylight.controller.cluster.raft.VotingState;
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 scala.concurrent.duration.FiniteDuration;
53 * The behavior of a RaftActor when it is in the Leader state
57 * <li> Upon election: send initial empty AppendEntries RPCs
58 * (heartbeat) to each server; repeat during idle periods to
59 * prevent election timeouts (§5.2)
60 * <li> If command received from client: append entry to local log,
61 * respond after entry applied to state machine (§5.3)
62 * <li> If last log index ≥ nextIndex for a follower: send
63 * AppendEntries RPC with log entries starting at nextIndex
65 * <li> If successful: update nextIndex and matchIndex for
67 * <li> If AppendEntries fails because of log inconsistency:
68 * decrement nextIndex and retry (§5.3)
70 * <li> If there exists an N such that N > commitIndex, a majority
71 * of matchIndex[i] ≥ N, and log[N].term == currentTerm:
72 * set commitIndex = N (§5.3, §5.4).
74 public abstract class AbstractLeader extends AbstractRaftActorBehavior {
76 // The index of the first chunk that is sent when installing a snapshot
77 public static final int FIRST_CHUNK_INDEX = 1;
79 // The index that the follower should respond with if it needs the install snapshot to be reset
80 public static final int INVALID_CHUNK_INDEX = -1;
82 // This would be passed as the hash code of the last chunk when sending the first chunk
83 public static final int INITIAL_LAST_CHUNK_HASH_CODE = -1;
85 private final Map<String, FollowerLogInformation> followerToLog = new HashMap<>();
86 private final Map<String, FollowerToSnapshot> mapFollowerToSnapshot = new HashMap<>();
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.
92 * TODO: Evaluate the use of ArrayDeque(), as that has lower memory overhead. Non-head removals are more costly,
93 * but we already expect those to be far from frequent.
95 private final Queue<ClientRequestTracker> trackers = new LinkedList<>();
97 private Cancellable heartbeatSchedule = null;
98 private Optional<SnapshotHolder> snapshot;
99 private int minReplicationCount;
101 protected AbstractLeader(RaftActorContext context, RaftState state) {
102 super(context, state);
104 for(PeerInfo peerInfo: context.getPeers()) {
105 FollowerLogInformation followerLogInformation = new FollowerLogInformationImpl(peerInfo, -1, context);
106 followerToLog.put(peerInfo.getId(), followerLogInformation);
109 LOG.debug("{}: Election: Leader has following peers: {}", logName(), getFollowerIds());
111 updateMinReplicaCount();
113 snapshot = Optional.absent();
115 // Immediately schedule a heartbeat
116 // Upon election: send initial empty AppendEntries RPCs
117 // (heartbeat) to each server; repeat during idle periods to
118 // prevent election timeouts (§5.2)
119 sendAppendEntries(0, false);
121 // It is important to schedule this heartbeat here
122 scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
126 * Return an immutable collection of follower identifiers.
128 * @return Collection of follower IDs
130 public final Collection<String> getFollowerIds() {
131 return followerToLog.keySet();
134 public void addFollower(String followerId) {
135 FollowerLogInformation followerLogInformation = new FollowerLogInformationImpl(
136 context.getPeerInfo(followerId), -1, context);
137 followerToLog.put(followerId, followerLogInformation);
139 if(heartbeatSchedule == null) {
140 scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
144 public void removeFollower(String followerId) {
145 followerToLog.remove(followerId);
146 mapFollowerToSnapshot.remove(followerId);
149 public void updateMinReplicaCount() {
151 for(PeerInfo peer: context.getPeers()) {
152 if(peer.isVoting()) {
157 minReplicationCount = getMajorityVoteCount(numVoting);
160 protected int getMinIsolatedLeaderPeerCount(){
161 //the isolated Leader peer count will be 1 less than the majority vote count.
162 //this is because the vote count has the self vote counted in it
164 //0 peers = 1 votesRequired , minIsolatedLeaderPeerCount = 0
165 //2 peers = 2 votesRequired , minIsolatedLeaderPeerCount = 1
166 //4 peers = 3 votesRequired, minIsolatedLeaderPeerCount = 2
168 return minReplicationCount > 0 ? (minReplicationCount - 1) : 0;
172 void setSnapshot(@Nullable Snapshot snapshot) {
173 if(snapshot != null) {
174 this.snapshot = Optional.of(new SnapshotHolder(snapshot));
176 this.snapshot = Optional.absent();
181 protected RaftActorBehavior handleAppendEntries(ActorRef sender,
182 AppendEntries appendEntries) {
184 LOG.debug("{}: handleAppendEntries: {}", logName(), appendEntries);
190 protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
191 AppendEntriesReply appendEntriesReply) {
193 if(LOG.isTraceEnabled()) {
194 LOG.trace("{}: handleAppendEntriesReply: {}", logName(), appendEntriesReply);
197 // Update the FollowerLogInformation
198 String followerId = appendEntriesReply.getFollowerId();
199 FollowerLogInformation followerLogInformation =
200 followerToLog.get(followerId);
202 if(followerLogInformation == null){
203 LOG.error("{}: handleAppendEntriesReply - unknown follower {}", logName(), followerId);
207 if(followerLogInformation.timeSinceLastActivity() >
208 context.getConfigParams().getElectionTimeOutInterval().toMillis()) {
209 LOG.warn("{} : handleAppendEntriesReply delayed beyond election timeout, " +
210 "appendEntriesReply : {}, timeSinceLastActivity : {}, lastApplied : {}, commitIndex : {}",
211 logName(), appendEntriesReply, followerLogInformation.timeSinceLastActivity(),
212 context.getLastApplied(), context.getCommitIndex());
215 followerLogInformation.markFollowerActive();
216 followerLogInformation.setPayloadVersion(appendEntriesReply.getPayloadVersion());
217 followerLogInformation.setRaftVersion(appendEntriesReply.getRaftVersion());
219 boolean updated = false;
220 if (appendEntriesReply.isSuccess()) {
221 updated = updateFollowerLogInformation(followerLogInformation, appendEntriesReply);
223 LOG.debug("{}: handleAppendEntriesReply: received unsuccessful reply: {}", logName(), appendEntriesReply);
225 long followerLastLogIndex = appendEntriesReply.getLogLastIndex();
226 ReplicatedLogEntry followersLastLogEntry = context.getReplicatedLog().get(followerLastLogIndex);
227 if(appendEntriesReply.isForceInstallSnapshot()) {
228 // Reset the followers match and next index. This is to signal that this follower has nothing
229 // in common with this Leader and so would require a snapshot to be installed
230 followerLogInformation.setMatchIndex(-1);
231 followerLogInformation.setNextIndex(-1);
233 // Force initiate a snapshot capture
234 initiateCaptureSnapshot(followerId);
235 } else if(followerLastLogIndex < 0 || (followersLastLogEntry != null &&
236 followersLastLogEntry.getTerm() == appendEntriesReply.getLogLastTerm())) {
237 // The follower's log is empty or the last entry is present in the leader's journal
238 // and the terms match so the follower is just behind the leader's journal from
239 // the last snapshot, if any. We'll catch up the follower quickly by starting at the
240 // follower's last log index.
242 updated = updateFollowerLogInformation(followerLogInformation, appendEntriesReply);
244 // TODO: When we find that the follower is out of sync with the
245 // Leader we simply decrement that followers next index by 1.
246 // Would it be possible to do better than this? The RAFT spec
247 // does not explicitly deal with it but may be something for us to
250 followerLogInformation.decrNextIndex();
254 // Now figure out if this reply warrants a change in the commitIndex
255 // If there exists an N such that N > commitIndex, a majority
256 // of matchIndex[i] ≥ N, and log[N].term == currentTerm:
257 // set commitIndex = N (§5.3, §5.4).
258 if(LOG.isTraceEnabled()) {
259 LOG.trace("{}: handleAppendEntriesReply from {}: commitIndex: {}, lastAppliedIndex: {}, currentTerm: {}",
260 logName(), followerId, context.getCommitIndex(), context.getLastApplied(), currentTerm());
263 for (long N = context.getCommitIndex() + 1; ; N++) {
264 int replicatedCount = 1;
266 LOG.trace("{}: checking Nth index {}", logName(), N);
267 for (FollowerLogInformation info : followerToLog.values()) {
268 final PeerInfo peerInfo = context.getPeerInfo(info.getId());
269 if(info.getMatchIndex() >= N && (peerInfo != null && peerInfo.isVoting())) {
271 } else if(LOG.isDebugEnabled()) {
272 LOG.debug("{}: Not counting follower {} - matchIndex: {}, {}", logName(), info.getId(),
273 info.getMatchIndex(), peerInfo);
277 if(LOG.isTraceEnabled()) {
278 LOG.trace("{}: replicatedCount {}, minReplicationCount: {}", logName(), replicatedCount, minReplicationCount);
281 if (replicatedCount >= minReplicationCount) {
282 ReplicatedLogEntry replicatedLogEntry = context.getReplicatedLog().get(N);
283 if (replicatedLogEntry == null) {
284 LOG.debug("{}: ReplicatedLogEntry not found for index {} - snapshotIndex: {}, journal size: {}",
285 logName(), N, context.getReplicatedLog().getSnapshotIndex(), context.getReplicatedLog().size());
289 // Don't update the commit index if the log entry is from a previous term, as per §5.4.1:
290 // "Raft never commits log entries from previous terms by counting replicas".
291 // However we keep looping so we can make progress when new entries in the current term
292 // reach consensus, as per §5.4.1: "once an entry from the current term is committed by
293 // counting replicas, then all prior entries are committed indirectly".
294 if (replicatedLogEntry.getTerm() == currentTerm()) {
295 LOG.trace("{}: Setting commit index to {}", logName(), N);
296 context.setCommitIndex(N);
298 LOG.debug("{}: Not updating commit index to {} - retrieved log entry with index {}, term {} does not match the current term {}",
299 logName(), N, replicatedLogEntry.getIndex(), replicatedLogEntry.getTerm(), currentTerm());
302 LOG.trace("{}: minReplicationCount not reached - breaking", logName());
307 // Apply the change to the state machine
308 if (context.getCommitIndex() > context.getLastApplied()) {
309 if(LOG.isDebugEnabled()) {
310 LOG.debug("{}: handleAppendEntriesReply from {}: applying to log - commitIndex: {}, lastAppliedIndex: {}",
311 logName(), followerId, context.getCommitIndex(), context.getLastApplied());
314 applyLogToStateMachine(context.getCommitIndex());
317 if (!context.getSnapshotManager().isCapturing()) {
321 //Send the next log entry immediately, if possible, no need to wait for heartbeat to trigger that event
322 sendUpdatesToFollower(followerId, followerLogInformation, false, !updated);
326 private boolean updateFollowerLogInformation(FollowerLogInformation followerLogInformation,
327 AppendEntriesReply appendEntriesReply) {
328 boolean updated = followerLogInformation.setMatchIndex(appendEntriesReply.getLogLastIndex());
329 updated = followerLogInformation.setNextIndex(appendEntriesReply.getLogLastIndex() + 1) || updated;
331 if(updated && LOG.isDebugEnabled()) {
332 LOG.debug("{}: handleAppendEntriesReply - FollowerLogInformation for {} updated: matchIndex: {}, nextIndex: {}",
333 logName(), followerLogInformation.getId(), followerLogInformation.getMatchIndex(),
334 followerLogInformation.getNextIndex());
339 private void purgeInMemoryLog() {
340 //find the lowest index across followers which has been replicated to all.
341 // lastApplied if there are no followers, so that we keep clearing the log for single-node
342 // we would delete the in-mem log from that index on, in-order to minimize mem usage
343 // we would also share this info thru AE with the followers so that they can delete their log entries as well.
344 long minReplicatedToAllIndex = followerToLog.isEmpty() ? context.getLastApplied() : Long.MAX_VALUE;
345 for (FollowerLogInformation info : followerToLog.values()) {
346 minReplicatedToAllIndex = Math.min(minReplicatedToAllIndex, info.getMatchIndex());
349 super.performSnapshotWithoutCapture(minReplicatedToAllIndex);
353 protected ClientRequestTracker removeClientRequestTracker(long logIndex) {
354 final Iterator<ClientRequestTracker> it = trackers.iterator();
355 while (it.hasNext()) {
356 final ClientRequestTracker t = it.next();
357 if (t.getIndex() == logIndex) {
367 protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
368 RequestVoteReply requestVoteReply) {
372 protected void beforeSendHeartbeat(){}
375 public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) {
376 Preconditions.checkNotNull(sender, "sender should not be null");
378 Object message = fromSerializableMessage(originalMessage);
380 if (message instanceof RaftRPC) {
381 RaftRPC rpc = (RaftRPC) message;
382 // If RPC request or response contains term T > currentTerm:
383 // set currentTerm = T, convert to follower (§5.1)
384 // This applies to all RPC messages and responses
385 if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
386 LOG.debug("{}: Term {} in \"{}\" message is greater than leader's term {} - switching to Follower",
387 logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm());
389 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
391 return internalSwitchBehavior(RaftState.Follower);
395 if (message instanceof SendHeartBeat) {
396 beforeSendHeartbeat();
398 scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
399 } else if(message instanceof SendInstallSnapshot) {
400 // received from RaftActor
401 setSnapshot(((SendInstallSnapshot) message).getSnapshot());
402 sendInstallSnapshot();
403 } else if (message instanceof Replicate) {
404 replicate((Replicate) message);
405 } else if (message instanceof InstallSnapshotReply) {
406 handleInstallSnapshotReply((InstallSnapshotReply) message);
408 return super.handleMessage(sender, message);
414 private void handleInstallSnapshotReply(InstallSnapshotReply reply) {
415 LOG.debug("{}: handleInstallSnapshotReply: {}", logName(), reply);
417 String followerId = reply.getFollowerId();
418 FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
420 if (followerToSnapshot == null) {
421 LOG.error("{}: FollowerToSnapshot not found for follower {} in InstallSnapshotReply",
422 logName(), followerId);
426 FollowerLogInformation followerLogInformation = followerToLog.get(followerId);
427 if(followerLogInformation == null) {
428 // This can happen during AddServer if it times out.
429 LOG.error("{}: FollowerLogInformation not found for follower {} in InstallSnapshotReply",
430 logName(), followerId);
431 mapFollowerToSnapshot.remove(followerId);
435 followerLogInformation.markFollowerActive();
437 if (followerToSnapshot.getChunkIndex() == reply.getChunkIndex()) {
438 boolean wasLastChunk = false;
439 if (reply.isSuccess()) {
440 if(followerToSnapshot.isLastChunk(reply.getChunkIndex())) {
441 //this was the last chunk reply
442 if(LOG.isDebugEnabled()) {
443 LOG.debug("{}: InstallSnapshotReply received, " +
444 "last chunk received, Chunk: {}. Follower: {} Setting nextIndex: {}",
445 logName(), reply.getChunkIndex(), followerId,
446 context.getReplicatedLog().getSnapshotIndex() + 1
450 long followerMatchIndex = snapshot.get().getLastIncludedIndex();
451 followerLogInformation.setMatchIndex(followerMatchIndex);
452 followerLogInformation.setNextIndex(followerMatchIndex + 1);
453 mapFollowerToSnapshot.remove(followerId);
455 LOG.debug("{}: follower: {}, matchIndex set to {}, nextIndex set to {}",
456 logName(), followerId, followerLogInformation.getMatchIndex(),
457 followerLogInformation.getNextIndex());
459 if (mapFollowerToSnapshot.isEmpty()) {
460 // once there are no pending followers receiving snapshots
461 // we can remove snapshot from the memory
465 if(context.getPeerInfo(followerId).getVotingState() == VotingState.VOTING_NOT_INITIALIZED){
466 UnInitializedFollowerSnapshotReply unInitFollowerSnapshotSuccess =
467 new UnInitializedFollowerSnapshotReply(followerId);
468 context.getActor().tell(unInitFollowerSnapshotSuccess, context.getActor());
469 LOG.debug("Sent message UnInitializedFollowerSnapshotReply to self");
472 followerToSnapshot.markSendStatus(true);
475 LOG.info("{}: InstallSnapshotReply received sending snapshot chunk failed, Will retry, Chunk: {}",
476 logName(), reply.getChunkIndex());
478 followerToSnapshot.markSendStatus(false);
481 if (wasLastChunk && !context.getSnapshotManager().isCapturing()) {
482 // Since the follower is now caught up try to purge the log.
484 } else if (!wasLastChunk && followerToSnapshot.canSendNextChunk()) {
485 ActorSelection followerActor = context.getPeerActorSelection(followerId);
486 if(followerActor != null) {
487 sendSnapshotChunk(followerActor, followerId);
492 LOG.error("{}: Chunk index {} in InstallSnapshotReply from follower {} does not match expected index {}",
493 logName(), reply.getChunkIndex(), followerId,
494 followerToSnapshot.getChunkIndex());
496 if(reply.getChunkIndex() == INVALID_CHUNK_INDEX){
497 // Since the Follower did not find this index to be valid we should reset the follower snapshot
498 // so that Installing the snapshot can resume from the beginning
499 followerToSnapshot.reset();
504 private void replicate(Replicate replicate) {
505 long logIndex = replicate.getReplicatedLogEntry().getIndex();
507 LOG.debug("{}: Replicate message: identifier: {}, logIndex: {}", logName(),
508 replicate.getIdentifier(), logIndex);
510 // Create a tracker entry we will use this later to notify the
513 new ClientRequestTrackerImpl(replicate.getClientActor(),
514 replicate.getIdentifier(),
518 boolean applyModificationToState = followerToLog.isEmpty()
519 || context.getRaftPolicy().applyModificationToStateBeforeConsensus();
521 if(applyModificationToState){
522 context.setCommitIndex(logIndex);
523 applyLogToStateMachine(logIndex);
526 if (!followerToLog.isEmpty()) {
527 sendAppendEntries(0, false);
531 protected void sendAppendEntries(long timeSinceLastActivityInterval, boolean isHeartbeat) {
532 // Send an AppendEntries to all followers
533 for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
534 final String followerId = e.getKey();
535 final FollowerLogInformation followerLogInformation = e.getValue();
536 // This checks helps not to send a repeat message to the follower
537 if(!followerLogInformation.isFollowerActive() ||
538 followerLogInformation.timeSinceLastActivity() >= timeSinceLastActivityInterval) {
539 sendUpdatesToFollower(followerId, followerLogInformation, true, isHeartbeat);
546 * This method checks if any update needs to be sent to the given follower. This includes append log entries,
547 * sending next snapshot chunk, and initiating a snapshot.
548 * @return true if any update is sent, false otherwise
551 private void sendUpdatesToFollower(String followerId, FollowerLogInformation followerLogInformation,
552 boolean sendHeartbeat, boolean isHeartbeat) {
554 ActorSelection followerActor = context.getPeerActorSelection(followerId);
555 if (followerActor != null) {
556 long followerNextIndex = followerLogInformation.getNextIndex();
557 boolean isFollowerActive = followerLogInformation.isFollowerActive();
558 boolean sendAppendEntries = false;
559 List<ReplicatedLogEntry> entries = Collections.emptyList();
561 if (mapFollowerToSnapshot.get(followerId) != null) {
562 // if install snapshot is in process , then sent next chunk if possible
563 if (isFollowerActive && mapFollowerToSnapshot.get(followerId).canSendNextChunk()) {
564 sendSnapshotChunk(followerActor, followerId);
565 } else if(sendHeartbeat) {
566 // we send a heartbeat even if we have not received a reply for the last chunk
567 sendAppendEntries = true;
570 long leaderLastIndex = context.getReplicatedLog().lastIndex();
571 long leaderSnapShotIndex = context.getReplicatedLog().getSnapshotIndex();
573 if((!isHeartbeat && LOG.isDebugEnabled()) || LOG.isTraceEnabled()) {
574 LOG.debug("{}: Checking sendAppendEntries for follower {}: active: {}, followerNextIndex: {}, leaderLastIndex: {}, leaderSnapShotIndex: {}",
575 logName(), followerId, isFollowerActive, followerNextIndex, leaderLastIndex, leaderSnapShotIndex);
578 if (isFollowerActive && context.getReplicatedLog().isPresent(followerNextIndex)) {
580 LOG.debug("{}: sendAppendEntries: {} is present for follower {}", logName(),
581 followerNextIndex, followerId);
583 if(followerLogInformation.okToReplicate()) {
584 // Try to send all the entries in the journal but not exceeding the max data size
585 // for a single AppendEntries message.
586 int maxEntries = (int) context.getReplicatedLog().size();
587 entries = context.getReplicatedLog().getFrom(followerNextIndex, maxEntries,
588 context.getConfigParams().getSnapshotChunkSize());
589 sendAppendEntries = true;
591 } else if (isFollowerActive && followerNextIndex >= 0 &&
592 leaderLastIndex > followerNextIndex && !context.getSnapshotManager().isCapturing()) {
593 // if the followers next index is not present in the leaders log, and
594 // if the follower is just not starting and if leader's index is more than followers index
595 // then snapshot should be sent
597 if (LOG.isDebugEnabled()) {
598 LOG.debug(String.format("%s: InitiateInstallSnapshot to follower: %s," +
599 "follower-nextIndex: %d, leader-snapshot-index: %d, " +
600 "leader-last-index: %d", logName(), followerId,
601 followerNextIndex, leaderSnapShotIndex, leaderLastIndex));
604 // Send heartbeat to follower whenever install snapshot is initiated.
605 sendAppendEntries = true;
606 if (canInstallSnapshot(followerNextIndex)) {
607 initiateCaptureSnapshot(followerId);
610 } else if(sendHeartbeat) {
611 // we send an AppendEntries, even if the follower is inactive
612 // in-order to update the followers timestamp, in case it becomes active again
613 sendAppendEntries = true;
618 if(sendAppendEntries) {
619 sendAppendEntriesToFollower(followerActor, followerNextIndex,
620 entries, followerId);
625 private void sendAppendEntriesToFollower(ActorSelection followerActor, long followerNextIndex,
626 List<ReplicatedLogEntry> entries, String followerId) {
627 AppendEntries appendEntries = new AppendEntries(currentTerm(), context.getId(),
628 prevLogIndex(followerNextIndex),
629 prevLogTerm(followerNextIndex), entries,
630 context.getCommitIndex(), super.getReplicatedToAllIndex(), context.getPayloadVersion());
632 if(!entries.isEmpty() || LOG.isTraceEnabled()) {
633 LOG.debug("{}: Sending AppendEntries to follower {}: {}", logName(), followerId,
637 followerActor.tell(appendEntries, actor());
641 * Install Snapshot works as follows
642 * 1. Leader initiates the capture snapshot by sending a CaptureSnapshot message to actor
643 * 2. RaftActor on receipt of the CaptureSnapshotReply (from Shard), stores the received snapshot in the replicated log
644 * and makes a call to Leader's handleMessage , with SendInstallSnapshot message.
645 * 3. Leader , picks the snapshot from im-mem ReplicatedLog and sends it in chunks to the Follower
646 * 4. On complete, Follower sends back a InstallSnapshotReply.
647 * 5. On receipt of the InstallSnapshotReply for the last chunk, Leader marks the install complete for that follower
648 * and replenishes the memory by deleting the snapshot in Replicated log.
649 * 6. If another follower requires a snapshot and a snapshot has been collected (via CaptureSnapshotReply)
650 * then send the existing snapshot in chunks to the follower.
653 public boolean initiateCaptureSnapshot(String followerId) {
654 if (snapshot.isPresent()) {
655 // if a snapshot is present in the memory, most likely another install is in progress
656 // no need to capture snapshot.
657 // This could happen if another follower needs an install when one is going on.
658 final ActorSelection followerActor = context.getPeerActorSelection(followerId);
659 sendSnapshotChunk(followerActor, followerId);
662 return context.getSnapshotManager().captureToInstall(context.getReplicatedLog().last(),
663 this.getReplicatedToAllIndex(), followerId);
667 private boolean canInstallSnapshot(long nextIndex){
668 // If the follower's nextIndex is -1 then we might as well send it a snapshot
669 // Otherwise send it a snapshot only if the nextIndex is not present in the log but is present
671 return (nextIndex == -1 ||
672 (!context.getReplicatedLog().isPresent(nextIndex)
673 && context.getReplicatedLog().isInSnapshot(nextIndex)));
678 private void sendInstallSnapshot() {
679 LOG.debug("{}: sendInstallSnapshot", logName());
680 for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
681 String followerId = e.getKey();
682 ActorSelection followerActor = context.getPeerActorSelection(followerId);
683 FollowerLogInformation followerLogInfo = e.getValue();
685 if (followerActor != null) {
686 long nextIndex = followerLogInfo.getNextIndex();
687 if (context.getPeerInfo(followerId).getVotingState() == VotingState.VOTING_NOT_INITIALIZED ||
688 canInstallSnapshot(nextIndex)) {
689 sendSnapshotChunk(followerActor, followerId);
696 * Sends a snapshot chunk to a given follower
697 * InstallSnapshot should qualify as a heartbeat too.
699 private void sendSnapshotChunk(ActorSelection followerActor, String followerId) {
701 if (snapshot.isPresent()) {
702 byte[] nextSnapshotChunk = getNextSnapshotChunk(followerId, snapshot.get().getSnapshotBytes());
704 // Note: the previous call to getNextSnapshotChunk has the side-effect of adding
705 // followerId to the followerToSnapshot map.
706 FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
709 new InstallSnapshot(currentTerm(), context.getId(),
710 snapshot.get().getLastIncludedIndex(),
711 snapshot.get().getLastIncludedTerm(),
713 followerToSnapshot.incrementChunkIndex(),
714 followerToSnapshot.getTotalChunks(),
715 Optional.of(followerToSnapshot.getLastChunkHashCode())
716 ).toSerializable(followerToLog.get(followerId).getRaftVersion()),
720 if(LOG.isDebugEnabled()) {
721 LOG.debug("{}: InstallSnapshot sent to follower {}, Chunk: {}/{}",
722 logName(), followerActor.path(), followerToSnapshot.getChunkIndex(),
723 followerToSnapshot.getTotalChunks());
726 } catch (IOException e) {
727 LOG.error("{}: InstallSnapshot failed for Leader.", logName(), e);
732 * Acccepts snaphot as ByteString, enters into map for future chunks
733 * creates and return a ByteString chunk
735 private byte[] getNextSnapshotChunk(String followerId, ByteString snapshotBytes) throws IOException {
736 FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
737 if (followerToSnapshot == null) {
738 followerToSnapshot = new FollowerToSnapshot(snapshotBytes);
739 mapFollowerToSnapshot.put(followerId, followerToSnapshot);
741 byte[] nextChunk = followerToSnapshot.getNextChunk();
743 LOG.debug("{}: next snapshot chunk size for follower {}: {}", logName(), followerId, nextChunk.length);
748 private void sendHeartBeat() {
749 if (!followerToLog.isEmpty()) {
750 LOG.trace("{}: Sending heartbeat", logName());
751 sendAppendEntries(context.getConfigParams().getHeartBeatInterval().toMillis(), true);
755 private void stopHeartBeat() {
756 if (heartbeatSchedule != null && !heartbeatSchedule.isCancelled()) {
757 heartbeatSchedule.cancel();
761 private void scheduleHeartBeat(FiniteDuration interval) {
762 if (followerToLog.isEmpty()) {
763 // Optimization - do not bother scheduling a heartbeat as there are
770 // Schedule a heartbeat. When the scheduler triggers a SendHeartbeat
771 // message is sent to itself.
772 // Scheduling the heartbeat only once here because heartbeats do not
773 // need to be sent if there are other messages being sent to the remote
775 heartbeatSchedule = context.getActorSystem().scheduler().scheduleOnce(
776 interval, context.getActor(), SendHeartBeat.INSTANCE,
777 context.getActorSystem().dispatcher(), context.getActor());
781 public void close() {
786 public final String getLeaderId() {
787 return context.getId();
791 public final short getLeaderPayloadVersion() {
792 return context.getPayloadVersion();
795 protected boolean isLeaderIsolated() {
796 int minPresent = getMinIsolatedLeaderPeerCount();
797 for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
798 final PeerInfo peerInfo = context.getPeerInfo(followerLogInformation.getId());
799 if(peerInfo != null && peerInfo.isVoting() && followerLogInformation.isFollowerActive()) {
801 if (minPresent == 0) {
806 return (minPresent != 0);
810 * Encapsulates the snapshot bytestring and handles the logic of sending
813 protected class FollowerToSnapshot {
814 private final ByteString snapshotBytes;
815 private int offset = 0;
816 // the next snapshot chunk is sent only if the replyReceivedForOffset matches offset
817 private int replyReceivedForOffset;
818 // if replyStatus is false, the previous chunk is attempted
819 private boolean replyStatus = false;
820 private int chunkIndex;
821 private final int totalChunks;
822 private int lastChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
823 private int nextChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
825 public FollowerToSnapshot(ByteString snapshotBytes) {
826 this.snapshotBytes = snapshotBytes;
827 int size = snapshotBytes.size();
828 totalChunks = ( size / context.getConfigParams().getSnapshotChunkSize()) +
829 ((size % context.getConfigParams().getSnapshotChunkSize()) > 0 ? 1 : 0);
830 if(LOG.isDebugEnabled()) {
831 LOG.debug("{}: Snapshot {} bytes, total chunks to send:{}",
832 logName(), size, totalChunks);
834 replyReceivedForOffset = -1;
835 chunkIndex = AbstractLeader.FIRST_CHUNK_INDEX;
838 public ByteString getSnapshotBytes() {
839 return snapshotBytes;
842 public int incrementOffset() {
844 // if prev chunk failed, we would want to sent the same chunk again
845 offset = offset + context.getConfigParams().getSnapshotChunkSize();
850 public int incrementChunkIndex() {
852 // if prev chunk failed, we would want to sent the same chunk again
853 chunkIndex = chunkIndex + 1;
858 public int getChunkIndex() {
862 public int getTotalChunks() {
866 public boolean canSendNextChunk() {
867 // we only send a false if a chunk is sent but we have not received a reply yet
868 return replyReceivedForOffset == offset;
871 public boolean isLastChunk(int chunkIndex) {
872 return totalChunks == chunkIndex;
875 public void markSendStatus(boolean success) {
877 // if the chunk sent was successful
878 replyReceivedForOffset = offset;
880 lastChunkHashCode = nextChunkHashCode;
882 // if the chunk sent was failure
883 replyReceivedForOffset = offset;
888 public byte[] getNextChunk() {
889 int snapshotLength = getSnapshotBytes().size();
890 int start = incrementOffset();
891 int size = context.getConfigParams().getSnapshotChunkSize();
892 if (context.getConfigParams().getSnapshotChunkSize() > snapshotLength) {
893 size = snapshotLength;
894 } else if ((start + context.getConfigParams().getSnapshotChunkSize()) > snapshotLength) {
895 size = snapshotLength - start;
898 byte[] nextChunk = new byte[size];
899 getSnapshotBytes().copyTo(nextChunk, start, 0, size);
900 nextChunkHashCode = Arrays.hashCode(nextChunk);
902 LOG.debug("{}: Next chunk: total length={}, offset={}, size={}, hashCode={}", logName(),
903 snapshotLength, start, size, nextChunkHashCode);
908 * reset should be called when the Follower needs to be sent the snapshot from the beginning
913 replyReceivedForOffset = offset;
914 chunkIndex = AbstractLeader.FIRST_CHUNK_INDEX;
915 lastChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
918 public int getLastChunkHashCode() {
919 return lastChunkHashCode;
923 // called from example-actor for printing the follower-states
924 public String printFollowerStates() {
925 final StringBuilder sb = new StringBuilder();
928 for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
930 sb.append(followerLogInformation.getId());
931 sb.append(" state:");
932 sb.append(followerLogInformation.isFollowerActive());
937 return sb.toString();
941 public FollowerLogInformation getFollower(String followerId) {
942 return followerToLog.get(followerId);
946 protected void setFollowerSnapshot(String followerId, FollowerToSnapshot snapshot) {
947 mapFollowerToSnapshot.put(followerId, snapshot);
951 public int followerSnapshotSize() {
952 return mapFollowerToSnapshot.size();
956 public int followerLogSize() {
957 return followerToLog.size();
960 private static class SnapshotHolder {
961 private final long lastIncludedTerm;
962 private final long lastIncludedIndex;
963 private final ByteString snapshotBytes;
965 SnapshotHolder(Snapshot snapshot) {
966 this.lastIncludedTerm = snapshot.getLastAppliedTerm();
967 this.lastIncludedIndex = snapshot.getLastAppliedIndex();
968 this.snapshotBytes = ByteString.copyFrom(snapshot.getState());
971 long getLastIncludedTerm() {
972 return lastIncludedTerm;
975 long getLastIncludedIndex() {
976 return lastIncludedIndex;
979 ByteString getSnapshotBytes() {
980 return snapshotBytes;