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