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