Merge "Add (un)lock ops to netconf testtool"
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / behaviors / AbstractLeader.java
1 /*
2  * Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
3  *
4  * This program and the accompanying materials are made available under the
5  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
6  * and is available at http://www.eclipse.org/legal/epl-v10.html
7  */
8
9 package org.opendaylight.controller.cluster.raft.behaviors;
10
11 import akka.actor.ActorRef;
12 import akka.actor.ActorSelection;
13 import akka.actor.Cancellable;
14 import com.google.common.annotations.VisibleForTesting;
15 import com.google.common.base.Optional;
16 import com.google.common.base.Preconditions;
17 import com.google.common.collect.ImmutableMap;
18 import com.google.common.collect.ImmutableMap.Builder;
19 import com.google.protobuf.ByteString;
20 import java.io.IOException;
21 import java.util.Collection;
22 import java.util.Collections;
23 import java.util.HashMap;
24 import java.util.Iterator;
25 import java.util.LinkedList;
26 import java.util.List;
27 import java.util.Map;
28 import java.util.Map.Entry;
29 import java.util.concurrent.TimeUnit;
30 import org.opendaylight.controller.cluster.raft.ClientRequestTracker;
31 import org.opendaylight.controller.cluster.raft.ClientRequestTrackerImpl;
32 import org.opendaylight.controller.cluster.raft.FollowerLogInformation;
33 import org.opendaylight.controller.cluster.raft.FollowerLogInformationImpl;
34 import org.opendaylight.controller.cluster.raft.RaftActorContext;
35 import org.opendaylight.controller.cluster.raft.RaftState;
36 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
37 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshot;
38 import org.opendaylight.controller.cluster.raft.base.messages.InitiateInstallSnapshot;
39 import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
40 import org.opendaylight.controller.cluster.raft.base.messages.SendHeartBeat;
41 import org.opendaylight.controller.cluster.raft.base.messages.SendInstallSnapshot;
42 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
43 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
44 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot;
45 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshotReply;
46 import org.opendaylight.controller.cluster.raft.messages.RaftRPC;
47 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
48 import scala.concurrent.duration.FiniteDuration;
49
50 /**
51  * The behavior of a RaftActor when it is in the Leader state
52  * <p/>
53  * Leaders:
54  * <ul>
55  * <li> Upon election: send initial empty AppendEntries RPCs
56  * (heartbeat) to each server; repeat during idle periods to
57  * prevent election timeouts (§5.2)
58  * <li> If command received from client: append entry to local log,
59  * respond after entry applied to state machine (§5.3)
60  * <li> If last log index ≥ nextIndex for a follower: send
61  * AppendEntries RPC with log entries starting at nextIndex
62  * <ul>
63  * <li> If successful: update nextIndex and matchIndex for
64  * follower (§5.3)
65  * <li> If AppendEntries fails because of log inconsistency:
66  * decrement nextIndex and retry (§5.3)
67  * </ul>
68  * <li> If there exists an N such that N > commitIndex, a majority
69  * of matchIndex[i] ≥ N, and log[N].term == currentTerm:
70  * set commitIndex = N (§5.3, §5.4).
71  */
72 public abstract class AbstractLeader extends AbstractRaftActorBehavior {
73
74     // The index of the first chunk that is sent when installing a snapshot
75     public static final int FIRST_CHUNK_INDEX = 1;
76
77     // The index that the follower should respond with if it needs the install snapshot to be reset
78     public static final int INVALID_CHUNK_INDEX = -1;
79
80     // This would be passed as the hash code of the last chunk when sending the first chunk
81     public static final int INITIAL_LAST_CHUNK_HASH_CODE = -1;
82
83     private final Map<String, FollowerLogInformation> followerToLog;
84     private final Map<String, FollowerToSnapshot> mapFollowerToSnapshot = new HashMap<>();
85
86     private Cancellable heartbeatSchedule = null;
87
88     private final Collection<ClientRequestTracker> trackerList = new LinkedList<>();
89
90     protected final int minReplicationCount;
91
92     protected final int minIsolatedLeaderPeerCount;
93
94     private Optional<ByteString> snapshot;
95
96     public AbstractLeader(RaftActorContext context) {
97         super(context);
98
99         final Builder<String, FollowerLogInformation> ftlBuilder = ImmutableMap.builder();
100         for (String followerId : context.getPeerAddresses().keySet()) {
101             FollowerLogInformation followerLogInformation =
102                 new FollowerLogInformationImpl(followerId,
103                     context.getCommitIndex(), -1,
104                     context.getConfigParams().getElectionTimeOutInterval());
105
106             ftlBuilder.put(followerId, followerLogInformation);
107         }
108         followerToLog = ftlBuilder.build();
109
110         leaderId = context.getId();
111
112         LOG.debug("{}: Election: Leader has following peers: {}", context.getId(), getFollowerIds());
113
114         minReplicationCount = getMajorityVoteCount(getFollowerIds().size());
115
116         // the isolated Leader peer count will be 1 less than the majority vote count.
117         // this is because the vote count has the self vote counted in it
118         // for e.g
119         // 0 peers = 1 votesRequired , minIsolatedLeaderPeerCount = 0
120         // 2 peers = 2 votesRequired , minIsolatedLeaderPeerCount = 1
121         // 4 peers = 3 votesRequired, minIsolatedLeaderPeerCount = 2
122         minIsolatedLeaderPeerCount = minReplicationCount > 0 ? (minReplicationCount - 1) : 0;
123
124         snapshot = Optional.absent();
125
126         // Immediately schedule a heartbeat
127         // Upon election: send initial empty AppendEntries RPCs
128         // (heartbeat) to each server; repeat during idle periods to
129         // prevent election timeouts (§5.2)
130         scheduleHeartBeat(new FiniteDuration(0, TimeUnit.SECONDS));
131     }
132
133     /**
134      * Return an immutable collection of follower identifiers.
135      *
136      * @return Collection of follower IDs
137      */
138     protected final Collection<String> getFollowerIds() {
139         return followerToLog.keySet();
140     }
141
142     private Optional<ByteString> getSnapshot() {
143         return snapshot;
144     }
145
146     @VisibleForTesting
147     void setSnapshot(Optional<ByteString> snapshot) {
148         this.snapshot = snapshot;
149     }
150
151     @Override
152     protected RaftActorBehavior handleAppendEntries(ActorRef sender,
153         AppendEntries appendEntries) {
154
155         if(LOG.isDebugEnabled()) {
156             LOG.debug("{}: handleAppendEntries: {}", context.getId(), appendEntries);
157         }
158
159         return this;
160     }
161
162     @Override
163     protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
164         AppendEntriesReply appendEntriesReply) {
165
166         if(! appendEntriesReply.isSuccess()) {
167             if(LOG.isDebugEnabled()) {
168                 LOG.debug("{}: handleAppendEntriesReply: {}", context.getId(), appendEntriesReply);
169             }
170         }
171
172         // Update the FollowerLogInformation
173         String followerId = appendEntriesReply.getFollowerId();
174         FollowerLogInformation followerLogInformation =
175             followerToLog.get(followerId);
176
177         if(followerLogInformation == null){
178             LOG.error("{}: handleAppendEntriesReply - unknown follower {}", context.getId(), followerId);
179             return this;
180         }
181
182         followerLogInformation.markFollowerActive();
183
184         if (appendEntriesReply.isSuccess()) {
185             followerLogInformation
186                 .setMatchIndex(appendEntriesReply.getLogLastIndex());
187             followerLogInformation
188                 .setNextIndex(appendEntriesReply.getLogLastIndex() + 1);
189         } else {
190
191             // TODO: When we find that the follower is out of sync with the
192             // Leader we simply decrement that followers next index by 1.
193             // Would it be possible to do better than this? The RAFT spec
194             // does not explicitly deal with it but may be something for us to
195             // think about
196
197             followerLogInformation.decrNextIndex();
198         }
199
200         // Now figure out if this reply warrants a change in the commitIndex
201         // If there exists an N such that N > commitIndex, a majority
202         // of matchIndex[i] ≥ N, and log[N].term == currentTerm:
203         // set commitIndex = N (§5.3, §5.4).
204         for (long N = context.getCommitIndex() + 1; ; N++) {
205             int replicatedCount = 1;
206
207             for (FollowerLogInformation info : followerToLog.values()) {
208                 if (info.getMatchIndex() >= N) {
209                     replicatedCount++;
210                 }
211             }
212
213             if (replicatedCount >= minReplicationCount) {
214                 ReplicatedLogEntry replicatedLogEntry = context.getReplicatedLog().get(N);
215                 if (replicatedLogEntry != null &&
216                     replicatedLogEntry.getTerm() == currentTerm()) {
217                     context.setCommitIndex(N);
218                 }
219             } else {
220                 break;
221             }
222         }
223
224         // Apply the change to the state machine
225         if (context.getCommitIndex() > context.getLastApplied()) {
226             applyLogToStateMachine(context.getCommitIndex());
227         }
228
229         return this;
230     }
231
232     @Override
233     protected ClientRequestTracker removeClientRequestTracker(long logIndex) {
234         final Iterator<ClientRequestTracker> it = trackerList.iterator();
235         while (it.hasNext()) {
236             final ClientRequestTracker t = it.next();
237             if (t.getIndex() == logIndex) {
238                 it.remove();
239                 return t;
240             }
241         }
242
243         return null;
244     }
245
246     @Override
247     protected ClientRequestTracker findClientRequestTracker(long logIndex) {
248         for (ClientRequestTracker tracker : trackerList) {
249             if (tracker.getIndex() == logIndex) {
250                 return tracker;
251             }
252         }
253         return null;
254     }
255
256     @Override
257     protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
258         RequestVoteReply requestVoteReply) {
259         return this;
260     }
261
262     @Override
263     public RaftState state() {
264         return RaftState.Leader;
265     }
266
267     @Override
268     public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) {
269         Preconditions.checkNotNull(sender, "sender should not be null");
270
271         Object message = fromSerializableMessage(originalMessage);
272
273         if (message instanceof RaftRPC) {
274             RaftRPC rpc = (RaftRPC) message;
275             // If RPC request or response contains term T > currentTerm:
276             // set currentTerm = T, convert to follower (§5.1)
277             // This applies to all RPC messages and responses
278             if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
279                 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
280
281                 return switchBehavior(new Follower(context));
282             }
283         }
284
285         try {
286             if (message instanceof SendHeartBeat) {
287                 sendHeartBeat();
288                 return this;
289
290             } else if(message instanceof InitiateInstallSnapshot) {
291                 installSnapshotIfNeeded();
292
293             } else if(message instanceof SendInstallSnapshot) {
294                 // received from RaftActor
295                 setSnapshot(Optional.of(((SendInstallSnapshot) message).getSnapshot()));
296                 sendInstallSnapshot();
297
298             } else if (message instanceof Replicate) {
299                 replicate((Replicate) message);
300
301             } else if (message instanceof InstallSnapshotReply){
302                 handleInstallSnapshotReply((InstallSnapshotReply) message);
303
304             }
305         } finally {
306             scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
307         }
308
309         return super.handleMessage(sender, message);
310     }
311
312     private void handleInstallSnapshotReply(InstallSnapshotReply reply) {
313         String followerId = reply.getFollowerId();
314         FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
315         FollowerLogInformation followerLogInformation = followerToLog.get(followerId);
316         followerLogInformation.markFollowerActive();
317
318         if (followerToSnapshot != null &&
319             followerToSnapshot.getChunkIndex() == reply.getChunkIndex()) {
320
321             if (reply.isSuccess()) {
322                 if(followerToSnapshot.isLastChunk(reply.getChunkIndex())) {
323                     //this was the last chunk reply
324                     if(LOG.isDebugEnabled()) {
325                         LOG.debug("{}: InstallSnapshotReply received, " +
326                                 "last chunk received, Chunk:{}. Follower:{} Setting nextIndex:{}",
327                                 context.getId(), reply.getChunkIndex(), followerId,
328                             context.getReplicatedLog().getSnapshotIndex() + 1
329                         );
330                     }
331
332                     followerLogInformation.setMatchIndex(
333                         context.getReplicatedLog().getSnapshotIndex());
334                     followerLogInformation.setNextIndex(
335                         context.getReplicatedLog().getSnapshotIndex() + 1);
336                     mapFollowerToSnapshot.remove(followerId);
337
338                     if(LOG.isDebugEnabled()) {
339                         LOG.debug("{}: followerToLog.get(followerId).getNextIndex()=" +
340                                 context.getId(), followerToLog.get(followerId).getNextIndex());
341                     }
342
343                     if (mapFollowerToSnapshot.isEmpty()) {
344                         // once there are no pending followers receiving snapshots
345                         // we can remove snapshot from the memory
346                         setSnapshot(Optional.<ByteString>absent());
347                     }
348
349                 } else {
350                     followerToSnapshot.markSendStatus(true);
351                 }
352             } else {
353                 LOG.info("{}: InstallSnapshotReply received sending snapshot chunk failed, Will retry, Chunk: {}",
354                         context.getId(), reply.getChunkIndex());
355
356                 followerToSnapshot.markSendStatus(false);
357             }
358
359         } else {
360             LOG.error("{}: FollowerId in InstallSnapshotReply not known to Leader" +
361                     " or Chunk Index in InstallSnapshotReply not matching {} != {}",
362                     context.getId(), followerToSnapshot.getChunkIndex(), reply.getChunkIndex()
363             );
364
365             if(reply.getChunkIndex() == INVALID_CHUNK_INDEX){
366                 // Since the Follower did not find this index to be valid we should reset the follower snapshot
367                 // so that Installing the snapshot can resume from the beginning
368                 followerToSnapshot.reset();
369             }
370         }
371     }
372
373     private void replicate(Replicate replicate) {
374         long logIndex = replicate.getReplicatedLogEntry().getIndex();
375
376         if(LOG.isDebugEnabled()) {
377             LOG.debug("{}: Replicate message {}", context.getId(), logIndex);
378         }
379
380         // Create a tracker entry we will use this later to notify the
381         // client actor
382         trackerList.add(
383             new ClientRequestTrackerImpl(replicate.getClientActor(),
384                 replicate.getIdentifier(),
385                 logIndex)
386         );
387
388         if (followerToLog.isEmpty()) {
389             context.setCommitIndex(logIndex);
390             applyLogToStateMachine(logIndex);
391         } else {
392             sendAppendEntries();
393         }
394     }
395
396     private void sendAppendEntries() {
397         // Send an AppendEntries to all followers
398         for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
399             final String followerId = e.getKey();
400             ActorSelection followerActor = context.getPeerActorSelection(followerId);
401
402             if (followerActor != null) {
403                 FollowerLogInformation followerLogInformation = followerToLog.get(followerId);
404                 long followerNextIndex = followerLogInformation.getNextIndex();
405                 boolean isFollowerActive = followerLogInformation.isFollowerActive();
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                     final List<ReplicatedLogEntry> entries;
421
422                     if (isFollowerActive &&
423                         context.getReplicatedLog().isPresent(followerNextIndex)) {
424                         // FIXME : Sending one entry at a time
425                         entries = context.getReplicatedLog().getFrom(followerNextIndex, 1);
426
427                     } else if (isFollowerActive && followerNextIndex >= 0 &&
428                         leaderLastIndex >= followerNextIndex ) {
429                         // if the followers next index is not present in the leaders log, and
430                         // if the follower is just not starting and if leader's index is more than followers index
431                         // then snapshot should be sent
432
433                         if(LOG.isDebugEnabled()) {
434                             LOG.debug(String.format("%s: InitiateInstallSnapshot to follower: %s," +
435                                     "follower-nextIndex: %s, leader-snapshot-index: %s,  " +
436                                     "leader-last-index: %s", context.getId(), followerId,
437                                 followerNextIndex, leaderSnapShotIndex, leaderLastIndex));
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", context.getId(), 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:{}", context.getId(), 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                         context.getId(), followerActor.path(),
570                         mapFollowerToSnapshot.get(followerId).getChunkIndex(),
571                         mapFollowerToSnapshot.get(followerId).getTotalChunks());
572             }
573         } catch (IOException e) {
574             LOG.error(e, "{}: InstallSnapshot failed for Leader.", context.getId());
575         }
576     }
577
578     /**
579      * Acccepts snaphot as ByteString, enters into map for future chunks
580      * creates and return a ByteString chunk
581      */
582     private ByteString getNextSnapshotChunk(String followerId, ByteString snapshotBytes) throws IOException {
583         FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
584         if (followerToSnapshot == null) {
585             followerToSnapshot = new FollowerToSnapshot(snapshotBytes);
586             mapFollowerToSnapshot.put(followerId, followerToSnapshot);
587         }
588         ByteString nextChunk = followerToSnapshot.getNextChunk();
589         if (LOG.isDebugEnabled()) {
590             LOG.debug("{}: Leader's snapshot nextChunk size:{}", context.getId(), nextChunk.size());
591         }
592         return nextChunk;
593     }
594
595     private void sendHeartBeat() {
596         if (!followerToLog.isEmpty()) {
597             sendAppendEntries();
598         }
599     }
600
601     private void stopHeartBeat() {
602         if (heartbeatSchedule != null && !heartbeatSchedule.isCancelled()) {
603             heartbeatSchedule.cancel();
604         }
605     }
606
607     private void scheduleHeartBeat(FiniteDuration interval) {
608         if (followerToLog.isEmpty()) {
609             // Optimization - do not bother scheduling a heartbeat as there are
610             // no followers
611             return;
612         }
613
614         stopHeartBeat();
615
616         // Schedule a heartbeat. When the scheduler triggers a SendHeartbeat
617         // message is sent to itself.
618         // Scheduling the heartbeat only once here because heartbeats do not
619         // need to be sent if there are other messages being sent to the remote
620         // actor.
621         heartbeatSchedule = context.getActorSystem().scheduler().scheduleOnce(
622             interval, context.getActor(), new SendHeartBeat(),
623             context.getActorSystem().dispatcher(), context.getActor());
624     }
625
626     @Override
627     public void close() throws Exception {
628         stopHeartBeat();
629     }
630
631     @Override
632     public String getLeaderId() {
633         return context.getId();
634     }
635
636     protected boolean isLeaderIsolated() {
637         int minPresent = minIsolatedLeaderPeerCount;
638         for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
639             if (followerLogInformation.isFollowerActive()) {
640                 --minPresent;
641                 if (minPresent == 0) {
642                     break;
643                 }
644             }
645         }
646         return (minPresent != 0);
647     }
648
649     /**
650      * Encapsulates the snapshot bytestring and handles the logic of sending
651      * snapshot chunks
652      */
653     protected class FollowerToSnapshot {
654         private final ByteString snapshotBytes;
655         private int offset = 0;
656         // the next snapshot chunk is sent only if the replyReceivedForOffset matches offset
657         private int replyReceivedForOffset;
658         // if replyStatus is false, the previous chunk is attempted
659         private boolean replyStatus = false;
660         private int chunkIndex;
661         private final int totalChunks;
662         private int lastChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
663         private int nextChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
664
665         public FollowerToSnapshot(ByteString snapshotBytes) {
666             this.snapshotBytes = snapshotBytes;
667             int size = snapshotBytes.size();
668             totalChunks = ( size / context.getConfigParams().getSnapshotChunkSize()) +
669                 ((size % context.getConfigParams().getSnapshotChunkSize()) > 0 ? 1 : 0);
670             if(LOG.isDebugEnabled()) {
671                 LOG.debug("{}: Snapshot {} bytes, total chunks to send:{}",
672                         context.getId(), size, totalChunks);
673             }
674             replyReceivedForOffset = -1;
675             chunkIndex = AbstractLeader.FIRST_CHUNK_INDEX;
676         }
677
678         public ByteString getSnapshotBytes() {
679             return snapshotBytes;
680         }
681
682         public int incrementOffset() {
683             if(replyStatus) {
684                 // if prev chunk failed, we would want to sent the same chunk again
685                 offset = offset + context.getConfigParams().getSnapshotChunkSize();
686             }
687             return offset;
688         }
689
690         public int incrementChunkIndex() {
691             if (replyStatus) {
692                 // if prev chunk failed, we would want to sent the same chunk again
693                 chunkIndex =  chunkIndex + 1;
694             }
695             return chunkIndex;
696         }
697
698         public int getChunkIndex() {
699             return chunkIndex;
700         }
701
702         public int getTotalChunks() {
703             return totalChunks;
704         }
705
706         public boolean canSendNextChunk() {
707             // we only send a false if a chunk is sent but we have not received a reply yet
708             return replyReceivedForOffset == offset;
709         }
710
711         public boolean isLastChunk(int chunkIndex) {
712             return totalChunks == chunkIndex;
713         }
714
715         public void markSendStatus(boolean success) {
716             if (success) {
717                 // if the chunk sent was successful
718                 replyReceivedForOffset = offset;
719                 replyStatus = true;
720                 lastChunkHashCode = nextChunkHashCode;
721             } else {
722                 // if the chunk sent was failure
723                 replyReceivedForOffset = offset;
724                 replyStatus = false;
725             }
726         }
727
728         public ByteString getNextChunk() {
729             int snapshotLength = getSnapshotBytes().size();
730             int start = incrementOffset();
731             int size = context.getConfigParams().getSnapshotChunkSize();
732             if (context.getConfigParams().getSnapshotChunkSize() > snapshotLength) {
733                 size = snapshotLength;
734             } else {
735                 if ((start + context.getConfigParams().getSnapshotChunkSize()) > snapshotLength) {
736                     size = snapshotLength - start;
737                 }
738             }
739
740             if(LOG.isDebugEnabled()) {
741                 LOG.debug("{}: Next chunk: length={}, offset={},size={}", context.getId(),
742                     snapshotLength, start, size);
743             }
744             ByteString substring = getSnapshotBytes().substring(start, start + size);
745             nextChunkHashCode = substring.hashCode();
746             return substring;
747         }
748
749         /**
750          * reset should be called when the Follower needs to be sent the snapshot from the beginning
751          */
752         public void reset(){
753             offset = 0;
754             replyStatus = false;
755             replyReceivedForOffset = offset;
756             chunkIndex = AbstractLeader.FIRST_CHUNK_INDEX;
757             lastChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
758         }
759
760         public int getLastChunkHashCode() {
761             return lastChunkHashCode;
762         }
763     }
764
765     // called from example-actor for printing the follower-states
766     public String printFollowerStates() {
767         final StringBuilder sb = new StringBuilder();
768
769         sb.append('[');
770         for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
771             sb.append('{');
772             sb.append(followerLogInformation.getId());
773             sb.append(" state:");
774             sb.append(followerLogInformation.isFollowerActive());
775             sb.append("},");
776         }
777         sb.append(']');
778
779         return sb.toString();
780     }
781
782     @VisibleForTesting
783     public FollowerLogInformation getFollower(String followerId) {
784         return followerToLog.get(followerId);
785     }
786
787     @VisibleForTesting
788     protected void setFollowerSnapshot(String followerId, FollowerToSnapshot snapshot) {
789         mapFollowerToSnapshot.put(followerId, snapshot);
790     }
791
792     @VisibleForTesting
793     public int followerSnapshotSize() {
794         return mapFollowerToSnapshot.size();
795     }
796
797     @VisibleForTesting
798     public int followerLogSize() {
799         return followerToLog.size();
800     }
801 }