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