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 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.RaftActorContext;
34 import org.opendaylight.controller.cluster.raft.RaftState;
35 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
36 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshot;
37 import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
38 import org.opendaylight.controller.cluster.raft.base.messages.SendHeartBeat;
39 import org.opendaylight.controller.cluster.raft.base.messages.SendInstallSnapshot;
40 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
41 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
42 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot;
43 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshotReply;
44 import org.opendaylight.controller.cluster.raft.messages.RaftRPC;
45 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
46 import scala.concurrent.duration.FiniteDuration;
49 * The behavior of a RaftActor when it is in the Leader state
53 * <li> Upon election: send initial empty AppendEntries RPCs
54 * (heartbeat) to each server; repeat during idle periods to
55 * prevent election timeouts (§5.2)
56 * <li> If command received from client: append entry to local log,
57 * respond after entry applied to state machine (§5.3)
58 * <li> If last log index ≥ nextIndex for a follower: send
59 * AppendEntries RPC with log entries starting at nextIndex
61 * <li> If successful: update nextIndex and matchIndex for
63 * <li> If AppendEntries fails because of log inconsistency:
64 * decrement nextIndex and retry (§5.3)
66 * <li> If there exists an N such that N > commitIndex, a majority
67 * of matchIndex[i] ≥ N, and log[N].term == currentTerm:
68 * set commitIndex = N (§5.3, §5.4).
70 public abstract class AbstractLeader extends AbstractRaftActorBehavior {
72 // The index of the first chunk that is sent when installing a snapshot
73 public static final int FIRST_CHUNK_INDEX = 1;
75 // The index that the follower should respond with if it needs the install snapshot to be reset
76 public static final int INVALID_CHUNK_INDEX = -1;
78 // This would be passed as the hash code of the last chunk when sending the first chunk
79 public static final int INITIAL_LAST_CHUNK_HASH_CODE = -1;
81 private final Map<String, FollowerLogInformation> followerToLog;
82 private final Map<String, FollowerToSnapshot> mapFollowerToSnapshot = new HashMap<>();
84 private Cancellable heartbeatSchedule = null;
86 private final Collection<ClientRequestTracker> trackerList = new LinkedList<>();
88 protected final int minReplicationCount;
90 protected final int minIsolatedLeaderPeerCount;
92 private Optional<ByteString> snapshot;
94 public AbstractLeader(RaftActorContext context) {
95 super(context, RaftState.Leader);
97 final Builder<String, FollowerLogInformation> ftlBuilder = ImmutableMap.builder();
98 for (String followerId : context.getPeerAddresses().keySet()) {
99 FollowerLogInformation followerLogInformation =
100 new FollowerLogInformationImpl(followerId, -1, context);
102 ftlBuilder.put(followerId, followerLogInformation);
104 followerToLog = ftlBuilder.build();
106 leaderId = context.getId();
108 LOG.debug("{}: Election: Leader has following peers: {}", logName(), getFollowerIds());
110 minReplicationCount = getMajorityVoteCount(getFollowerIds().size());
112 // the isolated Leader peer count will be 1 less than the majority vote count.
113 // this is because the vote count has the self vote counted in it
115 // 0 peers = 1 votesRequired , minIsolatedLeaderPeerCount = 0
116 // 2 peers = 2 votesRequired , minIsolatedLeaderPeerCount = 1
117 // 4 peers = 3 votesRequired, minIsolatedLeaderPeerCount = 2
118 minIsolatedLeaderPeerCount = minReplicationCount > 0 ? (minReplicationCount - 1) : 0;
120 snapshot = Optional.absent();
122 // Immediately schedule a heartbeat
123 // Upon election: send initial empty AppendEntries RPCs
124 // (heartbeat) to each server; repeat during idle periods to
125 // prevent election timeouts (§5.2)
126 sendAppendEntries(0, false);
128 // It is important to schedule this heartbeat here
129 scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
133 * Return an immutable collection of follower identifiers.
135 * @return Collection of follower IDs
137 protected final Collection<String> getFollowerIds() {
138 return followerToLog.keySet();
142 void setSnapshot(Optional<ByteString> snapshot) {
143 this.snapshot = snapshot;
147 protected RaftActorBehavior handleAppendEntries(ActorRef sender,
148 AppendEntries appendEntries) {
150 LOG.debug("{}: handleAppendEntries: {}", logName(), appendEntries);
156 protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
157 AppendEntriesReply appendEntriesReply) {
159 if(LOG.isTraceEnabled()) {
160 LOG.trace("{}: handleAppendEntriesReply: {}", logName(), appendEntriesReply);
161 } else if(LOG.isDebugEnabled() && !appendEntriesReply.isSuccess()) {
162 LOG.debug("{}: handleAppendEntriesReply: {}", logName(), appendEntriesReply);
165 // Update the FollowerLogInformation
166 String followerId = appendEntriesReply.getFollowerId();
167 FollowerLogInformation followerLogInformation =
168 followerToLog.get(followerId);
170 if(followerLogInformation == null){
171 LOG.error("{}: handleAppendEntriesReply - unknown follower {}", logName(), followerId);
175 if(followerLogInformation.timeSinceLastActivity() >
176 context.getConfigParams().getElectionTimeOutInterval().toMillis()) {
177 LOG.error("{} : handleAppendEntriesReply delayed beyond election timeout, " +
178 "appendEntriesReply : {}, timeSinceLastActivity : {}, lastApplied : {}, commitIndex : {}",
179 logName(), appendEntriesReply, followerLogInformation.timeSinceLastActivity(),
180 context.getLastApplied(), context.getCommitIndex());
183 followerLogInformation.markFollowerActive();
185 if (appendEntriesReply.isSuccess()) {
186 followerLogInformation
187 .setMatchIndex(appendEntriesReply.getLogLastIndex());
188 followerLogInformation
189 .setNextIndex(appendEntriesReply.getLogLastIndex() + 1);
192 // TODO: When we find that the follower is out of sync with the
193 // Leader we simply decrement that followers next index by 1.
194 // Would it be possible to do better than this? The RAFT spec
195 // does not explicitly deal with it but may be something for us to
198 followerLogInformation.decrNextIndex();
201 // Now figure out if this reply warrants a change in the commitIndex
202 // If there exists an N such that N > commitIndex, a majority
203 // of matchIndex[i] ≥ N, and log[N].term == currentTerm:
204 // set commitIndex = N (§5.3, §5.4).
205 for (long N = context.getCommitIndex() + 1; ; N++) {
206 int replicatedCount = 1;
208 for (FollowerLogInformation info : followerToLog.values()) {
209 if (info.getMatchIndex() >= N) {
214 if (replicatedCount >= minReplicationCount) {
215 ReplicatedLogEntry replicatedLogEntry = context.getReplicatedLog().get(N);
216 if (replicatedLogEntry != null &&
217 replicatedLogEntry.getTerm() == currentTerm()) {
218 context.setCommitIndex(N);
225 // Apply the change to the state machine
226 if (context.getCommitIndex() > context.getLastApplied()) {
227 LOG.debug("{}: handleAppendEntriesReply: applying to log - commitIndex: {}, lastAppliedIndex: {}",
228 logName(), context.getCommitIndex(), context.getLastApplied());
230 applyLogToStateMachine(context.getCommitIndex());
233 if (!context.isSnapshotCaptureInitiated()) {
237 //Send the next log entry immediately, if possible, no need to wait for heartbeat to trigger that event
238 sendUpdatesToFollower(followerId, followerLogInformation, false, false);
242 private void purgeInMemoryLog() {
243 //find the lowest index across followers which has been replicated to all.
244 // lastApplied if there are no followers, so that we keep clearing the log for single-node
245 // we would delete the in-mem log from that index on, in-order to minimize mem usage
246 // we would also share this info thru AE with the followers so that they can delete their log entries as well.
247 long minReplicatedToAllIndex = followerToLog.isEmpty() ? context.getLastApplied() : Long.MAX_VALUE;
248 for (FollowerLogInformation info : followerToLog.values()) {
249 minReplicatedToAllIndex = Math.min(minReplicatedToAllIndex, info.getMatchIndex());
252 super.performSnapshotWithoutCapture(minReplicatedToAllIndex);
256 protected ClientRequestTracker removeClientRequestTracker(long logIndex) {
257 final Iterator<ClientRequestTracker> it = trackerList.iterator();
258 while (it.hasNext()) {
259 final ClientRequestTracker t = it.next();
260 if (t.getIndex() == logIndex) {
270 protected ClientRequestTracker findClientRequestTracker(long logIndex) {
271 for (ClientRequestTracker tracker : trackerList) {
272 if (tracker.getIndex() == logIndex) {
280 protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
281 RequestVoteReply requestVoteReply) {
285 protected void beforeSendHeartbeat(){}
288 public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) {
289 Preconditions.checkNotNull(sender, "sender should not be null");
291 Object message = fromSerializableMessage(originalMessage);
293 if (message instanceof RaftRPC) {
294 RaftRPC rpc = (RaftRPC) message;
295 // If RPC request or response contains term T > currentTerm:
296 // set currentTerm = T, convert to follower (§5.1)
297 // This applies to all RPC messages and responses
298 if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
299 LOG.debug("{}: Term {} in \"{}\" message is greater than leader's term {} - switching to Follower",
300 logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm());
302 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
304 return switchBehavior(new Follower(context));
308 if (message instanceof SendHeartBeat) {
309 beforeSendHeartbeat();
311 scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
314 } else if(message instanceof SendInstallSnapshot) {
315 // received from RaftActor
316 setSnapshot(Optional.of(((SendInstallSnapshot) message).getSnapshot()));
317 sendInstallSnapshot();
319 } else if (message instanceof Replicate) {
320 replicate((Replicate) message);
322 } else if (message instanceof InstallSnapshotReply){
323 handleInstallSnapshotReply((InstallSnapshotReply) message);
328 return super.handleMessage(sender, message);
331 private void handleInstallSnapshotReply(InstallSnapshotReply reply) {
332 LOG.debug("{}: handleInstallSnapshotReply: {}", logName(), reply);
334 String followerId = reply.getFollowerId();
335 FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
337 if (followerToSnapshot == null) {
338 LOG.error("{}: FollowerId {} in InstallSnapshotReply not known to Leader",
339 logName(), followerId);
343 FollowerLogInformation followerLogInformation = followerToLog.get(followerId);
344 followerLogInformation.markFollowerActive();
346 if (followerToSnapshot.getChunkIndex() == reply.getChunkIndex()) {
347 boolean wasLastChunk = false;
348 if (reply.isSuccess()) {
349 if(followerToSnapshot.isLastChunk(reply.getChunkIndex())) {
350 //this was the last chunk reply
351 if(LOG.isDebugEnabled()) {
352 LOG.debug("{}: InstallSnapshotReply received, " +
353 "last chunk received, Chunk: {}. Follower: {} Setting nextIndex: {}",
354 logName(), reply.getChunkIndex(), followerId,
355 context.getReplicatedLog().getSnapshotIndex() + 1
359 followerLogInformation.setMatchIndex(
360 context.getReplicatedLog().getSnapshotIndex());
361 followerLogInformation.setNextIndex(
362 context.getReplicatedLog().getSnapshotIndex() + 1);
363 mapFollowerToSnapshot.remove(followerId);
365 LOG.debug("{}: follower: {}, matchIndex set to {}, nextIndex set to {}",
366 logName(), followerId, followerLogInformation.getMatchIndex(),
367 followerLogInformation.getNextIndex());
369 if (mapFollowerToSnapshot.isEmpty()) {
370 // once there are no pending followers receiving snapshots
371 // we can remove snapshot from the memory
372 setSnapshot(Optional.<ByteString>absent());
377 followerToSnapshot.markSendStatus(true);
380 LOG.info("{}: InstallSnapshotReply received sending snapshot chunk failed, Will retry, Chunk: {}",
381 logName(), reply.getChunkIndex());
383 followerToSnapshot.markSendStatus(false);
386 if (wasLastChunk && !context.isSnapshotCaptureInitiated()) {
387 // Since the follower is now caught up try to purge the log.
389 } else if (!wasLastChunk && followerToSnapshot.canSendNextChunk()) {
390 ActorSelection followerActor = context.getPeerActorSelection(followerId);
391 if(followerActor != null) {
392 sendSnapshotChunk(followerActor, followerId);
397 LOG.error("{}: Chunk index {} in InstallSnapshotReply from follower {} does not match expected index {}",
398 logName(), reply.getChunkIndex(), followerId,
399 followerToSnapshot.getChunkIndex());
401 if(reply.getChunkIndex() == INVALID_CHUNK_INDEX){
402 // Since the Follower did not find this index to be valid we should reset the follower snapshot
403 // so that Installing the snapshot can resume from the beginning
404 followerToSnapshot.reset();
409 private void replicate(Replicate replicate) {
410 long logIndex = replicate.getReplicatedLogEntry().getIndex();
412 LOG.debug("{}: Replicate message: identifier: {}, logIndex: {}", logName(),
413 replicate.getIdentifier(), logIndex);
415 // Create a tracker entry we will use this later to notify the
418 new ClientRequestTrackerImpl(replicate.getClientActor(),
419 replicate.getIdentifier(),
423 if (followerToLog.isEmpty()) {
424 context.setCommitIndex(logIndex);
425 applyLogToStateMachine(logIndex);
427 sendAppendEntries(0, false);
431 private void sendAppendEntries(long timeSinceLastActivityInterval, boolean isHeartbeat) {
432 // Send an AppendEntries to all followers
433 for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
434 final String followerId = e.getKey();
435 final FollowerLogInformation followerLogInformation = e.getValue();
436 // This checks helps not to send a repeat message to the follower
437 if(!followerLogInformation.isFollowerActive() ||
438 followerLogInformation.timeSinceLastActivity() >= timeSinceLastActivityInterval) {
439 sendUpdatesToFollower(followerId, followerLogInformation, true, isHeartbeat);
446 * This method checks if any update needs to be sent to the given follower. This includes append log entries,
447 * sending next snapshot chunk, and initiating a snapshot.
448 * @return true if any update is sent, false otherwise
451 private void sendUpdatesToFollower(String followerId, FollowerLogInformation followerLogInformation,
452 boolean sendHeartbeat, boolean isHeartbeat) {
454 ActorSelection followerActor = context.getPeerActorSelection(followerId);
455 if (followerActor != null) {
456 long followerNextIndex = followerLogInformation.getNextIndex();
457 boolean isFollowerActive = followerLogInformation.isFollowerActive();
459 if (mapFollowerToSnapshot.get(followerId) != null) {
460 // if install snapshot is in process , then sent next chunk if possible
461 if (isFollowerActive && mapFollowerToSnapshot.get(followerId).canSendNextChunk()) {
462 sendSnapshotChunk(followerActor, followerId);
463 } else if(sendHeartbeat) {
464 // we send a heartbeat even if we have not received a reply for the last chunk
465 sendAppendEntriesToFollower(followerActor, followerLogInformation.getNextIndex(),
466 Collections.<ReplicatedLogEntry>emptyList(), followerId);
469 long leaderLastIndex = context.getReplicatedLog().lastIndex();
470 long leaderSnapShotIndex = context.getReplicatedLog().getSnapshotIndex();
472 if(!isHeartbeat || LOG.isTraceEnabled()) {
473 LOG.debug("{}: Checking sendAppendEntries for follower {}, leaderLastIndex: {}, leaderSnapShotIndex: {}",
474 logName(), followerId, leaderLastIndex, leaderSnapShotIndex);
477 if (isFollowerActive && context.getReplicatedLog().isPresent(followerNextIndex)) {
479 LOG.debug("{}: sendAppendEntries: {} is present for follower {}", logName(),
480 followerNextIndex, followerId);
482 // FIXME : Sending one entry at a time
483 final List<ReplicatedLogEntry> entries = context.getReplicatedLog().getFrom(followerNextIndex, 1);
485 sendAppendEntriesToFollower(followerActor, followerNextIndex, entries, followerId);
487 } else if (isFollowerActive && followerNextIndex >= 0 &&
488 leaderLastIndex > followerNextIndex && !context.isSnapshotCaptureInitiated()) {
489 // if the followers next index is not present in the leaders log, and
490 // if the follower is just not starting and if leader's index is more than followers index
491 // then snapshot should be sent
493 if (LOG.isDebugEnabled()) {
494 LOG.debug(String.format("%s: InitiateInstallSnapshot to follower: %s," +
495 "follower-nextIndex: %d, leader-snapshot-index: %d, " +
496 "leader-last-index: %d", logName(), followerId,
497 followerNextIndex, leaderSnapShotIndex, leaderLastIndex));
500 // Send heartbeat to follower whenever install snapshot is initiated.
501 sendAppendEntriesToFollower(followerActor, followerLogInformation.getNextIndex(),
502 Collections.<ReplicatedLogEntry>emptyList(), followerId);
504 initiateCaptureSnapshot(followerId, followerNextIndex);
506 } else if(sendHeartbeat) {
507 //we send an AppendEntries, even if the follower is inactive
508 // in-order to update the followers timestamp, in case it becomes active again
509 sendAppendEntriesToFollower(followerActor, followerLogInformation.getNextIndex(),
510 Collections.<ReplicatedLogEntry>emptyList(), followerId);
517 private void sendAppendEntriesToFollower(ActorSelection followerActor, long followerNextIndex,
518 List<ReplicatedLogEntry> entries, String followerId) {
519 AppendEntries appendEntries = new AppendEntries(currentTerm(), context.getId(),
520 prevLogIndex(followerNextIndex),
521 prevLogTerm(followerNextIndex), entries,
522 context.getCommitIndex(), super.getReplicatedToAllIndex());
524 if(!entries.isEmpty() || LOG.isTraceEnabled()) {
525 LOG.debug("{}: Sending AppendEntries to follower {}: {}", logName(), followerId,
529 followerActor.tell(appendEntries.toSerializable(), actor());
533 * Install Snapshot works as follows
534 * 1. Leader initiates the capture snapshot by sending a CaptureSnapshot message to actor
535 * 2. RaftActor on receipt of the CaptureSnapshotReply (from Shard), stores the received snapshot in the replicated log
536 * and makes a call to Leader's handleMessage , with SendInstallSnapshot message.
537 * 3. Leader , picks the snapshot from im-mem ReplicatedLog and sends it in chunks to the Follower
538 * 4. On complete, Follower sends back a InstallSnapshotReply.
539 * 5. On receipt of the InstallSnapshotReply for the last chunk, Leader marks the install complete for that follower
540 * and replenishes the memory by deleting the snapshot in Replicated log.
541 * 6. If another follower requires a snapshot and a snapshot has been collected (via CaptureSnapshotReply)
542 * then send the existing snapshot in chunks to the follower.
544 * @param followerNextIndex
546 private void initiateCaptureSnapshot(String followerId, long followerNextIndex) {
547 if (!context.getReplicatedLog().isPresent(followerNextIndex) &&
548 context.getReplicatedLog().isInSnapshot(followerNextIndex)) {
550 if (snapshot.isPresent()) {
551 // if a snapshot is present in the memory, most likely another install is in progress
552 // no need to capture snapshot.
553 // This could happen if another follower needs an install when one is going on.
554 final ActorSelection followerActor = context.getPeerActorSelection(followerId);
555 sendSnapshotChunk(followerActor, followerId);
557 } else if (!context.isSnapshotCaptureInitiated()) {
559 LOG.info("{}: Initiating Snapshot Capture to Install Snapshot, Leader:{}", logName(), getLeaderId());
560 ReplicatedLogEntry lastAppliedEntry = context.getReplicatedLog().get(context.getLastApplied());
561 long lastAppliedIndex = -1;
562 long lastAppliedTerm = -1;
564 if (lastAppliedEntry != null) {
565 lastAppliedIndex = lastAppliedEntry.getIndex();
566 lastAppliedTerm = lastAppliedEntry.getTerm();
567 } else if (context.getReplicatedLog().getSnapshotIndex() > -1) {
568 lastAppliedIndex = context.getReplicatedLog().getSnapshotIndex();
569 lastAppliedTerm = context.getReplicatedLog().getSnapshotTerm();
572 boolean isInstallSnapshotInitiated = true;
573 long replicatedToAllIndex = super.getReplicatedToAllIndex();
574 ReplicatedLogEntry replicatedToAllEntry = context.getReplicatedLog().get(replicatedToAllIndex);
575 actor().tell(new CaptureSnapshot(lastIndex(), lastTerm(), lastAppliedIndex, lastAppliedTerm,
576 (replicatedToAllEntry != null ? replicatedToAllEntry.getIndex() : -1),
577 (replicatedToAllEntry != null ? replicatedToAllEntry.getTerm() : -1),
578 isInstallSnapshotInitiated), actor());
579 context.setSnapshotCaptureInitiated(true);
585 private void sendInstallSnapshot() {
586 LOG.debug("{}: sendInstallSnapshot", logName());
587 for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
588 ActorSelection followerActor = context.getPeerActorSelection(e.getKey());
590 if (followerActor != null) {
591 long nextIndex = e.getValue().getNextIndex();
593 if (!context.getReplicatedLog().isPresent(nextIndex) &&
594 context.getReplicatedLog().isInSnapshot(nextIndex)) {
595 sendSnapshotChunk(followerActor, e.getKey());
602 * Sends a snapshot chunk to a given follower
603 * InstallSnapshot should qualify as a heartbeat too.
605 private void sendSnapshotChunk(ActorSelection followerActor, String followerId) {
607 if (snapshot.isPresent()) {
608 ByteString nextSnapshotChunk = getNextSnapshotChunk(followerId,snapshot.get());
610 // Note: the previous call to getNextSnapshotChunk has the side-effect of adding
611 // followerId to the followerToSnapshot map.
612 FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
615 new InstallSnapshot(currentTerm(), context.getId(),
616 context.getReplicatedLog().getSnapshotIndex(),
617 context.getReplicatedLog().getSnapshotTerm(),
619 followerToSnapshot.incrementChunkIndex(),
620 followerToSnapshot.getTotalChunks(),
621 Optional.of(followerToSnapshot.getLastChunkHashCode())
625 LOG.info("{}: InstallSnapshot sent to follower {}, Chunk: {}/{}",
626 logName(), followerActor.path(),
627 followerToSnapshot.getChunkIndex(),
628 followerToSnapshot.getTotalChunks());
630 } catch (IOException e) {
631 LOG.error("{}: InstallSnapshot failed for Leader.", logName(), e);
636 * Acccepts snaphot as ByteString, enters into map for future chunks
637 * creates and return a ByteString chunk
639 private ByteString getNextSnapshotChunk(String followerId, ByteString snapshotBytes) throws IOException {
640 FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
641 if (followerToSnapshot == null) {
642 followerToSnapshot = new FollowerToSnapshot(snapshotBytes);
643 mapFollowerToSnapshot.put(followerId, followerToSnapshot);
645 ByteString nextChunk = followerToSnapshot.getNextChunk();
647 LOG.debug("{}: next snapshot chunk size for follower {}: {}", logName(), followerId, nextChunk.size());
652 private void sendHeartBeat() {
653 if (!followerToLog.isEmpty()) {
654 LOG.trace("{}: Sending heartbeat", logName());
655 sendAppendEntries(context.getConfigParams().getHeartBeatInterval().toMillis(), true);
659 private void stopHeartBeat() {
660 if (heartbeatSchedule != null && !heartbeatSchedule.isCancelled()) {
661 heartbeatSchedule.cancel();
665 private void scheduleHeartBeat(FiniteDuration interval) {
666 if (followerToLog.isEmpty()) {
667 // Optimization - do not bother scheduling a heartbeat as there are
674 // Schedule a heartbeat. When the scheduler triggers a SendHeartbeat
675 // message is sent to itself.
676 // Scheduling the heartbeat only once here because heartbeats do not
677 // need to be sent if there are other messages being sent to the remote
679 heartbeatSchedule = context.getActorSystem().scheduler().scheduleOnce(
680 interval, context.getActor(), new SendHeartBeat(),
681 context.getActorSystem().dispatcher(), context.getActor());
685 public void close() throws Exception {
690 public String getLeaderId() {
691 return context.getId();
694 protected boolean isLeaderIsolated() {
695 int minPresent = minIsolatedLeaderPeerCount;
696 for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
697 if (followerLogInformation.isFollowerActive()) {
699 if (minPresent == 0) {
704 return (minPresent != 0);
708 * Encapsulates the snapshot bytestring and handles the logic of sending
711 protected class FollowerToSnapshot {
712 private final ByteString snapshotBytes;
713 private int offset = 0;
714 // the next snapshot chunk is sent only if the replyReceivedForOffset matches offset
715 private int replyReceivedForOffset;
716 // if replyStatus is false, the previous chunk is attempted
717 private boolean replyStatus = false;
718 private int chunkIndex;
719 private final int totalChunks;
720 private int lastChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
721 private int nextChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
723 public FollowerToSnapshot(ByteString snapshotBytes) {
724 this.snapshotBytes = snapshotBytes;
725 int size = snapshotBytes.size();
726 totalChunks = ( size / context.getConfigParams().getSnapshotChunkSize()) +
727 ((size % context.getConfigParams().getSnapshotChunkSize()) > 0 ? 1 : 0);
728 if(LOG.isDebugEnabled()) {
729 LOG.debug("{}: Snapshot {} bytes, total chunks to send:{}",
730 logName(), size, totalChunks);
732 replyReceivedForOffset = -1;
733 chunkIndex = AbstractLeader.FIRST_CHUNK_INDEX;
736 public ByteString getSnapshotBytes() {
737 return snapshotBytes;
740 public int incrementOffset() {
742 // if prev chunk failed, we would want to sent the same chunk again
743 offset = offset + context.getConfigParams().getSnapshotChunkSize();
748 public int incrementChunkIndex() {
750 // if prev chunk failed, we would want to sent the same chunk again
751 chunkIndex = chunkIndex + 1;
756 public int getChunkIndex() {
760 public int getTotalChunks() {
764 public boolean canSendNextChunk() {
765 // we only send a false if a chunk is sent but we have not received a reply yet
766 return replyReceivedForOffset == offset;
769 public boolean isLastChunk(int chunkIndex) {
770 return totalChunks == chunkIndex;
773 public void markSendStatus(boolean success) {
775 // if the chunk sent was successful
776 replyReceivedForOffset = offset;
778 lastChunkHashCode = nextChunkHashCode;
780 // if the chunk sent was failure
781 replyReceivedForOffset = offset;
786 public ByteString getNextChunk() {
787 int snapshotLength = getSnapshotBytes().size();
788 int start = incrementOffset();
789 int size = context.getConfigParams().getSnapshotChunkSize();
790 if (context.getConfigParams().getSnapshotChunkSize() > snapshotLength) {
791 size = snapshotLength;
793 if ((start + context.getConfigParams().getSnapshotChunkSize()) > snapshotLength) {
794 size = snapshotLength - start;
799 LOG.debug("{}: Next chunk: length={}, offset={},size={}", logName(),
800 snapshotLength, start, size);
802 ByteString substring = getSnapshotBytes().substring(start, start + size);
803 nextChunkHashCode = substring.hashCode();
808 * reset should be called when the Follower needs to be sent the snapshot from the beginning
813 replyReceivedForOffset = offset;
814 chunkIndex = AbstractLeader.FIRST_CHUNK_INDEX;
815 lastChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
818 public int getLastChunkHashCode() {
819 return lastChunkHashCode;
823 // called from example-actor for printing the follower-states
824 public String printFollowerStates() {
825 final StringBuilder sb = new StringBuilder();
828 for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
830 sb.append(followerLogInformation.getId());
831 sb.append(" state:");
832 sb.append(followerLogInformation.isFollowerActive());
837 return sb.toString();
841 public FollowerLogInformation getFollower(String followerId) {
842 return followerToLog.get(followerId);
846 protected void setFollowerSnapshot(String followerId, FollowerToSnapshot snapshot) {
847 mapFollowerToSnapshot.put(followerId, snapshot);
851 public int followerSnapshotSize() {
852 return mapFollowerToSnapshot.size();
856 public int followerLogSize() {
857 return followerToLog.size();