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