Bug-2590: Clustering : Minimize usage of in-memory journal
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / behaviors / AbstractLeader.java
1 /*
2  * Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
3  *
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
7  */
8
9 package org.opendaylight.controller.cluster.raft.behaviors;
10
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;
27 import java.util.Map;
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;
49
50 /**
51  * The behavior of a RaftActor when it is in the Leader state
52  * <p/>
53  * Leaders:
54  * <ul>
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
62  * <ul>
63  * <li> If successful: update nextIndex and matchIndex for
64  * follower (§5.3)
65  * <li> If AppendEntries fails because of log inconsistency:
66  * decrement nextIndex and retry (§5.3)
67  * </ul>
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).
71  */
72 public abstract class AbstractLeader extends AbstractRaftActorBehavior {
73
74     // The index of the first chunk that is sent when installing a snapshot
75     public static final int FIRST_CHUNK_INDEX = 1;
76
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;
79
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;
82
83     private final Map<String, FollowerLogInformation> followerToLog;
84     private final Map<String, FollowerToSnapshot> mapFollowerToSnapshot = new HashMap<>();
85
86     private Cancellable heartbeatSchedule = null;
87
88     private final Collection<ClientRequestTracker> trackerList = new LinkedList<>();
89
90     protected final int minReplicationCount;
91
92     protected final int minIsolatedLeaderPeerCount;
93
94     private Optional<ByteString> snapshot;
95
96     private long replicatedToAllIndex = -1;
97
98     public AbstractLeader(RaftActorContext context) {
99         super(context);
100
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());
107
108             ftlBuilder.put(followerId, followerLogInformation);
109         }
110         followerToLog = ftlBuilder.build();
111
112         leaderId = context.getId();
113
114         LOG.debug("{}: Election: Leader has following peers: {}", context.getId(), getFollowerIds());
115
116         minReplicationCount = getMajorityVoteCount(getFollowerIds().size());
117
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
120         // for e.g
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;
125
126         snapshot = Optional.absent();
127
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));
133     }
134
135     /**
136      * Return an immutable collection of follower identifiers.
137      *
138      * @return Collection of follower IDs
139      */
140     protected final Collection<String> getFollowerIds() {
141         return followerToLog.keySet();
142     }
143
144     private Optional<ByteString> getSnapshot() {
145         return snapshot;
146     }
147
148     @VisibleForTesting
149     void setSnapshot(Optional<ByteString> snapshot) {
150         this.snapshot = snapshot;
151     }
152
153     @Override
154     protected RaftActorBehavior handleAppendEntries(ActorRef sender,
155         AppendEntries appendEntries) {
156
157         if(LOG.isDebugEnabled()) {
158             LOG.debug("{}: handleAppendEntries: {}", context.getId(), appendEntries);
159         }
160
161         return this;
162     }
163
164     @Override
165     protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
166         AppendEntriesReply appendEntriesReply) {
167
168         if(! appendEntriesReply.isSuccess()) {
169             if(LOG.isDebugEnabled()) {
170                 LOG.debug("{}: handleAppendEntriesReply: {}", context.getId(), appendEntriesReply);
171             }
172         }
173
174         // Update the FollowerLogInformation
175         String followerId = appendEntriesReply.getFollowerId();
176         FollowerLogInformation followerLogInformation =
177             followerToLog.get(followerId);
178
179         if(followerLogInformation == null){
180             LOG.error("{}: handleAppendEntriesReply - unknown follower {}", context.getId(), followerId);
181             return this;
182         }
183
184         followerLogInformation.markFollowerActive();
185
186         if (appendEntriesReply.isSuccess()) {
187             followerLogInformation
188                 .setMatchIndex(appendEntriesReply.getLogLastIndex());
189             followerLogInformation
190                 .setNextIndex(appendEntriesReply.getLogLastIndex() + 1);
191         } else {
192
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
197             // think about
198
199             followerLogInformation.decrNextIndex();
200         }
201
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;
208
209             for (FollowerLogInformation info : followerToLog.values()) {
210                 if (info.getMatchIndex() >= N) {
211                     replicatedCount++;
212                 }
213             }
214
215             if (replicatedCount >= minReplicationCount) {
216                 ReplicatedLogEntry replicatedLogEntry = context.getReplicatedLog().get(N);
217                 if (replicatedLogEntry != null &&
218                     replicatedLogEntry.getTerm() == currentTerm()) {
219                     context.setCommitIndex(N);
220                 }
221             } else {
222                 break;
223             }
224         }
225
226         // Apply the change to the state machine
227         if (context.getCommitIndex() > context.getLastApplied()) {
228             applyLogToStateMachine(context.getCommitIndex());
229         }
230
231         if (!context.isSnapshotCaptureInitiated()) {
232             purgeInMemoryLog();
233         }
234
235         return this;
236     }
237
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());
245         }
246
247         replicatedToAllIndex = fakeSnapshot(minReplicatedToAllIndex, replicatedToAllIndex);
248     }
249
250     @Override
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) {
256                 it.remove();
257                 return t;
258             }
259         }
260
261         return null;
262     }
263
264     @Override
265     protected ClientRequestTracker findClientRequestTracker(long logIndex) {
266         for (ClientRequestTracker tracker : trackerList) {
267             if (tracker.getIndex() == logIndex) {
268                 return tracker;
269             }
270         }
271         return null;
272     }
273
274     @Override
275     protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
276         RequestVoteReply requestVoteReply) {
277         return this;
278     }
279
280     @Override
281     public RaftState state() {
282         return RaftState.Leader;
283     }
284
285     @Override
286     public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) {
287         Preconditions.checkNotNull(sender, "sender should not be null");
288
289         Object message = fromSerializableMessage(originalMessage);
290
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                 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
298
299                 return switchBehavior(new Follower(context));
300             }
301         }
302
303         try {
304             if (message instanceof SendHeartBeat) {
305                 sendHeartBeat();
306                 return this;
307
308             } else if(message instanceof InitiateInstallSnapshot) {
309                 installSnapshotIfNeeded();
310
311             } else if(message instanceof SendInstallSnapshot) {
312                 // received from RaftActor
313                 setSnapshot(Optional.of(((SendInstallSnapshot) message).getSnapshot()));
314                 sendInstallSnapshot();
315
316             } else if (message instanceof Replicate) {
317                 replicate((Replicate) message);
318
319             } else if (message instanceof InstallSnapshotReply){
320                 handleInstallSnapshotReply((InstallSnapshotReply) message);
321
322             }
323         } finally {
324             scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
325         }
326
327         return super.handleMessage(sender, message);
328     }
329
330     private void handleInstallSnapshotReply(InstallSnapshotReply reply) {
331         String followerId = reply.getFollowerId();
332         FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
333         FollowerLogInformation followerLogInformation = followerToLog.get(followerId);
334         followerLogInformation.markFollowerActive();
335
336         if (followerToSnapshot != null &&
337             followerToSnapshot.getChunkIndex() == reply.getChunkIndex()) {
338
339             if (reply.isSuccess()) {
340                 if(followerToSnapshot.isLastChunk(reply.getChunkIndex())) {
341                     //this was the last chunk reply
342                     if(LOG.isDebugEnabled()) {
343                         LOG.debug("{}: InstallSnapshotReply received, " +
344                                 "last chunk received, Chunk:{}. Follower:{} Setting nextIndex:{}",
345                                 context.getId(), reply.getChunkIndex(), followerId,
346                             context.getReplicatedLog().getSnapshotIndex() + 1
347                         );
348                     }
349
350                     followerLogInformation.setMatchIndex(
351                         context.getReplicatedLog().getSnapshotIndex());
352                     followerLogInformation.setNextIndex(
353                         context.getReplicatedLog().getSnapshotIndex() + 1);
354                     mapFollowerToSnapshot.remove(followerId);
355
356                     if(LOG.isDebugEnabled()) {
357                         LOG.debug("{}: followerToLog.get(followerId).getNextIndex()=" +
358                                 context.getId(), followerToLog.get(followerId).getNextIndex());
359                     }
360
361                     if (mapFollowerToSnapshot.isEmpty()) {
362                         // once there are no pending followers receiving snapshots
363                         // we can remove snapshot from the memory
364                         setSnapshot(Optional.<ByteString>absent());
365                     }
366
367                 } else {
368                     followerToSnapshot.markSendStatus(true);
369                 }
370             } else {
371                 LOG.info("{}: InstallSnapshotReply received sending snapshot chunk failed, Will retry, Chunk: {}",
372                         context.getId(), reply.getChunkIndex());
373
374                 followerToSnapshot.markSendStatus(false);
375             }
376
377         } else {
378             LOG.error("{}: FollowerId in InstallSnapshotReply not known to Leader" +
379                     " or Chunk Index in InstallSnapshotReply not matching {} != {}",
380                     context.getId(), followerToSnapshot.getChunkIndex(), reply.getChunkIndex()
381             );
382
383             if(reply.getChunkIndex() == INVALID_CHUNK_INDEX){
384                 // Since the Follower did not find this index to be valid we should reset the follower snapshot
385                 // so that Installing the snapshot can resume from the beginning
386                 followerToSnapshot.reset();
387             }
388         }
389     }
390
391     private void replicate(Replicate replicate) {
392         long logIndex = replicate.getReplicatedLogEntry().getIndex();
393
394         if(LOG.isDebugEnabled()) {
395             LOG.debug("{}: Replicate message {}", context.getId(), logIndex);
396         }
397
398         // Create a tracker entry we will use this later to notify the
399         // client actor
400         trackerList.add(
401             new ClientRequestTrackerImpl(replicate.getClientActor(),
402                 replicate.getIdentifier(),
403                 logIndex)
404         );
405
406         if (followerToLog.isEmpty()) {
407             context.setCommitIndex(logIndex);
408             applyLogToStateMachine(logIndex);
409         } else {
410             sendAppendEntries();
411         }
412     }
413
414     private void sendAppendEntries() {
415         // Send an AppendEntries to all followers
416         for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
417             final String followerId = e.getKey();
418             ActorSelection followerActor = context.getPeerActorSelection(followerId);
419
420             if (followerActor != null) {
421                 FollowerLogInformation followerLogInformation = followerToLog.get(followerId);
422                 long followerNextIndex = followerLogInformation.getNextIndex();
423                 boolean isFollowerActive = followerLogInformation.isFollowerActive();
424
425                 if (mapFollowerToSnapshot.get(followerId) != null) {
426                     // if install snapshot is in process , then sent next chunk if possible
427                     if (isFollowerActive && mapFollowerToSnapshot.get(followerId).canSendNextChunk()) {
428                         sendSnapshotChunk(followerActor, followerId);
429                     } else {
430                         // we send a heartbeat even if we have not received a reply for the last chunk
431                         sendAppendEntriesToFollower(followerActor, followerNextIndex,
432                             Collections.<ReplicatedLogEntry>emptyList());
433                     }
434
435                 } else {
436                     long leaderLastIndex = context.getReplicatedLog().lastIndex();
437                     long leaderSnapShotIndex = context.getReplicatedLog().getSnapshotIndex();
438                     final List<ReplicatedLogEntry> entries;
439
440                     if (isFollowerActive &&
441                         context.getReplicatedLog().isPresent(followerNextIndex)) {
442                         // FIXME : Sending one entry at a time
443                         entries = context.getReplicatedLog().getFrom(followerNextIndex, 1);
444
445                     } else if (isFollowerActive && followerNextIndex >= 0 &&
446                         leaderLastIndex >= followerNextIndex ) {
447                         // if the followers next index is not present in the leaders log, and
448                         // if the follower is just not starting and if leader's index is more than followers index
449                         // then snapshot should be sent
450
451                         if(LOG.isDebugEnabled()) {
452                             LOG.debug(String.format("%s: InitiateInstallSnapshot to follower: %s," +
453                                     "follower-nextIndex: %s, leader-snapshot-index: %s,  " +
454                                     "leader-last-index: %s", context.getId(), followerId,
455                                 followerNextIndex, leaderSnapShotIndex, leaderLastIndex));
456                         }
457                         actor().tell(new InitiateInstallSnapshot(), actor());
458
459                         // we would want to sent AE as the capture snapshot might take time
460                         entries =  Collections.<ReplicatedLogEntry>emptyList();
461
462                     } else {
463                         //we send an AppendEntries, even if the follower is inactive
464                         // in-order to update the followers timestamp, in case it becomes active again
465                         entries =  Collections.<ReplicatedLogEntry>emptyList();
466                     }
467
468                     sendAppendEntriesToFollower(followerActor, followerNextIndex, entries);
469
470                 }
471             }
472         }
473     }
474
475     private void sendAppendEntriesToFollower(ActorSelection followerActor, long followerNextIndex,
476         List<ReplicatedLogEntry> entries) {
477         followerActor.tell(
478             new AppendEntries(currentTerm(), context.getId(),
479                 prevLogIndex(followerNextIndex),
480                 prevLogTerm(followerNextIndex), entries,
481                 context.getCommitIndex(),
482                 replicatedToAllIndex).toSerializable(),
483             actor()
484         );
485     }
486
487     /**
488      * An installSnapshot is scheduled at a interval that is a multiple of
489      * a HEARTBEAT_INTERVAL. This is to avoid the need to check for installing
490      * snapshots at every heartbeat.
491      *
492      * Install Snapshot works as follows
493      * 1. Leader sends a InitiateInstallSnapshot message to self
494      * 2. Leader then initiates the capture snapshot by sending a CaptureSnapshot message to actor
495      * 3. RaftActor on receipt of the CaptureSnapshotReply (from Shard), stores the received snapshot in the replicated log
496      * and makes a call to Leader's handleMessage , with SendInstallSnapshot message.
497      * 4. Leader , picks the snapshot from im-mem ReplicatedLog and sends it in chunks to the Follower
498      * 5. On complete, Follower sends back a InstallSnapshotReply.
499      * 6. On receipt of the InstallSnapshotReply for the last chunk, Leader marks the install complete for that follower
500      * and replenishes the memory by deleting the snapshot in Replicated log.
501      *
502      */
503     private void installSnapshotIfNeeded() {
504         for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
505             final ActorSelection followerActor = context.getPeerActorSelection(e.getKey());
506
507             if (followerActor != null) {
508                 long nextIndex = e.getValue().getNextIndex();
509
510                 if (!context.getReplicatedLog().isPresent(nextIndex) &&
511                     context.getReplicatedLog().isInSnapshot(nextIndex)) {
512                     LOG.info("{}: {} follower needs a snapshot install", context.getId(), e.getKey());
513                     if (snapshot.isPresent()) {
514                         // if a snapshot is present in the memory, most likely another install is in progress
515                         // no need to capture snapshot
516                         sendSnapshotChunk(followerActor, e.getKey());
517
518                     } else {
519                         initiateCaptureSnapshot();
520                         //we just need 1 follower who would need snapshot to be installed.
521                         // when we have the snapshot captured, we would again check (in SendInstallSnapshot)
522                         // who needs an install and send to all who need
523                         break;
524                     }
525
526                 }
527             }
528         }
529     }
530
531     // on every install snapshot, we try to capture the snapshot.
532     // Once a capture is going on, another one issued will get ignored by RaftActor.
533     private void initiateCaptureSnapshot() {
534         LOG.info("{}: Initiating Snapshot Capture to Install Snapshot, Leader:{}", context.getId(), getLeaderId());
535         ReplicatedLogEntry lastAppliedEntry = context.getReplicatedLog().get(context.getLastApplied());
536         long lastAppliedIndex = -1;
537         long lastAppliedTerm = -1;
538
539         if (lastAppliedEntry != null) {
540             lastAppliedIndex = lastAppliedEntry.getIndex();
541             lastAppliedTerm = lastAppliedEntry.getTerm();
542         } else if (context.getReplicatedLog().getSnapshotIndex() > -1)  {
543             lastAppliedIndex = context.getReplicatedLog().getSnapshotIndex();
544             lastAppliedTerm = context.getReplicatedLog().getSnapshotTerm();
545         }
546
547         boolean isInstallSnapshotInitiated = true;
548         actor().tell(new CaptureSnapshot(lastIndex(), lastTerm(),
549                 lastAppliedIndex, lastAppliedTerm, isInstallSnapshotInitiated),
550             actor());
551     }
552
553
554     private void sendInstallSnapshot() {
555         for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
556             ActorSelection followerActor = context.getPeerActorSelection(e.getKey());
557
558             if (followerActor != null) {
559                 long nextIndex = e.getValue().getNextIndex();
560
561                 if (!context.getReplicatedLog().isPresent(nextIndex) &&
562                     context.getReplicatedLog().isInSnapshot(nextIndex)) {
563                     sendSnapshotChunk(followerActor, e.getKey());
564                 }
565             }
566         }
567     }
568
569     /**
570      *  Sends a snapshot chunk to a given follower
571      *  InstallSnapshot should qualify as a heartbeat too.
572      */
573     private void sendSnapshotChunk(ActorSelection followerActor, String followerId) {
574         try {
575             if (snapshot.isPresent()) {
576                 followerActor.tell(
577                     new InstallSnapshot(currentTerm(), context.getId(),
578                         context.getReplicatedLog().getSnapshotIndex(),
579                         context.getReplicatedLog().getSnapshotTerm(),
580                         getNextSnapshotChunk(followerId,snapshot.get()),
581                         mapFollowerToSnapshot.get(followerId).incrementChunkIndex(),
582                         mapFollowerToSnapshot.get(followerId).getTotalChunks(),
583                         Optional.of(mapFollowerToSnapshot.get(followerId).getLastChunkHashCode())
584                     ).toSerializable(),
585                     actor()
586                 );
587                 LOG.info("{}: InstallSnapshot sent to follower {}, Chunk: {}/{}",
588                         context.getId(), followerActor.path(),
589                         mapFollowerToSnapshot.get(followerId).getChunkIndex(),
590                         mapFollowerToSnapshot.get(followerId).getTotalChunks());
591             }
592         } catch (IOException e) {
593             LOG.error(e, "{}: InstallSnapshot failed for Leader.", context.getId());
594         }
595     }
596
597     /**
598      * Acccepts snaphot as ByteString, enters into map for future chunks
599      * creates and return a ByteString chunk
600      */
601     private ByteString getNextSnapshotChunk(String followerId, ByteString snapshotBytes) throws IOException {
602         FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
603         if (followerToSnapshot == null) {
604             followerToSnapshot = new FollowerToSnapshot(snapshotBytes);
605             mapFollowerToSnapshot.put(followerId, followerToSnapshot);
606         }
607         ByteString nextChunk = followerToSnapshot.getNextChunk();
608         if (LOG.isDebugEnabled()) {
609             LOG.debug("{}: Leader's snapshot nextChunk size:{}", context.getId(), nextChunk.size());
610         }
611         return nextChunk;
612     }
613
614     private void sendHeartBeat() {
615         if (!followerToLog.isEmpty()) {
616             sendAppendEntries();
617         }
618     }
619
620     private void stopHeartBeat() {
621         if (heartbeatSchedule != null && !heartbeatSchedule.isCancelled()) {
622             heartbeatSchedule.cancel();
623         }
624     }
625
626     private void scheduleHeartBeat(FiniteDuration interval) {
627         if (followerToLog.isEmpty()) {
628             // Optimization - do not bother scheduling a heartbeat as there are
629             // no followers
630             return;
631         }
632
633         stopHeartBeat();
634
635         // Schedule a heartbeat. When the scheduler triggers a SendHeartbeat
636         // message is sent to itself.
637         // Scheduling the heartbeat only once here because heartbeats do not
638         // need to be sent if there are other messages being sent to the remote
639         // actor.
640         heartbeatSchedule = context.getActorSystem().scheduler().scheduleOnce(
641             interval, context.getActor(), new SendHeartBeat(),
642             context.getActorSystem().dispatcher(), context.getActor());
643     }
644
645     @Override
646     public void close() throws Exception {
647         stopHeartBeat();
648     }
649
650     @Override
651     public String getLeaderId() {
652         return context.getId();
653     }
654
655     protected boolean isLeaderIsolated() {
656         int minPresent = minIsolatedLeaderPeerCount;
657         for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
658             if (followerLogInformation.isFollowerActive()) {
659                 --minPresent;
660                 if (minPresent == 0) {
661                     break;
662                 }
663             }
664         }
665         return (minPresent != 0);
666     }
667
668     /**
669      * Encapsulates the snapshot bytestring and handles the logic of sending
670      * snapshot chunks
671      */
672     protected class FollowerToSnapshot {
673         private final ByteString snapshotBytes;
674         private int offset = 0;
675         // the next snapshot chunk is sent only if the replyReceivedForOffset matches offset
676         private int replyReceivedForOffset;
677         // if replyStatus is false, the previous chunk is attempted
678         private boolean replyStatus = false;
679         private int chunkIndex;
680         private final int totalChunks;
681         private int lastChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
682         private int nextChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
683
684         public FollowerToSnapshot(ByteString snapshotBytes) {
685             this.snapshotBytes = snapshotBytes;
686             int size = snapshotBytes.size();
687             totalChunks = ( size / context.getConfigParams().getSnapshotChunkSize()) +
688                 ((size % context.getConfigParams().getSnapshotChunkSize()) > 0 ? 1 : 0);
689             if(LOG.isDebugEnabled()) {
690                 LOG.debug("{}: Snapshot {} bytes, total chunks to send:{}",
691                         context.getId(), size, totalChunks);
692             }
693             replyReceivedForOffset = -1;
694             chunkIndex = AbstractLeader.FIRST_CHUNK_INDEX;
695         }
696
697         public ByteString getSnapshotBytes() {
698             return snapshotBytes;
699         }
700
701         public int incrementOffset() {
702             if(replyStatus) {
703                 // if prev chunk failed, we would want to sent the same chunk again
704                 offset = offset + context.getConfigParams().getSnapshotChunkSize();
705             }
706             return offset;
707         }
708
709         public int incrementChunkIndex() {
710             if (replyStatus) {
711                 // if prev chunk failed, we would want to sent the same chunk again
712                 chunkIndex =  chunkIndex + 1;
713             }
714             return chunkIndex;
715         }
716
717         public int getChunkIndex() {
718             return chunkIndex;
719         }
720
721         public int getTotalChunks() {
722             return totalChunks;
723         }
724
725         public boolean canSendNextChunk() {
726             // we only send a false if a chunk is sent but we have not received a reply yet
727             return replyReceivedForOffset == offset;
728         }
729
730         public boolean isLastChunk(int chunkIndex) {
731             return totalChunks == chunkIndex;
732         }
733
734         public void markSendStatus(boolean success) {
735             if (success) {
736                 // if the chunk sent was successful
737                 replyReceivedForOffset = offset;
738                 replyStatus = true;
739                 lastChunkHashCode = nextChunkHashCode;
740             } else {
741                 // if the chunk sent was failure
742                 replyReceivedForOffset = offset;
743                 replyStatus = false;
744             }
745         }
746
747         public ByteString getNextChunk() {
748             int snapshotLength = getSnapshotBytes().size();
749             int start = incrementOffset();
750             int size = context.getConfigParams().getSnapshotChunkSize();
751             if (context.getConfigParams().getSnapshotChunkSize() > snapshotLength) {
752                 size = snapshotLength;
753             } else {
754                 if ((start + context.getConfigParams().getSnapshotChunkSize()) > snapshotLength) {
755                     size = snapshotLength - start;
756                 }
757             }
758
759             if(LOG.isDebugEnabled()) {
760                 LOG.debug("{}: Next chunk: length={}, offset={},size={}", context.getId(),
761                     snapshotLength, start, size);
762             }
763             ByteString substring = getSnapshotBytes().substring(start, start + size);
764             nextChunkHashCode = substring.hashCode();
765             return substring;
766         }
767
768         /**
769          * reset should be called when the Follower needs to be sent the snapshot from the beginning
770          */
771         public void reset(){
772             offset = 0;
773             replyStatus = false;
774             replyReceivedForOffset = offset;
775             chunkIndex = AbstractLeader.FIRST_CHUNK_INDEX;
776             lastChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
777         }
778
779         public int getLastChunkHashCode() {
780             return lastChunkHashCode;
781         }
782     }
783
784     // called from example-actor for printing the follower-states
785     public String printFollowerStates() {
786         final StringBuilder sb = new StringBuilder();
787
788         sb.append('[');
789         for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
790             sb.append('{');
791             sb.append(followerLogInformation.getId());
792             sb.append(" state:");
793             sb.append(followerLogInformation.isFollowerActive());
794             sb.append("},");
795         }
796         sb.append(']');
797
798         return sb.toString();
799     }
800
801     @VisibleForTesting
802     public FollowerLogInformation getFollower(String followerId) {
803         return followerToLog.get(followerId);
804     }
805
806     @VisibleForTesting
807     protected void setFollowerSnapshot(String followerId, FollowerToSnapshot snapshot) {
808         mapFollowerToSnapshot.put(followerId, snapshot);
809     }
810
811     @VisibleForTesting
812     public int followerSnapshotSize() {
813         return mapFollowerToSnapshot.size();
814     }
815
816     @VisibleForTesting
817     public int followerLogSize() {
818         return followerToLog.size();
819     }
820 }