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