Bug 2187: AddServer unit test and bug fixes
[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.Collection;
20 import java.util.Collections;
21 import java.util.HashMap;
22 import java.util.Iterator;
23 import java.util.LinkedList;
24 import java.util.List;
25 import java.util.Map;
26 import java.util.Map.Entry;
27 import javax.annotation.Nullable;
28 import org.opendaylight.controller.cluster.raft.ClientRequestTracker;
29 import org.opendaylight.controller.cluster.raft.ClientRequestTrackerImpl;
30 import org.opendaylight.controller.cluster.raft.FollowerLogInformation;
31 import org.opendaylight.controller.cluster.raft.FollowerLogInformation.FollowerState;
32 import org.opendaylight.controller.cluster.raft.FollowerLogInformationImpl;
33 import org.opendaylight.controller.cluster.raft.RaftActorContext;
34 import org.opendaylight.controller.cluster.raft.RaftState;
35 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
36 import org.opendaylight.controller.cluster.raft.Snapshot;
37 import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
38 import org.opendaylight.controller.cluster.raft.base.messages.SendHeartBeat;
39 import org.opendaylight.controller.cluster.raft.base.messages.SendInstallSnapshot;
40 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
41 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
42 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot;
43 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshotReply;
44 import org.opendaylight.controller.cluster.raft.messages.RaftRPC;
45 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
46 import org.opendaylight.controller.cluster.raft.messages.UnInitializedFollowerSnapshotReply;
47 import scala.concurrent.duration.FiniteDuration;
48
49 /**
50  * The behavior of a RaftActor when it is in the Leader state
51  * <p/>
52  * Leaders:
53  * <ul>
54  * <li> Upon election: send initial empty AppendEntries RPCs
55  * (heartbeat) to each server; repeat during idle periods to
56  * prevent election timeouts (§5.2)
57  * <li> If command received from client: append entry to local log,
58  * respond after entry applied to state machine (§5.3)
59  * <li> If last log index ≥ nextIndex for a follower: send
60  * AppendEntries RPC with log entries starting at nextIndex
61  * <ul>
62  * <li> If successful: update nextIndex and matchIndex for
63  * follower (§5.3)
64  * <li> If AppendEntries fails because of log inconsistency:
65  * decrement nextIndex and retry (§5.3)
66  * </ul>
67  * <li> If there exists an N such that N > commitIndex, a majority
68  * of matchIndex[i] ≥ N, and log[N].term == currentTerm:
69  * set commitIndex = N (§5.3, §5.4).
70  */
71 public abstract class AbstractLeader extends AbstractRaftActorBehavior {
72
73     // The index of the first chunk that is sent when installing a snapshot
74     public static final int FIRST_CHUNK_INDEX = 1;
75
76     // The index that the follower should respond with if it needs the install snapshot to be reset
77     public static final int INVALID_CHUNK_INDEX = -1;
78
79     // This would be passed as the hash code of the last chunk when sending the first chunk
80     public static final int INITIAL_LAST_CHUNK_HASH_CODE = -1;
81
82     private final Map<String, FollowerLogInformation> followerToLog = new HashMap<>();
83     private final Map<String, FollowerToSnapshot> mapFollowerToSnapshot = new HashMap<>();
84
85     private Cancellable heartbeatSchedule = null;
86
87     private final Collection<ClientRequestTracker> trackerList = new LinkedList<>();
88
89     private int minReplicationCount;
90
91     private int minIsolatedLeaderPeerCount;
92
93     private Optional<SnapshotHolder> snapshot;
94
95     public AbstractLeader(RaftActorContext context) {
96         super(context, RaftState.Leader);
97
98         setLeaderPayloadVersion(context.getPayloadVersion());
99
100         for (String followerId : context.getPeerAddresses().keySet()) {
101             FollowerLogInformation followerLogInformation =
102                 new FollowerLogInformationImpl(followerId, -1, context);
103
104             followerToLog.put(followerId, followerLogInformation);
105         }
106
107         leaderId = context.getId();
108
109         LOG.debug("{}: Election: Leader has following peers: {}", logName(), getFollowerIds());
110
111         updateMinReplicaCountAndMinIsolatedLeaderPeerCount();
112
113         snapshot = Optional.absent();
114
115         // Immediately schedule a heartbeat
116         // Upon election: send initial empty AppendEntries RPCs
117         // (heartbeat) to each server; repeat during idle periods to
118         // prevent election timeouts (§5.2)
119         sendAppendEntries(0, false);
120
121         // It is important to schedule this heartbeat here
122         scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
123     }
124
125     /**
126      * Return an immutable collection of follower identifiers.
127      *
128      * @return Collection of follower IDs
129      */
130     public final Collection<String> getFollowerIds() {
131         return followerToLog.keySet();
132     }
133
134     public void addFollower(String followerId, FollowerState followerState) {
135         FollowerLogInformation followerLogInformation = new FollowerLogInformationImpl(followerId, -1, context);
136         followerLogInformation.setFollowerState(followerState);
137         followerToLog.put(followerId, followerLogInformation);
138
139         if(heartbeatSchedule == null) {
140             scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
141         }
142     }
143
144     public void removeFollower(String followerId) {
145         followerToLog.remove(followerId);
146     }
147
148     public void updateMinReplicaCountAndMinIsolatedLeaderPeerCount(){
149         minReplicationCount = getMajorityVoteCount(getFollowerIds().size());
150
151         //the isolated Leader peer count will be 1 less than the majority vote count.
152         //this is because the vote count has the self vote counted in it
153         //for e.g
154         //0 peers = 1 votesRequired , minIsolatedLeaderPeerCount = 0
155         //2 peers = 2 votesRequired , minIsolatedLeaderPeerCount = 1
156         //4 peers = 3 votesRequired, minIsolatedLeaderPeerCount = 2
157         minIsolatedLeaderPeerCount = minReplicationCount > 0 ? (minReplicationCount - 1) : 0;
158     }
159
160     public int getMinIsolatedLeaderPeerCount(){
161         return minIsolatedLeaderPeerCount;
162     }
163
164     @VisibleForTesting
165     void setSnapshot(@Nullable Snapshot snapshot) {
166         if(snapshot != null) {
167             this.snapshot = Optional.of(new SnapshotHolder(snapshot));
168         } else {
169             this.snapshot = Optional.absent();
170         }
171     }
172
173     @Override
174     protected RaftActorBehavior handleAppendEntries(ActorRef sender,
175         AppendEntries appendEntries) {
176
177         LOG.debug("{}: handleAppendEntries: {}", logName(), appendEntries);
178
179         return this;
180     }
181
182     @Override
183     protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
184         AppendEntriesReply appendEntriesReply) {
185
186         if(LOG.isTraceEnabled()) {
187             LOG.trace("{}: handleAppendEntriesReply: {}", logName(), appendEntriesReply);
188         }
189
190         // Update the FollowerLogInformation
191         String followerId = appendEntriesReply.getFollowerId();
192         FollowerLogInformation followerLogInformation =
193             followerToLog.get(followerId);
194
195         if(followerLogInformation == null){
196             LOG.error("{}: handleAppendEntriesReply - unknown follower {}", logName(), followerId);
197             return this;
198         }
199
200         if(followerLogInformation.timeSinceLastActivity() >
201                 context.getConfigParams().getElectionTimeOutInterval().toMillis()) {
202             LOG.warn("{} : handleAppendEntriesReply delayed beyond election timeout, " +
203                             "appendEntriesReply : {}, timeSinceLastActivity : {}, lastApplied : {}, commitIndex : {}",
204                     logName(), appendEntriesReply, followerLogInformation.timeSinceLastActivity(),
205                     context.getLastApplied(), context.getCommitIndex());
206         }
207
208         followerLogInformation.markFollowerActive();
209         followerLogInformation.setPayloadVersion(appendEntriesReply.getPayloadVersion());
210
211         boolean updated = false;
212         if (appendEntriesReply.isSuccess()) {
213             updated = updateFollowerLogInformation(followerLogInformation, appendEntriesReply);
214         } else {
215             LOG.debug("{}: handleAppendEntriesReply: received unsuccessful reply: {}", logName(), appendEntriesReply);
216
217             long followerLastLogIndex = appendEntriesReply.getLogLastIndex();
218             ReplicatedLogEntry followersLastLogEntry = context.getReplicatedLog().get(followerLastLogIndex);
219             if(appendEntriesReply.isForceInstallSnapshot()) {
220                 // Reset the followers match and next index. This is to signal that this follower has nothing
221                 // in common with this Leader and so would require a snapshot to be installed
222                 followerLogInformation.setMatchIndex(-1);
223                 followerLogInformation.setNextIndex(-1);
224
225                 // Force initiate a snapshot capture
226                 initiateCaptureSnapshot(followerId);
227             } else if(followerLastLogIndex < 0 || (followersLastLogEntry != null &&
228                     followersLastLogEntry.getTerm() == appendEntriesReply.getLogLastTerm())) {
229                 // The follower's log is empty or the last entry is present in the leader's journal
230                 // and the terms match so the follower is just behind the leader's journal from
231                 // the last snapshot, if any. We'll catch up the follower quickly by starting at the
232                 // follower's last log index.
233
234                 updated = updateFollowerLogInformation(followerLogInformation, appendEntriesReply);
235             } else {
236                 // TODO: When we find that the follower is out of sync with the
237                 // Leader we simply decrement that followers next index by 1.
238                 // Would it be possible to do better than this? The RAFT spec
239                 // does not explicitly deal with it but may be something for us to
240                 // think about.
241
242                 followerLogInformation.decrNextIndex();
243             }
244         }
245
246         // Now figure out if this reply warrants a change in the commitIndex
247         // If there exists an N such that N > commitIndex, a majority
248         // of matchIndex[i] ≥ N, and log[N].term == currentTerm:
249         // set commitIndex = N (§5.3, §5.4).
250         for (long N = context.getCommitIndex() + 1; ; N++) {
251             int replicatedCount = 1;
252
253             for (FollowerLogInformation info : followerToLog.values()) {
254                 if (info.getMatchIndex() >= N) {
255                     replicatedCount++;
256                 }
257             }
258
259             if (replicatedCount >= minReplicationCount) {
260                 ReplicatedLogEntry replicatedLogEntry = context.getReplicatedLog().get(N);
261                 if (replicatedLogEntry != null && replicatedLogEntry.getTerm() == currentTerm()) {
262                     context.setCommitIndex(N);
263                 } else {
264                     break;
265                 }
266             } else {
267                 break;
268             }
269         }
270
271         // Apply the change to the state machine
272         if (context.getCommitIndex() > context.getLastApplied()) {
273             if(LOG.isDebugEnabled()) {
274                 LOG.debug("{}: handleAppendEntriesReply from {}: applying to log - commitIndex: {}, lastAppliedIndex: {}",
275                         logName(), followerId, context.getCommitIndex(), context.getLastApplied());
276             }
277
278             applyLogToStateMachine(context.getCommitIndex());
279         }
280
281         if (!context.getSnapshotManager().isCapturing()) {
282             purgeInMemoryLog();
283         }
284
285         //Send the next log entry immediately, if possible, no need to wait for heartbeat to trigger that event
286         sendUpdatesToFollower(followerId, followerLogInformation, false, !updated);
287         return this;
288     }
289
290     private boolean updateFollowerLogInformation(FollowerLogInformation followerLogInformation,
291             AppendEntriesReply appendEntriesReply) {
292         boolean updated = followerLogInformation.setMatchIndex(appendEntriesReply.getLogLastIndex());
293         updated = followerLogInformation.setNextIndex(appendEntriesReply.getLogLastIndex() + 1) || updated;
294
295         if(updated && LOG.isDebugEnabled()) {
296             LOG.debug("{}: handleAppendEntriesReply - FollowerLogInformation for {} updated: matchIndex: {}, nextIndex: {}",
297                     logName(), followerLogInformation.getId(), followerLogInformation.getMatchIndex(),
298                     followerLogInformation.getNextIndex());
299         }
300         return updated;
301     }
302
303     private void purgeInMemoryLog() {
304         //find the lowest index across followers which has been replicated to all.
305         // lastApplied if there are no followers, so that we keep clearing the log for single-node
306         // we would delete the in-mem log from that index on, in-order to minimize mem usage
307         // we would also share this info thru AE with the followers so that they can delete their log entries as well.
308         long minReplicatedToAllIndex = followerToLog.isEmpty() ? context.getLastApplied() : Long.MAX_VALUE;
309         for (FollowerLogInformation info : followerToLog.values()) {
310             minReplicatedToAllIndex = Math.min(minReplicatedToAllIndex, info.getMatchIndex());
311         }
312
313         super.performSnapshotWithoutCapture(minReplicatedToAllIndex);
314     }
315
316     @Override
317     protected ClientRequestTracker removeClientRequestTracker(long logIndex) {
318         final Iterator<ClientRequestTracker> it = trackerList.iterator();
319         while (it.hasNext()) {
320             final ClientRequestTracker t = it.next();
321             if (t.getIndex() == logIndex) {
322                 it.remove();
323                 return t;
324             }
325         }
326
327         return null;
328     }
329
330     @Override
331     protected ClientRequestTracker findClientRequestTracker(long logIndex) {
332         for (ClientRequestTracker tracker : trackerList) {
333             if (tracker.getIndex() == logIndex) {
334                 return tracker;
335             }
336         }
337         return null;
338     }
339
340     @Override
341     protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
342         RequestVoteReply requestVoteReply) {
343         return this;
344     }
345
346     protected void beforeSendHeartbeat(){}
347
348     @Override
349     public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) {
350         Preconditions.checkNotNull(sender, "sender should not be null");
351
352         Object message = fromSerializableMessage(originalMessage);
353
354         if (message instanceof RaftRPC) {
355             RaftRPC rpc = (RaftRPC) message;
356             // If RPC request or response contains term T > currentTerm:
357             // set currentTerm = T, convert to follower (§5.1)
358             // This applies to all RPC messages and responses
359             if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
360                 LOG.debug("{}: Term {} in \"{}\" message is greater than leader's term {} - switching to Follower",
361                         logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm());
362
363                 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
364
365                 return internalSwitchBehavior(RaftState.Follower);
366             }
367         }
368
369         if (message instanceof SendHeartBeat) {
370             beforeSendHeartbeat();
371             sendHeartBeat();
372             scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
373             return this;
374
375         } else if(message instanceof SendInstallSnapshot) {
376             // received from RaftActor
377             setSnapshot(((SendInstallSnapshot) message).getSnapshot());
378             sendInstallSnapshot();
379
380         } else if (message instanceof Replicate) {
381             replicate((Replicate) message);
382
383         } else if (message instanceof InstallSnapshotReply){
384             handleInstallSnapshotReply((InstallSnapshotReply) message);
385
386         }
387
388
389         return super.handleMessage(sender, message);
390     }
391
392     private void handleInstallSnapshotReply(InstallSnapshotReply reply) {
393         LOG.debug("{}: handleInstallSnapshotReply: {}", logName(), reply);
394
395         String followerId = reply.getFollowerId();
396         FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
397
398         if (followerToSnapshot == null) {
399             LOG.error("{}: FollowerId {} in InstallSnapshotReply not known to Leader",
400                     logName(), followerId);
401             return;
402         }
403
404         FollowerLogInformation followerLogInformation = followerToLog.get(followerId);
405         followerLogInformation.markFollowerActive();
406
407         if (followerToSnapshot.getChunkIndex() == reply.getChunkIndex()) {
408             boolean wasLastChunk = false;
409             if (reply.isSuccess()) {
410                 if(followerToSnapshot.isLastChunk(reply.getChunkIndex())) {
411                     //this was the last chunk reply
412                     if(LOG.isDebugEnabled()) {
413                         LOG.debug("{}: InstallSnapshotReply received, " +
414                                 "last chunk received, Chunk: {}. Follower: {} Setting nextIndex: {}",
415                                 logName(), reply.getChunkIndex(), followerId,
416                             context.getReplicatedLog().getSnapshotIndex() + 1
417                         );
418                     }
419
420                     long followerMatchIndex = snapshot.get().getLastIncludedIndex();
421                     followerLogInformation.setMatchIndex(followerMatchIndex);
422                     followerLogInformation.setNextIndex(followerMatchIndex + 1);
423                     mapFollowerToSnapshot.remove(followerId);
424
425                     LOG.debug("{}: follower: {}, matchIndex set to {}, nextIndex set to {}",
426                         logName(), followerId, followerLogInformation.getMatchIndex(),
427                         followerLogInformation.getNextIndex());
428
429                     if (mapFollowerToSnapshot.isEmpty()) {
430                         // once there are no pending followers receiving snapshots
431                         // we can remove snapshot from the memory
432                         setSnapshot(null);
433                     }
434                     wasLastChunk = true;
435                     FollowerState followerState = followerLogInformation.getFollowerState();
436                     if(followerState ==  FollowerState.VOTING_NOT_INITIALIZED){
437                         UnInitializedFollowerSnapshotReply unInitFollowerSnapshotSuccess =
438                                              new UnInitializedFollowerSnapshotReply(followerId);
439                         context.getActor().tell(unInitFollowerSnapshotSuccess, context.getActor());
440                         LOG.debug("Sent message UnInitializedFollowerSnapshotReply to self");
441                     }
442                 } else {
443                     followerToSnapshot.markSendStatus(true);
444                 }
445             } else {
446                 LOG.info("{}: InstallSnapshotReply received sending snapshot chunk failed, Will retry, Chunk: {}",
447                         logName(), reply.getChunkIndex());
448
449                 followerToSnapshot.markSendStatus(false);
450             }
451
452             if (wasLastChunk && !context.getSnapshotManager().isCapturing()) {
453                 // Since the follower is now caught up try to purge the log.
454                 purgeInMemoryLog();
455             } else if (!wasLastChunk && followerToSnapshot.canSendNextChunk()) {
456                 ActorSelection followerActor = context.getPeerActorSelection(followerId);
457                 if(followerActor != null) {
458                     sendSnapshotChunk(followerActor, followerId);
459                 }
460             }
461
462         } else {
463             LOG.error("{}: Chunk index {} in InstallSnapshotReply from follower {} does not match expected index {}",
464                     logName(), reply.getChunkIndex(), followerId,
465                     followerToSnapshot.getChunkIndex());
466
467             if(reply.getChunkIndex() == INVALID_CHUNK_INDEX){
468                 // Since the Follower did not find this index to be valid we should reset the follower snapshot
469                 // so that Installing the snapshot can resume from the beginning
470                 followerToSnapshot.reset();
471             }
472         }
473     }
474
475     private void replicate(Replicate replicate) {
476         long logIndex = replicate.getReplicatedLogEntry().getIndex();
477
478         LOG.debug("{}: Replicate message: identifier: {}, logIndex: {}", logName(),
479                 replicate.getIdentifier(), logIndex);
480
481         // Create a tracker entry we will use this later to notify the
482         // client actor
483         trackerList.add(
484             new ClientRequestTrackerImpl(replicate.getClientActor(),
485                 replicate.getIdentifier(),
486                 logIndex)
487         );
488
489         boolean applyModificationToState = followerToLog.isEmpty()
490                 || context.getRaftPolicy().applyModificationToStateBeforeConsensus();
491
492         if(applyModificationToState){
493             context.setCommitIndex(logIndex);
494             applyLogToStateMachine(logIndex);
495         }
496
497         if (!followerToLog.isEmpty()) {
498             sendAppendEntries(0, false);
499         }
500     }
501
502     private void sendAppendEntries(long timeSinceLastActivityInterval, boolean isHeartbeat) {
503         // Send an AppendEntries to all followers
504         for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
505             final String followerId = e.getKey();
506             final FollowerLogInformation followerLogInformation = e.getValue();
507             // This checks helps not to send a repeat message to the follower
508             if(!followerLogInformation.isFollowerActive() ||
509                     followerLogInformation.timeSinceLastActivity() >= timeSinceLastActivityInterval) {
510                 sendUpdatesToFollower(followerId, followerLogInformation, true, isHeartbeat);
511             }
512         }
513     }
514
515     /**
516      *
517      * This method checks if any update needs to be sent to the given follower. This includes append log entries,
518      * sending next snapshot chunk, and initiating a snapshot.
519      * @return true if any update is sent, false otherwise
520      */
521
522     private void sendUpdatesToFollower(String followerId, FollowerLogInformation followerLogInformation,
523                                        boolean sendHeartbeat, boolean isHeartbeat) {
524
525         ActorSelection followerActor = context.getPeerActorSelection(followerId);
526         if (followerActor != null) {
527             long followerNextIndex = followerLogInformation.getNextIndex();
528             boolean isFollowerActive = followerLogInformation.isFollowerActive();
529             boolean sendAppendEntries = false;
530             List<ReplicatedLogEntry> entries = Collections.emptyList();
531
532             if (mapFollowerToSnapshot.get(followerId) != null) {
533                 // if install snapshot is in process , then sent next chunk if possible
534                 if (isFollowerActive && mapFollowerToSnapshot.get(followerId).canSendNextChunk()) {
535                     sendSnapshotChunk(followerActor, followerId);
536                 } else if(sendHeartbeat) {
537                     // we send a heartbeat even if we have not received a reply for the last chunk
538                     sendAppendEntries = true;
539                 }
540             } else {
541                 long leaderLastIndex = context.getReplicatedLog().lastIndex();
542                 long leaderSnapShotIndex = context.getReplicatedLog().getSnapshotIndex();
543
544                 if((!isHeartbeat && LOG.isDebugEnabled()) || LOG.isTraceEnabled()) {
545                     LOG.debug("{}: Checking sendAppendEntries for follower {}: active: {}, followerNextIndex: {}, leaderLastIndex: {}, leaderSnapShotIndex: {}",
546                             logName(), followerId, isFollowerActive, followerNextIndex, leaderLastIndex, leaderSnapShotIndex);
547                 }
548
549                 if (isFollowerActive && context.getReplicatedLog().isPresent(followerNextIndex)) {
550
551                     LOG.debug("{}: sendAppendEntries: {} is present for follower {}", logName(),
552                             followerNextIndex, followerId);
553
554                     if(followerLogInformation.okToReplicate()) {
555                         // Try to send all the entries in the journal but not exceeding the max data size
556                         // for a single AppendEntries message.
557                         int maxEntries = (int) context.getReplicatedLog().size();
558                         entries = context.getReplicatedLog().getFrom(followerNextIndex, maxEntries,
559                                 context.getConfigParams().getSnapshotChunkSize());
560                         sendAppendEntries = true;
561                     }
562                 } else if (isFollowerActive && followerNextIndex >= 0 &&
563                     leaderLastIndex > followerNextIndex && !context.getSnapshotManager().isCapturing()) {
564                     // if the followers next index is not present in the leaders log, and
565                     // if the follower is just not starting and if leader's index is more than followers index
566                     // then snapshot should be sent
567
568                     if (LOG.isDebugEnabled()) {
569                         LOG.debug(String.format("%s: InitiateInstallSnapshot to follower: %s," +
570                                     "follower-nextIndex: %d, leader-snapshot-index: %d,  " +
571                                     "leader-last-index: %d", logName(), followerId,
572                                     followerNextIndex, leaderSnapShotIndex, leaderLastIndex));
573                     }
574
575                     // Send heartbeat to follower whenever install snapshot is initiated.
576                     sendAppendEntries = true;
577                     if (canInstallSnapshot(followerNextIndex)) {
578                         initiateCaptureSnapshot(followerId);
579                     }
580
581                 } else if(sendHeartbeat) {
582                     // we send an AppendEntries, even if the follower is inactive
583                     // in-order to update the followers timestamp, in case it becomes active again
584                     sendAppendEntries = true;
585                 }
586
587             }
588
589             if(sendAppendEntries) {
590                 sendAppendEntriesToFollower(followerActor, followerNextIndex,
591                         entries, followerId);
592             }
593         }
594     }
595
596     private void sendAppendEntriesToFollower(ActorSelection followerActor, long followerNextIndex,
597         List<ReplicatedLogEntry> entries, String followerId) {
598         AppendEntries appendEntries = new AppendEntries(currentTerm(), context.getId(),
599             prevLogIndex(followerNextIndex),
600             prevLogTerm(followerNextIndex), entries,
601             context.getCommitIndex(), super.getReplicatedToAllIndex(), context.getPayloadVersion());
602
603         if(!entries.isEmpty() || LOG.isTraceEnabled()) {
604             LOG.debug("{}: Sending AppendEntries to follower {}: {}", logName(), followerId,
605                     appendEntries);
606         }
607
608         followerActor.tell(appendEntries.toSerializable(), actor());
609     }
610
611     /**
612      * Install Snapshot works as follows
613      * 1. Leader initiates the capture snapshot by sending a CaptureSnapshot message to actor
614      * 2. RaftActor on receipt of the CaptureSnapshotReply (from Shard), stores the received snapshot in the replicated log
615      * and makes a call to Leader's handleMessage , with SendInstallSnapshot message.
616      * 3. Leader , picks the snapshot from im-mem ReplicatedLog and sends it in chunks to the Follower
617      * 4. On complete, Follower sends back a InstallSnapshotReply.
618      * 5. On receipt of the InstallSnapshotReply for the last chunk, Leader marks the install complete for that follower
619      * and replenishes the memory by deleting the snapshot in Replicated log.
620      * 6. If another follower requires a snapshot and a snapshot has been collected (via CaptureSnapshotReply)
621      * then send the existing snapshot in chunks to the follower.
622      * @param followerId
623      */
624     public void initiateCaptureSnapshot(String followerId) {
625         if (snapshot.isPresent()) {
626             // if a snapshot is present in the memory, most likely another install is in progress
627             // no need to capture snapshot.
628             // This could happen if another follower needs an install when one is going on.
629             final ActorSelection followerActor = context.getPeerActorSelection(followerId);
630             sendSnapshotChunk(followerActor, followerId);
631
632
633         } else {
634             context.getSnapshotManager().captureToInstall(context.getReplicatedLog().last(),
635                     this.getReplicatedToAllIndex(), followerId);
636         }
637     }
638
639     private boolean canInstallSnapshot(long nextIndex){
640         // If the follower's nextIndex is -1 then we might as well send it a snapshot
641         // Otherwise send it a snapshot only if the nextIndex is not present in the log but is present
642         // in the snapshot
643         return (nextIndex == -1 ||
644                 (!context.getReplicatedLog().isPresent(nextIndex)
645                         && context.getReplicatedLog().isInSnapshot(nextIndex)));
646
647     }
648
649
650     private void sendInstallSnapshot() {
651         LOG.debug("{}: sendInstallSnapshot", logName());
652         for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
653             ActorSelection followerActor = context.getPeerActorSelection(e.getKey());
654             FollowerLogInformation followerLogInfo = e.getValue();
655
656             if (followerActor != null) {
657                 long nextIndex = e.getValue().getNextIndex();
658                 if (followerLogInfo.getFollowerState() == FollowerState.VOTING_NOT_INITIALIZED ||
659                         canInstallSnapshot(nextIndex)) {
660                     sendSnapshotChunk(followerActor, e.getKey());
661                 }
662             }
663         }
664     }
665
666     /**
667      *  Sends a snapshot chunk to a given follower
668      *  InstallSnapshot should qualify as a heartbeat too.
669      */
670     private void sendSnapshotChunk(ActorSelection followerActor, String followerId) {
671         try {
672             if (snapshot.isPresent()) {
673                 ByteString nextSnapshotChunk = getNextSnapshotChunk(followerId, snapshot.get().getSnapshotBytes());
674
675                 // Note: the previous call to getNextSnapshotChunk has the side-effect of adding
676                 // followerId to the followerToSnapshot map.
677                 FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
678
679                 followerActor.tell(
680                     new InstallSnapshot(currentTerm(), context.getId(),
681                         snapshot.get().getLastIncludedIndex(),
682                         snapshot.get().getLastIncludedTerm(),
683                         nextSnapshotChunk,
684                         followerToSnapshot.incrementChunkIndex(),
685                         followerToSnapshot.getTotalChunks(),
686                         Optional.of(followerToSnapshot.getLastChunkHashCode())
687                     ).toSerializable(),
688                     actor()
689                 );
690
691                 if(LOG.isDebugEnabled()) {
692                     LOG.debug("{}: InstallSnapshot sent to follower {}, Chunk: {}/{}",
693                             logName(), followerActor.path(), followerToSnapshot.getChunkIndex(),
694                             followerToSnapshot.getTotalChunks());
695                 }
696             }
697         } catch (IOException e) {
698             LOG.error("{}: InstallSnapshot failed for Leader.", logName(), e);
699         }
700     }
701
702     /**
703      * Acccepts snaphot as ByteString, enters into map for future chunks
704      * creates and return a ByteString chunk
705      */
706     private ByteString getNextSnapshotChunk(String followerId, ByteString snapshotBytes) throws IOException {
707         FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
708         if (followerToSnapshot == null) {
709             followerToSnapshot = new FollowerToSnapshot(snapshotBytes);
710             mapFollowerToSnapshot.put(followerId, followerToSnapshot);
711         }
712         ByteString nextChunk = followerToSnapshot.getNextChunk();
713
714         LOG.debug("{}: next snapshot chunk size for follower {}: {}", logName(), followerId, nextChunk.size());
715
716         return nextChunk;
717     }
718
719     private void sendHeartBeat() {
720         if (!followerToLog.isEmpty()) {
721             LOG.trace("{}: Sending heartbeat", logName());
722             sendAppendEntries(context.getConfigParams().getHeartBeatInterval().toMillis(), true);
723         }
724     }
725
726     private void stopHeartBeat() {
727         if (heartbeatSchedule != null && !heartbeatSchedule.isCancelled()) {
728             heartbeatSchedule.cancel();
729         }
730     }
731
732     private void scheduleHeartBeat(FiniteDuration interval) {
733         if (followerToLog.isEmpty()) {
734             // Optimization - do not bother scheduling a heartbeat as there are
735             // no followers
736             return;
737         }
738
739         stopHeartBeat();
740
741         // Schedule a heartbeat. When the scheduler triggers a SendHeartbeat
742         // message is sent to itself.
743         // Scheduling the heartbeat only once here because heartbeats do not
744         // need to be sent if there are other messages being sent to the remote
745         // actor.
746         heartbeatSchedule = context.getActorSystem().scheduler().scheduleOnce(
747             interval, context.getActor(), new SendHeartBeat(),
748             context.getActorSystem().dispatcher(), context.getActor());
749     }
750
751     @Override
752     public void close() throws Exception {
753         stopHeartBeat();
754     }
755
756     @Override
757     public String getLeaderId() {
758         return context.getId();
759     }
760
761     protected boolean isLeaderIsolated() {
762         int minPresent = minIsolatedLeaderPeerCount;
763         for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
764             if (followerLogInformation.isFollowerActive()) {
765                 --minPresent;
766                 if (minPresent == 0) {
767                     break;
768                 }
769             }
770         }
771         return (minPresent != 0);
772     }
773
774     /**
775      * Encapsulates the snapshot bytestring and handles the logic of sending
776      * snapshot chunks
777      */
778     protected class FollowerToSnapshot {
779         private final ByteString snapshotBytes;
780         private int offset = 0;
781         // the next snapshot chunk is sent only if the replyReceivedForOffset matches offset
782         private int replyReceivedForOffset;
783         // if replyStatus is false, the previous chunk is attempted
784         private boolean replyStatus = false;
785         private int chunkIndex;
786         private final int totalChunks;
787         private int lastChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
788         private int nextChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
789
790         public FollowerToSnapshot(ByteString snapshotBytes) {
791             this.snapshotBytes = snapshotBytes;
792             int size = snapshotBytes.size();
793             totalChunks = ( size / context.getConfigParams().getSnapshotChunkSize()) +
794                 ((size % context.getConfigParams().getSnapshotChunkSize()) > 0 ? 1 : 0);
795             if(LOG.isDebugEnabled()) {
796                 LOG.debug("{}: Snapshot {} bytes, total chunks to send:{}",
797                         logName(), size, totalChunks);
798             }
799             replyReceivedForOffset = -1;
800             chunkIndex = AbstractLeader.FIRST_CHUNK_INDEX;
801         }
802
803         public ByteString getSnapshotBytes() {
804             return snapshotBytes;
805         }
806
807         public int incrementOffset() {
808             if(replyStatus) {
809                 // if prev chunk failed, we would want to sent the same chunk again
810                 offset = offset + context.getConfigParams().getSnapshotChunkSize();
811             }
812             return offset;
813         }
814
815         public int incrementChunkIndex() {
816             if (replyStatus) {
817                 // if prev chunk failed, we would want to sent the same chunk again
818                 chunkIndex =  chunkIndex + 1;
819             }
820             return chunkIndex;
821         }
822
823         public int getChunkIndex() {
824             return chunkIndex;
825         }
826
827         public int getTotalChunks() {
828             return totalChunks;
829         }
830
831         public boolean canSendNextChunk() {
832             // we only send a false if a chunk is sent but we have not received a reply yet
833             return replyReceivedForOffset == offset;
834         }
835
836         public boolean isLastChunk(int chunkIndex) {
837             return totalChunks == chunkIndex;
838         }
839
840         public void markSendStatus(boolean success) {
841             if (success) {
842                 // if the chunk sent was successful
843                 replyReceivedForOffset = offset;
844                 replyStatus = true;
845                 lastChunkHashCode = nextChunkHashCode;
846             } else {
847                 // if the chunk sent was failure
848                 replyReceivedForOffset = offset;
849                 replyStatus = false;
850             }
851         }
852
853         public ByteString getNextChunk() {
854             int snapshotLength = getSnapshotBytes().size();
855             int start = incrementOffset();
856             int size = context.getConfigParams().getSnapshotChunkSize();
857             if (context.getConfigParams().getSnapshotChunkSize() > snapshotLength) {
858                 size = snapshotLength;
859             } else {
860                 if ((start + context.getConfigParams().getSnapshotChunkSize()) > snapshotLength) {
861                     size = snapshotLength - start;
862                 }
863             }
864
865
866             LOG.debug("{}: Next chunk: length={}, offset={},size={}", logName(),
867                     snapshotLength, start, size);
868
869             ByteString substring = getSnapshotBytes().substring(start, start + size);
870             nextChunkHashCode = substring.hashCode();
871             return substring;
872         }
873
874         /**
875          * reset should be called when the Follower needs to be sent the snapshot from the beginning
876          */
877         public void reset(){
878             offset = 0;
879             replyStatus = false;
880             replyReceivedForOffset = offset;
881             chunkIndex = AbstractLeader.FIRST_CHUNK_INDEX;
882             lastChunkHashCode = AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE;
883         }
884
885         public int getLastChunkHashCode() {
886             return lastChunkHashCode;
887         }
888     }
889
890     // called from example-actor for printing the follower-states
891     public String printFollowerStates() {
892         final StringBuilder sb = new StringBuilder();
893
894         sb.append('[');
895         for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
896             sb.append('{');
897             sb.append(followerLogInformation.getId());
898             sb.append(" state:");
899             sb.append(followerLogInformation.isFollowerActive());
900             sb.append("},");
901         }
902         sb.append(']');
903
904         return sb.toString();
905     }
906
907     @VisibleForTesting
908     public FollowerLogInformation getFollower(String followerId) {
909         return followerToLog.get(followerId);
910     }
911
912     @VisibleForTesting
913     protected void setFollowerSnapshot(String followerId, FollowerToSnapshot snapshot) {
914         mapFollowerToSnapshot.put(followerId, snapshot);
915     }
916
917     @VisibleForTesting
918     public int followerSnapshotSize() {
919         return mapFollowerToSnapshot.size();
920     }
921
922     @VisibleForTesting
923     public int followerLogSize() {
924         return followerToLog.size();
925     }
926
927     private static class SnapshotHolder {
928         private final long lastIncludedTerm;
929         private final long lastIncludedIndex;
930         private final ByteString snapshotBytes;
931
932         SnapshotHolder(Snapshot snapshot) {
933             this.lastIncludedTerm = snapshot.getLastAppliedTerm();
934             this.lastIncludedIndex = snapshot.getLastAppliedIndex();
935             this.snapshotBytes = ByteString.copyFrom(snapshot.getState());
936         }
937
938         long getLastIncludedTerm() {
939             return lastIncludedTerm;
940         }
941
942         long getLastIncludedIndex() {
943             return lastIncludedIndex;
944         }
945
946         ByteString getSnapshotBytes() {
947             return snapshotBytes;
948         }
949     }
950 }