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