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