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.Collection;
20 import java.util.Collections;
21 import java.util.HashMap;
22 import java.util.Iterator;
23 import java.util.LinkedList;
24 import java.util.List;
26 import java.util.Map.Entry;
27 import javax.annotation.Nullable;
28 import org.opendaylight.controller.cluster.raft.ClientRequestTracker;
29 import org.opendaylight.controller.cluster.raft.ClientRequestTrackerImpl;
30 import org.opendaylight.controller.cluster.raft.FollowerLogInformation;
31 import org.opendaylight.controller.cluster.raft.FollowerLogInformationImpl;
32 import org.opendaylight.controller.cluster.raft.PeerInfo;
33 import org.opendaylight.controller.cluster.raft.RaftActorContext;
34 import org.opendaylight.controller.cluster.raft.RaftState;
35 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
36 import org.opendaylight.controller.cluster.raft.Snapshot;
37 import org.opendaylight.controller.cluster.raft.VotingState;
38 import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
39 import org.opendaylight.controller.cluster.raft.base.messages.SendHeartBeat;
40 import org.opendaylight.controller.cluster.raft.base.messages.SendInstallSnapshot;
41 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
42 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
43 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot;
44 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshotReply;
45 import org.opendaylight.controller.cluster.raft.messages.RaftRPC;
46 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
47 import org.opendaylight.controller.cluster.raft.messages.UnInitializedFollowerSnapshotReply;
48 import scala.concurrent.duration.FiniteDuration;
51 * The behavior of a RaftActor when it is in the Leader state
55 * <li> Upon election: send initial empty AppendEntries RPCs
56 * (heartbeat) to each server; repeat during idle periods to
57 * prevent election timeouts (§5.2)
58 * <li> If command received from client: append entry to local log,
59 * respond after entry applied to state machine (§5.3)
60 * <li> If last log index ≥ nextIndex for a follower: send
61 * AppendEntries RPC with log entries starting at nextIndex
63 * <li> If successful: update nextIndex and matchIndex for
65 * <li> If AppendEntries fails because of log inconsistency:
66 * decrement nextIndex and retry (§5.3)
68 * <li> If there exists an N such that N > commitIndex, a majority
69 * of matchIndex[i] ≥ N, and log[N].term == currentTerm:
70 * set commitIndex = N (§5.3, §5.4).
72 public abstract class AbstractLeader extends AbstractRaftActorBehavior {
74 // The index of the first chunk that is sent when installing a snapshot
75 public static final int FIRST_CHUNK_INDEX = 1;
77 // The index that the follower should respond with if it needs the install snapshot to be reset
78 public static final int INVALID_CHUNK_INDEX = -1;
80 // This would be passed as the hash code of the last chunk when sending the first chunk
81 public static final int INITIAL_LAST_CHUNK_HASH_CODE = -1;
83 private final Map<String, FollowerLogInformation> followerToLog = new HashMap<>();
84 private final Map<String, FollowerToSnapshot> mapFollowerToSnapshot = new HashMap<>();
86 private Cancellable heartbeatSchedule = null;
88 private final Collection<ClientRequestTracker> trackerList = new LinkedList<>();
90 private int minReplicationCount;
92 private Optional<SnapshotHolder> snapshot;
94 public AbstractLeader(RaftActorContext context) {
95 super(context, RaftState.Leader);
97 setLeaderPayloadVersion(context.getPayloadVersion());
99 for(PeerInfo peerInfo: context.getPeers()) {
100 FollowerLogInformation followerLogInformation = new FollowerLogInformationImpl(peerInfo, -1, context);
101 followerToLog.put(peerInfo.getId(), followerLogInformation);
104 leaderId = context.getId();
106 LOG.debug("{}: Election: Leader has following peers: {}", logName(), getFollowerIds());
108 updateMinReplicaCount();
110 snapshot = Optional.absent();
112 // Immediately schedule a heartbeat
113 // Upon election: send initial empty AppendEntries RPCs
114 // (heartbeat) to each server; repeat during idle periods to
115 // prevent election timeouts (§5.2)
116 sendAppendEntries(0, false);
118 // It is important to schedule this heartbeat here
119 scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
123 * Return an immutable collection of follower identifiers.
125 * @return Collection of follower IDs
127 public final Collection<String> getFollowerIds() {
128 return followerToLog.keySet();
131 public void addFollower(String followerId) {
132 FollowerLogInformation followerLogInformation = new FollowerLogInformationImpl(
133 context.getPeerInfo(followerId), -1, context);
134 followerToLog.put(followerId, followerLogInformation);
136 if(heartbeatSchedule == null) {
137 scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
141 public void removeFollower(String followerId) {
142 followerToLog.remove(followerId);
143 mapFollowerToSnapshot.remove(followerId);
146 public void updateMinReplicaCount() {
148 for(PeerInfo peer: context.getPeers()) {
149 if(peer.isVoting()) {
154 minReplicationCount = getMajorityVoteCount(numVoting);
157 protected int getMinIsolatedLeaderPeerCount(){
158 //the isolated Leader peer count will be 1 less than the majority vote count.
159 //this is because the vote count has the self vote counted in it
161 //0 peers = 1 votesRequired , minIsolatedLeaderPeerCount = 0
162 //2 peers = 2 votesRequired , minIsolatedLeaderPeerCount = 1
163 //4 peers = 3 votesRequired, minIsolatedLeaderPeerCount = 2
165 return minReplicationCount > 0 ? (minReplicationCount - 1) : 0;
169 void setSnapshot(@Nullable Snapshot snapshot) {
170 if(snapshot != null) {
171 this.snapshot = Optional.of(new SnapshotHolder(snapshot));
173 this.snapshot = Optional.absent();
178 protected RaftActorBehavior handleAppendEntries(ActorRef sender,
179 AppendEntries appendEntries) {
181 LOG.debug("{}: handleAppendEntries: {}", logName(), appendEntries);
187 protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
188 AppendEntriesReply appendEntriesReply) {
190 if(LOG.isTraceEnabled()) {
191 LOG.trace("{}: handleAppendEntriesReply: {}", logName(), appendEntriesReply);
194 // Update the FollowerLogInformation
195 String followerId = appendEntriesReply.getFollowerId();
196 FollowerLogInformation followerLogInformation =
197 followerToLog.get(followerId);
199 if(followerLogInformation == null){
200 LOG.error("{}: handleAppendEntriesReply - unknown follower {}", logName(), followerId);
204 if(followerLogInformation.timeSinceLastActivity() >
205 context.getConfigParams().getElectionTimeOutInterval().toMillis()) {
206 LOG.warn("{} : handleAppendEntriesReply delayed beyond election timeout, " +
207 "appendEntriesReply : {}, timeSinceLastActivity : {}, lastApplied : {}, commitIndex : {}",
208 logName(), appendEntriesReply, followerLogInformation.timeSinceLastActivity(),
209 context.getLastApplied(), context.getCommitIndex());
212 followerLogInformation.markFollowerActive();
213 followerLogInformation.setPayloadVersion(appendEntriesReply.getPayloadVersion());
214 followerLogInformation.setRaftVersion(appendEntriesReply.getRaftVersion());
216 boolean updated = false;
217 if (appendEntriesReply.isSuccess()) {
218 updated = updateFollowerLogInformation(followerLogInformation, appendEntriesReply);
220 LOG.debug("{}: handleAppendEntriesReply: received unsuccessful reply: {}", logName(), appendEntriesReply);
222 long followerLastLogIndex = appendEntriesReply.getLogLastIndex();
223 ReplicatedLogEntry followersLastLogEntry = context.getReplicatedLog().get(followerLastLogIndex);
224 if(appendEntriesReply.isForceInstallSnapshot()) {
225 // Reset the followers match and next index. This is to signal that this follower has nothing
226 // in common with this Leader and so would require a snapshot to be installed
227 followerLogInformation.setMatchIndex(-1);
228 followerLogInformation.setNextIndex(-1);
230 // Force initiate a snapshot capture
231 initiateCaptureSnapshot(followerId);
232 } else if(followerLastLogIndex < 0 || (followersLastLogEntry != null &&
233 followersLastLogEntry.getTerm() == appendEntriesReply.getLogLastTerm())) {
234 // The follower's log is empty or the last entry is present in the leader's journal
235 // and the terms match so the follower is just behind the leader's journal from
236 // the last snapshot, if any. We'll catch up the follower quickly by starting at the
237 // follower's last log index.
239 updated = updateFollowerLogInformation(followerLogInformation, appendEntriesReply);
241 // TODO: When we find that the follower is out of sync with the
242 // Leader we simply decrement that followers next index by 1.
243 // Would it be possible to do better than this? The RAFT spec
244 // does not explicitly deal with it but may be something for us to
247 followerLogInformation.decrNextIndex();
251 // Now figure out if this reply warrants a change in the commitIndex
252 // If there exists an N such that N > commitIndex, a majority
253 // of matchIndex[i] ≥ N, and log[N].term == currentTerm:
254 // set commitIndex = N (§5.3, §5.4).
255 for (long N = context.getCommitIndex() + 1; ; N++) {
256 int replicatedCount = 1;
258 for (FollowerLogInformation info : followerToLog.values()) {
259 final PeerInfo peerInfo = context.getPeerInfo(info.getId());
260 if(info.getMatchIndex() >= N && (peerInfo != null && peerInfo.isVoting())) {
265 if (replicatedCount >= minReplicationCount) {
266 ReplicatedLogEntry replicatedLogEntry = context.getReplicatedLog().get(N);
267 if (replicatedLogEntry == null) {
271 // Don't update the commit index if the log entry is from a previous term, as per §5.4.1:
272 // "Raft never commits log entries from previous terms by counting replicas".
273 // However we keep looping so we can make progress when new entries in the current term
274 // reach consensus, as per §5.4.1: "once an entry from the current term is committed by
275 // counting replicas, then all prior entries are committed indirectly".
276 if (replicatedLogEntry.getTerm() == currentTerm()) {
277 context.setCommitIndex(N);
284 // Apply the change to the state machine
285 if (context.getCommitIndex() > context.getLastApplied()) {
286 if(LOG.isDebugEnabled()) {
287 LOG.debug("{}: handleAppendEntriesReply from {}: applying to log - commitIndex: {}, lastAppliedIndex: {}",
288 logName(), followerId, context.getCommitIndex(), context.getLastApplied());
291 applyLogToStateMachine(context.getCommitIndex());
294 if (!context.getSnapshotManager().isCapturing()) {
298 //Send the next log entry immediately, if possible, no need to wait for heartbeat to trigger that event
299 sendUpdatesToFollower(followerId, followerLogInformation, false, !updated);
303 private boolean updateFollowerLogInformation(FollowerLogInformation followerLogInformation,
304 AppendEntriesReply appendEntriesReply) {
305 boolean updated = followerLogInformation.setMatchIndex(appendEntriesReply.getLogLastIndex());
306 updated = followerLogInformation.setNextIndex(appendEntriesReply.getLogLastIndex() + 1) || updated;
308 if(updated && LOG.isDebugEnabled()) {
309 LOG.debug("{}: handleAppendEntriesReply - FollowerLogInformation for {} updated: matchIndex: {}, nextIndex: {}",
310 logName(), followerLogInformation.getId(), followerLogInformation.getMatchIndex(),
311 followerLogInformation.getNextIndex());
316 private void purgeInMemoryLog() {
317 //find the lowest index across followers which has been replicated to all.
318 // lastApplied if there are no followers, so that we keep clearing the log for single-node
319 // we would delete the in-mem log from that index on, in-order to minimize mem usage
320 // we would also share this info thru AE with the followers so that they can delete their log entries as well.
321 long minReplicatedToAllIndex = followerToLog.isEmpty() ? context.getLastApplied() : Long.MAX_VALUE;
322 for (FollowerLogInformation info : followerToLog.values()) {
323 minReplicatedToAllIndex = Math.min(minReplicatedToAllIndex, info.getMatchIndex());
326 super.performSnapshotWithoutCapture(minReplicatedToAllIndex);
330 protected ClientRequestTracker removeClientRequestTracker(long logIndex) {
331 final Iterator<ClientRequestTracker> it = trackerList.iterator();
332 while (it.hasNext()) {
333 final ClientRequestTracker t = it.next();
334 if (t.getIndex() == logIndex) {
344 protected ClientRequestTracker findClientRequestTracker(long logIndex) {
345 for (ClientRequestTracker tracker : trackerList) {
346 if (tracker.getIndex() == logIndex) {
354 protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
355 RequestVoteReply requestVoteReply) {
359 protected void beforeSendHeartbeat(){}
362 public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) {
363 Preconditions.checkNotNull(sender, "sender should not be null");
365 Object message = fromSerializableMessage(originalMessage);
367 if (message instanceof RaftRPC) {
368 RaftRPC rpc = (RaftRPC) message;
369 // If RPC request or response contains term T > currentTerm:
370 // set currentTerm = T, convert to follower (§5.1)
371 // This applies to all RPC messages and responses
372 if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
373 LOG.debug("{}: Term {} in \"{}\" message is greater than leader's term {} - switching to Follower",
374 logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm());
376 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
378 return internalSwitchBehavior(RaftState.Follower);
382 if (message instanceof SendHeartBeat) {
383 beforeSendHeartbeat();
385 scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
388 } else if(message instanceof SendInstallSnapshot) {
389 // received from RaftActor
390 setSnapshot(((SendInstallSnapshot) message).getSnapshot());
391 sendInstallSnapshot();
393 } else if (message instanceof Replicate) {
394 replicate((Replicate) message);
396 } else if (message instanceof InstallSnapshotReply){
397 handleInstallSnapshotReply((InstallSnapshotReply) message);
402 return super.handleMessage(sender, message);
405 private void handleInstallSnapshotReply(InstallSnapshotReply reply) {
406 LOG.debug("{}: handleInstallSnapshotReply: {}", logName(), reply);
408 String followerId = reply.getFollowerId();
409 FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
411 if (followerToSnapshot == null) {
412 LOG.error("{}: FollowerToSnapshot not found for follower {} in InstallSnapshotReply",
413 logName(), followerId);
417 FollowerLogInformation followerLogInformation = followerToLog.get(followerId);
418 if(followerLogInformation == null) {
419 // This can happen during AddServer if it times out.
420 LOG.error("{}: FollowerLogInformation not found for follower {} in InstallSnapshotReply",
421 logName(), followerId);
422 mapFollowerToSnapshot.remove(followerId);
426 followerLogInformation.markFollowerActive();
428 if (followerToSnapshot.getChunkIndex() == reply.getChunkIndex()) {
429 boolean wasLastChunk = false;
430 if (reply.isSuccess()) {
431 if(followerToSnapshot.isLastChunk(reply.getChunkIndex())) {
432 //this was the last chunk reply
433 if(LOG.isDebugEnabled()) {
434 LOG.debug("{}: InstallSnapshotReply received, " +
435 "last chunk received, Chunk: {}. Follower: {} Setting nextIndex: {}",
436 logName(), reply.getChunkIndex(), followerId,
437 context.getReplicatedLog().getSnapshotIndex() + 1
441 long followerMatchIndex = snapshot.get().getLastIncludedIndex();
442 followerLogInformation.setMatchIndex(followerMatchIndex);
443 followerLogInformation.setNextIndex(followerMatchIndex + 1);
444 mapFollowerToSnapshot.remove(followerId);
446 LOG.debug("{}: follower: {}, matchIndex set to {}, nextIndex set to {}",
447 logName(), followerId, followerLogInformation.getMatchIndex(),
448 followerLogInformation.getNextIndex());
450 if (mapFollowerToSnapshot.isEmpty()) {
451 // once there are no pending followers receiving snapshots
452 // we can remove snapshot from the memory
456 if(context.getPeerInfo(followerId).getVotingState() == VotingState.VOTING_NOT_INITIALIZED){
457 UnInitializedFollowerSnapshotReply unInitFollowerSnapshotSuccess =
458 new UnInitializedFollowerSnapshotReply(followerId);
459 context.getActor().tell(unInitFollowerSnapshotSuccess, context.getActor());
460 LOG.debug("Sent message UnInitializedFollowerSnapshotReply to self");
463 followerToSnapshot.markSendStatus(true);
466 LOG.info("{}: InstallSnapshotReply received sending snapshot chunk failed, Will retry, Chunk: {}",
467 logName(), reply.getChunkIndex());
469 followerToSnapshot.markSendStatus(false);
472 if (wasLastChunk && !context.getSnapshotManager().isCapturing()) {
473 // Since the follower is now caught up try to purge the log.
475 } else if (!wasLastChunk && followerToSnapshot.canSendNextChunk()) {
476 ActorSelection followerActor = context.getPeerActorSelection(followerId);
477 if(followerActor != null) {
478 sendSnapshotChunk(followerActor, followerId);
483 LOG.error("{}: Chunk index {} in InstallSnapshotReply from follower {} does not match expected index {}",
484 logName(), reply.getChunkIndex(), followerId,
485 followerToSnapshot.getChunkIndex());
487 if(reply.getChunkIndex() == INVALID_CHUNK_INDEX){
488 // Since the Follower did not find this index to be valid we should reset the follower snapshot
489 // so that Installing the snapshot can resume from the beginning
490 followerToSnapshot.reset();
495 private void replicate(Replicate replicate) {
496 long logIndex = replicate.getReplicatedLogEntry().getIndex();
498 LOG.debug("{}: Replicate message: identifier: {}, logIndex: {}", logName(),
499 replicate.getIdentifier(), logIndex);
501 // Create a tracker entry we will use this later to notify the
504 new ClientRequestTrackerImpl(replicate.getClientActor(),
505 replicate.getIdentifier(),
509 boolean applyModificationToState = followerToLog.isEmpty()
510 || context.getRaftPolicy().applyModificationToStateBeforeConsensus();
512 if(applyModificationToState){
513 context.setCommitIndex(logIndex);
514 applyLogToStateMachine(logIndex);
517 if (!followerToLog.isEmpty()) {
518 sendAppendEntries(0, false);
522 protected void sendAppendEntries(long timeSinceLastActivityInterval, boolean isHeartbeat) {
523 // Send an AppendEntries to all followers
524 for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
525 final String followerId = e.getKey();
526 final FollowerLogInformation followerLogInformation = e.getValue();
527 // This checks helps not to send a repeat message to the follower
528 if(!followerLogInformation.isFollowerActive() ||
529 followerLogInformation.timeSinceLastActivity() >= timeSinceLastActivityInterval) {
530 sendUpdatesToFollower(followerId, followerLogInformation, true, isHeartbeat);
537 * This method checks if any update needs to be sent to the given follower. This includes append log entries,
538 * sending next snapshot chunk, and initiating a snapshot.
539 * @return true if any update is sent, false otherwise
542 private void sendUpdatesToFollower(String followerId, FollowerLogInformation followerLogInformation,
543 boolean sendHeartbeat, boolean isHeartbeat) {
545 ActorSelection followerActor = context.getPeerActorSelection(followerId);
546 if (followerActor != null) {
547 long followerNextIndex = followerLogInformation.getNextIndex();
548 boolean isFollowerActive = followerLogInformation.isFollowerActive();
549 boolean sendAppendEntries = false;
550 List<ReplicatedLogEntry> entries = Collections.emptyList();
552 if (mapFollowerToSnapshot.get(followerId) != null) {
553 // if install snapshot is in process , then sent next chunk if possible
554 if (isFollowerActive && mapFollowerToSnapshot.get(followerId).canSendNextChunk()) {
555 sendSnapshotChunk(followerActor, followerId);
556 } else if(sendHeartbeat) {
557 // we send a heartbeat even if we have not received a reply for the last chunk
558 sendAppendEntries = true;
561 long leaderLastIndex = context.getReplicatedLog().lastIndex();
562 long leaderSnapShotIndex = context.getReplicatedLog().getSnapshotIndex();
564 if((!isHeartbeat && LOG.isDebugEnabled()) || LOG.isTraceEnabled()) {
565 LOG.debug("{}: Checking sendAppendEntries for follower {}: active: {}, followerNextIndex: {}, leaderLastIndex: {}, leaderSnapShotIndex: {}",
566 logName(), followerId, isFollowerActive, followerNextIndex, leaderLastIndex, leaderSnapShotIndex);
569 if (isFollowerActive && context.getReplicatedLog().isPresent(followerNextIndex)) {
571 LOG.debug("{}: sendAppendEntries: {} is present for follower {}", logName(),
572 followerNextIndex, followerId);
574 if(followerLogInformation.okToReplicate()) {
575 // Try to send all the entries in the journal but not exceeding the max data size
576 // for a single AppendEntries message.
577 int maxEntries = (int) context.getReplicatedLog().size();
578 entries = context.getReplicatedLog().getFrom(followerNextIndex, maxEntries,
579 context.getConfigParams().getSnapshotChunkSize());
580 sendAppendEntries = true;
582 } else if (isFollowerActive && followerNextIndex >= 0 &&
583 leaderLastIndex > followerNextIndex && !context.getSnapshotManager().isCapturing()) {
584 // if the followers next index is not present in the leaders log, and
585 // if the follower is just not starting and if leader's index is more than followers index
586 // then snapshot should be sent
588 if (LOG.isDebugEnabled()) {
589 LOG.debug(String.format("%s: InitiateInstallSnapshot to follower: %s," +
590 "follower-nextIndex: %d, leader-snapshot-index: %d, " +
591 "leader-last-index: %d", logName(), followerId,
592 followerNextIndex, leaderSnapShotIndex, leaderLastIndex));
595 // Send heartbeat to follower whenever install snapshot is initiated.
596 sendAppendEntries = true;
597 if (canInstallSnapshot(followerNextIndex)) {
598 initiateCaptureSnapshot(followerId);
601 } else if(sendHeartbeat) {
602 // we send an AppendEntries, even if the follower is inactive
603 // in-order to update the followers timestamp, in case it becomes active again
604 sendAppendEntries = true;
609 if(sendAppendEntries) {
610 sendAppendEntriesToFollower(followerActor, followerNextIndex,
611 entries, followerId);
616 private void sendAppendEntriesToFollower(ActorSelection followerActor, long followerNextIndex,
617 List<ReplicatedLogEntry> entries, String followerId) {
618 AppendEntries appendEntries = new AppendEntries(currentTerm(), context.getId(),
619 prevLogIndex(followerNextIndex),
620 prevLogTerm(followerNextIndex), entries,
621 context.getCommitIndex(), super.getReplicatedToAllIndex(), context.getPayloadVersion());
623 if(!entries.isEmpty() || LOG.isTraceEnabled()) {
624 LOG.debug("{}: Sending AppendEntries to follower {}: {}", logName(), followerId,
628 followerActor.tell(appendEntries, actor());
632 * Install Snapshot works as follows
633 * 1. Leader initiates the capture snapshot by sending a CaptureSnapshot message to actor
634 * 2. RaftActor on receipt of the CaptureSnapshotReply (from Shard), stores the received snapshot in the replicated log
635 * and makes a call to Leader's handleMessage , with SendInstallSnapshot message.
636 * 3. Leader , picks the snapshot from im-mem ReplicatedLog and sends it in chunks to the Follower
637 * 4. On complete, Follower sends back a InstallSnapshotReply.
638 * 5. On receipt of the InstallSnapshotReply for the last chunk, Leader marks the install complete for that follower
639 * and replenishes the memory by deleting the snapshot in Replicated log.
640 * 6. If another follower requires a snapshot and a snapshot has been collected (via CaptureSnapshotReply)
641 * then send the existing snapshot in chunks to the follower.
644 public boolean initiateCaptureSnapshot(String followerId) {
645 if (snapshot.isPresent()) {
646 // if a snapshot is present in the memory, most likely another install is in progress
647 // no need to capture snapshot.
648 // This could happen if another follower needs an install when one is going on.
649 final ActorSelection followerActor = context.getPeerActorSelection(followerId);
650 sendSnapshotChunk(followerActor, followerId);
653 return context.getSnapshotManager().captureToInstall(context.getReplicatedLog().last(),
654 this.getReplicatedToAllIndex(), followerId);
658 private boolean canInstallSnapshot(long nextIndex){
659 // If the follower's nextIndex is -1 then we might as well send it a snapshot
660 // Otherwise send it a snapshot only if the nextIndex is not present in the log but is present
662 return (nextIndex == -1 ||
663 (!context.getReplicatedLog().isPresent(nextIndex)
664 && context.getReplicatedLog().isInSnapshot(nextIndex)));
669 private void sendInstallSnapshot() {
670 LOG.debug("{}: sendInstallSnapshot", logName());
671 for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
672 String followerId = e.getKey();
673 ActorSelection followerActor = context.getPeerActorSelection(followerId);
674 FollowerLogInformation followerLogInfo = e.getValue();
676 if (followerActor != null) {
677 long nextIndex = followerLogInfo.getNextIndex();
678 if (context.getPeerInfo(followerId).getVotingState() == VotingState.VOTING_NOT_INITIALIZED ||
679 canInstallSnapshot(nextIndex)) {
680 sendSnapshotChunk(followerActor, followerId);
687 * Sends a snapshot chunk to a given follower
688 * InstallSnapshot should qualify as a heartbeat too.
690 private void sendSnapshotChunk(ActorSelection followerActor, String followerId) {
692 if (snapshot.isPresent()) {
693 ByteString nextSnapshotChunk = getNextSnapshotChunk(followerId, snapshot.get().getSnapshotBytes());
695 // Note: the previous call to getNextSnapshotChunk has the side-effect of adding
696 // followerId to the followerToSnapshot map.
697 FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
700 new InstallSnapshot(currentTerm(), context.getId(),
701 snapshot.get().getLastIncludedIndex(),
702 snapshot.get().getLastIncludedTerm(),
704 followerToSnapshot.incrementChunkIndex(),
705 followerToSnapshot.getTotalChunks(),
706 Optional.of(followerToSnapshot.getLastChunkHashCode())
711 if(LOG.isDebugEnabled()) {
712 LOG.debug("{}: InstallSnapshot sent to follower {}, Chunk: {}/{}",
713 logName(), followerActor.path(), followerToSnapshot.getChunkIndex(),
714 followerToSnapshot.getTotalChunks());
717 } catch (IOException e) {
718 LOG.error("{}: InstallSnapshot failed for Leader.", logName(), e);
723 * Acccepts snaphot as ByteString, enters into map for future chunks
724 * creates and return a ByteString chunk
726 private ByteString getNextSnapshotChunk(String followerId, ByteString snapshotBytes) throws IOException {
727 FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
728 if (followerToSnapshot == null) {
729 followerToSnapshot = new FollowerToSnapshot(snapshotBytes);
730 mapFollowerToSnapshot.put(followerId, followerToSnapshot);
732 ByteString nextChunk = followerToSnapshot.getNextChunk();
734 LOG.debug("{}: next snapshot chunk size for follower {}: {}", logName(), followerId, nextChunk.size());
739 private void sendHeartBeat() {
740 if (!followerToLog.isEmpty()) {
741 LOG.trace("{}: Sending heartbeat", logName());
742 sendAppendEntries(context.getConfigParams().getHeartBeatInterval().toMillis(), true);
746 private void stopHeartBeat() {
747 if (heartbeatSchedule != null && !heartbeatSchedule.isCancelled()) {
748 heartbeatSchedule.cancel();
752 private void scheduleHeartBeat(FiniteDuration interval) {
753 if (followerToLog.isEmpty()) {
754 // Optimization - do not bother scheduling a heartbeat as there are
761 // Schedule a heartbeat. When the scheduler triggers a SendHeartbeat
762 // message is sent to itself.
763 // Scheduling the heartbeat only once here because heartbeats do not
764 // need to be sent if there are other messages being sent to the remote
766 heartbeatSchedule = context.getActorSystem().scheduler().scheduleOnce(
767 interval, context.getActor(), new SendHeartBeat(),
768 context.getActorSystem().dispatcher(), context.getActor());
772 public void close() throws Exception {
777 public String getLeaderId() {
778 return context.getId();
781 protected boolean isLeaderIsolated() {
782 int minPresent = getMinIsolatedLeaderPeerCount();
783 for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
784 if (followerLogInformation.isFollowerActive()) {
786 if (minPresent == 0) {
791 return (minPresent != 0);
795 * Encapsulates the snapshot bytestring and handles the logic of sending
798 protected class FollowerToSnapshot {
799 private final ByteString snapshotBytes;
800 private int offset = 0;
801 // the next snapshot chunk is sent only if the replyReceivedForOffset matches offset
802 private int replyReceivedForOffset;
803 // if replyStatus is false, the previous chunk is attempted
804 private boolean replyStatus = false;
805 private int chunkIndex;
806 private final int totalChunks;
807 private int lastChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
808 private int nextChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
810 public FollowerToSnapshot(ByteString snapshotBytes) {
811 this.snapshotBytes = snapshotBytes;
812 int size = snapshotBytes.size();
813 totalChunks = ( size / context.getConfigParams().getSnapshotChunkSize()) +
814 ((size % context.getConfigParams().getSnapshotChunkSize()) > 0 ? 1 : 0);
815 if(LOG.isDebugEnabled()) {
816 LOG.debug("{}: Snapshot {} bytes, total chunks to send:{}",
817 logName(), size, totalChunks);
819 replyReceivedForOffset = -1;
820 chunkIndex = AbstractLeader.FIRST_CHUNK_INDEX;
823 public ByteString getSnapshotBytes() {
824 return snapshotBytes;
827 public int incrementOffset() {
829 // if prev chunk failed, we would want to sent the same chunk again
830 offset = offset + context.getConfigParams().getSnapshotChunkSize();
835 public int incrementChunkIndex() {
837 // if prev chunk failed, we would want to sent the same chunk again
838 chunkIndex = chunkIndex + 1;
843 public int getChunkIndex() {
847 public int getTotalChunks() {
851 public boolean canSendNextChunk() {
852 // we only send a false if a chunk is sent but we have not received a reply yet
853 return replyReceivedForOffset == offset;
856 public boolean isLastChunk(int chunkIndex) {
857 return totalChunks == chunkIndex;
860 public void markSendStatus(boolean success) {
862 // if the chunk sent was successful
863 replyReceivedForOffset = offset;
865 lastChunkHashCode = nextChunkHashCode;
867 // if the chunk sent was failure
868 replyReceivedForOffset = offset;
873 public ByteString getNextChunk() {
874 int snapshotLength = getSnapshotBytes().size();
875 int start = incrementOffset();
876 int size = context.getConfigParams().getSnapshotChunkSize();
877 if (context.getConfigParams().getSnapshotChunkSize() > snapshotLength) {
878 size = snapshotLength;
880 if ((start + context.getConfigParams().getSnapshotChunkSize()) > snapshotLength) {
881 size = snapshotLength - start;
886 LOG.debug("{}: Next chunk: length={}, offset={},size={}", logName(),
887 snapshotLength, start, size);
889 ByteString substring = getSnapshotBytes().substring(start, start + size);
890 nextChunkHashCode = substring.hashCode();
895 * reset should be called when the Follower needs to be sent the snapshot from the beginning
900 replyReceivedForOffset = offset;
901 chunkIndex = AbstractLeader.FIRST_CHUNK_INDEX;
902 lastChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
905 public int getLastChunkHashCode() {
906 return lastChunkHashCode;
910 // called from example-actor for printing the follower-states
911 public String printFollowerStates() {
912 final StringBuilder sb = new StringBuilder();
915 for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
917 sb.append(followerLogInformation.getId());
918 sb.append(" state:");
919 sb.append(followerLogInformation.isFollowerActive());
924 return sb.toString();
928 public FollowerLogInformation getFollower(String followerId) {
929 return followerToLog.get(followerId);
933 protected void setFollowerSnapshot(String followerId, FollowerToSnapshot snapshot) {
934 mapFollowerToSnapshot.put(followerId, snapshot);
938 public int followerSnapshotSize() {
939 return mapFollowerToSnapshot.size();
943 public int followerLogSize() {
944 return followerToLog.size();
947 private static class SnapshotHolder {
948 private final long lastIncludedTerm;
949 private final long lastIncludedIndex;
950 private final ByteString snapshotBytes;
952 SnapshotHolder(Snapshot snapshot) {
953 this.lastIncludedTerm = snapshot.getLastAppliedTerm();
954 this.lastIncludedIndex = snapshot.getLastAppliedIndex();
955 this.snapshotBytes = ByteString.copyFrom(snapshot.getState());
958 long getLastIncludedTerm() {
959 return lastIncludedTerm;
962 long getLastIncludedIndex() {
963 return lastIncludedIndex;
966 ByteString getSnapshotBytes() {
967 return snapshotBytes;