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