Bug 8206: Prevent decr follower next index beyong -1
[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                 if (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(),
296                             followersLastLogTermInLeadersLog, followerLogInformation.getNextIndex());
297                 }
298             }
299         }
300
301         if (log.isTraceEnabled()) {
302             log.trace("{}: handleAppendEntriesReply from {}: commitIndex: {}, lastAppliedIndex: {}, currentTerm: {}",
303                     logName(), followerId, context.getCommitIndex(), context.getLastApplied(), currentTerm());
304         }
305
306         possiblyUpdateCommitIndex();
307
308         //Send the next log entry immediately, if possible, no need to wait for heartbeat to trigger that event
309         sendUpdatesToFollower(followerId, followerLogInformation, false, !updated);
310
311         return this;
312     }
313
314     private void possiblyUpdateCommitIndex() {
315         // Figure out if we can update the the commitIndex as follows:
316         //   If there exists an index N such that N > commitIndex, a majority of matchIndex[i] ≥ N,
317         //     and log[N].term == currentTerm:
318         //   set commitIndex = N (§5.3, §5.4).
319         for (long index = context.getCommitIndex() + 1; ; index++) {
320             ReplicatedLogEntry replicatedLogEntry = context.getReplicatedLog().get(index);
321             if (replicatedLogEntry == null) {
322                 log.trace("{}: ReplicatedLogEntry not found for index {} - snapshotIndex: {}, journal size: {}",
323                         logName(), index, context.getReplicatedLog().getSnapshotIndex(),
324                         context.getReplicatedLog().size());
325                 break;
326             }
327
328             // Count our entry if it has been persisted.
329             int replicatedCount = replicatedLogEntry.isPersistencePending() ? 0 : 1;
330
331             if (replicatedCount == 0) {
332                 // We don't commit and apply a log entry until we've gotten the ack from our local persistence,
333                 // even though there *shouldn't* be any issue with updating the commit index if we get a consensus
334                 // amongst the followers w/o the local persistence ack.
335                 break;
336             }
337
338             log.trace("{}: checking Nth index {}", logName(), index);
339             for (FollowerLogInformation info : followerToLog.values()) {
340                 final PeerInfo peerInfo = context.getPeerInfo(info.getId());
341                 if (info.getMatchIndex() >= index && peerInfo != null && peerInfo.isVoting()) {
342                     replicatedCount++;
343                 } else if (log.isTraceEnabled()) {
344                     log.trace("{}: Not counting follower {} - matchIndex: {}, {}", logName(), info.getId(),
345                             info.getMatchIndex(), peerInfo);
346                 }
347             }
348
349             if (log.isTraceEnabled()) {
350                 log.trace("{}: replicatedCount {}, minReplicationCount: {}", logName(), replicatedCount,
351                         minReplicationCount);
352             }
353
354             if (replicatedCount >= minReplicationCount) {
355                 // Don't update the commit index if the log entry is from a previous term, as per §5.4.1:
356                 // "Raft never commits log entries from previous terms by counting replicas".
357                 // However we keep looping so we can make progress when new entries in the current term
358                 // reach consensus, as per §5.4.1: "once an entry from the current term is committed by
359                 // counting replicas, then all prior entries are committed indirectly".
360                 if (replicatedLogEntry.getTerm() == currentTerm()) {
361                     log.trace("{}: Setting commit index to {}", logName(), index);
362                     context.setCommitIndex(index);
363                 } else {
364                     log.debug("{}: Not updating commit index to {} - retrieved log entry with index {}, "
365                             + "term {} does not match the current term {}", logName(), index,
366                             replicatedLogEntry.getIndex(), replicatedLogEntry.getTerm(), currentTerm());
367                 }
368             } else {
369                 log.trace("{}: minReplicationCount not reached, actual {} - breaking", logName(), replicatedCount);
370                 break;
371             }
372         }
373
374         // Apply the change to the state machine
375         if (context.getCommitIndex() > context.getLastApplied()) {
376             log.debug("{}: Applying to log - commitIndex: {}, lastAppliedIndex: {}", logName(),
377                     context.getCommitIndex(), context.getLastApplied());
378
379             applyLogToStateMachine(context.getCommitIndex());
380         }
381
382         if (!context.getSnapshotManager().isCapturing()) {
383             purgeInMemoryLog();
384         }
385     }
386
387     private boolean updateFollowerLogInformation(FollowerLogInformation followerLogInformation,
388             AppendEntriesReply appendEntriesReply) {
389         boolean updated = followerLogInformation.setMatchIndex(appendEntriesReply.getLogLastIndex());
390         updated = followerLogInformation.setNextIndex(appendEntriesReply.getLogLastIndex() + 1) || updated;
391
392         if (updated && log.isDebugEnabled()) {
393             log.debug(
394                 "{}: handleAppendEntriesReply - FollowerLogInformation for {} updated: matchIndex: {}, nextIndex: {}",
395                 logName(), followerLogInformation.getId(), followerLogInformation.getMatchIndex(),
396                 followerLogInformation.getNextIndex());
397         }
398         return updated;
399     }
400
401     private void purgeInMemoryLog() {
402         //find the lowest index across followers which has been replicated to all.
403         // lastApplied if there are no followers, so that we keep clearing the log for single-node
404         // we would delete the in-mem log from that index on, in-order to minimize mem usage
405         // we would also share this info thru AE with the followers so that they can delete their log entries as well.
406         long minReplicatedToAllIndex = followerToLog.isEmpty() ? context.getLastApplied() : Long.MAX_VALUE;
407         for (FollowerLogInformation info : followerToLog.values()) {
408             minReplicatedToAllIndex = Math.min(minReplicatedToAllIndex, info.getMatchIndex());
409         }
410
411         super.performSnapshotWithoutCapture(minReplicatedToAllIndex);
412     }
413
414     @Override
415     protected ClientRequestTracker removeClientRequestTracker(long logIndex) {
416         final Iterator<ClientRequestTracker> it = trackers.iterator();
417         while (it.hasNext()) {
418             final ClientRequestTracker t = it.next();
419             if (t.getIndex() == logIndex) {
420                 it.remove();
421                 return t;
422             }
423         }
424
425         return null;
426     }
427
428     @Override
429     protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
430         RequestVoteReply requestVoteReply) {
431         return this;
432     }
433
434     protected void beforeSendHeartbeat(){}
435
436     @Override
437     public RaftActorBehavior handleMessage(ActorRef sender, Object message) {
438         Preconditions.checkNotNull(sender, "sender should not be null");
439
440         if (message instanceof RaftRPC) {
441             RaftRPC rpc = (RaftRPC) message;
442             // If RPC request or response contains term T > currentTerm:
443             // set currentTerm = T, convert to follower (§5.1)
444             // This applies to all RPC messages and responses
445             if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
446                 log.info("{}: Term {} in \"{}\" message is greater than leader's term {} - switching to Follower",
447                         logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm());
448
449                 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
450
451                 return internalSwitchBehavior(RaftState.Follower);
452             }
453         }
454
455         if (message instanceof SendHeartBeat) {
456             beforeSendHeartbeat();
457             sendHeartBeat();
458             scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
459         } else if (message instanceof SendInstallSnapshot) {
460             SendInstallSnapshot sendInstallSnapshot = (SendInstallSnapshot) message;
461             setSnapshot(new SnapshotHolder(sendInstallSnapshot.getSnapshot(), sendInstallSnapshot.getSnapshotBytes()));
462             sendInstallSnapshot();
463         } else if (message instanceof Replicate) {
464             replicate((Replicate) message);
465         } else if (message instanceof InstallSnapshotReply) {
466             handleInstallSnapshotReply((InstallSnapshotReply) message);
467         } else if (message instanceof CheckConsensusReached) {
468             possiblyUpdateCommitIndex();
469         } else {
470             return super.handleMessage(sender, message);
471         }
472
473         return this;
474     }
475
476     private void handleInstallSnapshotReply(InstallSnapshotReply reply) {
477         log.debug("{}: handleInstallSnapshotReply: {}", logName(), reply);
478
479         String followerId = reply.getFollowerId();
480         FollowerLogInformation followerLogInformation = followerToLog.get(followerId);
481         if (followerLogInformation == null) {
482             // This can happen during AddServer if it times out.
483             log.error("{}: FollowerLogInformation not found for follower {} in InstallSnapshotReply",
484                     logName(), followerId);
485             return;
486         }
487
488         LeaderInstallSnapshotState installSnapshotState = followerLogInformation.getInstallSnapshotState();
489         if (installSnapshotState == null) {
490             log.error("{}: LeaderInstallSnapshotState not found for follower {} in InstallSnapshotReply",
491                     logName(), followerId);
492             return;
493         }
494
495         followerLogInformation.markFollowerActive();
496
497         if (installSnapshotState.getChunkIndex() == reply.getChunkIndex()) {
498             boolean wasLastChunk = false;
499             if (reply.isSuccess()) {
500                 if (installSnapshotState.isLastChunk(reply.getChunkIndex())) {
501                     //this was the last chunk reply
502
503                     long followerMatchIndex = snapshotHolder.get().getLastIncludedIndex();
504                     followerLogInformation.setMatchIndex(followerMatchIndex);
505                     followerLogInformation.setNextIndex(followerMatchIndex + 1);
506                     followerLogInformation.clearLeaderInstallSnapshotState();
507
508                     log.info("{}: Snapshot successfully installed on follower {} (last chunk {}) - "
509                         + "matchIndex set to {}, nextIndex set to {}", logName(), followerId, reply.getChunkIndex(),
510                         followerLogInformation.getMatchIndex(), followerLogInformation.getNextIndex());
511
512                     if (!anyFollowersInstallingSnapshot()) {
513                         // once there are no pending followers receiving snapshots
514                         // we can remove snapshot from the memory
515                         setSnapshot(null);
516                     }
517
518                     wasLastChunk = true;
519                     if (context.getPeerInfo(followerId).getVotingState() == VotingState.VOTING_NOT_INITIALIZED) {
520                         UnInitializedFollowerSnapshotReply unInitFollowerSnapshotSuccess =
521                                              new UnInitializedFollowerSnapshotReply(followerId);
522                         context.getActor().tell(unInitFollowerSnapshotSuccess, context.getActor());
523                         log.debug("Sent message UnInitializedFollowerSnapshotReply to self");
524                     }
525                 } else {
526                     installSnapshotState.markSendStatus(true);
527                 }
528             } else {
529                 log.warn("{}: Received failed InstallSnapshotReply - will retry: {}", logName(), reply);
530
531                 installSnapshotState.markSendStatus(false);
532             }
533
534             if (wasLastChunk && !context.getSnapshotManager().isCapturing()) {
535                 // Since the follower is now caught up try to purge the log.
536                 purgeInMemoryLog();
537             } else if (!wasLastChunk && installSnapshotState.canSendNextChunk()) {
538                 ActorSelection followerActor = context.getPeerActorSelection(followerId);
539                 if (followerActor != null) {
540                     sendSnapshotChunk(followerActor, followerLogInformation);
541                 }
542             }
543
544         } else {
545             log.error("{}: Chunk index {} in InstallSnapshotReply from follower {} does not match expected index {}",
546                     logName(), reply.getChunkIndex(), followerId,
547                     installSnapshotState.getChunkIndex());
548
549             if (reply.getChunkIndex() == LeaderInstallSnapshotState.INVALID_CHUNK_INDEX) {
550                 // Since the Follower did not find this index to be valid we should reset the follower snapshot
551                 // so that Installing the snapshot can resume from the beginning
552                 installSnapshotState.reset();
553             }
554         }
555     }
556
557     private boolean anyFollowersInstallingSnapshot() {
558         for (FollowerLogInformation info: followerToLog.values()) {
559             if (info.getInstallSnapshotState() != null) {
560                 return true;
561             }
562
563         }
564
565         return false;
566     }
567
568     private void replicate(Replicate replicate) {
569         long logIndex = replicate.getReplicatedLogEntry().getIndex();
570
571         log.debug("{}: Replicate message: identifier: {}, logIndex: {}, payload: {}, isSendImmediate: {}", logName(),
572                 replicate.getIdentifier(), logIndex, replicate.getReplicatedLogEntry().getData().getClass(),
573                 replicate.isSendImmediate());
574
575         // Create a tracker entry we will use this later to notify the
576         // client actor
577         if (replicate.getClientActor() != null) {
578             trackers.add(new ClientRequestTrackerImpl(replicate.getClientActor(), replicate.getIdentifier(),
579                     logIndex));
580         }
581
582         boolean applyModificationToState = !context.anyVotingPeers()
583                 || context.getRaftPolicy().applyModificationToStateBeforeConsensus();
584
585         if (applyModificationToState) {
586             context.setCommitIndex(logIndex);
587             applyLogToStateMachine(logIndex);
588         }
589
590         if (replicate.isSendImmediate() && !followerToLog.isEmpty()) {
591             sendAppendEntries(0, false);
592         }
593     }
594
595     protected void sendAppendEntries(long timeSinceLastActivityInterval, boolean isHeartbeat) {
596         // Send an AppendEntries to all followers
597         for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
598             final String followerId = e.getKey();
599             final FollowerLogInformation followerLogInformation = e.getValue();
600             // This checks helps not to send a repeat message to the follower
601             if (!followerLogInformation.isFollowerActive()
602                     || followerLogInformation.timeSinceLastActivity() >= timeSinceLastActivityInterval) {
603                 sendUpdatesToFollower(followerId, followerLogInformation, true, isHeartbeat);
604             }
605         }
606     }
607
608     /**
609      * This method checks if any update needs to be sent to the given follower. This includes append log entries,
610      * sending next snapshot chunk, and initiating a snapshot.
611      */
612     private void sendUpdatesToFollower(String followerId, FollowerLogInformation followerLogInformation,
613                                        boolean sendHeartbeat, boolean isHeartbeat) {
614
615         ActorSelection followerActor = context.getPeerActorSelection(followerId);
616         if (followerActor != null) {
617             long followerNextIndex = followerLogInformation.getNextIndex();
618             boolean isFollowerActive = followerLogInformation.isFollowerActive();
619             boolean sendAppendEntries = false;
620             List<ReplicatedLogEntry> entries = Collections.emptyList();
621
622             LeaderInstallSnapshotState installSnapshotState = followerLogInformation.getInstallSnapshotState();
623             if (installSnapshotState != null) {
624                 // if install snapshot is in process , then sent next chunk if possible
625                 if (isFollowerActive && installSnapshotState.canSendNextChunk()) {
626                     sendSnapshotChunk(followerActor, followerLogInformation);
627                 } else if (sendHeartbeat) {
628                     // we send a heartbeat even if we have not received a reply for the last chunk
629                     sendAppendEntries = true;
630                 }
631             } else {
632                 long leaderLastIndex = context.getReplicatedLog().lastIndex();
633                 long leaderSnapShotIndex = context.getReplicatedLog().getSnapshotIndex();
634
635                 if (!isHeartbeat && log.isDebugEnabled() || log.isTraceEnabled()) {
636                     log.debug("{}: Checking sendAppendEntries for follower {}: active: {}, followerNextIndex: {}, "
637                             + "leaderLastIndex: {}, leaderSnapShotIndex: {}", logName(), followerId, isFollowerActive,
638                             followerNextIndex, leaderLastIndex, leaderSnapShotIndex);
639                 }
640
641                 if (isFollowerActive && context.getReplicatedLog().isPresent(followerNextIndex)) {
642
643                     log.debug("{}: sendAppendEntries: {} is present for follower {}", logName(),
644                             followerNextIndex, followerId);
645
646                     if (followerLogInformation.okToReplicate()) {
647                         // Try to send all the entries in the journal but not exceeding the max data size
648                         // for a single AppendEntries message.
649                         int maxEntries = (int) context.getReplicatedLog().size();
650                         entries = context.getReplicatedLog().getFrom(followerNextIndex, maxEntries,
651                                 context.getConfigParams().getSnapshotChunkSize());
652                         sendAppendEntries = true;
653                     }
654                 } else if (isFollowerActive && followerNextIndex >= 0
655                         && leaderLastIndex > followerNextIndex && !context.getSnapshotManager().isCapturing()) {
656                     // if the followers next index is not present in the leaders log, and
657                     // if the follower is just not starting and if leader's index is more than followers index
658                     // then snapshot should be sent
659
660                     // Send heartbeat to follower whenever install snapshot is initiated.
661                     sendAppendEntries = true;
662                     if (canInstallSnapshot(followerNextIndex)) {
663                         log.info("{}: Initiating install snapshot to follower {}: follower nextIndex: {}, leader "
664                                 + "snapshotIndex: {}, leader lastIndex: {}, leader log size: {}", logName(), followerId,
665                                 followerNextIndex, leaderSnapShotIndex, leaderLastIndex,
666                                 context.getReplicatedLog().size());
667
668                         initiateCaptureSnapshot(followerId);
669                     } else {
670                         // It doesn't seem like we should ever reach here - most likely indicates sonething is
671                         // wrong.
672                         log.info("{}: Follower {} is behind but cannot install snapshot: follower nextIndex: {}, "
673                                 + "leader snapshotIndex: {}, leader lastIndex: {}, leader log size: {}", logName(),
674                                 followerId, followerNextIndex, leaderSnapShotIndex, leaderLastIndex,
675                                 context.getReplicatedLog().size());
676                     }
677
678                 } else if (sendHeartbeat) {
679                     // we send an AppendEntries, even if the follower is inactive
680                     // in-order to update the followers timestamp, in case it becomes active again
681                     sendAppendEntries = true;
682                 }
683
684             }
685
686             if (sendAppendEntries) {
687                 sendAppendEntriesToFollower(followerActor, entries, followerLogInformation);
688             }
689         }
690     }
691
692     private void sendAppendEntriesToFollower(ActorSelection followerActor, List<ReplicatedLogEntry> entries,
693             FollowerLogInformation followerLogInformation) {
694         // In certain cases outlined below we don't want to send the actual commit index to prevent the follower from
695         // possibly committing and applying conflicting entries (those with same index, different term) from a prior
696         // term that weren't replicated to a majority, which would be a violation of raft.
697         //     - if the follower isn't active. In this case we don't know the state of the follower and we send an
698         //       empty AppendEntries as a heart beat to prevent election.
699         //     - if we're in the process of installing a snapshot. In this case we don't send any new entries but still
700         //       need to send AppendEntries to prevent election.
701         boolean isInstallingSnaphot = followerLogInformation.getInstallSnapshotState() != null;
702         long leaderCommitIndex = isInstallingSnaphot || !followerLogInformation.isFollowerActive() ? -1 :
703             context.getCommitIndex();
704
705         long followerNextIndex = followerLogInformation.getNextIndex();
706         AppendEntries appendEntries = new AppendEntries(currentTerm(), context.getId(),
707             getLogEntryIndex(followerNextIndex - 1),
708             getLogEntryTerm(followerNextIndex - 1), entries,
709             leaderCommitIndex, super.getReplicatedToAllIndex(), context.getPayloadVersion());
710
711         if (!entries.isEmpty() || log.isTraceEnabled()) {
712             log.debug("{}: Sending AppendEntries to follower {}: {}", logName(), followerLogInformation.getId(),
713                     appendEntries);
714         }
715
716         followerActor.tell(appendEntries, actor());
717     }
718
719     /**
720      * Initiates a snapshot capture to install on a follower.
721      *
722      * <p>
723      * Install Snapshot works as follows
724      *   1. Leader initiates the capture snapshot by calling createSnapshot on the RaftActor.
725      *   2. On receipt of the CaptureSnapshotReply message, the RaftActor persists the snapshot and makes a call to
726      *      the Leader's handleMessage with a SendInstallSnapshot message.
727      *   3. The Leader obtains and stores the Snapshot from the SendInstallSnapshot message and sends it in chunks to
728      *      the Follower via InstallSnapshot messages.
729      *   4. For each chunk, the Follower sends back an InstallSnapshotReply.
730      *   5. On receipt of the InstallSnapshotReply for the last chunk, the Leader marks the install complete for that
731      *      follower.
732      *   6. If another follower requires a snapshot and a snapshot has been collected (via SendInstallSnapshot)
733      *      then send the existing snapshot in chunks to the follower.
734      *
735      * @param followerId the id of the follower.
736      * @return true if capture was initiated, false otherwise.
737      */
738     public boolean initiateCaptureSnapshot(String followerId) {
739         FollowerLogInformation followerLogInfo = followerToLog.get(followerId);
740         if (snapshotHolder.isPresent()) {
741             // If a snapshot is present in the memory, most likely another install is in progress no need to capture
742             // snapshot. This could happen if another follower needs an install when one is going on.
743             final ActorSelection followerActor = context.getPeerActorSelection(followerId);
744
745             // Note: sendSnapshotChunk will set the LeaderInstallSnapshotState.
746             sendSnapshotChunk(followerActor, followerLogInfo);
747             return true;
748         }
749
750         boolean captureInitiated = context.getSnapshotManager().captureToInstall(context.getReplicatedLog().last(),
751             this.getReplicatedToAllIndex(), followerId);
752         if (captureInitiated) {
753             followerLogInfo.setLeaderInstallSnapshotState(new LeaderInstallSnapshotState(
754                 context.getConfigParams().getSnapshotChunkSize(), logName()));
755         }
756
757         return captureInitiated;
758     }
759
760     private boolean canInstallSnapshot(long nextIndex) {
761         // If the follower's nextIndex is -1 then we might as well send it a snapshot
762         // Otherwise send it a snapshot only if the nextIndex is not present in the log but is present
763         // in the snapshot
764         return nextIndex == -1 || !context.getReplicatedLog().isPresent(nextIndex)
765                 && context.getReplicatedLog().isInSnapshot(nextIndex);
766
767     }
768
769
770     private void sendInstallSnapshot() {
771         log.debug("{}: sendInstallSnapshot", logName());
772         for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
773             String followerId = e.getKey();
774             ActorSelection followerActor = context.getPeerActorSelection(followerId);
775             FollowerLogInformation followerLogInfo = e.getValue();
776
777             if (followerActor != null) {
778                 long nextIndex = followerLogInfo.getNextIndex();
779                 if (followerLogInfo.getInstallSnapshotState() != null
780                         || context.getPeerInfo(followerId).getVotingState() == VotingState.VOTING_NOT_INITIALIZED
781                         || canInstallSnapshot(nextIndex)) {
782                     sendSnapshotChunk(followerActor, followerLogInfo);
783                 }
784             }
785         }
786     }
787
788     /**
789      *  Sends a snapshot chunk to a given follower
790      *  InstallSnapshot should qualify as a heartbeat too.
791      */
792     private void sendSnapshotChunk(ActorSelection followerActor, FollowerLogInformation followerLogInfo) {
793         if (snapshotHolder.isPresent()) {
794             LeaderInstallSnapshotState installSnapshotState = followerLogInfo.getInstallSnapshotState();
795             if (installSnapshotState == null) {
796                 installSnapshotState = new LeaderInstallSnapshotState(context.getConfigParams().getSnapshotChunkSize(),
797                         logName());
798                 followerLogInfo.setLeaderInstallSnapshotState(installSnapshotState);
799             }
800
801             try {
802                 // Ensure the snapshot bytes are set - this is a no-op.
803                 installSnapshotState.setSnapshotBytes(snapshotHolder.get().getSnapshotBytes());
804
805                 byte[] nextSnapshotChunk = installSnapshotState.getNextChunk();
806
807                 log.debug("{}: next snapshot chunk size for follower {}: {}", logName(), followerLogInfo.getId(),
808                         nextSnapshotChunk.length);
809
810                 int nextChunkIndex = installSnapshotState.incrementChunkIndex();
811                 Optional<ServerConfigurationPayload> serverConfig = Optional.absent();
812                 if (installSnapshotState.isLastChunk(nextChunkIndex)) {
813                     serverConfig = Optional.fromNullable(context.getPeerServerInfo(true));
814                 }
815
816                 followerActor.tell(
817                     new InstallSnapshot(currentTerm(), context.getId(),
818                         snapshotHolder.get().getLastIncludedIndex(),
819                         snapshotHolder.get().getLastIncludedTerm(),
820                         nextSnapshotChunk,
821                         nextChunkIndex,
822                         installSnapshotState.getTotalChunks(),
823                         Optional.of(installSnapshotState.getLastChunkHashCode()),
824                         serverConfig
825                     ).toSerializable(followerLogInfo.getRaftVersion()),
826                     actor()
827                 );
828
829                 log.debug("{}: InstallSnapshot sent to follower {}, Chunk: {}/{}", logName(), followerActor.path(),
830                         installSnapshotState.getChunkIndex(), installSnapshotState.getTotalChunks());
831             } catch (IOException e) {
832                 throw Throwables.propagate(e);
833             }
834         }
835     }
836
837     private void sendHeartBeat() {
838         if (!followerToLog.isEmpty()) {
839             log.trace("{}: Sending heartbeat", logName());
840             sendAppendEntries(context.getConfigParams().getHeartBeatInterval().toMillis(), true);
841         }
842     }
843
844     private void stopHeartBeat() {
845         if (heartbeatSchedule != null && !heartbeatSchedule.isCancelled()) {
846             heartbeatSchedule.cancel();
847         }
848     }
849
850     private void scheduleHeartBeat(FiniteDuration interval) {
851         if (followerToLog.isEmpty()) {
852             // Optimization - do not bother scheduling a heartbeat as there are
853             // no followers
854             return;
855         }
856
857         stopHeartBeat();
858
859         // Schedule a heartbeat. When the scheduler triggers a SendHeartbeat
860         // message is sent to itself.
861         // Scheduling the heartbeat only once here because heartbeats do not
862         // need to be sent if there are other messages being sent to the remote
863         // actor.
864         heartbeatSchedule = context.getActorSystem().scheduler().scheduleOnce(
865             interval, context.getActor(), SendHeartBeat.INSTANCE,
866             context.getActorSystem().dispatcher(), context.getActor());
867     }
868
869     @Override
870     public void close() {
871         stopHeartBeat();
872     }
873
874     @Override
875     public final String getLeaderId() {
876         return context.getId();
877     }
878
879     @Override
880     public final short getLeaderPayloadVersion() {
881         return context.getPayloadVersion();
882     }
883
884     protected boolean isLeaderIsolated() {
885         int minPresent = getMinIsolatedLeaderPeerCount();
886         for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
887             final PeerInfo peerInfo = context.getPeerInfo(followerLogInformation.getId());
888             if (peerInfo != null && peerInfo.isVoting() && followerLogInformation.isFollowerActive()) {
889                 --minPresent;
890                 if (minPresent == 0) {
891                     return false;
892                 }
893             }
894         }
895         return minPresent != 0;
896     }
897
898     // called from example-actor for printing the follower-states
899     public String printFollowerStates() {
900         final StringBuilder sb = new StringBuilder();
901
902         sb.append('[');
903         for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
904             sb.append('{');
905             sb.append(followerLogInformation.getId());
906             sb.append(" state:");
907             sb.append(followerLogInformation.isFollowerActive());
908             sb.append("},");
909         }
910         sb.append(']');
911
912         return sb.toString();
913     }
914
915     @VisibleForTesting
916     public FollowerLogInformation getFollower(String followerId) {
917         return followerToLog.get(followerId);
918     }
919
920     @VisibleForTesting
921     public int followerLogSize() {
922         return followerToLog.size();
923     }
924
925     static class SnapshotHolder {
926         private final long lastIncludedTerm;
927         private final long lastIncludedIndex;
928         private final ByteSource snapshotBytes;
929
930         SnapshotHolder(Snapshot snapshot, ByteSource snapshotBytes) {
931             this.lastIncludedTerm = snapshot.getLastAppliedTerm();
932             this.lastIncludedIndex = snapshot.getLastAppliedIndex();
933             this.snapshotBytes = snapshotBytes;
934         }
935
936         long getLastIncludedTerm() {
937             return lastIncludedTerm;
938         }
939
940         long getLastIncludedIndex() {
941             return lastIncludedIndex;
942         }
943
944         ByteSource getSnapshotBytes() {
945             return snapshotBytes;
946         }
947     }
948 }