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 javax.annotation.Nullable;
29 import org.opendaylight.controller.cluster.raft.ClientRequestTracker;
30 import org.opendaylight.controller.cluster.raft.ClientRequestTrackerImpl;
31 import org.opendaylight.controller.cluster.raft.FollowerLogInformation;
32 import org.opendaylight.controller.cluster.raft.FollowerLogInformationImpl;
33 import org.opendaylight.controller.cluster.raft.PeerInfo;
34 import org.opendaylight.controller.cluster.raft.RaftActorContext;
35 import org.opendaylight.controller.cluster.raft.RaftState;
36 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
37 import org.opendaylight.controller.cluster.raft.Snapshot;
38 import org.opendaylight.controller.cluster.raft.VotingState;
39 import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
40 import org.opendaylight.controller.cluster.raft.base.messages.SendHeartBeat;
41 import org.opendaylight.controller.cluster.raft.base.messages.SendInstallSnapshot;
42 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
43 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
44 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot;
45 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshotReply;
46 import org.opendaylight.controller.cluster.raft.messages.RaftRPC;
47 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
48 import org.opendaylight.controller.cluster.raft.messages.UnInitializedFollowerSnapshotReply;
49 import scala.concurrent.duration.FiniteDuration;
52 * The behavior of a RaftActor when it is in the Leader state
56 * <li> Upon election: send initial empty AppendEntries RPCs
57 * (heartbeat) to each server; repeat during idle periods to
58 * prevent election timeouts (§5.2)
59 * <li> If command received from client: append entry to local log,
60 * respond after entry applied to state machine (§5.3)
61 * <li> If last log index ≥ nextIndex for a follower: send
62 * AppendEntries RPC with log entries starting at nextIndex
64 * <li> If successful: update nextIndex and matchIndex for
66 * <li> If AppendEntries fails because of log inconsistency:
67 * decrement nextIndex and retry (§5.3)
69 * <li> If there exists an N such that N > commitIndex, a majority
70 * of matchIndex[i] ≥ N, and log[N].term == currentTerm:
71 * set commitIndex = N (§5.3, §5.4).
73 public abstract class AbstractLeader extends AbstractRaftActorBehavior {
75 // The index of the first chunk that is sent when installing a snapshot
76 public static final int FIRST_CHUNK_INDEX = 1;
78 // The index that the follower should respond with if it needs the install snapshot to be reset
79 public static final int INVALID_CHUNK_INDEX = -1;
81 // This would be passed as the hash code of the last chunk when sending the first chunk
82 public static final int INITIAL_LAST_CHUNK_HASH_CODE = -1;
84 private final Map<String, FollowerLogInformation> followerToLog = new HashMap<>();
85 private final Map<String, FollowerToSnapshot> mapFollowerToSnapshot = new HashMap<>();
87 private Cancellable heartbeatSchedule = null;
89 private final Collection<ClientRequestTracker> trackerList = new LinkedList<>();
91 private int minReplicationCount;
93 private Optional<SnapshotHolder> snapshot;
95 protected AbstractLeader(RaftActorContext context, RaftState state) {
96 super(context, state);
98 setLeaderPayloadVersion(context.getPayloadVersion());
100 for(PeerInfo peerInfo: context.getPeers()) {
101 FollowerLogInformation followerLogInformation = new FollowerLogInformationImpl(peerInfo, -1, context);
102 followerToLog.put(peerInfo.getId(), followerLogInformation);
105 leaderId = context.getId();
107 LOG.debug("{}: Election: Leader has following peers: {}", logName(), getFollowerIds());
109 updateMinReplicaCount();
111 snapshot = Optional.absent();
113 // Immediately schedule a heartbeat
114 // Upon election: send initial empty AppendEntries RPCs
115 // (heartbeat) to each server; repeat during idle periods to
116 // prevent election timeouts (§5.2)
117 sendAppendEntries(0, false);
119 // It is important to schedule this heartbeat here
120 scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
124 * Return an immutable collection of follower identifiers.
126 * @return Collection of follower IDs
128 public final Collection<String> getFollowerIds() {
129 return followerToLog.keySet();
132 public void addFollower(String followerId) {
133 FollowerLogInformation followerLogInformation = new FollowerLogInformationImpl(
134 context.getPeerInfo(followerId), -1, context);
135 followerToLog.put(followerId, followerLogInformation);
137 if(heartbeatSchedule == null) {
138 scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
142 public void removeFollower(String followerId) {
143 followerToLog.remove(followerId);
144 mapFollowerToSnapshot.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 Snapshot snapshot) {
171 if(snapshot != null) {
172 this.snapshot = Optional.of(new SnapshotHolder(snapshot));
174 this.snapshot = Optional.absent();
179 protected RaftActorBehavior handleAppendEntries(ActorRef sender,
180 AppendEntries appendEntries) {
182 LOG.debug("{}: handleAppendEntries: {}", logName(), appendEntries);
188 protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
189 AppendEntriesReply appendEntriesReply) {
191 if(LOG.isTraceEnabled()) {
192 LOG.trace("{}: handleAppendEntriesReply: {}", logName(), appendEntriesReply);
195 // Update the FollowerLogInformation
196 String followerId = appendEntriesReply.getFollowerId();
197 FollowerLogInformation followerLogInformation =
198 followerToLog.get(followerId);
200 if(followerLogInformation == null){
201 LOG.error("{}: handleAppendEntriesReply - unknown follower {}", logName(), followerId);
205 if(followerLogInformation.timeSinceLastActivity() >
206 context.getConfigParams().getElectionTimeOutInterval().toMillis()) {
207 LOG.warn("{} : handleAppendEntriesReply delayed beyond election timeout, " +
208 "appendEntriesReply : {}, timeSinceLastActivity : {}, lastApplied : {}, commitIndex : {}",
209 logName(), appendEntriesReply, followerLogInformation.timeSinceLastActivity(),
210 context.getLastApplied(), context.getCommitIndex());
213 followerLogInformation.markFollowerActive();
214 followerLogInformation.setPayloadVersion(appendEntriesReply.getPayloadVersion());
215 followerLogInformation.setRaftVersion(appendEntriesReply.getRaftVersion());
217 boolean updated = false;
218 if (appendEntriesReply.isSuccess()) {
219 updated = updateFollowerLogInformation(followerLogInformation, appendEntriesReply);
221 LOG.debug("{}: handleAppendEntriesReply: received unsuccessful reply: {}", logName(), appendEntriesReply);
223 long followerLastLogIndex = appendEntriesReply.getLogLastIndex();
224 ReplicatedLogEntry followersLastLogEntry = context.getReplicatedLog().get(followerLastLogIndex);
225 if(appendEntriesReply.isForceInstallSnapshot()) {
226 // Reset the followers match and next index. This is to signal that this follower has nothing
227 // in common with this Leader and so would require a snapshot to be installed
228 followerLogInformation.setMatchIndex(-1);
229 followerLogInformation.setNextIndex(-1);
231 // Force initiate a snapshot capture
232 initiateCaptureSnapshot(followerId);
233 } else if(followerLastLogIndex < 0 || (followersLastLogEntry != null &&
234 followersLastLogEntry.getTerm() == appendEntriesReply.getLogLastTerm())) {
235 // The follower's log is empty or the last entry is present in the leader's journal
236 // and the terms match so the follower is just behind the leader's journal from
237 // the last snapshot, if any. We'll catch up the follower quickly by starting at the
238 // follower's last log index.
240 updated = updateFollowerLogInformation(followerLogInformation, appendEntriesReply);
242 // TODO: When we find that the follower is out of sync with the
243 // Leader we simply decrement that followers next index by 1.
244 // Would it be possible to do better than this? The RAFT spec
245 // does not explicitly deal with it but may be something for us to
248 followerLogInformation.decrNextIndex();
252 // Now figure out if this reply warrants a change in the commitIndex
253 // If there exists an N such that N > commitIndex, a majority
254 // of matchIndex[i] ≥ N, and log[N].term == currentTerm:
255 // set commitIndex = N (§5.3, §5.4).
256 for (long N = context.getCommitIndex() + 1; ; N++) {
257 int replicatedCount = 1;
259 for (FollowerLogInformation info : followerToLog.values()) {
260 final PeerInfo peerInfo = context.getPeerInfo(info.getId());
261 if(info.getMatchIndex() >= N && (peerInfo != null && peerInfo.isVoting())) {
266 if (replicatedCount >= minReplicationCount) {
267 ReplicatedLogEntry replicatedLogEntry = context.getReplicatedLog().get(N);
268 if (replicatedLogEntry == null) {
272 // Don't update the commit index if the log entry is from a previous term, as per §5.4.1:
273 // "Raft never commits log entries from previous terms by counting replicas".
274 // However we keep looping so we can make progress when new entries in the current term
275 // reach consensus, as per §5.4.1: "once an entry from the current term is committed by
276 // counting replicas, then all prior entries are committed indirectly".
277 if (replicatedLogEntry.getTerm() == currentTerm()) {
278 context.setCommitIndex(N);
285 // Apply the change to the state machine
286 if (context.getCommitIndex() > context.getLastApplied()) {
287 if(LOG.isDebugEnabled()) {
288 LOG.debug("{}: handleAppendEntriesReply from {}: applying to log - commitIndex: {}, lastAppliedIndex: {}",
289 logName(), followerId, context.getCommitIndex(), context.getLastApplied());
292 applyLogToStateMachine(context.getCommitIndex());
295 if (!context.getSnapshotManager().isCapturing()) {
299 //Send the next log entry immediately, if possible, no need to wait for heartbeat to trigger that event
300 sendUpdatesToFollower(followerId, followerLogInformation, false, !updated);
304 private boolean updateFollowerLogInformation(FollowerLogInformation followerLogInformation,
305 AppendEntriesReply appendEntriesReply) {
306 boolean updated = followerLogInformation.setMatchIndex(appendEntriesReply.getLogLastIndex());
307 updated = followerLogInformation.setNextIndex(appendEntriesReply.getLogLastIndex() + 1) || updated;
309 if(updated && LOG.isDebugEnabled()) {
310 LOG.debug("{}: handleAppendEntriesReply - FollowerLogInformation for {} updated: matchIndex: {}, nextIndex: {}",
311 logName(), followerLogInformation.getId(), followerLogInformation.getMatchIndex(),
312 followerLogInformation.getNextIndex());
317 private void purgeInMemoryLog() {
318 //find the lowest index across followers which has been replicated to all.
319 // lastApplied if there are no followers, so that we keep clearing the log for single-node
320 // we would delete the in-mem log from that index on, in-order to minimize mem usage
321 // we would also share this info thru AE with the followers so that they can delete their log entries as well.
322 long minReplicatedToAllIndex = followerToLog.isEmpty() ? context.getLastApplied() : Long.MAX_VALUE;
323 for (FollowerLogInformation info : followerToLog.values()) {
324 minReplicatedToAllIndex = Math.min(minReplicatedToAllIndex, info.getMatchIndex());
327 super.performSnapshotWithoutCapture(minReplicatedToAllIndex);
331 protected ClientRequestTracker removeClientRequestTracker(long logIndex) {
332 final Iterator<ClientRequestTracker> it = trackerList.iterator();
333 while (it.hasNext()) {
334 final ClientRequestTracker t = it.next();
335 if (t.getIndex() == logIndex) {
345 protected ClientRequestTracker findClientRequestTracker(long logIndex) {
346 for (ClientRequestTracker tracker : trackerList) {
347 if (tracker.getIndex() == logIndex) {
355 protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
356 RequestVoteReply requestVoteReply) {
360 protected void beforeSendHeartbeat(){}
363 public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) {
364 Preconditions.checkNotNull(sender, "sender should not be null");
366 Object message = fromSerializableMessage(originalMessage);
368 if (message instanceof RaftRPC) {
369 RaftRPC rpc = (RaftRPC) message;
370 // If RPC request or response contains term T > currentTerm:
371 // set currentTerm = T, convert to follower (§5.1)
372 // This applies to all RPC messages and responses
373 if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
374 LOG.debug("{}: Term {} in \"{}\" message is greater than leader's term {} - switching to Follower",
375 logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm());
377 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
379 return internalSwitchBehavior(RaftState.Follower);
383 if (message instanceof SendHeartBeat) {
384 beforeSendHeartbeat();
386 scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
389 } else if(message instanceof SendInstallSnapshot) {
390 // received from RaftActor
391 setSnapshot(((SendInstallSnapshot) message).getSnapshot());
392 sendInstallSnapshot();
394 } else if (message instanceof Replicate) {
395 replicate((Replicate) message);
397 } else if (message instanceof InstallSnapshotReply){
398 handleInstallSnapshotReply((InstallSnapshotReply) message);
403 return super.handleMessage(sender, message);
406 private void handleInstallSnapshotReply(InstallSnapshotReply reply) {
407 LOG.debug("{}: handleInstallSnapshotReply: {}", logName(), reply);
409 String followerId = reply.getFollowerId();
410 FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
412 if (followerToSnapshot == null) {
413 LOG.error("{}: FollowerToSnapshot not found for follower {} in InstallSnapshotReply",
414 logName(), followerId);
418 FollowerLogInformation followerLogInformation = followerToLog.get(followerId);
419 if(followerLogInformation == null) {
420 // This can happen during AddServer if it times out.
421 LOG.error("{}: FollowerLogInformation not found for follower {} in InstallSnapshotReply",
422 logName(), followerId);
423 mapFollowerToSnapshot.remove(followerId);
427 followerLogInformation.markFollowerActive();
429 if (followerToSnapshot.getChunkIndex() == reply.getChunkIndex()) {
430 boolean wasLastChunk = false;
431 if (reply.isSuccess()) {
432 if(followerToSnapshot.isLastChunk(reply.getChunkIndex())) {
433 //this was the last chunk reply
434 if(LOG.isDebugEnabled()) {
435 LOG.debug("{}: InstallSnapshotReply received, " +
436 "last chunk received, Chunk: {}. Follower: {} Setting nextIndex: {}",
437 logName(), reply.getChunkIndex(), followerId,
438 context.getReplicatedLog().getSnapshotIndex() + 1
442 long followerMatchIndex = snapshot.get().getLastIncludedIndex();
443 followerLogInformation.setMatchIndex(followerMatchIndex);
444 followerLogInformation.setNextIndex(followerMatchIndex + 1);
445 mapFollowerToSnapshot.remove(followerId);
447 LOG.debug("{}: follower: {}, matchIndex set to {}, nextIndex set to {}",
448 logName(), followerId, followerLogInformation.getMatchIndex(),
449 followerLogInformation.getNextIndex());
451 if (mapFollowerToSnapshot.isEmpty()) {
452 // once there are no pending followers receiving snapshots
453 // we can remove snapshot from the memory
457 if(context.getPeerInfo(followerId).getVotingState() == VotingState.VOTING_NOT_INITIALIZED){
458 UnInitializedFollowerSnapshotReply unInitFollowerSnapshotSuccess =
459 new UnInitializedFollowerSnapshotReply(followerId);
460 context.getActor().tell(unInitFollowerSnapshotSuccess, context.getActor());
461 LOG.debug("Sent message UnInitializedFollowerSnapshotReply to self");
464 followerToSnapshot.markSendStatus(true);
467 LOG.info("{}: InstallSnapshotReply received sending snapshot chunk failed, Will retry, Chunk: {}",
468 logName(), reply.getChunkIndex());
470 followerToSnapshot.markSendStatus(false);
473 if (wasLastChunk && !context.getSnapshotManager().isCapturing()) {
474 // Since the follower is now caught up try to purge the log.
476 } else if (!wasLastChunk && followerToSnapshot.canSendNextChunk()) {
477 ActorSelection followerActor = context.getPeerActorSelection(followerId);
478 if(followerActor != null) {
479 sendSnapshotChunk(followerActor, followerId);
484 LOG.error("{}: Chunk index {} in InstallSnapshotReply from follower {} does not match expected index {}",
485 logName(), reply.getChunkIndex(), followerId,
486 followerToSnapshot.getChunkIndex());
488 if(reply.getChunkIndex() == INVALID_CHUNK_INDEX){
489 // Since the Follower did not find this index to be valid we should reset the follower snapshot
490 // so that Installing the snapshot can resume from the beginning
491 followerToSnapshot.reset();
496 private void replicate(Replicate replicate) {
497 long logIndex = replicate.getReplicatedLogEntry().getIndex();
499 LOG.debug("{}: Replicate message: identifier: {}, logIndex: {}", logName(),
500 replicate.getIdentifier(), logIndex);
502 // Create a tracker entry we will use this later to notify the
505 new ClientRequestTrackerImpl(replicate.getClientActor(),
506 replicate.getIdentifier(),
510 boolean applyModificationToState = followerToLog.isEmpty()
511 || context.getRaftPolicy().applyModificationToStateBeforeConsensus();
513 if(applyModificationToState){
514 context.setCommitIndex(logIndex);
515 applyLogToStateMachine(logIndex);
518 if (!followerToLog.isEmpty()) {
519 sendAppendEntries(0, false);
523 protected void sendAppendEntries(long timeSinceLastActivityInterval, boolean isHeartbeat) {
524 // Send an AppendEntries to all followers
525 for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
526 final String followerId = e.getKey();
527 final FollowerLogInformation followerLogInformation = e.getValue();
528 // This checks helps not to send a repeat message to the follower
529 if(!followerLogInformation.isFollowerActive() ||
530 followerLogInformation.timeSinceLastActivity() >= timeSinceLastActivityInterval) {
531 sendUpdatesToFollower(followerId, followerLogInformation, true, isHeartbeat);
538 * This method checks if any update needs to be sent to the given follower. This includes append log entries,
539 * sending next snapshot chunk, and initiating a snapshot.
540 * @return true if any update is sent, false otherwise
543 private void sendUpdatesToFollower(String followerId, FollowerLogInformation followerLogInformation,
544 boolean sendHeartbeat, boolean isHeartbeat) {
546 ActorSelection followerActor = context.getPeerActorSelection(followerId);
547 if (followerActor != null) {
548 long followerNextIndex = followerLogInformation.getNextIndex();
549 boolean isFollowerActive = followerLogInformation.isFollowerActive();
550 boolean sendAppendEntries = false;
551 List<ReplicatedLogEntry> entries = Collections.emptyList();
553 if (mapFollowerToSnapshot.get(followerId) != null) {
554 // if install snapshot is in process , then sent next chunk if possible
555 if (isFollowerActive && mapFollowerToSnapshot.get(followerId).canSendNextChunk()) {
556 sendSnapshotChunk(followerActor, followerId);
557 } else if(sendHeartbeat) {
558 // we send a heartbeat even if we have not received a reply for the last chunk
559 sendAppendEntries = true;
562 long leaderLastIndex = context.getReplicatedLog().lastIndex();
563 long leaderSnapShotIndex = context.getReplicatedLog().getSnapshotIndex();
565 if((!isHeartbeat && LOG.isDebugEnabled()) || LOG.isTraceEnabled()) {
566 LOG.debug("{}: Checking sendAppendEntries for follower {}: active: {}, followerNextIndex: {}, leaderLastIndex: {}, leaderSnapShotIndex: {}",
567 logName(), followerId, isFollowerActive, followerNextIndex, leaderLastIndex, leaderSnapShotIndex);
570 if (isFollowerActive && context.getReplicatedLog().isPresent(followerNextIndex)) {
572 LOG.debug("{}: sendAppendEntries: {} is present for follower {}", logName(),
573 followerNextIndex, followerId);
575 if(followerLogInformation.okToReplicate()) {
576 // Try to send all the entries in the journal but not exceeding the max data size
577 // for a single AppendEntries message.
578 int maxEntries = (int) context.getReplicatedLog().size();
579 entries = context.getReplicatedLog().getFrom(followerNextIndex, maxEntries,
580 context.getConfigParams().getSnapshotChunkSize());
581 sendAppendEntries = true;
583 } else if (isFollowerActive && followerNextIndex >= 0 &&
584 leaderLastIndex > followerNextIndex && !context.getSnapshotManager().isCapturing()) {
585 // if the followers next index is not present in the leaders log, and
586 // if the follower is just not starting and if leader's index is more than followers index
587 // then snapshot should be sent
589 if (LOG.isDebugEnabled()) {
590 LOG.debug(String.format("%s: InitiateInstallSnapshot to follower: %s," +
591 "follower-nextIndex: %d, leader-snapshot-index: %d, " +
592 "leader-last-index: %d", logName(), followerId,
593 followerNextIndex, leaderSnapShotIndex, leaderLastIndex));
596 // Send heartbeat to follower whenever install snapshot is initiated.
597 sendAppendEntries = true;
598 if (canInstallSnapshot(followerNextIndex)) {
599 initiateCaptureSnapshot(followerId);
602 } else if(sendHeartbeat) {
603 // we send an AppendEntries, even if the follower is inactive
604 // in-order to update the followers timestamp, in case it becomes active again
605 sendAppendEntries = true;
610 if(sendAppendEntries) {
611 sendAppendEntriesToFollower(followerActor, followerNextIndex,
612 entries, followerId);
617 private void sendAppendEntriesToFollower(ActorSelection followerActor, long followerNextIndex,
618 List<ReplicatedLogEntry> entries, String followerId) {
619 AppendEntries appendEntries = new AppendEntries(currentTerm(), context.getId(),
620 prevLogIndex(followerNextIndex),
621 prevLogTerm(followerNextIndex), entries,
622 context.getCommitIndex(), super.getReplicatedToAllIndex(), context.getPayloadVersion());
624 if(!entries.isEmpty() || LOG.isTraceEnabled()) {
625 LOG.debug("{}: Sending AppendEntries to follower {}: {}", logName(), followerId,
629 followerActor.tell(appendEntries, actor());
633 * Install Snapshot works as follows
634 * 1. Leader initiates the capture snapshot by sending a CaptureSnapshot message to actor
635 * 2. RaftActor on receipt of the CaptureSnapshotReply (from Shard), stores the received snapshot in the replicated log
636 * and makes a call to Leader's handleMessage , with SendInstallSnapshot message.
637 * 3. Leader , picks the snapshot from im-mem ReplicatedLog and sends it in chunks to the Follower
638 * 4. On complete, Follower sends back a InstallSnapshotReply.
639 * 5. On receipt of the InstallSnapshotReply for the last chunk, Leader marks the install complete for that follower
640 * and replenishes the memory by deleting the snapshot in Replicated log.
641 * 6. If another follower requires a snapshot and a snapshot has been collected (via CaptureSnapshotReply)
642 * then send the existing snapshot in chunks to the follower.
645 public boolean initiateCaptureSnapshot(String followerId) {
646 if (snapshot.isPresent()) {
647 // if a snapshot is present in the memory, most likely another install is in progress
648 // no need to capture snapshot.
649 // This could happen if another follower needs an install when one is going on.
650 final ActorSelection followerActor = context.getPeerActorSelection(followerId);
651 sendSnapshotChunk(followerActor, followerId);
654 return context.getSnapshotManager().captureToInstall(context.getReplicatedLog().last(),
655 this.getReplicatedToAllIndex(), followerId);
659 private boolean canInstallSnapshot(long nextIndex){
660 // If the follower's nextIndex is -1 then we might as well send it a snapshot
661 // Otherwise send it a snapshot only if the nextIndex is not present in the log but is present
663 return (nextIndex == -1 ||
664 (!context.getReplicatedLog().isPresent(nextIndex)
665 && context.getReplicatedLog().isInSnapshot(nextIndex)));
670 private void sendInstallSnapshot() {
671 LOG.debug("{}: sendInstallSnapshot", logName());
672 for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
673 String followerId = e.getKey();
674 ActorSelection followerActor = context.getPeerActorSelection(followerId);
675 FollowerLogInformation followerLogInfo = e.getValue();
677 if (followerActor != null) {
678 long nextIndex = followerLogInfo.getNextIndex();
679 if (context.getPeerInfo(followerId).getVotingState() == VotingState.VOTING_NOT_INITIALIZED ||
680 canInstallSnapshot(nextIndex)) {
681 sendSnapshotChunk(followerActor, followerId);
688 * Sends a snapshot chunk to a given follower
689 * InstallSnapshot should qualify as a heartbeat too.
691 private void sendSnapshotChunk(ActorSelection followerActor, String followerId) {
693 if (snapshot.isPresent()) {
694 byte[] nextSnapshotChunk = getNextSnapshotChunk(followerId, snapshot.get().getSnapshotBytes());
696 // Note: the previous call to getNextSnapshotChunk has the side-effect of adding
697 // followerId to the followerToSnapshot map.
698 FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
701 new InstallSnapshot(currentTerm(), context.getId(),
702 snapshot.get().getLastIncludedIndex(),
703 snapshot.get().getLastIncludedTerm(),
705 followerToSnapshot.incrementChunkIndex(),
706 followerToSnapshot.getTotalChunks(),
707 Optional.of(followerToSnapshot.getLastChunkHashCode())
708 ).toSerializable(followerToLog.get(followerId).getRaftVersion()),
712 if(LOG.isDebugEnabled()) {
713 LOG.debug("{}: InstallSnapshot sent to follower {}, Chunk: {}/{}",
714 logName(), followerActor.path(), followerToSnapshot.getChunkIndex(),
715 followerToSnapshot.getTotalChunks());
718 } catch (IOException e) {
719 LOG.error("{}: InstallSnapshot failed for Leader.", logName(), e);
724 * Acccepts snaphot as ByteString, enters into map for future chunks
725 * creates and return a ByteString chunk
727 private byte[] getNextSnapshotChunk(String followerId, ByteString snapshotBytes) throws IOException {
728 FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
729 if (followerToSnapshot == null) {
730 followerToSnapshot = new FollowerToSnapshot(snapshotBytes);
731 mapFollowerToSnapshot.put(followerId, followerToSnapshot);
733 byte[] nextChunk = followerToSnapshot.getNextChunk();
735 LOG.debug("{}: next snapshot chunk size for follower {}: {}", logName(), followerId, nextChunk.length);
740 private void sendHeartBeat() {
741 if (!followerToLog.isEmpty()) {
742 LOG.trace("{}: Sending heartbeat", logName());
743 sendAppendEntries(context.getConfigParams().getHeartBeatInterval().toMillis(), true);
747 private void stopHeartBeat() {
748 if (heartbeatSchedule != null && !heartbeatSchedule.isCancelled()) {
749 heartbeatSchedule.cancel();
753 private void scheduleHeartBeat(FiniteDuration interval) {
754 if (followerToLog.isEmpty()) {
755 // Optimization - do not bother scheduling a heartbeat as there are
762 // Schedule a heartbeat. When the scheduler triggers a SendHeartbeat
763 // message is sent to itself.
764 // Scheduling the heartbeat only once here because heartbeats do not
765 // need to be sent if there are other messages being sent to the remote
767 heartbeatSchedule = context.getActorSystem().scheduler().scheduleOnce(
768 interval, context.getActor(), SendHeartBeat.INSTANCE,
769 context.getActorSystem().dispatcher(), context.getActor());
773 public void close() {
778 public String getLeaderId() {
779 return context.getId();
782 protected boolean isLeaderIsolated() {
783 int minPresent = getMinIsolatedLeaderPeerCount();
784 for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
785 if (followerLogInformation.isFollowerActive()) {
787 if (minPresent == 0) {
792 return (minPresent != 0);
796 * Encapsulates the snapshot bytestring and handles the logic of sending
799 protected class FollowerToSnapshot {
800 private final ByteString snapshotBytes;
801 private int offset = 0;
802 // the next snapshot chunk is sent only if the replyReceivedForOffset matches offset
803 private int replyReceivedForOffset;
804 // if replyStatus is false, the previous chunk is attempted
805 private boolean replyStatus = false;
806 private int chunkIndex;
807 private final int totalChunks;
808 private int lastChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
809 private int nextChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
811 public FollowerToSnapshot(ByteString snapshotBytes) {
812 this.snapshotBytes = snapshotBytes;
813 int size = snapshotBytes.size();
814 totalChunks = ( size / context.getConfigParams().getSnapshotChunkSize()) +
815 ((size % context.getConfigParams().getSnapshotChunkSize()) > 0 ? 1 : 0);
816 if(LOG.isDebugEnabled()) {
817 LOG.debug("{}: Snapshot {} bytes, total chunks to send:{}",
818 logName(), size, totalChunks);
820 replyReceivedForOffset = -1;
821 chunkIndex = AbstractLeader.FIRST_CHUNK_INDEX;
824 public ByteString getSnapshotBytes() {
825 return snapshotBytes;
828 public int incrementOffset() {
830 // if prev chunk failed, we would want to sent the same chunk again
831 offset = offset + context.getConfigParams().getSnapshotChunkSize();
836 public int incrementChunkIndex() {
838 // if prev chunk failed, we would want to sent the same chunk again
839 chunkIndex = chunkIndex + 1;
844 public int getChunkIndex() {
848 public int getTotalChunks() {
852 public boolean canSendNextChunk() {
853 // we only send a false if a chunk is sent but we have not received a reply yet
854 return replyReceivedForOffset == offset;
857 public boolean isLastChunk(int chunkIndex) {
858 return totalChunks == chunkIndex;
861 public void markSendStatus(boolean success) {
863 // if the chunk sent was successful
864 replyReceivedForOffset = offset;
866 lastChunkHashCode = nextChunkHashCode;
868 // if the chunk sent was failure
869 replyReceivedForOffset = offset;
874 public byte[] getNextChunk() {
875 int snapshotLength = getSnapshotBytes().size();
876 int start = incrementOffset();
877 int size = context.getConfigParams().getSnapshotChunkSize();
878 if (context.getConfigParams().getSnapshotChunkSize() > snapshotLength) {
879 size = snapshotLength;
880 } else if ((start + context.getConfigParams().getSnapshotChunkSize()) > snapshotLength) {
881 size = snapshotLength - start;
884 byte[] nextChunk = new byte[size];
885 getSnapshotBytes().copyTo(nextChunk, start, 0, size);
886 nextChunkHashCode = Arrays.hashCode(nextChunk);
888 LOG.debug("{}: Next chunk: total length={}, offset={}, size={}, hashCode={}", logName(),
889 snapshotLength, start, size, nextChunkHashCode);
894 * reset should be called when the Follower needs to be sent the snapshot from the beginning
899 replyReceivedForOffset = offset;
900 chunkIndex = AbstractLeader.FIRST_CHUNK_INDEX;
901 lastChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
904 public int getLastChunkHashCode() {
905 return lastChunkHashCode;
909 // called from example-actor for printing the follower-states
910 public String printFollowerStates() {
911 final StringBuilder sb = new StringBuilder();
914 for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
916 sb.append(followerLogInformation.getId());
917 sb.append(" state:");
918 sb.append(followerLogInformation.isFollowerActive());
923 return sb.toString();
927 public FollowerLogInformation getFollower(String followerId) {
928 return followerToLog.get(followerId);
932 protected void setFollowerSnapshot(String followerId, FollowerToSnapshot snapshot) {
933 mapFollowerToSnapshot.put(followerId, snapshot);
937 public int followerSnapshotSize() {
938 return mapFollowerToSnapshot.size();
942 public int followerLogSize() {
943 return followerToLog.size();
946 private static class SnapshotHolder {
947 private final long lastIncludedTerm;
948 private final long lastIncludedIndex;
949 private final ByteString snapshotBytes;
951 SnapshotHolder(Snapshot snapshot) {
952 this.lastIncludedTerm = snapshot.getLastAppliedTerm();
953 this.lastIncludedIndex = snapshot.getLastAppliedIndex();
954 this.snapshotBytes = ByteString.copyFrom(snapshot.getState());
957 long getLastIncludedTerm() {
958 return lastIncludedTerm;
961 long getLastIncludedIndex() {
962 return lastIncludedIndex;
965 ByteString getSnapshotBytes() {
966 return snapshotBytes;