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