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