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