BUG 2676 : Use notification-dispatcher for DataChangeListener actors
[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, RaftState.Leader);
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: {}", logName(), 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, false);
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     @VisibleForTesting
141     void setSnapshot(Optional<ByteString> snapshot) {
142         this.snapshot = snapshot;
143     }
144
145     @Override
146     protected RaftActorBehavior handleAppendEntries(ActorRef sender,
147         AppendEntries appendEntries) {
148
149         LOG.debug("{}: handleAppendEntries: {}", logName(), appendEntries);
150
151         return this;
152     }
153
154     @Override
155     protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
156         AppendEntriesReply appendEntriesReply) {
157
158         if(LOG.isTraceEnabled()) {
159             LOG.trace("{}: handleAppendEntriesReply: {}", logName(), appendEntriesReply);
160         } else if(LOG.isDebugEnabled() && !appendEntriesReply.isSuccess()) {
161             LOG.debug("{}: handleAppendEntriesReply: {}", logName(), appendEntriesReply);
162         }
163
164         // Update the FollowerLogInformation
165         String followerId = appendEntriesReply.getFollowerId();
166         FollowerLogInformation followerLogInformation =
167             followerToLog.get(followerId);
168
169         if(followerLogInformation == null){
170             LOG.error("{}: handleAppendEntriesReply - unknown follower {}", logName(), followerId);
171             return this;
172         }
173
174         if(followerLogInformation.timeSinceLastActivity() >
175                 context.getConfigParams().getElectionTimeOutInterval().toMillis()) {
176             LOG.error("{} : handleAppendEntriesReply delayed beyond election timeout, " +
177                             "appendEntriesReply : {}, timeSinceLastActivity : {}, lastApplied : {}, commitIndex : {}",
178                     logName(), appendEntriesReply, followerLogInformation.timeSinceLastActivity(),
179                     context.getLastApplied(), context.getCommitIndex());
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             LOG.debug("{}: handleAppendEntriesReply: applying to log - commitIndex: {}, lastAppliedIndex: {}",
227                     logName(), context.getCommitIndex(), context.getLastApplied());
228
229             applyLogToStateMachine(context.getCommitIndex());
230         }
231
232         if (!context.isSnapshotCaptureInitiated()) {
233             purgeInMemoryLog();
234         }
235
236         //Send the next log entry immediately, if possible, no need to wait for heartbeat to trigger that event
237         sendUpdatesToFollower(followerId, followerLogInformation, false, false);
238         return this;
239     }
240
241     private void purgeInMemoryLog() {
242         //find the lowest index across followers which has been replicated to all.
243         // lastApplied if there are no followers, so that we keep clearing the log for single-node
244         // we would delete the in-mem log from that index on, in-order to minimize mem usage
245         // we would also share this info thru AE with the followers so that they can delete their log entries as well.
246         long minReplicatedToAllIndex = followerToLog.isEmpty() ? context.getLastApplied() : Long.MAX_VALUE;
247         for (FollowerLogInformation info : followerToLog.values()) {
248             minReplicatedToAllIndex = Math.min(minReplicatedToAllIndex, info.getMatchIndex());
249         }
250
251         super.performSnapshotWithoutCapture(minReplicatedToAllIndex);
252     }
253
254     @Override
255     protected ClientRequestTracker removeClientRequestTracker(long logIndex) {
256         final Iterator<ClientRequestTracker> it = trackerList.iterator();
257         while (it.hasNext()) {
258             final ClientRequestTracker t = it.next();
259             if (t.getIndex() == logIndex) {
260                 it.remove();
261                 return t;
262             }
263         }
264
265         return null;
266     }
267
268     @Override
269     protected ClientRequestTracker findClientRequestTracker(long logIndex) {
270         for (ClientRequestTracker tracker : trackerList) {
271             if (tracker.getIndex() == logIndex) {
272                 return tracker;
273             }
274         }
275         return null;
276     }
277
278     @Override
279     protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
280         RequestVoteReply requestVoteReply) {
281         return this;
282     }
283
284     protected void beforeSendHeartbeat(){}
285
286     @Override
287     public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) {
288         Preconditions.checkNotNull(sender, "sender should not be null");
289
290         Object message = fromSerializableMessage(originalMessage);
291
292         if (message instanceof RaftRPC) {
293             RaftRPC rpc = (RaftRPC) message;
294             // If RPC request or response contains term T > currentTerm:
295             // set currentTerm = T, convert to follower (§5.1)
296             // This applies to all RPC messages and responses
297             if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
298                 LOG.debug("{}: Term {} in \"{}\" message is greater than leader's term {} - switching to Follower",
299                         logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm());
300
301                 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
302
303                 return switchBehavior(new Follower(context));
304             }
305         }
306
307         try {
308             if (message instanceof SendHeartBeat) {
309                 beforeSendHeartbeat();
310                 sendHeartBeat();
311                 return this;
312
313             } else if(message instanceof SendInstallSnapshot) {
314                 // received from RaftActor
315                 setSnapshot(Optional.of(((SendInstallSnapshot) message).getSnapshot()));
316                 sendInstallSnapshot();
317
318             } else if (message instanceof Replicate) {
319                 replicate((Replicate) message);
320
321             } else if (message instanceof InstallSnapshotReply){
322                 handleInstallSnapshotReply((InstallSnapshotReply) message);
323
324             }
325         } finally {
326             scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
327         }
328
329         return super.handleMessage(sender, message);
330     }
331
332     private void handleInstallSnapshotReply(InstallSnapshotReply reply) {
333         LOG.debug("{}: handleInstallSnapshotReply: {}", logName(), reply);
334
335         String followerId = reply.getFollowerId();
336         FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
337
338         if (followerToSnapshot == null) {
339             LOG.error("{}: FollowerId {} in InstallSnapshotReply not known to Leader",
340                     logName(), followerId);
341             return;
342         }
343
344         FollowerLogInformation followerLogInformation = followerToLog.get(followerId);
345         followerLogInformation.markFollowerActive();
346
347         if (followerToSnapshot.getChunkIndex() == reply.getChunkIndex()) {
348             boolean wasLastChunk = false;
349             if (reply.isSuccess()) {
350                 if(followerToSnapshot.isLastChunk(reply.getChunkIndex())) {
351                     //this was the last chunk reply
352                     if(LOG.isDebugEnabled()) {
353                         LOG.debug("{}: InstallSnapshotReply received, " +
354                                 "last chunk received, Chunk: {}. Follower: {} Setting nextIndex: {}",
355                                 logName(), reply.getChunkIndex(), followerId,
356                             context.getReplicatedLog().getSnapshotIndex() + 1
357                         );
358                     }
359
360                     followerLogInformation.setMatchIndex(
361                         context.getReplicatedLog().getSnapshotIndex());
362                     followerLogInformation.setNextIndex(
363                         context.getReplicatedLog().getSnapshotIndex() + 1);
364                     mapFollowerToSnapshot.remove(followerId);
365
366                     LOG.debug("{}: follower: {}, matchIndex set to {}, nextIndex set to {}",
367                                 logName(), followerId, followerLogInformation.getMatchIndex(),
368                                 followerLogInformation.getNextIndex());
369
370                     if (mapFollowerToSnapshot.isEmpty()) {
371                         // once there are no pending followers receiving snapshots
372                         // we can remove snapshot from the memory
373                         setSnapshot(Optional.<ByteString>absent());
374                     }
375                     wasLastChunk = true;
376
377                 } else {
378                     followerToSnapshot.markSendStatus(true);
379                 }
380             } else {
381                 LOG.info("{}: InstallSnapshotReply received sending snapshot chunk failed, Will retry, Chunk: {}",
382                         logName(), reply.getChunkIndex());
383
384                 followerToSnapshot.markSendStatus(false);
385             }
386
387             if (!wasLastChunk && followerToSnapshot.canSendNextChunk()) {
388                 ActorSelection followerActor = context.getPeerActorSelection(followerId);
389                 if(followerActor != null) {
390                     sendSnapshotChunk(followerActor, followerId);
391                 }
392             }
393
394         } else {
395             LOG.error("{}: Chunk index {} in InstallSnapshotReply from follower {} does not match expected index {}",
396                     logName(), reply.getChunkIndex(), followerId,
397                     followerToSnapshot.getChunkIndex());
398
399             if(reply.getChunkIndex() == INVALID_CHUNK_INDEX){
400                 // Since the Follower did not find this index to be valid we should reset the follower snapshot
401                 // so that Installing the snapshot can resume from the beginning
402                 followerToSnapshot.reset();
403             }
404         }
405     }
406
407     private void replicate(Replicate replicate) {
408         long logIndex = replicate.getReplicatedLogEntry().getIndex();
409
410         LOG.debug("{}: Replicate message: identifier: {}, logIndex: {}", logName(),
411                 replicate.getIdentifier(), logIndex);
412
413         // Create a tracker entry we will use this later to notify the
414         // client actor
415         trackerList.add(
416             new ClientRequestTrackerImpl(replicate.getClientActor(),
417                 replicate.getIdentifier(),
418                 logIndex)
419         );
420
421         if (followerToLog.isEmpty()) {
422             context.setCommitIndex(logIndex);
423             applyLogToStateMachine(logIndex);
424         } else {
425             sendAppendEntries(0, false);
426         }
427     }
428
429     private void sendAppendEntries(long timeSinceLastActivityInterval, boolean isHeartbeat) {
430         // Send an AppendEntries to all followers
431         for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
432             final String followerId = e.getKey();
433             final FollowerLogInformation followerLogInformation = e.getValue();
434             // This checks helps not to send a repeat message to the follower
435             if(!followerLogInformation.isFollowerActive() ||
436                     followerLogInformation.timeSinceLastActivity() >= timeSinceLastActivityInterval) {
437                 sendUpdatesToFollower(followerId, followerLogInformation, true, isHeartbeat);
438             }
439         }
440     }
441
442     /**
443      *
444      * This method checks if any update needs to be sent to the given follower. This includes append log entries,
445      * sending next snapshot chunk, and initiating a snapshot.
446      * @return true if any update is sent, false otherwise
447      */
448
449     private void sendUpdatesToFollower(String followerId, FollowerLogInformation followerLogInformation,
450                                        boolean sendHeartbeat, boolean isHeartbeat) {
451
452         ActorSelection followerActor = context.getPeerActorSelection(followerId);
453         if (followerActor != null) {
454             long followerNextIndex = followerLogInformation.getNextIndex();
455             boolean isFollowerActive = followerLogInformation.isFollowerActive();
456
457             if (mapFollowerToSnapshot.get(followerId) != null) {
458                 // if install snapshot is in process , then sent next chunk if possible
459                 if (isFollowerActive && mapFollowerToSnapshot.get(followerId).canSendNextChunk()) {
460                     sendSnapshotChunk(followerActor, followerId);
461                 } else if(sendHeartbeat) {
462                     // we send a heartbeat even if we have not received a reply for the last chunk
463                     sendAppendEntriesToFollower(followerActor, followerLogInformation.getNextIndex(),
464                         Collections.<ReplicatedLogEntry>emptyList(), followerId);
465                 }
466             } else {
467                 long leaderLastIndex = context.getReplicatedLog().lastIndex();
468                 long leaderSnapShotIndex = context.getReplicatedLog().getSnapshotIndex();
469
470                 if(!isHeartbeat || LOG.isTraceEnabled()) {
471                     LOG.debug("{}: Checking sendAppendEntries for follower {}, leaderLastIndex: {}, leaderSnapShotIndex: {}",
472                             logName(), followerId, leaderLastIndex, leaderSnapShotIndex);
473                 }
474
475                 if (isFollowerActive && context.getReplicatedLog().isPresent(followerNextIndex)) {
476
477                     LOG.debug("{}: sendAppendEntries: {} is present for follower {}", logName(),
478                             followerNextIndex, followerId);
479
480                     // FIXME : Sending one entry at a time
481                     final List<ReplicatedLogEntry> entries = context.getReplicatedLog().getFrom(followerNextIndex, 1);
482
483                     sendAppendEntriesToFollower(followerActor, followerNextIndex, entries, followerId);
484
485                 } else if (isFollowerActive && followerNextIndex >= 0 &&
486                     leaderLastIndex > followerNextIndex && !context.isSnapshotCaptureInitiated()) {
487                     // if the followers next index is not present in the leaders log, and
488                     // if the follower is just not starting and if leader's index is more than followers index
489                     // then snapshot should be sent
490
491                     if (LOG.isDebugEnabled()) {
492                         LOG.debug(String.format("%s: InitiateInstallSnapshot to follower: %s," +
493                                     "follower-nextIndex: %d, leader-snapshot-index: %d,  " +
494                                     "leader-last-index: %d", logName(), followerId,
495                                     followerNextIndex, leaderSnapShotIndex, leaderLastIndex));
496                     }
497
498                     // Send heartbeat to follower whenever install snapshot is initiated.
499                     sendAppendEntriesToFollower(followerActor, followerLogInformation.getNextIndex(),
500                             Collections.<ReplicatedLogEntry>emptyList(), followerId);
501
502                     initiateCaptureSnapshot(followerId, followerNextIndex);
503
504                 } else if(sendHeartbeat) {
505                     //we send an AppendEntries, even if the follower is inactive
506                     // in-order to update the followers timestamp, in case it becomes active again
507                     sendAppendEntriesToFollower(followerActor, followerLogInformation.getNextIndex(),
508                         Collections.<ReplicatedLogEntry>emptyList(), followerId);
509                 }
510
511             }
512         }
513     }
514
515     private void sendAppendEntriesToFollower(ActorSelection followerActor, long followerNextIndex,
516         List<ReplicatedLogEntry> entries, String followerId) {
517         AppendEntries appendEntries = new AppendEntries(currentTerm(), context.getId(),
518             prevLogIndex(followerNextIndex),
519             prevLogTerm(followerNextIndex), entries,
520             context.getCommitIndex(), super.getReplicatedToAllIndex());
521
522         if(!entries.isEmpty() || LOG.isTraceEnabled()) {
523             LOG.debug("{}: Sending AppendEntries to follower {}: {}", logName(), followerId,
524                     appendEntries);
525         }
526
527         followerActor.tell(appendEntries.toSerializable(), actor());
528     }
529
530     /**
531      * Install Snapshot works as follows
532      * 1. Leader initiates the capture snapshot by sending a CaptureSnapshot message to actor
533      * 2. RaftActor on receipt of the CaptureSnapshotReply (from Shard), stores the received snapshot in the replicated log
534      * and makes a call to Leader's handleMessage , with SendInstallSnapshot message.
535      * 3. Leader , picks the snapshot from im-mem ReplicatedLog and sends it in chunks to the Follower
536      * 4. On complete, Follower sends back a InstallSnapshotReply.
537      * 5. On receipt of the InstallSnapshotReply for the last chunk, Leader marks the install complete for that follower
538      * and replenishes the memory by deleting the snapshot in Replicated log.
539      * 6. If another follower requires a snapshot and a snapshot has been collected (via CaptureSnapshotReply)
540      * then send the existing snapshot in chunks to the follower.
541      * @param followerId
542      * @param followerNextIndex
543      */
544     private void initiateCaptureSnapshot(String followerId, long followerNextIndex) {
545         if (!context.getReplicatedLog().isPresent(followerNextIndex) &&
546                 context.getReplicatedLog().isInSnapshot(followerNextIndex)) {
547
548             if (snapshot.isPresent()) {
549                 // if a snapshot is present in the memory, most likely another install is in progress
550                 // no need to capture snapshot.
551                 // This could happen if another follower needs an install when one is going on.
552                 final ActorSelection followerActor = context.getPeerActorSelection(followerId);
553                 sendSnapshotChunk(followerActor, followerId);
554
555             } else if (!context.isSnapshotCaptureInitiated()) {
556
557                 LOG.info("{}: Initiating Snapshot Capture to Install Snapshot, Leader:{}", logName(), getLeaderId());
558                 ReplicatedLogEntry lastAppliedEntry = context.getReplicatedLog().get(context.getLastApplied());
559                 long lastAppliedIndex = -1;
560                 long lastAppliedTerm = -1;
561
562                 if (lastAppliedEntry != null) {
563                     lastAppliedIndex = lastAppliedEntry.getIndex();
564                     lastAppliedTerm = lastAppliedEntry.getTerm();
565                 } else if (context.getReplicatedLog().getSnapshotIndex() > -1) {
566                     lastAppliedIndex = context.getReplicatedLog().getSnapshotIndex();
567                     lastAppliedTerm = context.getReplicatedLog().getSnapshotTerm();
568                 }
569
570                 boolean isInstallSnapshotInitiated = true;
571                 long replicatedToAllIndex = super.getReplicatedToAllIndex();
572                 ReplicatedLogEntry replicatedToAllEntry = context.getReplicatedLog().get(replicatedToAllIndex);
573                 actor().tell(new CaptureSnapshot(lastIndex(), lastTerm(), lastAppliedIndex, lastAppliedTerm,
574                     (replicatedToAllEntry != null ? replicatedToAllEntry.getIndex() : -1),
575                     (replicatedToAllEntry != null ? replicatedToAllEntry.getTerm() : -1),
576                     isInstallSnapshotInitiated), actor());
577                 context.setSnapshotCaptureInitiated(true);
578             }
579         }
580     }
581
582
583     private void sendInstallSnapshot() {
584         LOG.debug("{}: sendInstallSnapshot", logName());
585         for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
586             ActorSelection followerActor = context.getPeerActorSelection(e.getKey());
587
588             if (followerActor != null) {
589                 long nextIndex = e.getValue().getNextIndex();
590
591                 if (!context.getReplicatedLog().isPresent(nextIndex) &&
592                     context.getReplicatedLog().isInSnapshot(nextIndex)) {
593                     sendSnapshotChunk(followerActor, e.getKey());
594                 }
595             }
596         }
597     }
598
599     /**
600      *  Sends a snapshot chunk to a given follower
601      *  InstallSnapshot should qualify as a heartbeat too.
602      */
603     private void sendSnapshotChunk(ActorSelection followerActor, String followerId) {
604         try {
605             if (snapshot.isPresent()) {
606                 ByteString nextSnapshotChunk = getNextSnapshotChunk(followerId,snapshot.get());
607
608                 // Note: the previous call to getNextSnapshotChunk has the side-effect of adding
609                 // followerId to the followerToSnapshot map.
610                 FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
611
612                 followerActor.tell(
613                     new InstallSnapshot(currentTerm(), context.getId(),
614                         context.getReplicatedLog().getSnapshotIndex(),
615                         context.getReplicatedLog().getSnapshotTerm(),
616                         nextSnapshotChunk,
617                         followerToSnapshot.incrementChunkIndex(),
618                         followerToSnapshot.getTotalChunks(),
619                         Optional.of(followerToSnapshot.getLastChunkHashCode())
620                     ).toSerializable(),
621                     actor()
622                 );
623                 LOG.info("{}: InstallSnapshot sent to follower {}, Chunk: {}/{}",
624                         logName(), followerActor.path(),
625                         followerToSnapshot.getChunkIndex(),
626                         followerToSnapshot.getTotalChunks());
627             }
628         } catch (IOException e) {
629             LOG.error("{}: InstallSnapshot failed for Leader.", logName(), e);
630         }
631     }
632
633     /**
634      * Acccepts snaphot as ByteString, enters into map for future chunks
635      * creates and return a ByteString chunk
636      */
637     private ByteString getNextSnapshotChunk(String followerId, ByteString snapshotBytes) throws IOException {
638         FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
639         if (followerToSnapshot == null) {
640             followerToSnapshot = new FollowerToSnapshot(snapshotBytes);
641             mapFollowerToSnapshot.put(followerId, followerToSnapshot);
642         }
643         ByteString nextChunk = followerToSnapshot.getNextChunk();
644
645         LOG.debug("{}: next snapshot chunk size for follower {}: {}", logName(), followerId, nextChunk.size());
646
647         return nextChunk;
648     }
649
650     private void sendHeartBeat() {
651         if (!followerToLog.isEmpty()) {
652             LOG.trace("{}: Sending heartbeat", logName());
653             sendAppendEntries(context.getConfigParams().getHeartBeatInterval().toMillis(), true);
654         }
655     }
656
657     private void stopHeartBeat() {
658         if (heartbeatSchedule != null && !heartbeatSchedule.isCancelled()) {
659             heartbeatSchedule.cancel();
660         }
661     }
662
663     private void scheduleHeartBeat(FiniteDuration interval) {
664         if (followerToLog.isEmpty()) {
665             // Optimization - do not bother scheduling a heartbeat as there are
666             // no followers
667             return;
668         }
669
670         stopHeartBeat();
671
672         // Schedule a heartbeat. When the scheduler triggers a SendHeartbeat
673         // message is sent to itself.
674         // Scheduling the heartbeat only once here because heartbeats do not
675         // need to be sent if there are other messages being sent to the remote
676         // actor.
677         heartbeatSchedule = context.getActorSystem().scheduler().scheduleOnce(
678             interval, context.getActor(), new SendHeartBeat(),
679             context.getActorSystem().dispatcher(), context.getActor());
680     }
681
682     @Override
683     public void close() throws Exception {
684         stopHeartBeat();
685     }
686
687     @Override
688     public String getLeaderId() {
689         return context.getId();
690     }
691
692     protected boolean isLeaderIsolated() {
693         int minPresent = minIsolatedLeaderPeerCount;
694         for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
695             if (followerLogInformation.isFollowerActive()) {
696                 --minPresent;
697                 if (minPresent == 0) {
698                     break;
699                 }
700             }
701         }
702         return (minPresent != 0);
703     }
704
705     /**
706      * Encapsulates the snapshot bytestring and handles the logic of sending
707      * snapshot chunks
708      */
709     protected class FollowerToSnapshot {
710         private final ByteString snapshotBytes;
711         private int offset = 0;
712         // the next snapshot chunk is sent only if the replyReceivedForOffset matches offset
713         private int replyReceivedForOffset;
714         // if replyStatus is false, the previous chunk is attempted
715         private boolean replyStatus = false;
716         private int chunkIndex;
717         private final int totalChunks;
718         private int lastChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
719         private int nextChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
720
721         public FollowerToSnapshot(ByteString snapshotBytes) {
722             this.snapshotBytes = snapshotBytes;
723             int size = snapshotBytes.size();
724             totalChunks = ( size / context.getConfigParams().getSnapshotChunkSize()) +
725                 ((size % context.getConfigParams().getSnapshotChunkSize()) > 0 ? 1 : 0);
726             if(LOG.isDebugEnabled()) {
727                 LOG.debug("{}: Snapshot {} bytes, total chunks to send:{}",
728                         logName(), size, totalChunks);
729             }
730             replyReceivedForOffset = -1;
731             chunkIndex = AbstractLeader.FIRST_CHUNK_INDEX;
732         }
733
734         public ByteString getSnapshotBytes() {
735             return snapshotBytes;
736         }
737
738         public int incrementOffset() {
739             if(replyStatus) {
740                 // if prev chunk failed, we would want to sent the same chunk again
741                 offset = offset + context.getConfigParams().getSnapshotChunkSize();
742             }
743             return offset;
744         }
745
746         public int incrementChunkIndex() {
747             if (replyStatus) {
748                 // if prev chunk failed, we would want to sent the same chunk again
749                 chunkIndex =  chunkIndex + 1;
750             }
751             return chunkIndex;
752         }
753
754         public int getChunkIndex() {
755             return chunkIndex;
756         }
757
758         public int getTotalChunks() {
759             return totalChunks;
760         }
761
762         public boolean canSendNextChunk() {
763             // we only send a false if a chunk is sent but we have not received a reply yet
764             return replyReceivedForOffset == offset;
765         }
766
767         public boolean isLastChunk(int chunkIndex) {
768             return totalChunks == chunkIndex;
769         }
770
771         public void markSendStatus(boolean success) {
772             if (success) {
773                 // if the chunk sent was successful
774                 replyReceivedForOffset = offset;
775                 replyStatus = true;
776                 lastChunkHashCode = nextChunkHashCode;
777             } else {
778                 // if the chunk sent was failure
779                 replyReceivedForOffset = offset;
780                 replyStatus = false;
781             }
782         }
783
784         public ByteString getNextChunk() {
785             int snapshotLength = getSnapshotBytes().size();
786             int start = incrementOffset();
787             int size = context.getConfigParams().getSnapshotChunkSize();
788             if (context.getConfigParams().getSnapshotChunkSize() > snapshotLength) {
789                 size = snapshotLength;
790             } else {
791                 if ((start + context.getConfigParams().getSnapshotChunkSize()) > snapshotLength) {
792                     size = snapshotLength - start;
793                 }
794             }
795
796
797             LOG.debug("{}: Next chunk: length={}, offset={},size={}", logName(),
798                     snapshotLength, start, size);
799
800             ByteString substring = getSnapshotBytes().substring(start, start + size);
801             nextChunkHashCode = substring.hashCode();
802             return substring;
803         }
804
805         /**
806          * reset should be called when the Follower needs to be sent the snapshot from the beginning
807          */
808         public void reset(){
809             offset = 0;
810             replyStatus = false;
811             replyReceivedForOffset = offset;
812             chunkIndex = AbstractLeader.FIRST_CHUNK_INDEX;
813             lastChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
814         }
815
816         public int getLastChunkHashCode() {
817             return lastChunkHashCode;
818         }
819     }
820
821     // called from example-actor for printing the follower-states
822     public String printFollowerStates() {
823         final StringBuilder sb = new StringBuilder();
824
825         sb.append('[');
826         for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
827             sb.append('{');
828             sb.append(followerLogInformation.getId());
829             sb.append(" state:");
830             sb.append(followerLogInformation.isFollowerActive());
831             sb.append("},");
832         }
833         sb.append(']');
834
835         return sb.toString();
836     }
837
838     @VisibleForTesting
839     public FollowerLogInformation getFollower(String followerId) {
840         return followerToLog.get(followerId);
841     }
842
843     @VisibleForTesting
844     protected void setFollowerSnapshot(String followerId, FollowerToSnapshot snapshot) {
845         mapFollowerToSnapshot.put(followerId, snapshot);
846     }
847
848     @VisibleForTesting
849     public int followerSnapshotSize() {
850         return mapFollowerToSnapshot.size();
851     }
852
853     @VisibleForTesting
854     public int followerLogSize() {
855         return followerToLog.size();
856     }
857 }