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