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