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