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