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