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