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