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