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