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