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