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