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