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.InitiateInstallSnapshot;
38 import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
39 import org.opendaylight.controller.cluster.raft.base.messages.SendHeartBeat;
40 import org.opendaylight.controller.cluster.raft.base.messages.SendInstallSnapshot;
41 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
42 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
43 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot;
44 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshotReply;
45 import org.opendaylight.controller.cluster.raft.messages.RaftRPC;
46 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
47 import scala.concurrent.duration.FiniteDuration;
50 * The behavior of a RaftActor when it is in the Leader state
54 * <li> Upon election: send initial empty AppendEntries RPCs
55 * (heartbeat) to each server; repeat during idle periods to
56 * prevent election timeouts (§5.2)
57 * <li> If command received from client: append entry to local log,
58 * respond after entry applied to state machine (§5.3)
59 * <li> If last log index ≥ nextIndex for a follower: send
60 * AppendEntries RPC with log entries starting at nextIndex
62 * <li> If successful: update nextIndex and matchIndex for
64 * <li> If AppendEntries fails because of log inconsistency:
65 * decrement nextIndex and retry (§5.3)
67 * <li> If there exists an N such that N > commitIndex, a majority
68 * of matchIndex[i] ≥ N, and log[N].term == currentTerm:
69 * set commitIndex = N (§5.3, §5.4).
71 public abstract class AbstractLeader extends AbstractRaftActorBehavior {
73 // The index of the first chunk that is sent when installing a snapshot
74 public static final int FIRST_CHUNK_INDEX = 1;
76 // The index that the follower should respond with if it needs the install snapshot to be reset
77 public static final int INVALID_CHUNK_INDEX = -1;
79 // This would be passed as the hash code of the last chunk when sending the first chunk
80 public static final int INITIAL_LAST_CHUNK_HASH_CODE = -1;
82 private final Map<String, FollowerLogInformation> followerToLog;
83 private final Map<String, FollowerToSnapshot> mapFollowerToSnapshot = new HashMap<>();
85 private Cancellable heartbeatSchedule = null;
87 private final Collection<ClientRequestTracker> trackerList = new LinkedList<>();
89 protected final int minReplicationCount;
91 protected final int minIsolatedLeaderPeerCount;
93 private Optional<ByteString> snapshot;
95 private long replicatedToAllIndex = -1;
97 public AbstractLeader(RaftActorContext context) {
100 final Builder<String, FollowerLogInformation> ftlBuilder = ImmutableMap.builder();
101 for (String followerId : context.getPeerAddresses().keySet()) {
102 FollowerLogInformation followerLogInformation =
103 new FollowerLogInformationImpl(followerId,
104 context.getCommitIndex(), -1,
105 context.getConfigParams().getElectionTimeOutInterval());
107 ftlBuilder.put(followerId, followerLogInformation);
109 followerToLog = ftlBuilder.build();
111 leaderId = context.getId();
113 LOG.debug("{}: Election: Leader has following peers: {}", context.getId(), getFollowerIds());
115 minReplicationCount = getMajorityVoteCount(getFollowerIds().size());
117 // the isolated Leader peer count will be 1 less than the majority vote count.
118 // this is because the vote count has the self vote counted in it
120 // 0 peers = 1 votesRequired , minIsolatedLeaderPeerCount = 0
121 // 2 peers = 2 votesRequired , minIsolatedLeaderPeerCount = 1
122 // 4 peers = 3 votesRequired, minIsolatedLeaderPeerCount = 2
123 minIsolatedLeaderPeerCount = minReplicationCount > 0 ? (minReplicationCount - 1) : 0;
125 snapshot = Optional.absent();
127 // Immediately schedule a heartbeat
128 // Upon election: send initial empty AppendEntries RPCs
129 // (heartbeat) to each server; repeat during idle periods to
130 // prevent election timeouts (§5.2)
131 sendAppendEntries(0);
135 * Return an immutable collection of follower identifiers.
137 * @return Collection of follower IDs
139 protected final Collection<String> getFollowerIds() {
140 return followerToLog.keySet();
143 private Optional<ByteString> getSnapshot() {
148 void setSnapshot(Optional<ByteString> snapshot) {
149 this.snapshot = snapshot;
153 protected RaftActorBehavior handleAppendEntries(ActorRef sender,
154 AppendEntries appendEntries) {
156 if(LOG.isDebugEnabled()) {
157 LOG.debug("{}: handleAppendEntries: {}", context.getId(), appendEntries);
164 protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
165 AppendEntriesReply appendEntriesReply) {
167 if(! appendEntriesReply.isSuccess()) {
168 if(LOG.isDebugEnabled()) {
169 LOG.debug("{}: handleAppendEntriesReply: {}", context.getId(), appendEntriesReply);
173 // Update the FollowerLogInformation
174 String followerId = appendEntriesReply.getFollowerId();
175 FollowerLogInformation followerLogInformation =
176 followerToLog.get(followerId);
178 if(followerLogInformation == null){
179 LOG.error("{}: handleAppendEntriesReply - unknown follower {}", context.getId(), followerId);
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 applyLogToStateMachine(context.getCommitIndex());
230 if (!context.isSnapshotCaptureInitiated()) {
234 //Send the next log entry immediately, if possible, no need to wait for heartbeat to trigger that event
235 sendUpdatesToFollower(followerId, followerLogInformation, false);
239 private void purgeInMemoryLog() {
240 //find the lowest index across followers which has been replicated to all. -1 if there are no followers.
241 // we would delete the in-mem log from that index on, in-order to minimize mem usage
242 // we would also share this info thru AE with the followers so that they can delete their log entries as well.
243 long minReplicatedToAllIndex = followerToLog.isEmpty() ? -1 : Long.MAX_VALUE;
244 for (FollowerLogInformation info : followerToLog.values()) {
245 minReplicatedToAllIndex = Math.min(minReplicatedToAllIndex, info.getMatchIndex());
248 replicatedToAllIndex = fakeSnapshot(minReplicatedToAllIndex, replicatedToAllIndex);
252 protected ClientRequestTracker removeClientRequestTracker(long logIndex) {
253 final Iterator<ClientRequestTracker> it = trackerList.iterator();
254 while (it.hasNext()) {
255 final ClientRequestTracker t = it.next();
256 if (t.getIndex() == logIndex) {
266 protected ClientRequestTracker findClientRequestTracker(long logIndex) {
267 for (ClientRequestTracker tracker : trackerList) {
268 if (tracker.getIndex() == logIndex) {
276 protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
277 RequestVoteReply requestVoteReply) {
282 public RaftState state() {
283 return RaftState.Leader;
287 public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) {
288 Preconditions.checkNotNull(sender, "sender should not be null");
290 Object message = fromSerializableMessage(originalMessage);
292 if (message instanceof RaftRPC) {
293 RaftRPC rpc = (RaftRPC) message;
294 // If RPC request or response contains term T > currentTerm:
295 // set currentTerm = T, convert to follower (§5.1)
296 // This applies to all RPC messages and responses
297 if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
298 LOG.debug("{}: Term {} in \"{}\" message is greater than leader's term {}", context.getId(),
299 rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm());
301 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
303 return switchBehavior(new Follower(context));
308 if (message instanceof SendHeartBeat) {
312 } else if(message instanceof InitiateInstallSnapshot) {
313 installSnapshotIfNeeded();
315 } else if(message instanceof SendInstallSnapshot) {
316 // received from RaftActor
317 setSnapshot(Optional.of(((SendInstallSnapshot) message).getSnapshot()));
318 sendInstallSnapshot();
320 } else if (message instanceof Replicate) {
321 replicate((Replicate) message);
323 } else if (message instanceof InstallSnapshotReply){
324 handleInstallSnapshotReply((InstallSnapshotReply) message);
328 scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
331 return super.handleMessage(sender, message);
334 private void handleInstallSnapshotReply(InstallSnapshotReply reply) {
335 String followerId = reply.getFollowerId();
336 FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
338 if (followerToSnapshot == null) {
339 LOG.error("{}: FollowerId {} in InstallSnapshotReply not known to Leader",
340 context.getId(), followerId);
344 FollowerLogInformation followerLogInformation = followerToLog.get(followerId);
345 followerLogInformation.markFollowerActive();
347 if (followerToSnapshot.getChunkIndex() == reply.getChunkIndex()) {
348 boolean wasLastChunk = false;
349 if (reply.isSuccess()) {
350 if(followerToSnapshot.isLastChunk(reply.getChunkIndex())) {
351 //this was the last chunk reply
352 if(LOG.isDebugEnabled()) {
353 LOG.debug("{}: InstallSnapshotReply received, " +
354 "last chunk received, Chunk:{}. Follower:{} Setting nextIndex:{}",
355 context.getId(), reply.getChunkIndex(), followerId,
356 context.getReplicatedLog().getSnapshotIndex() + 1
360 followerLogInformation.setMatchIndex(
361 context.getReplicatedLog().getSnapshotIndex());
362 followerLogInformation.setNextIndex(
363 context.getReplicatedLog().getSnapshotIndex() + 1);
364 mapFollowerToSnapshot.remove(followerId);
366 if(LOG.isDebugEnabled()) {
367 LOG.debug("{}: followerToLog.get(followerId).getNextIndex()=" +
368 context.getId(), followerToLog.get(followerId).getNextIndex());
371 if (mapFollowerToSnapshot.isEmpty()) {
372 // once there are no pending followers receiving snapshots
373 // we can remove snapshot from the memory
374 setSnapshot(Optional.<ByteString>absent());
379 followerToSnapshot.markSendStatus(true);
382 LOG.info("{}: InstallSnapshotReply received sending snapshot chunk failed, Will retry, Chunk: {}",
383 context.getId(), reply.getChunkIndex());
385 followerToSnapshot.markSendStatus(false);
388 if (!wasLastChunk && followerToSnapshot.canSendNextChunk()) {
389 ActorSelection followerActor = context.getPeerActorSelection(followerId);
390 if(followerActor != null) {
391 sendSnapshotChunk(followerActor, followerId);
396 LOG.error("{}: Chunk index {} in InstallSnapshotReply from follower {} does not match expected index {}",
397 context.getId(), reply.getChunkIndex(), followerId,
398 followerToSnapshot.getChunkIndex());
400 if(reply.getChunkIndex() == INVALID_CHUNK_INDEX){
401 // Since the Follower did not find this index to be valid we should reset the follower snapshot
402 // so that Installing the snapshot can resume from the beginning
403 followerToSnapshot.reset();
408 private void replicate(Replicate replicate) {
409 long logIndex = replicate.getReplicatedLogEntry().getIndex();
411 if(LOG.isDebugEnabled()) {
412 LOG.debug("{}: Replicate message {}", context.getId(), 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);
431 private void sendAppendEntries(long timeSinceLastActivityInterval) {
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);
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) {
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();
471 if (isFollowerActive &&
472 context.getReplicatedLog().isPresent(followerNextIndex)) {
473 // FIXME : Sending one entry at a time
474 final List<ReplicatedLogEntry> entries = context.getReplicatedLog().getFrom(followerNextIndex, 1);
476 sendAppendEntriesToFollower(followerActor, followerNextIndex, entries, followerId);
478 } else if (isFollowerActive && followerNextIndex >= 0 &&
479 leaderLastIndex >= followerNextIndex) {
480 // if the followers next index is not present in the leaders log, and
481 // if the follower is just not starting and if leader's index is more than followers index
482 // then snapshot should be sent
484 if (LOG.isDebugEnabled()) {
485 LOG.debug("InitiateInstallSnapshot to follower:{}," +
486 "follower-nextIndex:{}, leader-snapshot-index:{}, " +
487 "leader-last-index:{}", followerId,
488 followerNextIndex, leaderSnapShotIndex, leaderLastIndex
491 actor().tell(new InitiateInstallSnapshot(), actor());
493 // Send heartbeat to follower whenever install snapshot is initiated.
494 sendAppendEntriesToFollower(followerActor, followerLogInformation.getNextIndex(),
495 Collections.<ReplicatedLogEntry>emptyList(), followerId);
497 } else if(sendHeartbeat) {
498 //we send an AppendEntries, even if the follower is inactive
499 // in-order to update the followers timestamp, in case it becomes active again
500 sendAppendEntriesToFollower(followerActor, followerLogInformation.getNextIndex(),
501 Collections.<ReplicatedLogEntry>emptyList(), followerId);
508 private void sendAppendEntriesToFollower(ActorSelection followerActor, long followerNextIndex,
509 List<ReplicatedLogEntry> entries, String followerId) {
510 AppendEntries appendEntries = new AppendEntries(currentTerm(), context.getId(),
511 prevLogIndex(followerNextIndex),
512 prevLogTerm(followerNextIndex), entries,
513 context.getCommitIndex(), replicatedToAllIndex);
515 if(!entries.isEmpty()) {
516 LOG.debug("{}: Sending AppendEntries to follower {}: {}", context.getId(), followerId,
520 followerActor.tell(appendEntries.toSerializable(), actor());
524 * An installSnapshot is scheduled at a interval that is a multiple of
525 * a HEARTBEAT_INTERVAL. This is to avoid the need to check for installing
526 * snapshots at every heartbeat.
528 * Install Snapshot works as follows
529 * 1. Leader sends a InitiateInstallSnapshot message to self
530 * 2. Leader then initiates the capture snapshot by sending a CaptureSnapshot message to actor
531 * 3. RaftActor on receipt of the CaptureSnapshotReply (from Shard), stores the received snapshot in the replicated log
532 * and makes a call to Leader's handleMessage , with SendInstallSnapshot message.
533 * 4. Leader , picks the snapshot from im-mem ReplicatedLog and sends it in chunks to the Follower
534 * 5. On complete, Follower sends back a InstallSnapshotReply.
535 * 6. On receipt of the InstallSnapshotReply for the last chunk, Leader marks the install complete for that follower
536 * and replenishes the memory by deleting the snapshot in Replicated log.
539 private void installSnapshotIfNeeded() {
540 if(LOG.isDebugEnabled()) {
541 LOG.debug("{}: installSnapshotIfNeeded, followers {}", context.getId(), followerToLog.keySet());
544 for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
545 final ActorSelection followerActor = context.getPeerActorSelection(e.getKey());
547 if (followerActor != null) {
548 long nextIndex = e.getValue().getNextIndex();
550 if (!context.getReplicatedLog().isPresent(nextIndex) &&
551 context.getReplicatedLog().isInSnapshot(nextIndex)) {
552 LOG.info("{}: {} follower needs a snapshot install", context.getId(), e.getKey());
553 if (snapshot.isPresent()) {
554 // if a snapshot is present in the memory, most likely another install is in progress
555 // no need to capture snapshot
556 sendSnapshotChunk(followerActor, e.getKey());
558 } else if (!context.isSnapshotCaptureInitiated()) {
559 initiateCaptureSnapshot();
560 //we just need 1 follower who would need snapshot to be installed.
561 // when we have the snapshot captured, we would again check (in SendInstallSnapshot)
562 // who needs an install and send to all who need
571 // on every install snapshot, we try to capture the snapshot.
572 // Once a capture is going on, another one issued will get ignored by RaftActor.
573 private void initiateCaptureSnapshot() {
574 LOG.info("{}: Initiating Snapshot Capture to Install Snapshot, Leader:{}", context.getId(), getLeaderId());
575 ReplicatedLogEntry lastAppliedEntry = context.getReplicatedLog().get(context.getLastApplied());
576 long lastAppliedIndex = -1;
577 long lastAppliedTerm = -1;
579 if (lastAppliedEntry != null) {
580 lastAppliedIndex = lastAppliedEntry.getIndex();
581 lastAppliedTerm = lastAppliedEntry.getTerm();
582 } else if (context.getReplicatedLog().getSnapshotIndex() > -1) {
583 lastAppliedIndex = context.getReplicatedLog().getSnapshotIndex();
584 lastAppliedTerm = context.getReplicatedLog().getSnapshotTerm();
587 boolean isInstallSnapshotInitiated = true;
588 actor().tell(new CaptureSnapshot(lastIndex(), lastTerm(),
589 lastAppliedIndex, lastAppliedTerm, isInstallSnapshotInitiated),
591 context.setSnapshotCaptureInitiated(true);
595 private void sendInstallSnapshot() {
596 for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
597 ActorSelection followerActor = context.getPeerActorSelection(e.getKey());
599 if (followerActor != null) {
600 long nextIndex = e.getValue().getNextIndex();
602 if (!context.getReplicatedLog().isPresent(nextIndex) &&
603 context.getReplicatedLog().isInSnapshot(nextIndex)) {
604 sendSnapshotChunk(followerActor, e.getKey());
611 * Sends a snapshot chunk to a given follower
612 * InstallSnapshot should qualify as a heartbeat too.
614 private void sendSnapshotChunk(ActorSelection followerActor, String followerId) {
616 if (snapshot.isPresent()) {
617 ByteString nextSnapshotChunk = getNextSnapshotChunk(followerId,snapshot.get());
619 // Note: the previous call to getNextSnapshotChunk has the side-effect of adding
620 // followerId to the followerToSnapshot map.
621 FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
624 new InstallSnapshot(currentTerm(), context.getId(),
625 context.getReplicatedLog().getSnapshotIndex(),
626 context.getReplicatedLog().getSnapshotTerm(),
628 followerToSnapshot.incrementChunkIndex(),
629 followerToSnapshot.getTotalChunks(),
630 Optional.of(followerToSnapshot.getLastChunkHashCode())
634 LOG.info("{}: InstallSnapshot sent to follower {}, Chunk: {}/{}",
635 context.getId(), followerActor.path(),
636 followerToSnapshot.getChunkIndex(),
637 followerToSnapshot.getTotalChunks());
639 } catch (IOException e) {
640 LOG.error("{}: InstallSnapshot failed for Leader.", context.getId(), e);
645 * Acccepts snaphot as ByteString, enters into map for future chunks
646 * creates and return a ByteString chunk
648 private ByteString getNextSnapshotChunk(String followerId, ByteString snapshotBytes) throws IOException {
649 FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
650 if (followerToSnapshot == null) {
651 followerToSnapshot = new FollowerToSnapshot(snapshotBytes);
652 mapFollowerToSnapshot.put(followerId, followerToSnapshot);
654 ByteString nextChunk = followerToSnapshot.getNextChunk();
655 if (LOG.isDebugEnabled()) {
656 LOG.debug("{}: Leader's snapshot nextChunk size:{}", context.getId(), nextChunk.size());
661 private void sendHeartBeat() {
662 if (!followerToLog.isEmpty()) {
663 sendAppendEntries(context.getConfigParams().getHeartBeatInterval().toMillis());
667 private void stopHeartBeat() {
668 if (heartbeatSchedule != null && !heartbeatSchedule.isCancelled()) {
669 heartbeatSchedule.cancel();
673 private void scheduleHeartBeat(FiniteDuration interval) {
674 if (followerToLog.isEmpty()) {
675 // Optimization - do not bother scheduling a heartbeat as there are
682 // Schedule a heartbeat. When the scheduler triggers a SendHeartbeat
683 // message is sent to itself.
684 // Scheduling the heartbeat only once here because heartbeats do not
685 // need to be sent if there are other messages being sent to the remote
687 heartbeatSchedule = context.getActorSystem().scheduler().scheduleOnce(
688 interval, context.getActor(), new SendHeartBeat(),
689 context.getActorSystem().dispatcher(), context.getActor());
693 public void close() throws Exception {
698 public String getLeaderId() {
699 return context.getId();
702 protected boolean isLeaderIsolated() {
703 int minPresent = minIsolatedLeaderPeerCount;
704 for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
705 if (followerLogInformation.isFollowerActive()) {
707 if (minPresent == 0) {
712 return (minPresent != 0);
716 * Encapsulates the snapshot bytestring and handles the logic of sending
719 protected class FollowerToSnapshot {
720 private final ByteString snapshotBytes;
721 private int offset = 0;
722 // the next snapshot chunk is sent only if the replyReceivedForOffset matches offset
723 private int replyReceivedForOffset;
724 // if replyStatus is false, the previous chunk is attempted
725 private boolean replyStatus = false;
726 private int chunkIndex;
727 private final int totalChunks;
728 private int lastChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
729 private int nextChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
731 public FollowerToSnapshot(ByteString snapshotBytes) {
732 this.snapshotBytes = snapshotBytes;
733 int size = snapshotBytes.size();
734 totalChunks = ( size / context.getConfigParams().getSnapshotChunkSize()) +
735 ((size % context.getConfigParams().getSnapshotChunkSize()) > 0 ? 1 : 0);
736 if(LOG.isDebugEnabled()) {
737 LOG.debug("{}: Snapshot {} bytes, total chunks to send:{}",
738 context.getId(), size, totalChunks);
740 replyReceivedForOffset = -1;
741 chunkIndex = AbstractLeader.FIRST_CHUNK_INDEX;
744 public ByteString getSnapshotBytes() {
745 return snapshotBytes;
748 public int incrementOffset() {
750 // if prev chunk failed, we would want to sent the same chunk again
751 offset = offset + context.getConfigParams().getSnapshotChunkSize();
756 public int incrementChunkIndex() {
758 // if prev chunk failed, we would want to sent the same chunk again
759 chunkIndex = chunkIndex + 1;
764 public int getChunkIndex() {
768 public int getTotalChunks() {
772 public boolean canSendNextChunk() {
773 // we only send a false if a chunk is sent but we have not received a reply yet
774 return replyReceivedForOffset == offset;
777 public boolean isLastChunk(int chunkIndex) {
778 return totalChunks == chunkIndex;
781 public void markSendStatus(boolean success) {
783 // if the chunk sent was successful
784 replyReceivedForOffset = offset;
786 lastChunkHashCode = nextChunkHashCode;
788 // if the chunk sent was failure
789 replyReceivedForOffset = offset;
794 public ByteString getNextChunk() {
795 int snapshotLength = getSnapshotBytes().size();
796 int start = incrementOffset();
797 int size = context.getConfigParams().getSnapshotChunkSize();
798 if (context.getConfigParams().getSnapshotChunkSize() > snapshotLength) {
799 size = snapshotLength;
801 if ((start + context.getConfigParams().getSnapshotChunkSize()) > snapshotLength) {
802 size = snapshotLength - start;
806 if(LOG.isDebugEnabled()) {
807 LOG.debug("{}: Next chunk: length={}, offset={},size={}", context.getId(),
808 snapshotLength, start, size);
810 ByteString substring = getSnapshotBytes().substring(start, start + size);
811 nextChunkHashCode = substring.hashCode();
816 * reset should be called when the Follower needs to be sent the snapshot from the beginning
821 replyReceivedForOffset = offset;
822 chunkIndex = AbstractLeader.FIRST_CHUNK_INDEX;
823 lastChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
826 public int getLastChunkHashCode() {
827 return lastChunkHashCode;
831 // called from example-actor for printing the follower-states
832 public String printFollowerStates() {
833 final StringBuilder sb = new StringBuilder();
836 for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
838 sb.append(followerLogInformation.getId());
839 sb.append(" state:");
840 sb.append(followerLogInformation.isFollowerActive());
845 return sb.toString();
849 public FollowerLogInformation getFollower(String followerId) {
850 return followerToLog.get(followerId);
854 protected void setFollowerSnapshot(String followerId, FollowerToSnapshot snapshot) {
855 mapFollowerToSnapshot.put(followerId, snapshot);
859 public int followerSnapshotSize() {
860 return mapFollowerToSnapshot.size();
864 public int followerLogSize() {
865 return followerToLog.size();