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 private long replicatedToAllIndex = -1;
96 public AbstractLeader(RaftActorContext context) {
99 final Builder<String, FollowerLogInformation> ftlBuilder = ImmutableMap.builder();
100 for (String followerId : context.getPeerAddresses().keySet()) {
101 FollowerLogInformation followerLogInformation =
102 new FollowerLogInformationImpl(followerId,
103 context.getCommitIndex(), -1,
104 context.getConfigParams().getElectionTimeOutInterval());
106 ftlBuilder.put(followerId, followerLogInformation);
108 followerToLog = ftlBuilder.build();
110 leaderId = context.getId();
112 LOG.debug("{}: Election: Leader has following peers: {}", context.getId(), getFollowerIds());
114 minReplicationCount = getMajorityVoteCount(getFollowerIds().size());
116 // the isolated Leader peer count will be 1 less than the majority vote count.
117 // this is because the vote count has the self vote counted in it
119 // 0 peers = 1 votesRequired , minIsolatedLeaderPeerCount = 0
120 // 2 peers = 2 votesRequired , minIsolatedLeaderPeerCount = 1
121 // 4 peers = 3 votesRequired, minIsolatedLeaderPeerCount = 2
122 minIsolatedLeaderPeerCount = minReplicationCount > 0 ? (minReplicationCount - 1) : 0;
124 snapshot = Optional.absent();
126 // Immediately schedule a heartbeat
127 // Upon election: send initial empty AppendEntries RPCs
128 // (heartbeat) to each server; repeat during idle periods to
129 // prevent election timeouts (§5.2)
130 sendAppendEntries(0);
134 * Return an immutable collection of follower identifiers.
136 * @return Collection of follower IDs
138 protected final Collection<String> getFollowerIds() {
139 return followerToLog.keySet();
142 private Optional<ByteString> getSnapshot() {
147 void setSnapshot(Optional<ByteString> snapshot) {
148 this.snapshot = snapshot;
152 protected RaftActorBehavior handleAppendEntries(ActorRef sender,
153 AppendEntries appendEntries) {
155 if(LOG.isDebugEnabled()) {
156 LOG.debug("{}: handleAppendEntries: {}", context.getId(), appendEntries);
163 protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
164 AppendEntriesReply appendEntriesReply) {
166 if(! appendEntriesReply.isSuccess()) {
167 if(LOG.isDebugEnabled()) {
168 LOG.debug("{}: handleAppendEntriesReply: {}", context.getId(), appendEntriesReply);
172 // Update the FollowerLogInformation
173 String followerId = appendEntriesReply.getFollowerId();
174 FollowerLogInformation followerLogInformation =
175 followerToLog.get(followerId);
177 if(followerLogInformation == null){
178 LOG.error("{}: handleAppendEntriesReply - unknown follower {}", context.getId(), followerId);
182 followerLogInformation.markFollowerActive();
184 if (appendEntriesReply.isSuccess()) {
185 followerLogInformation
186 .setMatchIndex(appendEntriesReply.getLogLastIndex());
187 followerLogInformation
188 .setNextIndex(appendEntriesReply.getLogLastIndex() + 1);
191 // TODO: When we find that the follower is out of sync with the
192 // Leader we simply decrement that followers next index by 1.
193 // Would it be possible to do better than this? The RAFT spec
194 // does not explicitly deal with it but may be something for us to
197 followerLogInformation.decrNextIndex();
200 // Now figure out if this reply warrants a change in the commitIndex
201 // If there exists an N such that N > commitIndex, a majority
202 // of matchIndex[i] ≥ N, and log[N].term == currentTerm:
203 // set commitIndex = N (§5.3, §5.4).
204 for (long N = context.getCommitIndex() + 1; ; N++) {
205 int replicatedCount = 1;
207 for (FollowerLogInformation info : followerToLog.values()) {
208 if (info.getMatchIndex() >= N) {
213 if (replicatedCount >= minReplicationCount) {
214 ReplicatedLogEntry replicatedLogEntry = context.getReplicatedLog().get(N);
215 if (replicatedLogEntry != null &&
216 replicatedLogEntry.getTerm() == currentTerm()) {
217 context.setCommitIndex(N);
224 // Apply the change to the state machine
225 if (context.getCommitIndex() > context.getLastApplied()) {
226 applyLogToStateMachine(context.getCommitIndex());
229 if (!context.isSnapshotCaptureInitiated()) {
233 //Send the next log entry immediately, if possible, no need to wait for heartbeat to trigger that event
234 sendUpdatesToFollower(followerId, followerLogInformation, false);
238 private void purgeInMemoryLog() {
239 //find the lowest index across followers which has been replicated to all. -1 if there are no followers.
240 // we would delete the in-mem log from that index on, in-order to minimize mem usage
241 // we would also share this info thru AE with the followers so that they can delete their log entries as well.
242 long minReplicatedToAllIndex = followerToLog.isEmpty() ? -1 : Long.MAX_VALUE;
243 for (FollowerLogInformation info : followerToLog.values()) {
244 minReplicatedToAllIndex = Math.min(minReplicatedToAllIndex, info.getMatchIndex());
247 replicatedToAllIndex = fakeSnapshot(minReplicatedToAllIndex, replicatedToAllIndex);
251 protected ClientRequestTracker removeClientRequestTracker(long logIndex) {
252 final Iterator<ClientRequestTracker> it = trackerList.iterator();
253 while (it.hasNext()) {
254 final ClientRequestTracker t = it.next();
255 if (t.getIndex() == logIndex) {
265 protected ClientRequestTracker findClientRequestTracker(long logIndex) {
266 for (ClientRequestTracker tracker : trackerList) {
267 if (tracker.getIndex() == logIndex) {
275 protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
276 RequestVoteReply requestVoteReply) {
281 public RaftState state() {
282 return RaftState.Leader;
286 public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) {
287 Preconditions.checkNotNull(sender, "sender should not be null");
289 Object message = fromSerializableMessage(originalMessage);
291 if (message instanceof RaftRPC) {
292 RaftRPC rpc = (RaftRPC) message;
293 // If RPC request or response contains term T > currentTerm:
294 // set currentTerm = T, convert to follower (§5.1)
295 // This applies to all RPC messages and responses
296 if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
297 LOG.debug("{}: Term {} in \"{}\" message is greater than leader's term {}", context.getId(),
298 rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm());
300 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
302 return switchBehavior(new Follower(context));
307 if (message instanceof SendHeartBeat) {
311 } else if(message instanceof SendInstallSnapshot) {
312 // received from RaftActor
313 setSnapshot(Optional.of(((SendInstallSnapshot) message).getSnapshot()));
314 sendInstallSnapshot();
316 } else if (message instanceof Replicate) {
317 replicate((Replicate) message);
319 } else if (message instanceof InstallSnapshotReply){
320 handleInstallSnapshotReply((InstallSnapshotReply) message);
324 scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
327 return super.handleMessage(sender, message);
330 private void handleInstallSnapshotReply(InstallSnapshotReply reply) {
331 String followerId = reply.getFollowerId();
332 FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
334 if (followerToSnapshot == null) {
335 LOG.error("{}: FollowerId {} in InstallSnapshotReply not known to Leader",
336 context.getId(), followerId);
340 FollowerLogInformation followerLogInformation = followerToLog.get(followerId);
341 followerLogInformation.markFollowerActive();
343 if (followerToSnapshot.getChunkIndex() == reply.getChunkIndex()) {
344 boolean wasLastChunk = false;
345 if (reply.isSuccess()) {
346 if(followerToSnapshot.isLastChunk(reply.getChunkIndex())) {
347 //this was the last chunk reply
348 if(LOG.isDebugEnabled()) {
349 LOG.debug("{}: InstallSnapshotReply received, " +
350 "last chunk received, Chunk:{}. Follower:{} Setting nextIndex:{}",
351 context.getId(), reply.getChunkIndex(), followerId,
352 context.getReplicatedLog().getSnapshotIndex() + 1
356 followerLogInformation.setMatchIndex(
357 context.getReplicatedLog().getSnapshotIndex());
358 followerLogInformation.setNextIndex(
359 context.getReplicatedLog().getSnapshotIndex() + 1);
360 mapFollowerToSnapshot.remove(followerId);
362 if(LOG.isDebugEnabled()) {
363 LOG.debug("{}: followerToLog.get(followerId).getNextIndex()=" +
364 context.getId(), followerToLog.get(followerId).getNextIndex());
367 if (mapFollowerToSnapshot.isEmpty()) {
368 // once there are no pending followers receiving snapshots
369 // we can remove snapshot from the memory
370 setSnapshot(Optional.<ByteString>absent());
375 followerToSnapshot.markSendStatus(true);
378 LOG.info("{}: InstallSnapshotReply received sending snapshot chunk failed, Will retry, Chunk: {}",
379 context.getId(), reply.getChunkIndex());
381 followerToSnapshot.markSendStatus(false);
384 if (!wasLastChunk && followerToSnapshot.canSendNextChunk()) {
385 ActorSelection followerActor = context.getPeerActorSelection(followerId);
386 if(followerActor != null) {
387 sendSnapshotChunk(followerActor, followerId);
392 LOG.error("{}: Chunk index {} in InstallSnapshotReply from follower {} does not match expected index {}",
393 context.getId(), reply.getChunkIndex(), followerId,
394 followerToSnapshot.getChunkIndex());
396 if(reply.getChunkIndex() == INVALID_CHUNK_INDEX){
397 // Since the Follower did not find this index to be valid we should reset the follower snapshot
398 // so that Installing the snapshot can resume from the beginning
399 followerToSnapshot.reset();
404 private void replicate(Replicate replicate) {
405 long logIndex = replicate.getReplicatedLogEntry().getIndex();
407 if(LOG.isDebugEnabled()) {
408 LOG.debug("{}: Replicate message {}", context.getId(), logIndex);
411 // Create a tracker entry we will use this later to notify the
414 new ClientRequestTrackerImpl(replicate.getClientActor(),
415 replicate.getIdentifier(),
419 if (followerToLog.isEmpty()) {
420 context.setCommitIndex(logIndex);
421 applyLogToStateMachine(logIndex);
423 sendAppendEntries(0);
427 private void sendAppendEntries(long timeSinceLastActivityInterval) {
428 // Send an AppendEntries to all followers
429 for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
430 final String followerId = e.getKey();
431 final FollowerLogInformation followerLogInformation = e.getValue();
432 // This checks helps not to send a repeat message to the follower
433 if(!followerLogInformation.isFollowerActive() ||
434 followerLogInformation.timeSinceLastActivity() >= timeSinceLastActivityInterval) {
435 sendUpdatesToFollower(followerId, followerLogInformation, true);
442 * This method checks if any update needs to be sent to the given follower. This includes append log entries,
443 * sending next snapshot chunk, and initiating a snapshot.
444 * @return true if any update is sent, false otherwise
447 private void sendUpdatesToFollower(String followerId, FollowerLogInformation followerLogInformation,
448 boolean sendHeartbeat) {
450 ActorSelection followerActor = context.getPeerActorSelection(followerId);
451 if (followerActor != null) {
452 long followerNextIndex = followerLogInformation.getNextIndex();
453 boolean isFollowerActive = followerLogInformation.isFollowerActive();
455 if (mapFollowerToSnapshot.get(followerId) != null) {
456 // if install snapshot is in process , then sent next chunk if possible
457 if (isFollowerActive && mapFollowerToSnapshot.get(followerId).canSendNextChunk()) {
458 sendSnapshotChunk(followerActor, followerId);
459 } else if(sendHeartbeat) {
460 // we send a heartbeat even if we have not received a reply for the last chunk
461 sendAppendEntriesToFollower(followerActor, followerLogInformation.getNextIndex(),
462 Collections.<ReplicatedLogEntry>emptyList(), followerId);
465 long leaderLastIndex = context.getReplicatedLog().lastIndex();
466 long leaderSnapShotIndex = context.getReplicatedLog().getSnapshotIndex();
467 if (isFollowerActive &&
468 context.getReplicatedLog().isPresent(followerNextIndex)) {
469 // FIXME : Sending one entry at a time
470 final List<ReplicatedLogEntry> entries = context.getReplicatedLog().getFrom(followerNextIndex, 1);
472 sendAppendEntriesToFollower(followerActor, followerNextIndex, entries, followerId);
474 } else if (isFollowerActive && followerNextIndex >= 0 &&
475 leaderLastIndex >= followerNextIndex) {
476 // if the followers next index is not present in the leaders log, and
477 // if the follower is just not starting and if leader's index is more than followers index
478 // then snapshot should be sent
480 if (LOG.isDebugEnabled()) {
481 LOG.debug("InitiateInstallSnapshot to follower:{}," +
482 "follower-nextIndex:{}, leader-snapshot-index:{}, " +
483 "leader-last-index:{}", followerId,
484 followerNextIndex, leaderSnapShotIndex, leaderLastIndex
488 // Send heartbeat to follower whenever install snapshot is initiated.
489 sendAppendEntriesToFollower(followerActor, followerLogInformation.getNextIndex(),
490 Collections.<ReplicatedLogEntry>emptyList(), followerId);
492 initiateCaptureSnapshot(followerId, followerNextIndex);
494 } else if(sendHeartbeat) {
495 //we send an AppendEntries, even if the follower is inactive
496 // in-order to update the followers timestamp, in case it becomes active again
497 sendAppendEntriesToFollower(followerActor, followerLogInformation.getNextIndex(),
498 Collections.<ReplicatedLogEntry>emptyList(), followerId);
505 private void sendAppendEntriesToFollower(ActorSelection followerActor, long followerNextIndex,
506 List<ReplicatedLogEntry> entries, String followerId) {
507 AppendEntries appendEntries = new AppendEntries(currentTerm(), context.getId(),
508 prevLogIndex(followerNextIndex),
509 prevLogTerm(followerNextIndex), entries,
510 context.getCommitIndex(), replicatedToAllIndex);
512 if(!entries.isEmpty()) {
513 LOG.debug("{}: Sending AppendEntries to follower {}: {}", context.getId(), followerId,
517 followerActor.tell(appendEntries.toSerializable(), actor());
522 * Install Snapshot works as follows
523 * 1. Leader initiates the capture snapshot by sending a CaptureSnapshot message to actor
524 * 2. RaftActor on receipt of the CaptureSnapshotReply (from Shard), stores the received snapshot in the replicated log
525 * and makes a call to Leader's handleMessage , with SendInstallSnapshot message.
526 * 3. Leader , picks the snapshot from im-mem ReplicatedLog and sends it in chunks to the Follower
527 * 4. On complete, Follower sends back a InstallSnapshotReply.
528 * 5. On receipt of the InstallSnapshotReply for the last chunk, Leader marks the install complete for that follower
529 * and replenishes the memory by deleting the snapshot in Replicated log.
530 * 6. If another follower requires a snapshot and a snapshot has been collected (via CaptureSnapshotReply)
531 * then send the existing snapshot in chunks to the follower.
533 * @param followerNextIndex
535 private void initiateCaptureSnapshot(String followerId, long followerNextIndex) {
536 if(LOG.isDebugEnabled()) {
537 LOG.debug("{}: initiateCaptureSnapshot, followers {}", context.getId(), followerToLog.keySet());
540 if (!context.getReplicatedLog().isPresent(followerNextIndex) &&
541 context.getReplicatedLog().isInSnapshot(followerNextIndex)) {
543 if (snapshot.isPresent()) {
544 // if a snapshot is present in the memory, most likely another install is in progress
545 // no need to capture snapshot.
546 // This could happen if another follower needs an install when one is going on.
547 final ActorSelection followerActor = context.getPeerActorSelection(followerId);
548 sendSnapshotChunk(followerActor, followerId);
550 } else if (!context.isSnapshotCaptureInitiated()) {
552 LOG.info("{}: Initiating Snapshot Capture to Install Snapshot, Leader:{}", context.getId(), getLeaderId());
553 ReplicatedLogEntry lastAppliedEntry = context.getReplicatedLog().get(context.getLastApplied());
554 long lastAppliedIndex = -1;
555 long lastAppliedTerm = -1;
557 if (lastAppliedEntry != null) {
558 lastAppliedIndex = lastAppliedEntry.getIndex();
559 lastAppliedTerm = lastAppliedEntry.getTerm();
560 } else if (context.getReplicatedLog().getSnapshotIndex() > -1) {
561 lastAppliedIndex = context.getReplicatedLog().getSnapshotIndex();
562 lastAppliedTerm = context.getReplicatedLog().getSnapshotTerm();
565 boolean isInstallSnapshotInitiated = true;
566 actor().tell(new CaptureSnapshot(lastIndex(), lastTerm(),
567 lastAppliedIndex, lastAppliedTerm, isInstallSnapshotInitiated),
569 context.setSnapshotCaptureInitiated(true);
575 private void sendInstallSnapshot() {
576 for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
577 ActorSelection followerActor = context.getPeerActorSelection(e.getKey());
579 if (followerActor != null) {
580 long nextIndex = e.getValue().getNextIndex();
582 if (!context.getReplicatedLog().isPresent(nextIndex) &&
583 context.getReplicatedLog().isInSnapshot(nextIndex)) {
584 sendSnapshotChunk(followerActor, e.getKey());
591 * Sends a snapshot chunk to a given follower
592 * InstallSnapshot should qualify as a heartbeat too.
594 private void sendSnapshotChunk(ActorSelection followerActor, String followerId) {
596 if (snapshot.isPresent()) {
597 ByteString nextSnapshotChunk = getNextSnapshotChunk(followerId,snapshot.get());
599 // Note: the previous call to getNextSnapshotChunk has the side-effect of adding
600 // followerId to the followerToSnapshot map.
601 FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
604 new InstallSnapshot(currentTerm(), context.getId(),
605 context.getReplicatedLog().getSnapshotIndex(),
606 context.getReplicatedLog().getSnapshotTerm(),
608 followerToSnapshot.incrementChunkIndex(),
609 followerToSnapshot.getTotalChunks(),
610 Optional.of(followerToSnapshot.getLastChunkHashCode())
614 LOG.info("{}: InstallSnapshot sent to follower {}, Chunk: {}/{}",
615 context.getId(), followerActor.path(),
616 followerToSnapshot.getChunkIndex(),
617 followerToSnapshot.getTotalChunks());
619 } catch (IOException e) {
620 LOG.error("{}: InstallSnapshot failed for Leader.", context.getId(), e);
625 * Acccepts snaphot as ByteString, enters into map for future chunks
626 * creates and return a ByteString chunk
628 private ByteString getNextSnapshotChunk(String followerId, ByteString snapshotBytes) throws IOException {
629 FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
630 if (followerToSnapshot == null) {
631 followerToSnapshot = new FollowerToSnapshot(snapshotBytes);
632 mapFollowerToSnapshot.put(followerId, followerToSnapshot);
634 ByteString nextChunk = followerToSnapshot.getNextChunk();
635 if (LOG.isDebugEnabled()) {
636 LOG.debug("{}: Leader's snapshot nextChunk size:{}", context.getId(), nextChunk.size());
641 private void sendHeartBeat() {
642 if (!followerToLog.isEmpty()) {
643 sendAppendEntries(context.getConfigParams().getHeartBeatInterval().toMillis());
647 private void stopHeartBeat() {
648 if (heartbeatSchedule != null && !heartbeatSchedule.isCancelled()) {
649 heartbeatSchedule.cancel();
653 private void scheduleHeartBeat(FiniteDuration interval) {
654 if (followerToLog.isEmpty()) {
655 // Optimization - do not bother scheduling a heartbeat as there are
662 // Schedule a heartbeat. When the scheduler triggers a SendHeartbeat
663 // message is sent to itself.
664 // Scheduling the heartbeat only once here because heartbeats do not
665 // need to be sent if there are other messages being sent to the remote
667 heartbeatSchedule = context.getActorSystem().scheduler().scheduleOnce(
668 interval, context.getActor(), new SendHeartBeat(),
669 context.getActorSystem().dispatcher(), context.getActor());
673 public void close() throws Exception {
678 public String getLeaderId() {
679 return context.getId();
682 protected boolean isLeaderIsolated() {
683 int minPresent = minIsolatedLeaderPeerCount;
684 for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
685 if (followerLogInformation.isFollowerActive()) {
687 if (minPresent == 0) {
692 return (minPresent != 0);
696 * Encapsulates the snapshot bytestring and handles the logic of sending
699 protected class FollowerToSnapshot {
700 private final ByteString snapshotBytes;
701 private int offset = 0;
702 // the next snapshot chunk is sent only if the replyReceivedForOffset matches offset
703 private int replyReceivedForOffset;
704 // if replyStatus is false, the previous chunk is attempted
705 private boolean replyStatus = false;
706 private int chunkIndex;
707 private final int totalChunks;
708 private int lastChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
709 private int nextChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
711 public FollowerToSnapshot(ByteString snapshotBytes) {
712 this.snapshotBytes = snapshotBytes;
713 int size = snapshotBytes.size();
714 totalChunks = ( size / context.getConfigParams().getSnapshotChunkSize()) +
715 ((size % context.getConfigParams().getSnapshotChunkSize()) > 0 ? 1 : 0);
716 if(LOG.isDebugEnabled()) {
717 LOG.debug("{}: Snapshot {} bytes, total chunks to send:{}",
718 context.getId(), size, totalChunks);
720 replyReceivedForOffset = -1;
721 chunkIndex = AbstractLeader.FIRST_CHUNK_INDEX;
724 public ByteString getSnapshotBytes() {
725 return snapshotBytes;
728 public int incrementOffset() {
730 // if prev chunk failed, we would want to sent the same chunk again
731 offset = offset + context.getConfigParams().getSnapshotChunkSize();
736 public int incrementChunkIndex() {
738 // if prev chunk failed, we would want to sent the same chunk again
739 chunkIndex = chunkIndex + 1;
744 public int getChunkIndex() {
748 public int getTotalChunks() {
752 public boolean canSendNextChunk() {
753 // we only send a false if a chunk is sent but we have not received a reply yet
754 return replyReceivedForOffset == offset;
757 public boolean isLastChunk(int chunkIndex) {
758 return totalChunks == chunkIndex;
761 public void markSendStatus(boolean success) {
763 // if the chunk sent was successful
764 replyReceivedForOffset = offset;
766 lastChunkHashCode = nextChunkHashCode;
768 // if the chunk sent was failure
769 replyReceivedForOffset = offset;
774 public ByteString getNextChunk() {
775 int snapshotLength = getSnapshotBytes().size();
776 int start = incrementOffset();
777 int size = context.getConfigParams().getSnapshotChunkSize();
778 if (context.getConfigParams().getSnapshotChunkSize() > snapshotLength) {
779 size = snapshotLength;
781 if ((start + context.getConfigParams().getSnapshotChunkSize()) > snapshotLength) {
782 size = snapshotLength - start;
786 if(LOG.isDebugEnabled()) {
787 LOG.debug("{}: Next chunk: length={}, offset={},size={}", context.getId(),
788 snapshotLength, start, size);
790 ByteString substring = getSnapshotBytes().substring(start, start + size);
791 nextChunkHashCode = substring.hashCode();
796 * reset should be called when the Follower needs to be sent the snapshot from the beginning
801 replyReceivedForOffset = offset;
802 chunkIndex = AbstractLeader.FIRST_CHUNK_INDEX;
803 lastChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
806 public int getLastChunkHashCode() {
807 return lastChunkHashCode;
811 // called from example-actor for printing the follower-states
812 public String printFollowerStates() {
813 final StringBuilder sb = new StringBuilder();
816 for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
818 sb.append(followerLogInformation.getId());
819 sb.append(" state:");
820 sb.append(followerLogInformation.isFollowerActive());
825 return sb.toString();
829 public FollowerLogInformation getFollower(String followerId) {
830 return followerToLog.get(followerId);
834 protected void setFollowerSnapshot(String followerId, FollowerToSnapshot snapshot) {
835 mapFollowerToSnapshot.put(followerId, snapshot);
839 public int followerSnapshotSize() {
840 return mapFollowerToSnapshot.size();
844 public int followerLogSize() {
845 return followerToLog.size();