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