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);
145 public void updateMinReplicaCount() {
147 for(PeerInfo peer: context.getPeers()) {
148 if(peer.isVoting()) {
153 minReplicationCount = getMajorityVoteCount(numVoting);
156 protected int getMinIsolatedLeaderPeerCount(){
157 //the isolated Leader peer count will be 1 less than the majority vote count.
158 //this is because the vote count has the self vote counted in it
160 //0 peers = 1 votesRequired , minIsolatedLeaderPeerCount = 0
161 //2 peers = 2 votesRequired , minIsolatedLeaderPeerCount = 1
162 //4 peers = 3 votesRequired, minIsolatedLeaderPeerCount = 2
164 return minReplicationCount > 0 ? (minReplicationCount - 1) : 0;
168 void setSnapshot(@Nullable Snapshot snapshot) {
169 if(snapshot != null) {
170 this.snapshot = Optional.of(new SnapshotHolder(snapshot));
172 this.snapshot = Optional.absent();
177 protected RaftActorBehavior handleAppendEntries(ActorRef sender,
178 AppendEntries appendEntries) {
180 LOG.debug("{}: handleAppendEntries: {}", logName(), appendEntries);
186 protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
187 AppendEntriesReply appendEntriesReply) {
189 if(LOG.isTraceEnabled()) {
190 LOG.trace("{}: handleAppendEntriesReply: {}", logName(), appendEntriesReply);
193 // Update the FollowerLogInformation
194 String followerId = appendEntriesReply.getFollowerId();
195 FollowerLogInformation followerLogInformation =
196 followerToLog.get(followerId);
198 if(followerLogInformation == null){
199 LOG.error("{}: handleAppendEntriesReply - unknown follower {}", logName(), followerId);
203 if(followerLogInformation.timeSinceLastActivity() >
204 context.getConfigParams().getElectionTimeOutInterval().toMillis()) {
205 LOG.warn("{} : handleAppendEntriesReply delayed beyond election timeout, " +
206 "appendEntriesReply : {}, timeSinceLastActivity : {}, lastApplied : {}, commitIndex : {}",
207 logName(), appendEntriesReply, followerLogInformation.timeSinceLastActivity(),
208 context.getLastApplied(), context.getCommitIndex());
211 followerLogInformation.markFollowerActive();
212 followerLogInformation.setPayloadVersion(appendEntriesReply.getPayloadVersion());
214 boolean updated = false;
215 if (appendEntriesReply.isSuccess()) {
216 updated = updateFollowerLogInformation(followerLogInformation, appendEntriesReply);
218 LOG.debug("{}: handleAppendEntriesReply: received unsuccessful reply: {}", logName(), appendEntriesReply);
220 long followerLastLogIndex = appendEntriesReply.getLogLastIndex();
221 ReplicatedLogEntry followersLastLogEntry = context.getReplicatedLog().get(followerLastLogIndex);
222 if(appendEntriesReply.isForceInstallSnapshot()) {
223 // Reset the followers match and next index. This is to signal that this follower has nothing
224 // in common with this Leader and so would require a snapshot to be installed
225 followerLogInformation.setMatchIndex(-1);
226 followerLogInformation.setNextIndex(-1);
228 // Force initiate a snapshot capture
229 initiateCaptureSnapshot(followerId);
230 } else if(followerLastLogIndex < 0 || (followersLastLogEntry != null &&
231 followersLastLogEntry.getTerm() == appendEntriesReply.getLogLastTerm())) {
232 // The follower's log is empty or the last entry is present in the leader's journal
233 // and the terms match so the follower is just behind the leader's journal from
234 // the last snapshot, if any. We'll catch up the follower quickly by starting at the
235 // follower's last log index.
237 updated = updateFollowerLogInformation(followerLogInformation, appendEntriesReply);
239 // TODO: When we find that the follower is out of sync with the
240 // Leader we simply decrement that followers next index by 1.
241 // Would it be possible to do better than this? The RAFT spec
242 // does not explicitly deal with it but may be something for us to
245 followerLogInformation.decrNextIndex();
249 // Now figure out if this reply warrants a change in the commitIndex
250 // If there exists an N such that N > commitIndex, a majority
251 // of matchIndex[i] ≥ N, and log[N].term == currentTerm:
252 // set commitIndex = N (§5.3, §5.4).
253 for (long N = context.getCommitIndex() + 1; ; N++) {
254 int replicatedCount = 1;
256 for (FollowerLogInformation info : followerToLog.values()) {
257 final PeerInfo peerInfo = context.getPeerInfo(info.getId());
258 if(info.getMatchIndex() >= N && (peerInfo != null && peerInfo.isVoting())) {
263 if (replicatedCount >= minReplicationCount) {
264 ReplicatedLogEntry replicatedLogEntry = context.getReplicatedLog().get(N);
265 if (replicatedLogEntry == null) {
269 // Don't update the commit index if the log entry is from a previous term, as per §5.4.1:
270 // "Raft never commits log entries from previous terms by counting replicas".
271 // However we keep looping so we can make progress when new entries in the current term
272 // reach consensus, as per §5.4.1: "once an entry from the current term is committed by
273 // counting replicas, then all prior entries are committed indirectly".
274 if (replicatedLogEntry.getTerm() == currentTerm()) {
275 context.setCommitIndex(N);
282 // Apply the change to the state machine
283 if (context.getCommitIndex() > context.getLastApplied()) {
284 if(LOG.isDebugEnabled()) {
285 LOG.debug("{}: handleAppendEntriesReply from {}: applying to log - commitIndex: {}, lastAppliedIndex: {}",
286 logName(), followerId, context.getCommitIndex(), context.getLastApplied());
289 applyLogToStateMachine(context.getCommitIndex());
292 if (!context.getSnapshotManager().isCapturing()) {
296 //Send the next log entry immediately, if possible, no need to wait for heartbeat to trigger that event
297 sendUpdatesToFollower(followerId, followerLogInformation, false, !updated);
301 private boolean updateFollowerLogInformation(FollowerLogInformation followerLogInformation,
302 AppendEntriesReply appendEntriesReply) {
303 boolean updated = followerLogInformation.setMatchIndex(appendEntriesReply.getLogLastIndex());
304 updated = followerLogInformation.setNextIndex(appendEntriesReply.getLogLastIndex() + 1) || updated;
306 if(updated && LOG.isDebugEnabled()) {
307 LOG.debug("{}: handleAppendEntriesReply - FollowerLogInformation for {} updated: matchIndex: {}, nextIndex: {}",
308 logName(), followerLogInformation.getId(), followerLogInformation.getMatchIndex(),
309 followerLogInformation.getNextIndex());
314 private void purgeInMemoryLog() {
315 //find the lowest index across followers which has been replicated to all.
316 // lastApplied if there are no followers, so that we keep clearing the log for single-node
317 // we would delete the in-mem log from that index on, in-order to minimize mem usage
318 // we would also share this info thru AE with the followers so that they can delete their log entries as well.
319 long minReplicatedToAllIndex = followerToLog.isEmpty() ? context.getLastApplied() : Long.MAX_VALUE;
320 for (FollowerLogInformation info : followerToLog.values()) {
321 minReplicatedToAllIndex = Math.min(minReplicatedToAllIndex, info.getMatchIndex());
324 super.performSnapshotWithoutCapture(minReplicatedToAllIndex);
328 protected ClientRequestTracker removeClientRequestTracker(long logIndex) {
329 final Iterator<ClientRequestTracker> it = trackerList.iterator();
330 while (it.hasNext()) {
331 final ClientRequestTracker t = it.next();
332 if (t.getIndex() == logIndex) {
342 protected ClientRequestTracker findClientRequestTracker(long logIndex) {
343 for (ClientRequestTracker tracker : trackerList) {
344 if (tracker.getIndex() == logIndex) {
352 protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
353 RequestVoteReply requestVoteReply) {
357 protected void beforeSendHeartbeat(){}
360 public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) {
361 Preconditions.checkNotNull(sender, "sender should not be null");
363 Object message = fromSerializableMessage(originalMessage);
365 if (message instanceof RaftRPC) {
366 RaftRPC rpc = (RaftRPC) message;
367 // If RPC request or response contains term T > currentTerm:
368 // set currentTerm = T, convert to follower (§5.1)
369 // This applies to all RPC messages and responses
370 if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
371 LOG.debug("{}: Term {} in \"{}\" message is greater than leader's term {} - switching to Follower",
372 logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm());
374 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
376 return internalSwitchBehavior(RaftState.Follower);
380 if (message instanceof SendHeartBeat) {
381 beforeSendHeartbeat();
383 scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
386 } else if(message instanceof SendInstallSnapshot) {
387 // received from RaftActor
388 setSnapshot(((SendInstallSnapshot) message).getSnapshot());
389 sendInstallSnapshot();
391 } else if (message instanceof Replicate) {
392 replicate((Replicate) message);
394 } else if (message instanceof InstallSnapshotReply){
395 handleInstallSnapshotReply((InstallSnapshotReply) message);
400 return super.handleMessage(sender, message);
403 private void handleInstallSnapshotReply(InstallSnapshotReply reply) {
404 LOG.debug("{}: handleInstallSnapshotReply: {}", logName(), reply);
406 String followerId = reply.getFollowerId();
407 FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
409 if (followerToSnapshot == null) {
410 LOG.error("{}: FollowerToSnapshot not found for follower {} in InstallSnapshotReply",
411 logName(), followerId);
415 FollowerLogInformation followerLogInformation = followerToLog.get(followerId);
416 if(followerLogInformation == null) {
417 // This can happen during AddServer if it times out.
418 LOG.error("{}: FollowerLogInformation not found for follower {} in InstallSnapshotReply",
419 logName(), followerId);
420 mapFollowerToSnapshot.remove(followerId);
424 followerLogInformation.markFollowerActive();
426 if (followerToSnapshot.getChunkIndex() == reply.getChunkIndex()) {
427 boolean wasLastChunk = false;
428 if (reply.isSuccess()) {
429 if(followerToSnapshot.isLastChunk(reply.getChunkIndex())) {
430 //this was the last chunk reply
431 if(LOG.isDebugEnabled()) {
432 LOG.debug("{}: InstallSnapshotReply received, " +
433 "last chunk received, Chunk: {}. Follower: {} Setting nextIndex: {}",
434 logName(), reply.getChunkIndex(), followerId,
435 context.getReplicatedLog().getSnapshotIndex() + 1
439 long followerMatchIndex = snapshot.get().getLastIncludedIndex();
440 followerLogInformation.setMatchIndex(followerMatchIndex);
441 followerLogInformation.setNextIndex(followerMatchIndex + 1);
442 mapFollowerToSnapshot.remove(followerId);
444 LOG.debug("{}: follower: {}, matchIndex set to {}, nextIndex set to {}",
445 logName(), followerId, followerLogInformation.getMatchIndex(),
446 followerLogInformation.getNextIndex());
448 if (mapFollowerToSnapshot.isEmpty()) {
449 // once there are no pending followers receiving snapshots
450 // we can remove snapshot from the memory
454 if(context.getPeerInfo(followerId).getVotingState() == VotingState.VOTING_NOT_INITIALIZED){
455 UnInitializedFollowerSnapshotReply unInitFollowerSnapshotSuccess =
456 new UnInitializedFollowerSnapshotReply(followerId);
457 context.getActor().tell(unInitFollowerSnapshotSuccess, context.getActor());
458 LOG.debug("Sent message UnInitializedFollowerSnapshotReply to self");
461 followerToSnapshot.markSendStatus(true);
464 LOG.info("{}: InstallSnapshotReply received sending snapshot chunk failed, Will retry, Chunk: {}",
465 logName(), reply.getChunkIndex());
467 followerToSnapshot.markSendStatus(false);
470 if (wasLastChunk && !context.getSnapshotManager().isCapturing()) {
471 // Since the follower is now caught up try to purge the log.
473 } else if (!wasLastChunk && followerToSnapshot.canSendNextChunk()) {
474 ActorSelection followerActor = context.getPeerActorSelection(followerId);
475 if(followerActor != null) {
476 sendSnapshotChunk(followerActor, followerId);
481 LOG.error("{}: Chunk index {} in InstallSnapshotReply from follower {} does not match expected index {}",
482 logName(), reply.getChunkIndex(), followerId,
483 followerToSnapshot.getChunkIndex());
485 if(reply.getChunkIndex() == INVALID_CHUNK_INDEX){
486 // Since the Follower did not find this index to be valid we should reset the follower snapshot
487 // so that Installing the snapshot can resume from the beginning
488 followerToSnapshot.reset();
493 private void replicate(Replicate replicate) {
494 long logIndex = replicate.getReplicatedLogEntry().getIndex();
496 LOG.debug("{}: Replicate message: identifier: {}, logIndex: {}", logName(),
497 replicate.getIdentifier(), logIndex);
499 // Create a tracker entry we will use this later to notify the
502 new ClientRequestTrackerImpl(replicate.getClientActor(),
503 replicate.getIdentifier(),
507 boolean applyModificationToState = followerToLog.isEmpty()
508 || context.getRaftPolicy().applyModificationToStateBeforeConsensus();
510 if(applyModificationToState){
511 context.setCommitIndex(logIndex);
512 applyLogToStateMachine(logIndex);
515 if (!followerToLog.isEmpty()) {
516 sendAppendEntries(0, false);
520 protected void sendAppendEntries(long timeSinceLastActivityInterval, boolean isHeartbeat) {
521 // Send an AppendEntries to all followers
522 for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
523 final String followerId = e.getKey();
524 final FollowerLogInformation followerLogInformation = e.getValue();
525 // This checks helps not to send a repeat message to the follower
526 if(!followerLogInformation.isFollowerActive() ||
527 followerLogInformation.timeSinceLastActivity() >= timeSinceLastActivityInterval) {
528 sendUpdatesToFollower(followerId, followerLogInformation, true, isHeartbeat);
535 * This method checks if any update needs to be sent to the given follower. This includes append log entries,
536 * sending next snapshot chunk, and initiating a snapshot.
537 * @return true if any update is sent, false otherwise
540 private void sendUpdatesToFollower(String followerId, FollowerLogInformation followerLogInformation,
541 boolean sendHeartbeat, boolean isHeartbeat) {
543 ActorSelection followerActor = context.getPeerActorSelection(followerId);
544 if (followerActor != null) {
545 long followerNextIndex = followerLogInformation.getNextIndex();
546 boolean isFollowerActive = followerLogInformation.isFollowerActive();
547 boolean sendAppendEntries = false;
548 List<ReplicatedLogEntry> entries = Collections.emptyList();
550 if (mapFollowerToSnapshot.get(followerId) != null) {
551 // if install snapshot is in process , then sent next chunk if possible
552 if (isFollowerActive && mapFollowerToSnapshot.get(followerId).canSendNextChunk()) {
553 sendSnapshotChunk(followerActor, followerId);
554 } else if(sendHeartbeat) {
555 // we send a heartbeat even if we have not received a reply for the last chunk
556 sendAppendEntries = true;
559 long leaderLastIndex = context.getReplicatedLog().lastIndex();
560 long leaderSnapShotIndex = context.getReplicatedLog().getSnapshotIndex();
562 if((!isHeartbeat && LOG.isDebugEnabled()) || LOG.isTraceEnabled()) {
563 LOG.debug("{}: Checking sendAppendEntries for follower {}: active: {}, followerNextIndex: {}, leaderLastIndex: {}, leaderSnapShotIndex: {}",
564 logName(), followerId, isFollowerActive, followerNextIndex, leaderLastIndex, leaderSnapShotIndex);
567 if (isFollowerActive && context.getReplicatedLog().isPresent(followerNextIndex)) {
569 LOG.debug("{}: sendAppendEntries: {} is present for follower {}", logName(),
570 followerNextIndex, followerId);
572 if(followerLogInformation.okToReplicate()) {
573 // Try to send all the entries in the journal but not exceeding the max data size
574 // for a single AppendEntries message.
575 int maxEntries = (int) context.getReplicatedLog().size();
576 entries = context.getReplicatedLog().getFrom(followerNextIndex, maxEntries,
577 context.getConfigParams().getSnapshotChunkSize());
578 sendAppendEntries = true;
580 } else if (isFollowerActive && followerNextIndex >= 0 &&
581 leaderLastIndex > followerNextIndex && !context.getSnapshotManager().isCapturing()) {
582 // if the followers next index is not present in the leaders log, and
583 // if the follower is just not starting and if leader's index is more than followers index
584 // then snapshot should be sent
586 if (LOG.isDebugEnabled()) {
587 LOG.debug(String.format("%s: InitiateInstallSnapshot to follower: %s," +
588 "follower-nextIndex: %d, leader-snapshot-index: %d, " +
589 "leader-last-index: %d", logName(), followerId,
590 followerNextIndex, leaderSnapShotIndex, leaderLastIndex));
593 // Send heartbeat to follower whenever install snapshot is initiated.
594 sendAppendEntries = true;
595 if (canInstallSnapshot(followerNextIndex)) {
596 initiateCaptureSnapshot(followerId);
599 } else if(sendHeartbeat) {
600 // we send an AppendEntries, even if the follower is inactive
601 // in-order to update the followers timestamp, in case it becomes active again
602 sendAppendEntries = true;
607 if(sendAppendEntries) {
608 sendAppendEntriesToFollower(followerActor, followerNextIndex,
609 entries, followerId);
614 private void sendAppendEntriesToFollower(ActorSelection followerActor, long followerNextIndex,
615 List<ReplicatedLogEntry> entries, String followerId) {
616 AppendEntries appendEntries = new AppendEntries(currentTerm(), context.getId(),
617 prevLogIndex(followerNextIndex),
618 prevLogTerm(followerNextIndex), entries,
619 context.getCommitIndex(), super.getReplicatedToAllIndex(), context.getPayloadVersion());
621 if(!entries.isEmpty() || LOG.isTraceEnabled()) {
622 LOG.debug("{}: Sending AppendEntries to follower {}: {}", logName(), followerId,
626 followerActor.tell(appendEntries.toSerializable(), actor());
630 * Install Snapshot works as follows
631 * 1. Leader initiates the capture snapshot by sending a CaptureSnapshot message to actor
632 * 2. RaftActor on receipt of the CaptureSnapshotReply (from Shard), stores the received snapshot in the replicated log
633 * and makes a call to Leader's handleMessage , with SendInstallSnapshot message.
634 * 3. Leader , picks the snapshot from im-mem ReplicatedLog and sends it in chunks to the Follower
635 * 4. On complete, Follower sends back a InstallSnapshotReply.
636 * 5. On receipt of the InstallSnapshotReply for the last chunk, Leader marks the install complete for that follower
637 * and replenishes the memory by deleting the snapshot in Replicated log.
638 * 6. If another follower requires a snapshot and a snapshot has been collected (via CaptureSnapshotReply)
639 * then send the existing snapshot in chunks to the follower.
642 public boolean initiateCaptureSnapshot(String followerId) {
643 if (snapshot.isPresent()) {
644 // if a snapshot is present in the memory, most likely another install is in progress
645 // no need to capture snapshot.
646 // This could happen if another follower needs an install when one is going on.
647 final ActorSelection followerActor = context.getPeerActorSelection(followerId);
648 sendSnapshotChunk(followerActor, followerId);
651 return context.getSnapshotManager().captureToInstall(context.getReplicatedLog().last(),
652 this.getReplicatedToAllIndex(), followerId);
656 private boolean canInstallSnapshot(long nextIndex){
657 // If the follower's nextIndex is -1 then we might as well send it a snapshot
658 // Otherwise send it a snapshot only if the nextIndex is not present in the log but is present
660 return (nextIndex == -1 ||
661 (!context.getReplicatedLog().isPresent(nextIndex)
662 && context.getReplicatedLog().isInSnapshot(nextIndex)));
667 private void sendInstallSnapshot() {
668 LOG.debug("{}: sendInstallSnapshot", logName());
669 for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
670 String followerId = e.getKey();
671 ActorSelection followerActor = context.getPeerActorSelection(followerId);
672 FollowerLogInformation followerLogInfo = e.getValue();
674 if (followerActor != null) {
675 long nextIndex = followerLogInfo.getNextIndex();
676 if (context.getPeerInfo(followerId).getVotingState() == VotingState.VOTING_NOT_INITIALIZED ||
677 canInstallSnapshot(nextIndex)) {
678 sendSnapshotChunk(followerActor, followerId);
685 * Sends a snapshot chunk to a given follower
686 * InstallSnapshot should qualify as a heartbeat too.
688 private void sendSnapshotChunk(ActorSelection followerActor, String followerId) {
690 if (snapshot.isPresent()) {
691 ByteString nextSnapshotChunk = getNextSnapshotChunk(followerId, snapshot.get().getSnapshotBytes());
693 // Note: the previous call to getNextSnapshotChunk has the side-effect of adding
694 // followerId to the followerToSnapshot map.
695 FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
698 new InstallSnapshot(currentTerm(), context.getId(),
699 snapshot.get().getLastIncludedIndex(),
700 snapshot.get().getLastIncludedTerm(),
702 followerToSnapshot.incrementChunkIndex(),
703 followerToSnapshot.getTotalChunks(),
704 Optional.of(followerToSnapshot.getLastChunkHashCode())
709 if(LOG.isDebugEnabled()) {
710 LOG.debug("{}: InstallSnapshot sent to follower {}, Chunk: {}/{}",
711 logName(), followerActor.path(), followerToSnapshot.getChunkIndex(),
712 followerToSnapshot.getTotalChunks());
715 } catch (IOException e) {
716 LOG.error("{}: InstallSnapshot failed for Leader.", logName(), e);
721 * Acccepts snaphot as ByteString, enters into map for future chunks
722 * creates and return a ByteString chunk
724 private ByteString getNextSnapshotChunk(String followerId, ByteString snapshotBytes) throws IOException {
725 FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
726 if (followerToSnapshot == null) {
727 followerToSnapshot = new FollowerToSnapshot(snapshotBytes);
728 mapFollowerToSnapshot.put(followerId, followerToSnapshot);
730 ByteString nextChunk = followerToSnapshot.getNextChunk();
732 LOG.debug("{}: next snapshot chunk size for follower {}: {}", logName(), followerId, nextChunk.size());
737 private void sendHeartBeat() {
738 if (!followerToLog.isEmpty()) {
739 LOG.trace("{}: Sending heartbeat", logName());
740 sendAppendEntries(context.getConfigParams().getHeartBeatInterval().toMillis(), true);
744 private void stopHeartBeat() {
745 if (heartbeatSchedule != null && !heartbeatSchedule.isCancelled()) {
746 heartbeatSchedule.cancel();
750 private void scheduleHeartBeat(FiniteDuration interval) {
751 if (followerToLog.isEmpty()) {
752 // Optimization - do not bother scheduling a heartbeat as there are
759 // Schedule a heartbeat. When the scheduler triggers a SendHeartbeat
760 // message is sent to itself.
761 // Scheduling the heartbeat only once here because heartbeats do not
762 // need to be sent if there are other messages being sent to the remote
764 heartbeatSchedule = context.getActorSystem().scheduler().scheduleOnce(
765 interval, context.getActor(), new SendHeartBeat(),
766 context.getActorSystem().dispatcher(), context.getActor());
770 public void close() throws Exception {
775 public String getLeaderId() {
776 return context.getId();
779 protected boolean isLeaderIsolated() {
780 int minPresent = getMinIsolatedLeaderPeerCount();
781 for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
782 if (followerLogInformation.isFollowerActive()) {
784 if (minPresent == 0) {
789 return (minPresent != 0);
793 * Encapsulates the snapshot bytestring and handles the logic of sending
796 protected class FollowerToSnapshot {
797 private final ByteString snapshotBytes;
798 private int offset = 0;
799 // the next snapshot chunk is sent only if the replyReceivedForOffset matches offset
800 private int replyReceivedForOffset;
801 // if replyStatus is false, the previous chunk is attempted
802 private boolean replyStatus = false;
803 private int chunkIndex;
804 private final int totalChunks;
805 private int lastChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
806 private int nextChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
808 public FollowerToSnapshot(ByteString snapshotBytes) {
809 this.snapshotBytes = snapshotBytes;
810 int size = snapshotBytes.size();
811 totalChunks = ( size / context.getConfigParams().getSnapshotChunkSize()) +
812 ((size % context.getConfigParams().getSnapshotChunkSize()) > 0 ? 1 : 0);
813 if(LOG.isDebugEnabled()) {
814 LOG.debug("{}: Snapshot {} bytes, total chunks to send:{}",
815 logName(), size, totalChunks);
817 replyReceivedForOffset = -1;
818 chunkIndex = AbstractLeader.FIRST_CHUNK_INDEX;
821 public ByteString getSnapshotBytes() {
822 return snapshotBytes;
825 public int incrementOffset() {
827 // if prev chunk failed, we would want to sent the same chunk again
828 offset = offset + context.getConfigParams().getSnapshotChunkSize();
833 public int incrementChunkIndex() {
835 // if prev chunk failed, we would want to sent the same chunk again
836 chunkIndex = chunkIndex + 1;
841 public int getChunkIndex() {
845 public int getTotalChunks() {
849 public boolean canSendNextChunk() {
850 // we only send a false if a chunk is sent but we have not received a reply yet
851 return replyReceivedForOffset == offset;
854 public boolean isLastChunk(int chunkIndex) {
855 return totalChunks == chunkIndex;
858 public void markSendStatus(boolean success) {
860 // if the chunk sent was successful
861 replyReceivedForOffset = offset;
863 lastChunkHashCode = nextChunkHashCode;
865 // if the chunk sent was failure
866 replyReceivedForOffset = offset;
871 public ByteString getNextChunk() {
872 int snapshotLength = getSnapshotBytes().size();
873 int start = incrementOffset();
874 int size = context.getConfigParams().getSnapshotChunkSize();
875 if (context.getConfigParams().getSnapshotChunkSize() > snapshotLength) {
876 size = snapshotLength;
878 if ((start + context.getConfigParams().getSnapshotChunkSize()) > snapshotLength) {
879 size = snapshotLength - start;
884 LOG.debug("{}: Next chunk: length={}, offset={},size={}", logName(),
885 snapshotLength, start, size);
887 ByteString substring = getSnapshotBytes().substring(start, start + size);
888 nextChunkHashCode = substring.hashCode();
893 * reset should be called when the Follower needs to be sent the snapshot from the beginning
898 replyReceivedForOffset = offset;
899 chunkIndex = AbstractLeader.FIRST_CHUNK_INDEX;
900 lastChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
903 public int getLastChunkHashCode() {
904 return lastChunkHashCode;
908 // called from example-actor for printing the follower-states
909 public String printFollowerStates() {
910 final StringBuilder sb = new StringBuilder();
913 for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
915 sb.append(followerLogInformation.getId());
916 sb.append(" state:");
917 sb.append(followerLogInformation.isFollowerActive());
922 return sb.toString();
926 public FollowerLogInformation getFollower(String followerId) {
927 return followerToLog.get(followerId);
931 protected void setFollowerSnapshot(String followerId, FollowerToSnapshot snapshot) {
932 mapFollowerToSnapshot.put(followerId, snapshot);
936 public int followerSnapshotSize() {
937 return mapFollowerToSnapshot.size();
941 public int followerLogSize() {
942 return followerToLog.size();
945 private static class SnapshotHolder {
946 private final long lastIncludedTerm;
947 private final long lastIncludedIndex;
948 private final ByteString snapshotBytes;
950 SnapshotHolder(Snapshot snapshot) {
951 this.lastIncludedTerm = snapshot.getLastAppliedTerm();
952 this.lastIncludedIndex = snapshot.getLastAppliedIndex();
953 this.snapshotBytes = ByteString.copyFrom(snapshot.getState());
956 long getLastIncludedTerm() {
957 return lastIncludedTerm;
960 long getLastIncludedIndex() {
961 return lastIncludedIndex;
964 ByteString getSnapshotBytes() {
965 return snapshotBytes;