2 * Copyright (c) 2014 Cisco Systems, Inc. and others. All rights reserved.
4 * This program and the accompanying materials are made available under the
5 * terms of the Eclipse Public License v1.0 which accompanies this distribution,
6 * and is available at http://www.eclipse.org/legal/epl-v10.html
9 package org.opendaylight.controller.cluster.raft.behaviors;
11 import akka.actor.ActorRef;
12 import akka.actor.ActorSelection;
13 import akka.actor.Cancellable;
14 import com.google.common.annotations.VisibleForTesting;
15 import com.google.common.base.Optional;
16 import com.google.common.base.Preconditions;
17 import com.google.common.collect.ImmutableMap;
18 import com.google.common.collect.ImmutableMap.Builder;
19 import com.google.protobuf.ByteString;
20 import java.io.IOException;
21 import java.util.Collection;
22 import java.util.Collections;
23 import java.util.HashMap;
24 import java.util.Iterator;
25 import java.util.LinkedList;
26 import java.util.List;
28 import java.util.Map.Entry;
29 import javax.annotation.Nullable;
30 import org.opendaylight.controller.cluster.raft.ClientRequestTracker;
31 import org.opendaylight.controller.cluster.raft.ClientRequestTrackerImpl;
32 import org.opendaylight.controller.cluster.raft.FollowerLogInformation;
33 import org.opendaylight.controller.cluster.raft.FollowerLogInformationImpl;
34 import org.opendaylight.controller.cluster.raft.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.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 scala.concurrent.duration.FiniteDuration;
50 * The behavior of a RaftActor when it is in the Leader state
54 * <li> Upon election: send initial empty AppendEntries RPCs
55 * (heartbeat) to each server; repeat during idle periods to
56 * prevent election timeouts (§5.2)
57 * <li> If command received from client: append entry to local log,
58 * respond after entry applied to state machine (§5.3)
59 * <li> If last log index ≥ nextIndex for a follower: send
60 * AppendEntries RPC with log entries starting at nextIndex
62 * <li> If successful: update nextIndex and matchIndex for
64 * <li> If AppendEntries fails because of log inconsistency:
65 * decrement nextIndex and retry (§5.3)
67 * <li> If there exists an N such that N > commitIndex, a majority
68 * of matchIndex[i] ≥ N, and log[N].term == currentTerm:
69 * set commitIndex = N (§5.3, §5.4).
71 public abstract class AbstractLeader extends AbstractRaftActorBehavior {
73 // The index of the first chunk that is sent when installing a snapshot
74 public static final int FIRST_CHUNK_INDEX = 1;
76 // The index that the follower should respond with if it needs the install snapshot to be reset
77 public static final int INVALID_CHUNK_INDEX = -1;
79 // This would be passed as the hash code of the last chunk when sending the first chunk
80 public static final int INITIAL_LAST_CHUNK_HASH_CODE = -1;
82 private final Map<String, FollowerLogInformation> followerToLog;
83 private final Map<String, FollowerToSnapshot> mapFollowerToSnapshot = new HashMap<>();
85 private Cancellable heartbeatSchedule = null;
87 private final Collection<ClientRequestTracker> trackerList = new LinkedList<>();
89 protected final int minReplicationCount;
91 protected final int minIsolatedLeaderPeerCount;
93 private Optional<SnapshotHolder> snapshot;
95 public AbstractLeader(RaftActorContext context) {
96 super(context, RaftState.Leader);
98 setLeaderPayloadVersion(context.getPayloadVersion());
100 final Builder<String, FollowerLogInformation> ftlBuilder = ImmutableMap.builder();
101 for (String followerId : context.getPeerAddresses().keySet()) {
102 FollowerLogInformation followerLogInformation =
103 new FollowerLogInformationImpl(followerId, -1, context);
105 ftlBuilder.put(followerId, followerLogInformation);
107 followerToLog = ftlBuilder.build();
109 leaderId = context.getId();
111 LOG.debug("{}: Election: Leader has following peers: {}", logName(), getFollowerIds());
113 minReplicationCount = getMajorityVoteCount(getFollowerIds().size());
115 // the isolated Leader peer count will be 1 less than the majority vote count.
116 // this is because the vote count has the self vote counted in it
118 // 0 peers = 1 votesRequired , minIsolatedLeaderPeerCount = 0
119 // 2 peers = 2 votesRequired , minIsolatedLeaderPeerCount = 1
120 // 4 peers = 3 votesRequired, minIsolatedLeaderPeerCount = 2
121 minIsolatedLeaderPeerCount = minReplicationCount > 0 ? (minReplicationCount - 1) : 0;
123 snapshot = Optional.absent();
125 // Immediately schedule a heartbeat
126 // Upon election: send initial empty AppendEntries RPCs
127 // (heartbeat) to each server; repeat during idle periods to
128 // prevent election timeouts (§5.2)
129 sendAppendEntries(0, false);
131 // It is important to schedule this heartbeat here
132 scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
136 * Return an immutable collection of follower identifiers.
138 * @return Collection of follower IDs
140 public final Collection<String> getFollowerIds() {
141 return followerToLog.keySet();
145 void setSnapshot(@Nullable Snapshot snapshot) {
146 if(snapshot != null) {
147 this.snapshot = Optional.of(new SnapshotHolder(snapshot));
149 this.snapshot = Optional.absent();
154 protected RaftActorBehavior handleAppendEntries(ActorRef sender,
155 AppendEntries appendEntries) {
157 LOG.debug("{}: handleAppendEntries: {}", logName(), appendEntries);
163 protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
164 AppendEntriesReply appendEntriesReply) {
166 if(LOG.isTraceEnabled()) {
167 LOG.trace("{}: handleAppendEntriesReply: {}", logName(), appendEntriesReply);
170 // Update the FollowerLogInformation
171 String followerId = appendEntriesReply.getFollowerId();
172 FollowerLogInformation followerLogInformation =
173 followerToLog.get(followerId);
175 if(followerLogInformation == null){
176 LOG.error("{}: handleAppendEntriesReply - unknown follower {}", logName(), followerId);
180 if(followerLogInformation.timeSinceLastActivity() >
181 context.getConfigParams().getElectionTimeOutInterval().toMillis()) {
182 LOG.warn("{} : handleAppendEntriesReply delayed beyond election timeout, " +
183 "appendEntriesReply : {}, timeSinceLastActivity : {}, lastApplied : {}, commitIndex : {}",
184 logName(), appendEntriesReply, followerLogInformation.timeSinceLastActivity(),
185 context.getLastApplied(), context.getCommitIndex());
188 followerLogInformation.markFollowerActive();
189 followerLogInformation.setPayloadVersion(appendEntriesReply.getPayloadVersion());
191 boolean updated = false;
192 if (appendEntriesReply.isSuccess()) {
193 updated = updateFollowerLogInformation(followerLogInformation, appendEntriesReply);
195 LOG.debug("{}: handleAppendEntriesReply: received unsuccessful reply: {}", logName(), appendEntriesReply);
197 long followerLastLogIndex = appendEntriesReply.getLogLastIndex();
198 ReplicatedLogEntry followersLastLogEntry = context.getReplicatedLog().get(followerLastLogIndex);
199 if(appendEntriesReply.isForceInstallSnapshot()) {
200 // Reset the followers match and next index. This is to signal that this follower has nothing
201 // in common with this Leader and so would require a snapshot to be installed
202 followerLogInformation.setMatchIndex(-1);
203 followerLogInformation.setNextIndex(-1);
205 // Force initiate a snapshot capture
206 initiateCaptureSnapshot(followerId);
207 } else if(followerLastLogIndex < 0 || (followersLastLogEntry != null &&
208 followersLastLogEntry.getTerm() == appendEntriesReply.getLogLastTerm())) {
209 // The follower's log is empty or the last entry is present in the leader's journal
210 // and the terms match so the follower is just behind the leader's journal from
211 // the last snapshot, if any. We'll catch up the follower quickly by starting at the
212 // follower's last log index.
214 updated = updateFollowerLogInformation(followerLogInformation, appendEntriesReply);
216 // TODO: When we find that the follower is out of sync with the
217 // Leader we simply decrement that followers next index by 1.
218 // Would it be possible to do better than this? The RAFT spec
219 // does not explicitly deal with it but may be something for us to
222 followerLogInformation.decrNextIndex();
226 // Now figure out if this reply warrants a change in the commitIndex
227 // If there exists an N such that N > commitIndex, a majority
228 // of matchIndex[i] ≥ N, and log[N].term == currentTerm:
229 // set commitIndex = N (§5.3, §5.4).
230 for (long N = context.getCommitIndex() + 1; ; N++) {
231 int replicatedCount = 1;
233 for (FollowerLogInformation info : followerToLog.values()) {
234 if (info.getMatchIndex() >= N) {
239 if (replicatedCount >= minReplicationCount) {
240 ReplicatedLogEntry replicatedLogEntry = context.getReplicatedLog().get(N);
241 if (replicatedLogEntry != null &&
242 replicatedLogEntry.getTerm() == currentTerm()) {
243 context.setCommitIndex(N);
250 // Apply the change to the state machine
251 if (context.getCommitIndex() > context.getLastApplied()) {
252 if(LOG.isDebugEnabled()) {
253 LOG.debug("{}: handleAppendEntriesReply from {}: applying to log - commitIndex: {}, lastAppliedIndex: {}",
254 logName(), followerId, context.getCommitIndex(), context.getLastApplied());
257 applyLogToStateMachine(context.getCommitIndex());
260 if (!context.getSnapshotManager().isCapturing()) {
264 //Send the next log entry immediately, if possible, no need to wait for heartbeat to trigger that event
265 sendUpdatesToFollower(followerId, followerLogInformation, false, !updated);
269 private boolean updateFollowerLogInformation(FollowerLogInformation followerLogInformation,
270 AppendEntriesReply appendEntriesReply) {
271 boolean updated = followerLogInformation.setMatchIndex(appendEntriesReply.getLogLastIndex());
272 updated = followerLogInformation.setNextIndex(appendEntriesReply.getLogLastIndex() + 1) || updated;
274 if(updated && LOG.isDebugEnabled()) {
275 LOG.debug("{}: handleAppendEntriesReply - FollowerLogInformation for {} updated: matchIndex: {}, nextIndex: {}",
276 logName(), followerLogInformation.getId(), followerLogInformation.getMatchIndex(),
277 followerLogInformation.getNextIndex());
282 private void purgeInMemoryLog() {
283 //find the lowest index across followers which has been replicated to all.
284 // lastApplied if there are no followers, so that we keep clearing the log for single-node
285 // we would delete the in-mem log from that index on, in-order to minimize mem usage
286 // we would also share this info thru AE with the followers so that they can delete their log entries as well.
287 long minReplicatedToAllIndex = followerToLog.isEmpty() ? context.getLastApplied() : Long.MAX_VALUE;
288 for (FollowerLogInformation info : followerToLog.values()) {
289 minReplicatedToAllIndex = Math.min(minReplicatedToAllIndex, info.getMatchIndex());
292 super.performSnapshotWithoutCapture(minReplicatedToAllIndex);
296 protected ClientRequestTracker removeClientRequestTracker(long logIndex) {
297 final Iterator<ClientRequestTracker> it = trackerList.iterator();
298 while (it.hasNext()) {
299 final ClientRequestTracker t = it.next();
300 if (t.getIndex() == logIndex) {
310 protected ClientRequestTracker findClientRequestTracker(long logIndex) {
311 for (ClientRequestTracker tracker : trackerList) {
312 if (tracker.getIndex() == logIndex) {
320 protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
321 RequestVoteReply requestVoteReply) {
325 protected void beforeSendHeartbeat(){}
328 public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) {
329 Preconditions.checkNotNull(sender, "sender should not be null");
331 Object message = fromSerializableMessage(originalMessage);
333 if (message instanceof RaftRPC) {
334 RaftRPC rpc = (RaftRPC) message;
335 // If RPC request or response contains term T > currentTerm:
336 // set currentTerm = T, convert to follower (§5.1)
337 // This applies to all RPC messages and responses
338 if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
339 LOG.debug("{}: Term {} in \"{}\" message is greater than leader's term {} - switching to Follower",
340 logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm());
342 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
344 return internalSwitchBehavior(RaftState.Follower);
348 if (message instanceof SendHeartBeat) {
349 beforeSendHeartbeat();
351 scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
354 } else if(message instanceof SendInstallSnapshot) {
355 // received from RaftActor
356 setSnapshot(((SendInstallSnapshot) message).getSnapshot());
357 sendInstallSnapshot();
359 } else if (message instanceof Replicate) {
360 replicate((Replicate) message);
362 } else if (message instanceof InstallSnapshotReply){
363 handleInstallSnapshotReply((InstallSnapshotReply) message);
368 return super.handleMessage(sender, message);
371 private void handleInstallSnapshotReply(InstallSnapshotReply reply) {
372 LOG.debug("{}: handleInstallSnapshotReply: {}", logName(), reply);
374 String followerId = reply.getFollowerId();
375 FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
377 if (followerToSnapshot == null) {
378 LOG.error("{}: FollowerId {} in InstallSnapshotReply not known to Leader",
379 logName(), followerId);
383 FollowerLogInformation followerLogInformation = followerToLog.get(followerId);
384 followerLogInformation.markFollowerActive();
386 if (followerToSnapshot.getChunkIndex() == reply.getChunkIndex()) {
387 boolean wasLastChunk = false;
388 if (reply.isSuccess()) {
389 if(followerToSnapshot.isLastChunk(reply.getChunkIndex())) {
390 //this was the last chunk reply
391 if(LOG.isDebugEnabled()) {
392 LOG.debug("{}: InstallSnapshotReply received, " +
393 "last chunk received, Chunk: {}. Follower: {} Setting nextIndex: {}",
394 logName(), reply.getChunkIndex(), followerId,
395 context.getReplicatedLog().getSnapshotIndex() + 1
399 long followerMatchIndex = snapshot.get().getLastIncludedIndex();
400 followerLogInformation.setMatchIndex(followerMatchIndex);
401 followerLogInformation.setNextIndex(followerMatchIndex + 1);
402 mapFollowerToSnapshot.remove(followerId);
404 LOG.debug("{}: follower: {}, matchIndex set to {}, nextIndex set to {}",
405 logName(), followerId, followerLogInformation.getMatchIndex(),
406 followerLogInformation.getNextIndex());
408 if (mapFollowerToSnapshot.isEmpty()) {
409 // once there are no pending followers receiving snapshots
410 // we can remove snapshot from the memory
416 followerToSnapshot.markSendStatus(true);
419 LOG.info("{}: InstallSnapshotReply received sending snapshot chunk failed, Will retry, Chunk: {}",
420 logName(), reply.getChunkIndex());
422 followerToSnapshot.markSendStatus(false);
425 if (wasLastChunk && !context.getSnapshotManager().isCapturing()) {
426 // Since the follower is now caught up try to purge the log.
428 } else if (!wasLastChunk && followerToSnapshot.canSendNextChunk()) {
429 ActorSelection followerActor = context.getPeerActorSelection(followerId);
430 if(followerActor != null) {
431 sendSnapshotChunk(followerActor, followerId);
436 LOG.error("{}: Chunk index {} in InstallSnapshotReply from follower {} does not match expected index {}",
437 logName(), reply.getChunkIndex(), followerId,
438 followerToSnapshot.getChunkIndex());
440 if(reply.getChunkIndex() == INVALID_CHUNK_INDEX){
441 // Since the Follower did not find this index to be valid we should reset the follower snapshot
442 // so that Installing the snapshot can resume from the beginning
443 followerToSnapshot.reset();
448 private void replicate(Replicate replicate) {
449 long logIndex = replicate.getReplicatedLogEntry().getIndex();
451 LOG.debug("{}: Replicate message: identifier: {}, logIndex: {}", logName(),
452 replicate.getIdentifier(), logIndex);
454 // Create a tracker entry we will use this later to notify the
457 new ClientRequestTrackerImpl(replicate.getClientActor(),
458 replicate.getIdentifier(),
462 boolean applyModificationToState = followerToLog.isEmpty()
463 || context.getRaftPolicy().applyModificationToStateBeforeConsensus();
465 if(applyModificationToState){
466 context.setCommitIndex(logIndex);
467 applyLogToStateMachine(logIndex);
470 if (!followerToLog.isEmpty()) {
471 sendAppendEntries(0, false);
475 private void sendAppendEntries(long timeSinceLastActivityInterval, boolean isHeartbeat) {
476 // Send an AppendEntries to all followers
477 for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
478 final String followerId = e.getKey();
479 final FollowerLogInformation followerLogInformation = e.getValue();
480 // This checks helps not to send a repeat message to the follower
481 if(!followerLogInformation.isFollowerActive() ||
482 followerLogInformation.timeSinceLastActivity() >= timeSinceLastActivityInterval) {
483 sendUpdatesToFollower(followerId, followerLogInformation, true, isHeartbeat);
490 * This method checks if any update needs to be sent to the given follower. This includes append log entries,
491 * sending next snapshot chunk, and initiating a snapshot.
492 * @return true if any update is sent, false otherwise
495 private void sendUpdatesToFollower(String followerId, FollowerLogInformation followerLogInformation,
496 boolean sendHeartbeat, boolean isHeartbeat) {
498 ActorSelection followerActor = context.getPeerActorSelection(followerId);
499 if (followerActor != null) {
500 long followerNextIndex = followerLogInformation.getNextIndex();
501 boolean isFollowerActive = followerLogInformation.isFollowerActive();
502 boolean sendAppendEntries = false;
503 List<ReplicatedLogEntry> entries = Collections.emptyList();
505 if (mapFollowerToSnapshot.get(followerId) != null) {
506 // if install snapshot is in process , then sent next chunk if possible
507 if (isFollowerActive && mapFollowerToSnapshot.get(followerId).canSendNextChunk()) {
508 sendSnapshotChunk(followerActor, followerId);
509 } else if(sendHeartbeat) {
510 // we send a heartbeat even if we have not received a reply for the last chunk
511 sendAppendEntries = true;
514 long leaderLastIndex = context.getReplicatedLog().lastIndex();
515 long leaderSnapShotIndex = context.getReplicatedLog().getSnapshotIndex();
517 if((!isHeartbeat && LOG.isDebugEnabled()) || LOG.isTraceEnabled()) {
518 LOG.debug("{}: Checking sendAppendEntries for follower {}: active: {}, followerNextIndex: {}, leaderLastIndex: {}, leaderSnapShotIndex: {}",
519 logName(), followerId, isFollowerActive, followerNextIndex, leaderLastIndex, leaderSnapShotIndex);
522 if (isFollowerActive && context.getReplicatedLog().isPresent(followerNextIndex)) {
524 LOG.debug("{}: sendAppendEntries: {} is present for follower {}", logName(),
525 followerNextIndex, followerId);
527 if(followerLogInformation.okToReplicate()) {
528 // Try to send all the entries in the journal but not exceeding the max data size
529 // for a single AppendEntries message.
530 int maxEntries = (int) context.getReplicatedLog().size();
531 entries = context.getReplicatedLog().getFrom(followerNextIndex, maxEntries,
532 context.getConfigParams().getSnapshotChunkSize());
533 sendAppendEntries = true;
535 } else if (isFollowerActive && followerNextIndex >= 0 &&
536 leaderLastIndex > followerNextIndex && !context.getSnapshotManager().isCapturing()) {
537 // if the followers next index is not present in the leaders log, and
538 // if the follower is just not starting and if leader's index is more than followers index
539 // then snapshot should be sent
541 if (LOG.isDebugEnabled()) {
542 LOG.debug(String.format("%s: InitiateInstallSnapshot to follower: %s," +
543 "follower-nextIndex: %d, leader-snapshot-index: %d, " +
544 "leader-last-index: %d", logName(), followerId,
545 followerNextIndex, leaderSnapShotIndex, leaderLastIndex));
548 // Send heartbeat to follower whenever install snapshot is initiated.
549 sendAppendEntries = true;
550 if (canInstallSnapshot(followerNextIndex)) {
551 initiateCaptureSnapshot(followerId);
554 } else if(sendHeartbeat) {
555 // we send an AppendEntries, even if the follower is inactive
556 // in-order to update the followers timestamp, in case it becomes active again
557 sendAppendEntries = true;
562 if(sendAppendEntries) {
563 sendAppendEntriesToFollower(followerActor, followerNextIndex,
564 entries, followerId);
569 private void sendAppendEntriesToFollower(ActorSelection followerActor, long followerNextIndex,
570 List<ReplicatedLogEntry> entries, String followerId) {
571 AppendEntries appendEntries = new AppendEntries(currentTerm(), context.getId(),
572 prevLogIndex(followerNextIndex),
573 prevLogTerm(followerNextIndex), entries,
574 context.getCommitIndex(), super.getReplicatedToAllIndex(), context.getPayloadVersion());
576 if(!entries.isEmpty() || LOG.isTraceEnabled()) {
577 LOG.debug("{}: Sending AppendEntries to follower {}: {}", logName(), followerId,
581 followerActor.tell(appendEntries.toSerializable(), actor());
585 * Install Snapshot works as follows
586 * 1. Leader initiates the capture snapshot by sending a CaptureSnapshot message to actor
587 * 2. RaftActor on receipt of the CaptureSnapshotReply (from Shard), stores the received snapshot in the replicated log
588 * and makes a call to Leader's handleMessage , with SendInstallSnapshot message.
589 * 3. Leader , picks the snapshot from im-mem ReplicatedLog and sends it in chunks to the Follower
590 * 4. On complete, Follower sends back a InstallSnapshotReply.
591 * 5. On receipt of the InstallSnapshotReply for the last chunk, Leader marks the install complete for that follower
592 * and replenishes the memory by deleting the snapshot in Replicated log.
593 * 6. If another follower requires a snapshot and a snapshot has been collected (via CaptureSnapshotReply)
594 * then send the existing snapshot in chunks to the follower.
597 private void initiateCaptureSnapshot(String followerId) {
598 if (snapshot.isPresent()) {
599 // if a snapshot is present in the memory, most likely another install is in progress
600 // no need to capture snapshot.
601 // This could happen if another follower needs an install when one is going on.
602 final ActorSelection followerActor = context.getPeerActorSelection(followerId);
603 sendSnapshotChunk(followerActor, followerId);
607 context.getSnapshotManager().captureToInstall(context.getReplicatedLog().last(),
608 this.getReplicatedToAllIndex(), followerId);
612 private boolean canInstallSnapshot(long nextIndex){
613 // If the follower's nextIndex is -1 then we might as well send it a snapshot
614 // Otherwise send it a snapshot only if the nextIndex is not present in the log but is present
616 return (nextIndex == -1 ||
617 (!context.getReplicatedLog().isPresent(nextIndex)
618 && context.getReplicatedLog().isInSnapshot(nextIndex)));
623 private void sendInstallSnapshot() {
624 LOG.debug("{}: sendInstallSnapshot", logName());
625 for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
626 ActorSelection followerActor = context.getPeerActorSelection(e.getKey());
628 if (followerActor != null) {
629 long nextIndex = e.getValue().getNextIndex();
630 if (canInstallSnapshot(nextIndex)) {
631 sendSnapshotChunk(followerActor, e.getKey());
638 * Sends a snapshot chunk to a given follower
639 * InstallSnapshot should qualify as a heartbeat too.
641 private void sendSnapshotChunk(ActorSelection followerActor, String followerId) {
643 if (snapshot.isPresent()) {
644 ByteString nextSnapshotChunk = getNextSnapshotChunk(followerId, snapshot.get().getSnapshotBytes());
646 // Note: the previous call to getNextSnapshotChunk has the side-effect of adding
647 // followerId to the followerToSnapshot map.
648 FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
651 new InstallSnapshot(currentTerm(), context.getId(),
652 snapshot.get().getLastIncludedIndex(),
653 snapshot.get().getLastIncludedTerm(),
655 followerToSnapshot.incrementChunkIndex(),
656 followerToSnapshot.getTotalChunks(),
657 Optional.of(followerToSnapshot.getLastChunkHashCode())
662 if(LOG.isDebugEnabled()) {
663 LOG.debug("{}: InstallSnapshot sent to follower {}, Chunk: {}/{}",
664 logName(), followerActor.path(), followerToSnapshot.getChunkIndex(),
665 followerToSnapshot.getTotalChunks());
668 } catch (IOException e) {
669 LOG.error("{}: InstallSnapshot failed for Leader.", logName(), e);
674 * Acccepts snaphot as ByteString, enters into map for future chunks
675 * creates and return a ByteString chunk
677 private ByteString getNextSnapshotChunk(String followerId, ByteString snapshotBytes) throws IOException {
678 FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
679 if (followerToSnapshot == null) {
680 followerToSnapshot = new FollowerToSnapshot(snapshotBytes);
681 mapFollowerToSnapshot.put(followerId, followerToSnapshot);
683 ByteString nextChunk = followerToSnapshot.getNextChunk();
685 LOG.debug("{}: next snapshot chunk size for follower {}: {}", logName(), followerId, nextChunk.size());
690 private void sendHeartBeat() {
691 if (!followerToLog.isEmpty()) {
692 LOG.trace("{}: Sending heartbeat", logName());
693 sendAppendEntries(context.getConfigParams().getHeartBeatInterval().toMillis(), true);
697 private void stopHeartBeat() {
698 if (heartbeatSchedule != null && !heartbeatSchedule.isCancelled()) {
699 heartbeatSchedule.cancel();
703 private void scheduleHeartBeat(FiniteDuration interval) {
704 if (followerToLog.isEmpty()) {
705 // Optimization - do not bother scheduling a heartbeat as there are
712 // Schedule a heartbeat. When the scheduler triggers a SendHeartbeat
713 // message is sent to itself.
714 // Scheduling the heartbeat only once here because heartbeats do not
715 // need to be sent if there are other messages being sent to the remote
717 heartbeatSchedule = context.getActorSystem().scheduler().scheduleOnce(
718 interval, context.getActor(), new SendHeartBeat(),
719 context.getActorSystem().dispatcher(), context.getActor());
723 public void close() throws Exception {
728 public String getLeaderId() {
729 return context.getId();
732 protected boolean isLeaderIsolated() {
733 int minPresent = minIsolatedLeaderPeerCount;
734 for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
735 if (followerLogInformation.isFollowerActive()) {
737 if (minPresent == 0) {
742 return (minPresent != 0);
746 * Encapsulates the snapshot bytestring and handles the logic of sending
749 protected class FollowerToSnapshot {
750 private final ByteString snapshotBytes;
751 private int offset = 0;
752 // the next snapshot chunk is sent only if the replyReceivedForOffset matches offset
753 private int replyReceivedForOffset;
754 // if replyStatus is false, the previous chunk is attempted
755 private boolean replyStatus = false;
756 private int chunkIndex;
757 private final int totalChunks;
758 private int lastChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
759 private int nextChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
761 public FollowerToSnapshot(ByteString snapshotBytes) {
762 this.snapshotBytes = snapshotBytes;
763 int size = snapshotBytes.size();
764 totalChunks = ( size / context.getConfigParams().getSnapshotChunkSize()) +
765 ((size % context.getConfigParams().getSnapshotChunkSize()) > 0 ? 1 : 0);
766 if(LOG.isDebugEnabled()) {
767 LOG.debug("{}: Snapshot {} bytes, total chunks to send:{}",
768 logName(), size, totalChunks);
770 replyReceivedForOffset = -1;
771 chunkIndex = AbstractLeader.FIRST_CHUNK_INDEX;
774 public ByteString getSnapshotBytes() {
775 return snapshotBytes;
778 public int incrementOffset() {
780 // if prev chunk failed, we would want to sent the same chunk again
781 offset = offset + context.getConfigParams().getSnapshotChunkSize();
786 public int incrementChunkIndex() {
788 // if prev chunk failed, we would want to sent the same chunk again
789 chunkIndex = chunkIndex + 1;
794 public int getChunkIndex() {
798 public int getTotalChunks() {
802 public boolean canSendNextChunk() {
803 // we only send a false if a chunk is sent but we have not received a reply yet
804 return replyReceivedForOffset == offset;
807 public boolean isLastChunk(int chunkIndex) {
808 return totalChunks == chunkIndex;
811 public void markSendStatus(boolean success) {
813 // if the chunk sent was successful
814 replyReceivedForOffset = offset;
816 lastChunkHashCode = nextChunkHashCode;
818 // if the chunk sent was failure
819 replyReceivedForOffset = offset;
824 public ByteString getNextChunk() {
825 int snapshotLength = getSnapshotBytes().size();
826 int start = incrementOffset();
827 int size = context.getConfigParams().getSnapshotChunkSize();
828 if (context.getConfigParams().getSnapshotChunkSize() > snapshotLength) {
829 size = snapshotLength;
831 if ((start + context.getConfigParams().getSnapshotChunkSize()) > snapshotLength) {
832 size = snapshotLength - start;
837 LOG.debug("{}: Next chunk: length={}, offset={},size={}", logName(),
838 snapshotLength, start, size);
840 ByteString substring = getSnapshotBytes().substring(start, start + size);
841 nextChunkHashCode = substring.hashCode();
846 * reset should be called when the Follower needs to be sent the snapshot from the beginning
851 replyReceivedForOffset = offset;
852 chunkIndex = AbstractLeader.FIRST_CHUNK_INDEX;
853 lastChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
856 public int getLastChunkHashCode() {
857 return lastChunkHashCode;
861 // called from example-actor for printing the follower-states
862 public String printFollowerStates() {
863 final StringBuilder sb = new StringBuilder();
866 for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
868 sb.append(followerLogInformation.getId());
869 sb.append(" state:");
870 sb.append(followerLogInformation.isFollowerActive());
875 return sb.toString();
879 public FollowerLogInformation getFollower(String followerId) {
880 return followerToLog.get(followerId);
884 protected void setFollowerSnapshot(String followerId, FollowerToSnapshot snapshot) {
885 mapFollowerToSnapshot.put(followerId, snapshot);
889 public int followerSnapshotSize() {
890 return mapFollowerToSnapshot.size();
894 public int followerLogSize() {
895 return followerToLog.size();
898 private static class SnapshotHolder {
899 private final long lastIncludedTerm;
900 private final long lastIncludedIndex;
901 private final ByteString snapshotBytes;
903 SnapshotHolder(Snapshot snapshot) {
904 this.lastIncludedTerm = snapshot.getLastAppliedTerm();
905 this.lastIncludedIndex = snapshot.getLastAppliedIndex();
906 this.snapshotBytes = ByteString.copyFrom(snapshot.getState());
909 long getLastIncludedTerm() {
910 return lastIncludedTerm;
913 long getLastIncludedIndex() {
914 return lastIncludedIndex;
917 ByteString getSnapshotBytes() {
918 return snapshotBytes;