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