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