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