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