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.ArrayList;
20 import java.util.Collections;
21 import java.util.HashMap;
22 import java.util.List;
25 import java.util.concurrent.TimeUnit;
26 import org.opendaylight.controller.cluster.raft.ClientRequestTracker;
27 import org.opendaylight.controller.cluster.raft.ClientRequestTrackerImpl;
28 import org.opendaylight.controller.cluster.raft.FollowerLogInformation;
29 import org.opendaylight.controller.cluster.raft.FollowerLogInformationImpl;
30 import org.opendaylight.controller.cluster.raft.RaftActorContext;
31 import org.opendaylight.controller.cluster.raft.RaftState;
32 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
33 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshot;
34 import org.opendaylight.controller.cluster.raft.base.messages.InitiateInstallSnapshot;
35 import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
36 import org.opendaylight.controller.cluster.raft.base.messages.SendHeartBeat;
37 import org.opendaylight.controller.cluster.raft.base.messages.SendInstallSnapshot;
38 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
39 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
40 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot;
41 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshotReply;
42 import org.opendaylight.controller.cluster.raft.messages.RaftRPC;
43 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
44 import scala.concurrent.duration.FiniteDuration;
47 * The behavior of a RaftActor when it is in the Leader state
51 * <li> Upon election: send initial empty AppendEntries RPCs
52 * (heartbeat) to each server; repeat during idle periods to
53 * prevent election timeouts (§5.2)
54 * <li> If command received from client: append entry to local log,
55 * respond after entry applied to state machine (§5.3)
56 * <li> If last log index ≥ nextIndex for a follower: send
57 * AppendEntries RPC with log entries starting at nextIndex
59 * <li> If successful: update nextIndex and matchIndex for
61 * <li> If AppendEntries fails because of log inconsistency:
62 * decrement nextIndex and retry (§5.3)
64 * <li> If there exists an N such that N > commitIndex, a majority
65 * of matchIndex[i] ≥ N, and log[N].term == currentTerm:
66 * set commitIndex = N (§5.3, §5.4).
68 public abstract class AbstractLeader extends AbstractRaftActorBehavior {
70 // The index of the first chunk that is sent when installing a snapshot
71 public static final int FIRST_CHUNK_INDEX = 1;
73 // The index that the follower should respond with if it needs the install snapshot to be reset
74 public static final int INVALID_CHUNK_INDEX = -1;
76 // This would be passed as the hash code of the last chunk when sending the first chunk
77 public static final int INITIAL_LAST_CHUNK_HASH_CODE = -1;
79 protected final Map<String, FollowerLogInformation> followerToLog = new HashMap<>();
80 protected final Map<String, FollowerToSnapshot> mapFollowerToSnapshot = new HashMap<>();
82 protected final Set<String> followers;
84 private Cancellable heartbeatSchedule = null;
86 private List<ClientRequestTracker> trackerList = new ArrayList<>();
88 protected final int minReplicationCount;
90 protected final int minIsolatedLeaderPeerCount;
92 private Optional<ByteString> snapshot;
94 public AbstractLeader(RaftActorContext context) {
97 followers = context.getPeerAddresses().keySet();
99 for (String followerId : followers) {
100 FollowerLogInformation followerLogInformation =
101 new FollowerLogInformationImpl(followerId,
102 context.getCommitIndex(), -1,
103 context.getConfigParams().getElectionTimeOutInterval());
105 followerToLog.put(followerId, followerLogInformation);
108 leaderId = context.getId();
110 if(LOG.isDebugEnabled()) {
111 LOG.debug("Election:Leader has following peers: {}", followers);
114 minReplicationCount = getMajorityVoteCount(followers.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 scheduleHeartBeat(new FiniteDuration(0, TimeUnit.SECONDS));
133 private Optional<ByteString> getSnapshot() {
138 void setSnapshot(Optional<ByteString> snapshot) {
139 this.snapshot = snapshot;
143 protected RaftActorBehavior handleAppendEntries(ActorRef sender,
144 AppendEntries appendEntries) {
146 if(LOG.isDebugEnabled()) {
147 LOG.debug(appendEntries.toString());
154 protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
155 AppendEntriesReply appendEntriesReply) {
157 if(! appendEntriesReply.isSuccess()) {
158 if(LOG.isDebugEnabled()) {
159 LOG.debug(appendEntriesReply.toString());
163 // Update the FollowerLogInformation
164 String followerId = appendEntriesReply.getFollowerId();
165 FollowerLogInformation followerLogInformation =
166 followerToLog.get(followerId);
168 if(followerLogInformation == null){
169 LOG.error("Unknown follower {}", followerId);
173 followerLogInformation.markFollowerActive();
175 if (appendEntriesReply.isSuccess()) {
176 followerLogInformation
177 .setMatchIndex(appendEntriesReply.getLogLastIndex());
178 followerLogInformation
179 .setNextIndex(appendEntriesReply.getLogLastIndex() + 1);
182 // TODO: When we find that the follower is out of sync with the
183 // Leader we simply decrement that followers next index by 1.
184 // Would it be possible to do better than this? The RAFT spec
185 // does not explicitly deal with it but may be something for us to
188 followerLogInformation.decrNextIndex();
191 // Now figure out if this reply warrants a change in the commitIndex
192 // If there exists an N such that N > commitIndex, a majority
193 // of matchIndex[i] ≥ N, and log[N].term == currentTerm:
194 // set commitIndex = N (§5.3, §5.4).
195 for (long N = context.getCommitIndex() + 1; ; N++) {
196 int replicatedCount = 1;
198 for (FollowerLogInformation info : followerToLog.values()) {
199 if (info.getMatchIndex().get() >= N) {
204 if (replicatedCount >= minReplicationCount) {
205 ReplicatedLogEntry replicatedLogEntry = context.getReplicatedLog().get(N);
206 if (replicatedLogEntry != null &&
207 replicatedLogEntry.getTerm() == currentTerm()) {
208 context.setCommitIndex(N);
215 // Apply the change to the state machine
216 if (context.getCommitIndex() > context.getLastApplied()) {
217 applyLogToStateMachine(context.getCommitIndex());
223 protected ClientRequestTracker removeClientRequestTracker(long logIndex) {
225 ClientRequestTracker toRemove = findClientRequestTracker(logIndex);
226 if(toRemove != null) {
227 trackerList.remove(toRemove);
233 protected ClientRequestTracker findClientRequestTracker(long logIndex) {
234 for (ClientRequestTracker tracker : trackerList) {
235 if (tracker.getIndex() == logIndex) {
243 protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
244 RequestVoteReply requestVoteReply) {
249 public RaftState state() {
250 return RaftState.Leader;
254 public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) {
255 Preconditions.checkNotNull(sender, "sender should not be null");
257 Object message = fromSerializableMessage(originalMessage);
259 if (message instanceof RaftRPC) {
260 RaftRPC rpc = (RaftRPC) message;
261 // If RPC request or response contains term T > currentTerm:
262 // set currentTerm = T, convert to follower (§5.1)
263 // This applies to all RPC messages and responses
264 if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
265 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
267 return switchBehavior(new Follower(context));
272 if (message instanceof SendHeartBeat) {
276 } else if(message instanceof InitiateInstallSnapshot) {
277 installSnapshotIfNeeded();
279 } else if(message instanceof SendInstallSnapshot) {
280 // received from RaftActor
281 setSnapshot(Optional.of(((SendInstallSnapshot) message).getSnapshot()));
282 sendInstallSnapshot();
284 } else if (message instanceof Replicate) {
285 replicate((Replicate) message);
287 } else if (message instanceof InstallSnapshotReply){
288 handleInstallSnapshotReply((InstallSnapshotReply) message);
292 scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
295 return super.handleMessage(sender, message);
298 private void handleInstallSnapshotReply(InstallSnapshotReply reply) {
299 String followerId = reply.getFollowerId();
300 FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
301 FollowerLogInformation followerLogInformation = followerToLog.get(followerId);
302 followerLogInformation.markFollowerActive();
304 if (followerToSnapshot != null &&
305 followerToSnapshot.getChunkIndex() == reply.getChunkIndex()) {
307 if (reply.isSuccess()) {
308 if(followerToSnapshot.isLastChunk(reply.getChunkIndex())) {
309 //this was the last chunk reply
310 if(LOG.isDebugEnabled()) {
311 LOG.debug("InstallSnapshotReply received, " +
312 "last chunk received, Chunk:{}. Follower:{} Setting nextIndex:{}",
313 reply.getChunkIndex(), followerId,
314 context.getReplicatedLog().getSnapshotIndex() + 1
318 followerLogInformation.setMatchIndex(
319 context.getReplicatedLog().getSnapshotIndex());
320 followerLogInformation.setNextIndex(
321 context.getReplicatedLog().getSnapshotIndex() + 1);
322 mapFollowerToSnapshot.remove(followerId);
324 if(LOG.isDebugEnabled()) {
325 LOG.debug("followerToLog.get(followerId).getNextIndex().get()=" +
326 followerToLog.get(followerId).getNextIndex().get());
329 if (mapFollowerToSnapshot.isEmpty()) {
330 // once there are no pending followers receiving snapshots
331 // we can remove snapshot from the memory
332 setSnapshot(Optional.<ByteString>absent());
336 followerToSnapshot.markSendStatus(true);
339 LOG.info("InstallSnapshotReply received, " +
340 "sending snapshot chunk failed, Will retry, Chunk:{}",
341 reply.getChunkIndex()
344 followerToSnapshot.markSendStatus(false);
348 LOG.error("ERROR!!" +
349 "FollowerId in InstallSnapshotReply not known to Leader" +
350 " or Chunk Index in InstallSnapshotReply not matching {} != {}",
351 followerToSnapshot.getChunkIndex(), reply.getChunkIndex()
354 if(reply.getChunkIndex() == INVALID_CHUNK_INDEX){
355 // Since the Follower did not find this index to be valid we should reset the follower snapshot
356 // so that Installing the snapshot can resume from the beginning
357 followerToSnapshot.reset();
362 private void replicate(Replicate replicate) {
363 long logIndex = replicate.getReplicatedLogEntry().getIndex();
365 if(LOG.isDebugEnabled()) {
366 LOG.debug("Replicate message {}", logIndex);
369 // Create a tracker entry we will use this later to notify the
372 new ClientRequestTrackerImpl(replicate.getClientActor(),
373 replicate.getIdentifier(),
377 if (followers.size() == 0) {
378 context.setCommitIndex(logIndex);
379 applyLogToStateMachine(logIndex);
385 private void sendAppendEntries() {
386 // Send an AppendEntries to all followers
387 for (String followerId : followers) {
388 ActorSelection followerActor = context.getPeerActorSelection(followerId);
390 if (followerActor != null) {
391 FollowerLogInformation followerLogInformation = followerToLog.get(followerId);
392 long followerNextIndex = followerLogInformation.getNextIndex().get();
393 boolean isFollowerActive = followerLogInformation.isFollowerActive();
394 List<ReplicatedLogEntry> entries = null;
396 if (mapFollowerToSnapshot.get(followerId) != null) {
397 // if install snapshot is in process , then sent next chunk if possible
398 if (isFollowerActive && mapFollowerToSnapshot.get(followerId).canSendNextChunk()) {
399 sendSnapshotChunk(followerActor, followerId);
401 // we send a heartbeat even if we have not received a reply for the last chunk
402 sendAppendEntriesToFollower(followerActor, followerNextIndex,
403 Collections.<ReplicatedLogEntry>emptyList());
407 long leaderLastIndex = context.getReplicatedLog().lastIndex();
408 long leaderSnapShotIndex = context.getReplicatedLog().getSnapshotIndex();
410 if (isFollowerActive &&
411 context.getReplicatedLog().isPresent(followerNextIndex)) {
412 // FIXME : Sending one entry at a time
413 entries = context.getReplicatedLog().getFrom(followerNextIndex, 1);
415 } else if (isFollowerActive && followerNextIndex >= 0 &&
416 leaderLastIndex >= followerNextIndex ) {
417 // if the followers next index is not present in the leaders log, and
418 // if the follower is just not starting and if leader's index is more than followers index
419 // then snapshot should be sent
421 if(LOG.isDebugEnabled()) {
422 LOG.debug("InitiateInstallSnapshot to follower:{}," +
423 "follower-nextIndex:{}, leader-snapshot-index:{}, " +
424 "leader-last-index:{}", followerId,
425 followerNextIndex, leaderSnapShotIndex, leaderLastIndex
428 actor().tell(new InitiateInstallSnapshot(), actor());
430 // we would want to sent AE as the capture snapshot might take time
431 entries = Collections.<ReplicatedLogEntry>emptyList();
434 //we send an AppendEntries, even if the follower is inactive
435 // in-order to update the followers timestamp, in case it becomes active again
436 entries = Collections.<ReplicatedLogEntry>emptyList();
439 sendAppendEntriesToFollower(followerActor, followerNextIndex, entries);
446 private void sendAppendEntriesToFollower(ActorSelection followerActor, long followerNextIndex,
447 List<ReplicatedLogEntry> entries) {
449 new AppendEntries(currentTerm(), context.getId(),
450 prevLogIndex(followerNextIndex),
451 prevLogTerm(followerNextIndex), entries,
452 context.getCommitIndex()).toSerializable(),
458 * An installSnapshot is scheduled at a interval that is a multiple of
459 * a HEARTBEAT_INTERVAL. This is to avoid the need to check for installing
460 * snapshots at every heartbeat.
462 * Install Snapshot works as follows
463 * 1. Leader sends a InitiateInstallSnapshot message to self
464 * 2. Leader then initiates the capture snapshot by sending a CaptureSnapshot message to actor
465 * 3. RaftActor on receipt of the CaptureSnapshotReply (from Shard), stores the received snapshot in the replicated log
466 * and makes a call to Leader's handleMessage , with SendInstallSnapshot message.
467 * 4. Leader , picks the snapshot from im-mem ReplicatedLog and sends it in chunks to the Follower
468 * 5. On complete, Follower sends back a InstallSnapshotReply.
469 * 6. On receipt of the InstallSnapshotReply for the last chunk, Leader marks the install complete for that follower
470 * and replenishes the memory by deleting the snapshot in Replicated log.
473 private void installSnapshotIfNeeded() {
474 for (String followerId : followers) {
475 ActorSelection followerActor =
476 context.getPeerActorSelection(followerId);
478 if(followerActor != null) {
479 FollowerLogInformation followerLogInformation =
480 followerToLog.get(followerId);
482 long nextIndex = followerLogInformation.getNextIndex().get();
484 if (!context.getReplicatedLog().isPresent(nextIndex) &&
485 context.getReplicatedLog().isInSnapshot(nextIndex)) {
486 LOG.info("{} follower needs a snapshot install", followerId);
487 if (snapshot.isPresent()) {
488 // if a snapshot is present in the memory, most likely another install is in progress
489 // no need to capture snapshot
490 sendSnapshotChunk(followerActor, followerId);
493 initiateCaptureSnapshot();
494 //we just need 1 follower who would need snapshot to be installed.
495 // when we have the snapshot captured, we would again check (in SendInstallSnapshot)
496 // who needs an install and send to all who need
505 // on every install snapshot, we try to capture the snapshot.
506 // Once a capture is going on, another one issued will get ignored by RaftActor.
507 private void initiateCaptureSnapshot() {
508 LOG.info("Initiating Snapshot Capture to Install Snapshot, Leader:{}", getLeaderId());
509 ReplicatedLogEntry lastAppliedEntry = context.getReplicatedLog().get(context.getLastApplied());
510 long lastAppliedIndex = -1;
511 long lastAppliedTerm = -1;
513 if (lastAppliedEntry != null) {
514 lastAppliedIndex = lastAppliedEntry.getIndex();
515 lastAppliedTerm = lastAppliedEntry.getTerm();
516 } else if (context.getReplicatedLog().getSnapshotIndex() > -1) {
517 lastAppliedIndex = context.getReplicatedLog().getSnapshotIndex();
518 lastAppliedTerm = context.getReplicatedLog().getSnapshotTerm();
521 boolean isInstallSnapshotInitiated = true;
522 actor().tell(new CaptureSnapshot(lastIndex(), lastTerm(),
523 lastAppliedIndex, lastAppliedTerm, isInstallSnapshotInitiated),
528 private void sendInstallSnapshot() {
529 for (String followerId : followers) {
530 ActorSelection followerActor = context.getPeerActorSelection(followerId);
532 if(followerActor != null) {
533 FollowerLogInformation followerLogInformation = followerToLog.get(followerId);
534 long nextIndex = followerLogInformation.getNextIndex().get();
536 if (!context.getReplicatedLog().isPresent(nextIndex) &&
537 context.getReplicatedLog().isInSnapshot(nextIndex)) {
538 sendSnapshotChunk(followerActor, followerId);
545 * Sends a snapshot chunk to a given follower
546 * InstallSnapshot should qualify as a heartbeat too.
548 private void sendSnapshotChunk(ActorSelection followerActor, String followerId) {
550 if (snapshot.isPresent()) {
552 new InstallSnapshot(currentTerm(), context.getId(),
553 context.getReplicatedLog().getSnapshotIndex(),
554 context.getReplicatedLog().getSnapshotTerm(),
555 getNextSnapshotChunk(followerId,snapshot.get()),
556 mapFollowerToSnapshot.get(followerId).incrementChunkIndex(),
557 mapFollowerToSnapshot.get(followerId).getTotalChunks(),
558 Optional.of(mapFollowerToSnapshot.get(followerId).getLastChunkHashCode())
562 LOG.info("InstallSnapshot sent to follower {}, Chunk: {}/{}",
563 followerActor.path(), mapFollowerToSnapshot.get(followerId).getChunkIndex(),
564 mapFollowerToSnapshot.get(followerId).getTotalChunks());
566 } catch (IOException e) {
567 LOG.error(e, "InstallSnapshot failed for Leader.");
572 * Acccepts snaphot as ByteString, enters into map for future chunks
573 * creates and return a ByteString chunk
575 private ByteString getNextSnapshotChunk(String followerId, ByteString snapshotBytes) throws IOException {
576 FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
577 if (followerToSnapshot == null) {
578 followerToSnapshot = new FollowerToSnapshot(snapshotBytes);
579 mapFollowerToSnapshot.put(followerId, followerToSnapshot);
581 ByteString nextChunk = followerToSnapshot.getNextChunk();
582 if (LOG.isDebugEnabled()) {
583 LOG.debug("Leader's snapshot nextChunk size:{}", nextChunk.size());
588 private void sendHeartBeat() {
589 if (followers.size() > 0) {
594 private void stopHeartBeat() {
595 if (heartbeatSchedule != null && !heartbeatSchedule.isCancelled()) {
596 heartbeatSchedule.cancel();
600 private void scheduleHeartBeat(FiniteDuration interval) {
601 if(followers.size() == 0){
602 // Optimization - do not bother scheduling a heartbeat as there are
609 // Schedule a heartbeat. When the scheduler triggers a SendHeartbeat
610 // message is sent to itself.
611 // Scheduling the heartbeat only once here because heartbeats do not
612 // need to be sent if there are other messages being sent to the remote
614 heartbeatSchedule = context.getActorSystem().scheduler().scheduleOnce(
615 interval, context.getActor(), new SendHeartBeat(),
616 context.getActorSystem().dispatcher(), context.getActor());
620 public void close() throws Exception {
625 public String getLeaderId() {
626 return context.getId();
629 protected boolean isLeaderIsolated() {
630 int minPresent = minIsolatedLeaderPeerCount;
631 for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
632 if (followerLogInformation.isFollowerActive()) {
634 if (minPresent == 0) {
639 return (minPresent != 0);
643 * Encapsulates the snapshot bytestring and handles the logic of sending
646 protected class FollowerToSnapshot {
647 private ByteString snapshotBytes;
648 private int offset = 0;
649 // the next snapshot chunk is sent only if the replyReceivedForOffset matches offset
650 private int replyReceivedForOffset;
651 // if replyStatus is false, the previous chunk is attempted
652 private boolean replyStatus = false;
653 private int chunkIndex;
654 private int totalChunks;
655 private int lastChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
656 private int nextChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
658 public FollowerToSnapshot(ByteString snapshotBytes) {
659 this.snapshotBytes = snapshotBytes;
660 int size = snapshotBytes.size();
661 totalChunks = ( size / context.getConfigParams().getSnapshotChunkSize()) +
662 ((size % context.getConfigParams().getSnapshotChunkSize()) > 0 ? 1 : 0);
663 if(LOG.isDebugEnabled()) {
664 LOG.debug("Snapshot {} bytes, total chunks to send:{}",
667 replyReceivedForOffset = -1;
668 chunkIndex = AbstractLeader.FIRST_CHUNK_INDEX;
671 public ByteString getSnapshotBytes() {
672 return snapshotBytes;
675 public int incrementOffset() {
677 // if prev chunk failed, we would want to sent the same chunk again
678 offset = offset + context.getConfigParams().getSnapshotChunkSize();
683 public int incrementChunkIndex() {
685 // if prev chunk failed, we would want to sent the same chunk again
686 chunkIndex = chunkIndex + 1;
691 public int getChunkIndex() {
695 public int getTotalChunks() {
699 public boolean canSendNextChunk() {
700 // we only send a false if a chunk is sent but we have not received a reply yet
701 return replyReceivedForOffset == offset;
704 public boolean isLastChunk(int chunkIndex) {
705 return totalChunks == chunkIndex;
708 public void markSendStatus(boolean success) {
710 // if the chunk sent was successful
711 replyReceivedForOffset = offset;
713 lastChunkHashCode = nextChunkHashCode;
715 // if the chunk sent was failure
716 replyReceivedForOffset = offset;
721 public ByteString getNextChunk() {
722 int snapshotLength = getSnapshotBytes().size();
723 int start = incrementOffset();
724 int size = context.getConfigParams().getSnapshotChunkSize();
725 if (context.getConfigParams().getSnapshotChunkSize() > snapshotLength) {
726 size = snapshotLength;
728 if ((start + context.getConfigParams().getSnapshotChunkSize()) > snapshotLength) {
729 size = snapshotLength - start;
733 if(LOG.isDebugEnabled()) {
734 LOG.debug("length={}, offset={},size={}",
735 snapshotLength, start, size);
737 ByteString substring = getSnapshotBytes().substring(start, start + size);
738 nextChunkHashCode = substring.hashCode();
743 * reset should be called when the Follower needs to be sent the snapshot from the beginning
748 replyReceivedForOffset = offset;
749 chunkIndex = AbstractLeader.FIRST_CHUNK_INDEX;
750 lastChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
753 public int getLastChunkHashCode() {
754 return lastChunkHashCode;
758 // called from example-actor for printing the follower-states
759 public String printFollowerStates() {
760 final StringBuilder sb = new StringBuilder();
763 for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
765 sb.append(followerLogInformation.getId());
766 sb.append(" state:");
767 sb.append(followerLogInformation.isFollowerActive());
772 return sb.toString();
776 void markFollowerActive(String followerId) {
777 followerToLog.get(followerId).markFollowerActive();