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 java.util.concurrent.TimeUnit;
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.base.messages.CaptureSnapshot;
38 import org.opendaylight.controller.cluster.raft.base.messages.InitiateInstallSnapshot;
39 import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
40 import org.opendaylight.controller.cluster.raft.base.messages.SendHeartBeat;
41 import org.opendaylight.controller.cluster.raft.base.messages.SendInstallSnapshot;
42 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
43 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
44 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot;
45 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshotReply;
46 import org.opendaylight.controller.cluster.raft.messages.RaftRPC;
47 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
48 import scala.concurrent.duration.FiniteDuration;
51 * The behavior of a RaftActor when it is in the Leader state
55 * <li> Upon election: send initial empty AppendEntries RPCs
56 * (heartbeat) to each server; repeat during idle periods to
57 * prevent election timeouts (§5.2)
58 * <li> If command received from client: append entry to local log,
59 * respond after entry applied to state machine (§5.3)
60 * <li> If last log index ≥ nextIndex for a follower: send
61 * AppendEntries RPC with log entries starting at nextIndex
63 * <li> If successful: update nextIndex and matchIndex for
65 * <li> If AppendEntries fails because of log inconsistency:
66 * decrement nextIndex and retry (§5.3)
68 * <li> If there exists an N such that N > commitIndex, a majority
69 * of matchIndex[i] ≥ N, and log[N].term == currentTerm:
70 * set commitIndex = N (§5.3, §5.4).
72 public abstract class AbstractLeader extends AbstractRaftActorBehavior {
74 // The index of the first chunk that is sent when installing a snapshot
75 public static final int FIRST_CHUNK_INDEX = 1;
77 // The index that the follower should respond with if it needs the install snapshot to be reset
78 public static final int INVALID_CHUNK_INDEX = -1;
80 // This would be passed as the hash code of the last chunk when sending the first chunk
81 public static final int INITIAL_LAST_CHUNK_HASH_CODE = -1;
83 private final Map<String, FollowerLogInformation> followerToLog;
84 private final Map<String, FollowerToSnapshot> mapFollowerToSnapshot = new HashMap<>();
86 private Cancellable heartbeatSchedule = null;
88 private final Collection<ClientRequestTracker> trackerList = new LinkedList<>();
90 protected final int minReplicationCount;
92 protected final int minIsolatedLeaderPeerCount;
94 private Optional<ByteString> snapshot;
96 private long replicatedToAllIndex = -1;
98 public AbstractLeader(RaftActorContext context) {
101 final Builder<String, FollowerLogInformation> ftlBuilder = ImmutableMap.builder();
102 for (String followerId : context.getPeerAddresses().keySet()) {
103 FollowerLogInformation followerLogInformation =
104 new FollowerLogInformationImpl(followerId,
105 context.getCommitIndex(), -1,
106 context.getConfigParams().getElectionTimeOutInterval());
108 ftlBuilder.put(followerId, followerLogInformation);
110 followerToLog = ftlBuilder.build();
112 leaderId = context.getId();
114 LOG.debug("{}: Election: Leader has following peers: {}", context.getId(), getFollowerIds());
116 minReplicationCount = getMajorityVoteCount(getFollowerIds().size());
118 // the isolated Leader peer count will be 1 less than the majority vote count.
119 // this is because the vote count has the self vote counted in it
121 // 0 peers = 1 votesRequired , minIsolatedLeaderPeerCount = 0
122 // 2 peers = 2 votesRequired , minIsolatedLeaderPeerCount = 1
123 // 4 peers = 3 votesRequired, minIsolatedLeaderPeerCount = 2
124 minIsolatedLeaderPeerCount = minReplicationCount > 0 ? (minReplicationCount - 1) : 0;
126 snapshot = Optional.absent();
128 // Immediately schedule a heartbeat
129 // Upon election: send initial empty AppendEntries RPCs
130 // (heartbeat) to each server; repeat during idle periods to
131 // prevent election timeouts (§5.2)
132 scheduleHeartBeat(new FiniteDuration(0, TimeUnit.SECONDS));
136 * Return an immutable collection of follower identifiers.
138 * @return Collection of follower IDs
140 protected final Collection<String> getFollowerIds() {
141 return followerToLog.keySet();
144 private Optional<ByteString> getSnapshot() {
149 void setSnapshot(Optional<ByteString> snapshot) {
150 this.snapshot = snapshot;
154 protected RaftActorBehavior handleAppendEntries(ActorRef sender,
155 AppendEntries appendEntries) {
157 if(LOG.isDebugEnabled()) {
158 LOG.debug("{}: handleAppendEntries: {}", context.getId(), appendEntries);
165 protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
166 AppendEntriesReply appendEntriesReply) {
168 if(! appendEntriesReply.isSuccess()) {
169 if(LOG.isDebugEnabled()) {
170 LOG.debug("{}: handleAppendEntriesReply: {}", context.getId(), appendEntriesReply);
174 // Update the FollowerLogInformation
175 String followerId = appendEntriesReply.getFollowerId();
176 FollowerLogInformation followerLogInformation =
177 followerToLog.get(followerId);
179 if(followerLogInformation == null){
180 LOG.error("{}: handleAppendEntriesReply - unknown follower {}", context.getId(), followerId);
184 followerLogInformation.markFollowerActive();
186 if (appendEntriesReply.isSuccess()) {
187 followerLogInformation
188 .setMatchIndex(appendEntriesReply.getLogLastIndex());
189 followerLogInformation
190 .setNextIndex(appendEntriesReply.getLogLastIndex() + 1);
193 // TODO: When we find that the follower is out of sync with the
194 // Leader we simply decrement that followers next index by 1.
195 // Would it be possible to do better than this? The RAFT spec
196 // does not explicitly deal with it but may be something for us to
199 followerLogInformation.decrNextIndex();
202 // Now figure out if this reply warrants a change in the commitIndex
203 // If there exists an N such that N > commitIndex, a majority
204 // of matchIndex[i] ≥ N, and log[N].term == currentTerm:
205 // set commitIndex = N (§5.3, §5.4).
206 for (long N = context.getCommitIndex() + 1; ; N++) {
207 int replicatedCount = 1;
209 for (FollowerLogInformation info : followerToLog.values()) {
210 if (info.getMatchIndex() >= N) {
215 if (replicatedCount >= minReplicationCount) {
216 ReplicatedLogEntry replicatedLogEntry = context.getReplicatedLog().get(N);
217 if (replicatedLogEntry != null &&
218 replicatedLogEntry.getTerm() == currentTerm()) {
219 context.setCommitIndex(N);
226 // Apply the change to the state machine
227 if (context.getCommitIndex() > context.getLastApplied()) {
228 applyLogToStateMachine(context.getCommitIndex());
231 if (!context.isSnapshotCaptureInitiated()) {
235 //Send the next log entry immediately, if possible, no need to wait for heartbeat to trigger that event
236 sendUpdatesToFollower(followerId, followerLogInformation, false);
240 private void purgeInMemoryLog() {
241 //find the lowest index across followers which has been replicated to all. -1 if there are no followers.
242 // we would delete the in-mem log from that index on, in-order to minimize mem usage
243 // we would also share this info thru AE with the followers so that they can delete their log entries as well.
244 long minReplicatedToAllIndex = followerToLog.isEmpty() ? -1 : Long.MAX_VALUE;
245 for (FollowerLogInformation info : followerToLog.values()) {
246 minReplicatedToAllIndex = Math.min(minReplicatedToAllIndex, info.getMatchIndex());
249 replicatedToAllIndex = fakeSnapshot(minReplicatedToAllIndex, replicatedToAllIndex);
253 protected ClientRequestTracker removeClientRequestTracker(long logIndex) {
254 final Iterator<ClientRequestTracker> it = trackerList.iterator();
255 while (it.hasNext()) {
256 final ClientRequestTracker t = it.next();
257 if (t.getIndex() == logIndex) {
267 protected ClientRequestTracker findClientRequestTracker(long logIndex) {
268 for (ClientRequestTracker tracker : trackerList) {
269 if (tracker.getIndex() == logIndex) {
277 protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
278 RequestVoteReply requestVoteReply) {
283 public RaftState state() {
284 return RaftState.Leader;
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 {}", context.getId(),
300 rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm());
302 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
304 return switchBehavior(new Follower(context));
309 if (message instanceof SendHeartBeat) {
313 } else if(message instanceof InitiateInstallSnapshot) {
314 installSnapshotIfNeeded();
316 } else if(message instanceof SendInstallSnapshot) {
317 // received from RaftActor
318 setSnapshot(Optional.of(((SendInstallSnapshot) message).getSnapshot()));
319 sendInstallSnapshot();
321 } else if (message instanceof Replicate) {
322 replicate((Replicate) message);
324 } else if (message instanceof InstallSnapshotReply){
325 handleInstallSnapshotReply((InstallSnapshotReply) message);
329 scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
332 return super.handleMessage(sender, message);
335 private void handleInstallSnapshotReply(InstallSnapshotReply reply) {
336 String followerId = reply.getFollowerId();
337 FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
339 if (followerToSnapshot == null) {
340 LOG.error("{}: FollowerId {} in InstallSnapshotReply not known to Leader",
341 context.getId(), followerId);
345 FollowerLogInformation followerLogInformation = followerToLog.get(followerId);
346 followerLogInformation.markFollowerActive();
348 if (followerToSnapshot.getChunkIndex() == reply.getChunkIndex()) {
349 boolean wasLastChunk = false;
350 if (reply.isSuccess()) {
351 if(followerToSnapshot.isLastChunk(reply.getChunkIndex())) {
352 //this was the last chunk reply
353 if(LOG.isDebugEnabled()) {
354 LOG.debug("{}: InstallSnapshotReply received, " +
355 "last chunk received, Chunk:{}. Follower:{} Setting nextIndex:{}",
356 context.getId(), reply.getChunkIndex(), followerId,
357 context.getReplicatedLog().getSnapshotIndex() + 1
361 followerLogInformation.setMatchIndex(
362 context.getReplicatedLog().getSnapshotIndex());
363 followerLogInformation.setNextIndex(
364 context.getReplicatedLog().getSnapshotIndex() + 1);
365 mapFollowerToSnapshot.remove(followerId);
367 if(LOG.isDebugEnabled()) {
368 LOG.debug("{}: followerToLog.get(followerId).getNextIndex()=" +
369 context.getId(), followerToLog.get(followerId).getNextIndex());
372 if (mapFollowerToSnapshot.isEmpty()) {
373 // once there are no pending followers receiving snapshots
374 // we can remove snapshot from the memory
375 setSnapshot(Optional.<ByteString>absent());
380 followerToSnapshot.markSendStatus(true);
383 LOG.info("{}: InstallSnapshotReply received sending snapshot chunk failed, Will retry, Chunk: {}",
384 context.getId(), reply.getChunkIndex());
386 followerToSnapshot.markSendStatus(false);
389 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 context.getId(), 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 if(LOG.isDebugEnabled()) {
413 LOG.debug("{}: Replicate message {}", context.getId(), logIndex);
416 // Create a tracker entry we will use this later to notify the
419 new ClientRequestTrackerImpl(replicate.getClientActor(),
420 replicate.getIdentifier(),
424 if (followerToLog.isEmpty()) {
425 context.setCommitIndex(logIndex);
426 applyLogToStateMachine(logIndex);
432 private void sendAppendEntries() {
433 // Send an AppendEntries to all followers
434 long heartbeatInterval = context.getConfigParams().getHeartBeatInterval().toMillis();
435 for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
436 final String followerId = e.getKey();
437 final FollowerLogInformation followerLogInformation = e.getValue();
438 // This checks helps not to send a repeat message to the follower
439 if(followerLogInformation.timeSinceLastActivity() >= heartbeatInterval) {
440 sendUpdatesToFollower(followerId, followerLogInformation, true);
447 * This method checks if any update needs to be sent to the given follower. This includes append log entries,
448 * sending next snapshot chunk, and initiating a snapshot.
449 * @return true if any update is sent, false otherwise
452 private void sendUpdatesToFollower(String followerId, FollowerLogInformation followerLogInformation,
453 boolean sendHeartbeat) {
455 ActorSelection followerActor = context.getPeerActorSelection(followerId);
456 if (followerActor != null) {
457 long followerNextIndex = followerLogInformation.getNextIndex();
458 boolean isFollowerActive = followerLogInformation.isFollowerActive();
460 if (mapFollowerToSnapshot.get(followerId) != null) {
461 // if install snapshot is in process , then sent next chunk if possible
462 if (isFollowerActive && mapFollowerToSnapshot.get(followerId).canSendNextChunk()) {
463 sendSnapshotChunk(followerActor, followerId);
464 } else if(sendHeartbeat) {
465 // we send a heartbeat even if we have not received a reply for the last chunk
466 sendAppendEntriesToFollower(followerActor, followerLogInformation.getNextIndex(),
467 Collections.<ReplicatedLogEntry>emptyList(), followerId);
470 long leaderLastIndex = context.getReplicatedLog().lastIndex();
471 long leaderSnapShotIndex = context.getReplicatedLog().getSnapshotIndex();
472 if (isFollowerActive &&
473 context.getReplicatedLog().isPresent(followerNextIndex)) {
474 // FIXME : Sending one entry at a time
475 final List<ReplicatedLogEntry> entries = context.getReplicatedLog().getFrom(followerNextIndex, 1);
477 sendAppendEntriesToFollower(followerActor, followerNextIndex, entries, followerId);
479 } else if (isFollowerActive && followerNextIndex >= 0 &&
480 leaderLastIndex >= followerNextIndex) {
481 // if the followers next index is not present in the leaders log, and
482 // if the follower is just not starting and if leader's index is more than followers index
483 // then snapshot should be sent
485 if (LOG.isDebugEnabled()) {
486 LOG.debug("InitiateInstallSnapshot to follower:{}," +
487 "follower-nextIndex:{}, leader-snapshot-index:{}, " +
488 "leader-last-index:{}", followerId,
489 followerNextIndex, leaderSnapShotIndex, leaderLastIndex
492 actor().tell(new InitiateInstallSnapshot(), actor());
494 // Send heartbeat to follower whenever install snapshot is initiated.
495 sendAppendEntriesToFollower(followerActor, followerLogInformation.getNextIndex(),
496 Collections.<ReplicatedLogEntry>emptyList(), followerId);
498 } else if(sendHeartbeat) {
499 //we send an AppendEntries, even if the follower is inactive
500 // in-order to update the followers timestamp, in case it becomes active again
501 sendAppendEntriesToFollower(followerActor, followerLogInformation.getNextIndex(),
502 Collections.<ReplicatedLogEntry>emptyList(), followerId);
509 private void sendAppendEntriesToFollower(ActorSelection followerActor, long followerNextIndex,
510 List<ReplicatedLogEntry> entries, String followerId) {
511 AppendEntries appendEntries = new AppendEntries(currentTerm(), context.getId(),
512 prevLogIndex(followerNextIndex),
513 prevLogTerm(followerNextIndex), entries,
514 context.getCommitIndex(), replicatedToAllIndex);
516 if(!entries.isEmpty()) {
517 LOG.debug("{}: Sending AppendEntries to follower {}: {}", context.getId(), followerId,
521 followerActor.tell(appendEntries.toSerializable(), actor());
525 * An installSnapshot is scheduled at a interval that is a multiple of
526 * a HEARTBEAT_INTERVAL. This is to avoid the need to check for installing
527 * snapshots at every heartbeat.
529 * Install Snapshot works as follows
530 * 1. Leader sends a InitiateInstallSnapshot message to self
531 * 2. Leader then initiates the capture snapshot by sending a CaptureSnapshot message to actor
532 * 3. RaftActor on receipt of the CaptureSnapshotReply (from Shard), stores the received snapshot in the replicated log
533 * and makes a call to Leader's handleMessage , with SendInstallSnapshot message.
534 * 4. Leader , picks the snapshot from im-mem ReplicatedLog and sends it in chunks to the Follower
535 * 5. On complete, Follower sends back a InstallSnapshotReply.
536 * 6. On receipt of the InstallSnapshotReply for the last chunk, Leader marks the install complete for that follower
537 * and replenishes the memory by deleting the snapshot in Replicated log.
540 private void installSnapshotIfNeeded() {
541 if(LOG.isDebugEnabled()) {
542 LOG.debug("{}: installSnapshotIfNeeded, followers {}", context.getId(), followerToLog.keySet());
545 for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
546 final ActorSelection followerActor = context.getPeerActorSelection(e.getKey());
548 if (followerActor != null) {
549 long nextIndex = e.getValue().getNextIndex();
551 if (!context.getReplicatedLog().isPresent(nextIndex) &&
552 context.getReplicatedLog().isInSnapshot(nextIndex)) {
553 LOG.info("{}: {} follower needs a snapshot install", context.getId(), e.getKey());
554 if (snapshot.isPresent()) {
555 // if a snapshot is present in the memory, most likely another install is in progress
556 // no need to capture snapshot
557 sendSnapshotChunk(followerActor, e.getKey());
559 } else if (!context.isSnapshotCaptureInitiated()) {
560 initiateCaptureSnapshot();
561 //we just need 1 follower who would need snapshot to be installed.
562 // when we have the snapshot captured, we would again check (in SendInstallSnapshot)
563 // who needs an install and send to all who need
572 // on every install snapshot, we try to capture the snapshot.
573 // Once a capture is going on, another one issued will get ignored by RaftActor.
574 private void initiateCaptureSnapshot() {
575 LOG.info("{}: Initiating Snapshot Capture to Install Snapshot, Leader:{}", context.getId(), getLeaderId());
576 ReplicatedLogEntry lastAppliedEntry = context.getReplicatedLog().get(context.getLastApplied());
577 long lastAppliedIndex = -1;
578 long lastAppliedTerm = -1;
580 if (lastAppliedEntry != null) {
581 lastAppliedIndex = lastAppliedEntry.getIndex();
582 lastAppliedTerm = lastAppliedEntry.getTerm();
583 } else if (context.getReplicatedLog().getSnapshotIndex() > -1) {
584 lastAppliedIndex = context.getReplicatedLog().getSnapshotIndex();
585 lastAppliedTerm = context.getReplicatedLog().getSnapshotTerm();
588 boolean isInstallSnapshotInitiated = true;
589 actor().tell(new CaptureSnapshot(lastIndex(), lastTerm(),
590 lastAppliedIndex, lastAppliedTerm, isInstallSnapshotInitiated),
592 context.setSnapshotCaptureInitiated(true);
596 private void sendInstallSnapshot() {
597 for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
598 ActorSelection followerActor = context.getPeerActorSelection(e.getKey());
600 if (followerActor != null) {
601 long nextIndex = e.getValue().getNextIndex();
603 if (!context.getReplicatedLog().isPresent(nextIndex) &&
604 context.getReplicatedLog().isInSnapshot(nextIndex)) {
605 sendSnapshotChunk(followerActor, e.getKey());
612 * Sends a snapshot chunk to a given follower
613 * InstallSnapshot should qualify as a heartbeat too.
615 private void sendSnapshotChunk(ActorSelection followerActor, String followerId) {
617 if (snapshot.isPresent()) {
618 ByteString nextSnapshotChunk = getNextSnapshotChunk(followerId,snapshot.get());
620 // Note: the previous call to getNextSnapshotChunk has the side-effect of adding
621 // followerId to the followerToSnapshot map.
622 FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
625 new InstallSnapshot(currentTerm(), context.getId(),
626 context.getReplicatedLog().getSnapshotIndex(),
627 context.getReplicatedLog().getSnapshotTerm(),
629 followerToSnapshot.incrementChunkIndex(),
630 followerToSnapshot.getTotalChunks(),
631 Optional.of(followerToSnapshot.getLastChunkHashCode())
635 LOG.info("{}: InstallSnapshot sent to follower {}, Chunk: {}/{}",
636 context.getId(), followerActor.path(),
637 followerToSnapshot.getChunkIndex(),
638 followerToSnapshot.getTotalChunks());
640 } catch (IOException e) {
641 LOG.error(e, "{}: InstallSnapshot failed for Leader.", context.getId());
646 * Acccepts snaphot as ByteString, enters into map for future chunks
647 * creates and return a ByteString chunk
649 private ByteString getNextSnapshotChunk(String followerId, ByteString snapshotBytes) throws IOException {
650 FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
651 if (followerToSnapshot == null) {
652 followerToSnapshot = new FollowerToSnapshot(snapshotBytes);
653 mapFollowerToSnapshot.put(followerId, followerToSnapshot);
655 ByteString nextChunk = followerToSnapshot.getNextChunk();
656 if (LOG.isDebugEnabled()) {
657 LOG.debug("{}: Leader's snapshot nextChunk size:{}", context.getId(), nextChunk.size());
662 private void sendHeartBeat() {
663 if (!followerToLog.isEmpty()) {
668 private void stopHeartBeat() {
669 if (heartbeatSchedule != null && !heartbeatSchedule.isCancelled()) {
670 heartbeatSchedule.cancel();
674 private void scheduleHeartBeat(FiniteDuration interval) {
675 if (followerToLog.isEmpty()) {
676 // Optimization - do not bother scheduling a heartbeat as there are
683 // Schedule a heartbeat. When the scheduler triggers a SendHeartbeat
684 // message is sent to itself.
685 // Scheduling the heartbeat only once here because heartbeats do not
686 // need to be sent if there are other messages being sent to the remote
688 heartbeatSchedule = context.getActorSystem().scheduler().scheduleOnce(
689 interval, context.getActor(), new SendHeartBeat(),
690 context.getActorSystem().dispatcher(), context.getActor());
694 public void close() throws Exception {
699 public String getLeaderId() {
700 return context.getId();
703 protected boolean isLeaderIsolated() {
704 int minPresent = minIsolatedLeaderPeerCount;
705 for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
706 if (followerLogInformation.isFollowerActive()) {
708 if (minPresent == 0) {
713 return (minPresent != 0);
717 * Encapsulates the snapshot bytestring and handles the logic of sending
720 protected class FollowerToSnapshot {
721 private final ByteString snapshotBytes;
722 private int offset = 0;
723 // the next snapshot chunk is sent only if the replyReceivedForOffset matches offset
724 private int replyReceivedForOffset;
725 // if replyStatus is false, the previous chunk is attempted
726 private boolean replyStatus = false;
727 private int chunkIndex;
728 private final int totalChunks;
729 private int lastChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
730 private int nextChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
732 public FollowerToSnapshot(ByteString snapshotBytes) {
733 this.snapshotBytes = snapshotBytes;
734 int size = snapshotBytes.size();
735 totalChunks = ( size / context.getConfigParams().getSnapshotChunkSize()) +
736 ((size % context.getConfigParams().getSnapshotChunkSize()) > 0 ? 1 : 0);
737 if(LOG.isDebugEnabled()) {
738 LOG.debug("{}: Snapshot {} bytes, total chunks to send:{}",
739 context.getId(), size, totalChunks);
741 replyReceivedForOffset = -1;
742 chunkIndex = AbstractLeader.FIRST_CHUNK_INDEX;
745 public ByteString getSnapshotBytes() {
746 return snapshotBytes;
749 public int incrementOffset() {
751 // if prev chunk failed, we would want to sent the same chunk again
752 offset = offset + context.getConfigParams().getSnapshotChunkSize();
757 public int incrementChunkIndex() {
759 // if prev chunk failed, we would want to sent the same chunk again
760 chunkIndex = chunkIndex + 1;
765 public int getChunkIndex() {
769 public int getTotalChunks() {
773 public boolean canSendNextChunk() {
774 // we only send a false if a chunk is sent but we have not received a reply yet
775 return replyReceivedForOffset == offset;
778 public boolean isLastChunk(int chunkIndex) {
779 return totalChunks == chunkIndex;
782 public void markSendStatus(boolean success) {
784 // if the chunk sent was successful
785 replyReceivedForOffset = offset;
787 lastChunkHashCode = nextChunkHashCode;
789 // if the chunk sent was failure
790 replyReceivedForOffset = offset;
795 public ByteString getNextChunk() {
796 int snapshotLength = getSnapshotBytes().size();
797 int start = incrementOffset();
798 int size = context.getConfigParams().getSnapshotChunkSize();
799 if (context.getConfigParams().getSnapshotChunkSize() > snapshotLength) {
800 size = snapshotLength;
802 if ((start + context.getConfigParams().getSnapshotChunkSize()) > snapshotLength) {
803 size = snapshotLength - start;
807 if(LOG.isDebugEnabled()) {
808 LOG.debug("{}: Next chunk: length={}, offset={},size={}", context.getId(),
809 snapshotLength, start, size);
811 ByteString substring = getSnapshotBytes().substring(start, start + size);
812 nextChunkHashCode = substring.hashCode();
817 * reset should be called when the Follower needs to be sent the snapshot from the beginning
822 replyReceivedForOffset = offset;
823 chunkIndex = AbstractLeader.FIRST_CHUNK_INDEX;
824 lastChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
827 public int getLastChunkHashCode() {
828 return lastChunkHashCode;
832 // called from example-actor for printing the follower-states
833 public String printFollowerStates() {
834 final StringBuilder sb = new StringBuilder();
837 for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
839 sb.append(followerLogInformation.getId());
840 sb.append(" state:");
841 sb.append(followerLogInformation.isFollowerActive());
846 return sb.toString();
850 public FollowerLogInformation getFollower(String followerId) {
851 return followerToLog.get(followerId);
855 protected void setFollowerSnapshot(String followerId, FollowerToSnapshot snapshot) {
856 mapFollowerToSnapshot.put(followerId, snapshot);
860 public int followerSnapshotSize() {
861 return mapFollowerToSnapshot.size();
865 public int followerLogSize() {
866 return followerToLog.size();