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