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