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