Add LeaderStateChanged notification
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / RaftActor.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;
10
11 import akka.actor.ActorRef;
12 import akka.actor.ActorSelection;
13 import akka.japi.Procedure;
14 import akka.persistence.RecoveryCompleted;
15 import akka.persistence.SaveSnapshotFailure;
16 import akka.persistence.SaveSnapshotSuccess;
17 import akka.persistence.SnapshotOffer;
18 import akka.persistence.SnapshotSelectionCriteria;
19 import com.google.common.annotations.VisibleForTesting;
20 import com.google.common.base.Objects;
21 import com.google.common.base.Optional;
22 import com.google.common.base.Stopwatch;
23 import com.google.common.collect.ImmutableMap;
24 import com.google.common.collect.Lists;
25 import com.google.protobuf.ByteString;
26 import java.io.Serializable;
27 import java.util.Collection;
28 import java.util.List;
29 import java.util.Map;
30 import java.util.concurrent.TimeUnit;
31 import org.apache.commons.lang3.time.DurationFormatUtils;
32 import org.opendaylight.controller.cluster.DataPersistenceProvider;
33 import org.opendaylight.controller.cluster.common.actor.AbstractUntypedPersistentActor;
34 import org.opendaylight.controller.cluster.notifications.LeaderStateChanged;
35 import org.opendaylight.controller.cluster.notifications.RoleChanged;
36 import org.opendaylight.controller.cluster.raft.base.messages.ApplyJournalEntries;
37 import org.opendaylight.controller.cluster.raft.base.messages.ApplyLogEntries;
38 import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
39 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
40 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshot;
41 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshotReply;
42 import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
43 import org.opendaylight.controller.cluster.raft.base.messages.SendInstallSnapshot;
44 import org.opendaylight.controller.cluster.raft.behaviors.AbstractLeader;
45 import org.opendaylight.controller.cluster.raft.behaviors.AbstractRaftActorBehavior;
46 import org.opendaylight.controller.cluster.raft.behaviors.Follower;
47 import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior;
48 import org.opendaylight.controller.cluster.raft.client.messages.FindLeader;
49 import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply;
50 import org.opendaylight.controller.cluster.raft.client.messages.FollowerInfo;
51 import org.opendaylight.controller.cluster.raft.client.messages.GetOnDemandRaftState;
52 import org.opendaylight.controller.cluster.raft.client.messages.OnDemandRaftState;
53 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
54 import org.slf4j.Logger;
55 import org.slf4j.LoggerFactory;
56
57 /**
58  * RaftActor encapsulates a state machine that needs to be kept synchronized
59  * in a cluster. It implements the RAFT algorithm as described in the paper
60  * <a href='https://ramcloud.stanford.edu/wiki/download/attachments/11370504/raft.pdf'>
61  * In Search of an Understandable Consensus Algorithm</a>
62  * <p/>
63  * RaftActor has 3 states and each state has a certain behavior associated
64  * with it. A Raft actor can behave as,
65  * <ul>
66  * <li> A Leader </li>
67  * <li> A Follower (or) </li>
68  * <li> A Candidate </li>
69  * </ul>
70  * <p/>
71  * <p/>
72  * A RaftActor MUST be a Leader in order to accept requests from clients to
73  * change the state of it's encapsulated state machine. Once a RaftActor becomes
74  * a Leader it is also responsible for ensuring that all followers ultimately
75  * have the same log and therefore the same state machine as itself.
76  * <p/>
77  * <p/>
78  * The current behavior of a RaftActor determines how election for leadership
79  * is initiated and how peer RaftActors react to request for votes.
80  * <p/>
81  * <p/>
82  * Each RaftActor also needs to know the current election term. It uses this
83  * information for a couple of things. One is to simply figure out who it
84  * voted for in the last election. Another is to figure out if the message
85  * it received to update it's state is stale.
86  * <p/>
87  * <p/>
88  * The RaftActor uses akka-persistence to store it's replicated log.
89  * Furthermore through it's behaviors a Raft Actor determines
90  * <p/>
91  * <ul>
92  * <li> when a log entry should be persisted </li>
93  * <li> when a log entry should be applied to the state machine (and) </li>
94  * <li> when a snapshot should be saved </li>
95  * </ul>
96  */
97 public abstract class RaftActor extends AbstractUntypedPersistentActor {
98
99     private static final long APPLY_STATE_DELAY_THRESHOLD_IN_NANOS = TimeUnit.MILLISECONDS.toNanos(50L); // 50 millis
100
101     private static final Procedure<ApplyJournalEntries> APPLY_JOURNAL_ENTRIES_PERSIST_CALLBACK =
102             new Procedure<ApplyJournalEntries>() {
103                 @Override
104                 public void apply(ApplyJournalEntries param) throws Exception {
105                 }
106             };
107
108     protected final Logger LOG = LoggerFactory.getLogger(getClass());
109
110     /**
111      * The current state determines the current behavior of a RaftActor
112      * A Raft Actor always starts off in the Follower State
113      */
114     private RaftActorBehavior currentBehavior;
115
116     /**
117      * This context should NOT be passed directly to any other actor it is
118      * only to be consumed by the RaftActorBehaviors
119      */
120     private final RaftActorContextImpl context;
121
122     /**
123      * The in-memory journal
124      */
125     private ReplicatedLogImpl replicatedLog = new ReplicatedLogImpl();
126
127     private CaptureSnapshot captureSnapshot = null;
128
129     private Stopwatch recoveryTimer;
130
131     private int currentRecoveryBatchCount;
132
133     private final BehaviorStateHolder reusableBehaviorStateHolder = new BehaviorStateHolder();
134
135     public RaftActor(String id, Map<String, String> peerAddresses) {
136         this(id, peerAddresses, Optional.<ConfigParams>absent());
137     }
138
139     public RaftActor(String id, Map<String, String> peerAddresses,
140          Optional<ConfigParams> configParams) {
141
142         context = new RaftActorContextImpl(this.getSelf(),
143             this.getContext(), id, new ElectionTermImpl(),
144             -1, -1, replicatedLog, peerAddresses,
145             (configParams.isPresent() ? configParams.get(): new DefaultConfigParamsImpl()),
146             LOG);
147     }
148
149     private void initRecoveryTimer() {
150         if(recoveryTimer == null) {
151             recoveryTimer = Stopwatch.createStarted();
152         }
153     }
154
155     @Override
156     public void preStart() throws Exception {
157         LOG.info("Starting recovery for {} with journal batch size {}", persistenceId(),
158                 context.getConfigParams().getJournalRecoveryLogBatchSize());
159
160         super.preStart();
161     }
162
163     @Override
164     public void postStop() {
165         if(currentBehavior != null) {
166             try {
167                 currentBehavior.close();
168             } catch (Exception e) {
169                 LOG.debug("{}: Error closing behavior {}", persistenceId(), currentBehavior.state());
170             }
171         }
172
173         super.postStop();
174     }
175
176     @Override
177     public void handleRecover(Object message) {
178         if(persistence().isRecoveryApplicable()) {
179             if (message instanceof SnapshotOffer) {
180                 onRecoveredSnapshot((SnapshotOffer) message);
181             } else if (message instanceof ReplicatedLogEntry) {
182                 onRecoveredJournalLogEntry((ReplicatedLogEntry) message);
183             } else if (message instanceof ApplyLogEntries) {
184                 // Handle this message for backwards compatibility with pre-Lithium versions.
185                 onRecoveredApplyLogEntries(((ApplyLogEntries) message).getToIndex());
186             } else if (message instanceof ApplyJournalEntries) {
187                 onRecoveredApplyLogEntries(((ApplyJournalEntries) message).getToIndex());
188             } else if (message instanceof DeleteEntries) {
189                 replicatedLog.removeFrom(((DeleteEntries) message).getFromIndex());
190             } else if (message instanceof UpdateElectionTerm) {
191                 context.getTermInformation().update(((UpdateElectionTerm) message).getCurrentTerm(),
192                         ((UpdateElectionTerm) message).getVotedFor());
193             } else if (message instanceof RecoveryCompleted) {
194                 onRecoveryCompletedMessage();
195             }
196         } else {
197             if (message instanceof RecoveryCompleted) {
198                 // Delete all the messages from the akka journal so that we do not end up with consistency issues
199                 // Note I am not using the dataPersistenceProvider and directly using the akka api here
200                 deleteMessages(lastSequenceNr());
201
202                 // Delete all the akka snapshots as they will not be needed
203                 deleteSnapshots(new SnapshotSelectionCriteria(scala.Long.MaxValue(), scala.Long.MaxValue()));
204
205                 onRecoveryComplete();
206
207                 initializeBehavior();
208             }
209         }
210     }
211
212     private void onRecoveredSnapshot(SnapshotOffer offer) {
213         if(LOG.isDebugEnabled()) {
214             LOG.debug("{}: SnapshotOffer called..", persistenceId());
215         }
216
217         initRecoveryTimer();
218
219         Snapshot snapshot = (Snapshot) offer.snapshot();
220
221         // Create a replicated log with the snapshot information
222         // The replicated log can be used later on to retrieve this snapshot
223         // when we need to install it on a peer
224         replicatedLog = new ReplicatedLogImpl(snapshot);
225
226         context.setReplicatedLog(replicatedLog);
227         context.setLastApplied(snapshot.getLastAppliedIndex());
228         context.setCommitIndex(snapshot.getLastAppliedIndex());
229
230         Stopwatch timer = Stopwatch.createStarted();
231
232         // Apply the snapshot to the actors state
233         applyRecoverySnapshot(snapshot.getState());
234
235         timer.stop();
236         LOG.info("Recovery snapshot applied for {} in {}: snapshotIndex={}, snapshotTerm={}, journal-size=" +
237                 replicatedLog.size(), persistenceId(), timer.toString(),
238                 replicatedLog.getSnapshotIndex(), replicatedLog.getSnapshotTerm());
239     }
240
241     private void onRecoveredJournalLogEntry(ReplicatedLogEntry logEntry) {
242         if(LOG.isDebugEnabled()) {
243             LOG.debug("{}: Received ReplicatedLogEntry for recovery: {}", persistenceId(), logEntry.getIndex());
244         }
245
246         replicatedLog.append(logEntry);
247     }
248
249     private void onRecoveredApplyLogEntries(long toIndex) {
250         if(LOG.isDebugEnabled()) {
251             LOG.debug("{}: Received ApplyLogEntries for recovery, applying to state: {} to {}",
252                     persistenceId(), context.getLastApplied() + 1, toIndex);
253         }
254
255         for (long i = context.getLastApplied() + 1; i <= toIndex; i++) {
256             batchRecoveredLogEntry(replicatedLog.get(i));
257         }
258
259         context.setLastApplied(toIndex);
260         context.setCommitIndex(toIndex);
261     }
262
263     private void batchRecoveredLogEntry(ReplicatedLogEntry logEntry) {
264         initRecoveryTimer();
265
266         int batchSize = context.getConfigParams().getJournalRecoveryLogBatchSize();
267         if(currentRecoveryBatchCount == 0) {
268             startLogRecoveryBatch(batchSize);
269         }
270
271         appendRecoveredLogEntry(logEntry.getData());
272
273         if(++currentRecoveryBatchCount >= batchSize) {
274             endCurrentLogRecoveryBatch();
275         }
276     }
277
278     private void endCurrentLogRecoveryBatch() {
279         applyCurrentLogRecoveryBatch();
280         currentRecoveryBatchCount = 0;
281     }
282
283     private void onRecoveryCompletedMessage() {
284         if(currentRecoveryBatchCount > 0) {
285             endCurrentLogRecoveryBatch();
286         }
287
288         onRecoveryComplete();
289
290         String recoveryTime = "";
291         if(recoveryTimer != null) {
292             recoveryTimer.stop();
293             recoveryTime = " in " + recoveryTimer.toString();
294             recoveryTimer = null;
295         }
296
297         LOG.info(
298             "Recovery completed" + recoveryTime + " - Switching actor to Follower - " +
299                 "Persistence Id =  " + persistenceId() +
300                 " Last index in log={}, snapshotIndex={}, snapshotTerm={}, " +
301                 "journal-size={}",
302             replicatedLog.lastIndex(), replicatedLog.getSnapshotIndex(),
303             replicatedLog.getSnapshotTerm(), replicatedLog.size());
304
305         initializeBehavior();
306     }
307
308     protected void initializeBehavior(){
309         changeCurrentBehavior(new Follower(context));
310     }
311
312     protected void changeCurrentBehavior(RaftActorBehavior newBehavior){
313         reusableBehaviorStateHolder.init(currentBehavior);
314         currentBehavior = newBehavior;
315         handleBehaviorChange(reusableBehaviorStateHolder, currentBehavior);
316     }
317
318     @Override public void handleCommand(Object message) {
319         if (message instanceof ApplyState){
320             ApplyState applyState = (ApplyState) message;
321
322             long elapsedTime = (System.nanoTime() - applyState.getStartTime());
323             if(elapsedTime >= APPLY_STATE_DELAY_THRESHOLD_IN_NANOS){
324                 LOG.warn("ApplyState took more time than expected. Elapsed Time = {} ms ApplyState = {}",
325                         TimeUnit.NANOSECONDS.toMillis(elapsedTime), applyState);
326             }
327
328             if(LOG.isDebugEnabled()) {
329                 LOG.debug("{}: Applying state for log index {} data {}",
330                     persistenceId(), applyState.getReplicatedLogEntry().getIndex(),
331                     applyState.getReplicatedLogEntry().getData());
332             }
333
334             applyState(applyState.getClientActor(), applyState.getIdentifier(),
335                 applyState.getReplicatedLogEntry().getData());
336
337         } else if (message instanceof ApplyJournalEntries){
338             ApplyJournalEntries applyEntries = (ApplyJournalEntries) message;
339             if(LOG.isDebugEnabled()) {
340                 LOG.debug("{}: Persisting ApplyLogEntries with index={}", persistenceId(), applyEntries.getToIndex());
341             }
342
343             persistence().persist(applyEntries, APPLY_JOURNAL_ENTRIES_PERSIST_CALLBACK);
344
345         } else if(message instanceof ApplySnapshot ) {
346             Snapshot snapshot = ((ApplySnapshot) message).getSnapshot();
347
348             if(LOG.isDebugEnabled()) {
349                 LOG.debug("{}: ApplySnapshot called on Follower Actor " +
350                         "snapshotIndex:{}, snapshotTerm:{}", persistenceId(), snapshot.getLastAppliedIndex(),
351                     snapshot.getLastAppliedTerm()
352                 );
353             }
354
355             applySnapshot(snapshot.getState());
356
357             //clears the followers log, sets the snapshot index to ensure adjusted-index works
358             replicatedLog = new ReplicatedLogImpl(snapshot);
359             context.setReplicatedLog(replicatedLog);
360             context.setLastApplied(snapshot.getLastAppliedIndex());
361
362         } else if (message instanceof FindLeader) {
363             getSender().tell(
364                 new FindLeaderReply(getLeaderAddress()),
365                 getSelf()
366             );
367
368         } else if (message instanceof SaveSnapshotSuccess) {
369             SaveSnapshotSuccess success = (SaveSnapshotSuccess) message;
370             LOG.info("{}: SaveSnapshotSuccess received for snapshot", persistenceId());
371
372             long sequenceNumber = success.metadata().sequenceNr();
373
374             commitSnapshot(sequenceNumber);
375
376         } else if (message instanceof SaveSnapshotFailure) {
377             SaveSnapshotFailure saveSnapshotFailure = (SaveSnapshotFailure) message;
378
379             LOG.error("{}: SaveSnapshotFailure received for snapshot Cause:",
380                     persistenceId(), saveSnapshotFailure.cause());
381
382             context.getReplicatedLog().snapshotRollback();
383
384             LOG.info("{}: Replicated Log rollbacked. Snapshot will be attempted in the next cycle." +
385                 "snapshotIndex:{}, snapshotTerm:{}, log-size:{}", persistenceId(),
386                 context.getReplicatedLog().getSnapshotIndex(),
387                 context.getReplicatedLog().getSnapshotTerm(),
388                 context.getReplicatedLog().size());
389
390         } else if (message instanceof CaptureSnapshot) {
391             LOG.debug("{}: CaptureSnapshot received by actor: {}", persistenceId(), message);
392
393             if(captureSnapshot == null) {
394                 captureSnapshot = (CaptureSnapshot)message;
395                 createSnapshot();
396             }
397
398         } else if (message instanceof CaptureSnapshotReply){
399             handleCaptureSnapshotReply(((CaptureSnapshotReply) message).getSnapshot());
400         } else if(message instanceof GetOnDemandRaftState) {
401             onGetOnDemandRaftStats();
402         } else {
403             reusableBehaviorStateHolder.init(currentBehavior);
404
405             currentBehavior = currentBehavior.handleMessage(getSender(), message);
406
407             handleBehaviorChange(reusableBehaviorStateHolder, currentBehavior);
408         }
409     }
410
411     private void onGetOnDemandRaftStats() {
412         // Debugging message to retrieve raft stats.
413
414         OnDemandRaftState.Builder builder = OnDemandRaftState.builder()
415                 .commitIndex(context.getCommitIndex())
416                 .currentTerm(context.getTermInformation().getCurrentTerm())
417                 .inMemoryJournalDataSize(replicatedLog.dataSize())
418                 .inMemoryJournalLogSize(replicatedLog.size())
419                 .isSnapshotCaptureInitiated(context.isSnapshotCaptureInitiated())
420                 .lastApplied(context.getLastApplied())
421                 .lastIndex(replicatedLog.lastIndex())
422                 .lastTerm(replicatedLog.lastTerm())
423                 .leader(getLeaderId())
424                 .raftState(currentBehavior.state().toString())
425                 .replicatedToAllIndex(currentBehavior.getReplicatedToAllIndex())
426                 .snapshotIndex(replicatedLog.getSnapshotIndex())
427                 .snapshotTerm(replicatedLog.getSnapshotTerm())
428                 .votedFor(context.getTermInformation().getVotedFor())
429                 .peerAddresses(ImmutableMap.copyOf(context.getPeerAddresses()));
430
431         ReplicatedLogEntry lastLogEntry = getLastLogEntry();
432         if (lastLogEntry != null) {
433             builder.lastLogIndex(lastLogEntry.getIndex());
434             builder.lastLogTerm(lastLogEntry.getTerm());
435         }
436
437         if(currentBehavior instanceof AbstractLeader) {
438             AbstractLeader leader = (AbstractLeader)currentBehavior;
439             Collection<String> followerIds = leader.getFollowerIds();
440             List<FollowerInfo> followerInfoList = Lists.newArrayListWithCapacity(followerIds.size());
441             for(String id: followerIds) {
442                 final FollowerLogInformation info = leader.getFollower(id);
443                 followerInfoList.add(new FollowerInfo(id, info.getNextIndex(), info.getMatchIndex(),
444                         info.isFollowerActive(), DurationFormatUtils.formatDurationHMS(info.timeSinceLastActivity())));
445             }
446
447             builder.followerInfoList(followerInfoList);
448         }
449
450         sender().tell(builder.build(), self());
451
452     }
453
454     private void handleBehaviorChange(BehaviorStateHolder oldBehaviorState, RaftActorBehavior currentBehavior) {
455         RaftActorBehavior oldBehavior = oldBehaviorState.getBehavior();
456
457         if (oldBehavior != currentBehavior){
458             onStateChanged();
459         }
460
461         String oldBehaviorLeaderId = oldBehavior == null ? null : oldBehaviorState.getLeaderId();
462         String oldBehaviorStateName = oldBehavior == null ? null : oldBehavior.state().name();
463
464         // it can happen that the state has not changed but the leader has changed.
465         Optional<ActorRef> roleChangeNotifier = getRoleChangeNotifier();
466         if(!Objects.equal(oldBehaviorLeaderId, currentBehavior.getLeaderId())) {
467             if(roleChangeNotifier.isPresent()) {
468                 roleChangeNotifier.get().tell(new LeaderStateChanged(getId(), currentBehavior.getLeaderId()), getSelf());
469             }
470
471             onLeaderChanged(oldBehaviorLeaderId, currentBehavior.getLeaderId());
472         }
473
474         if (roleChangeNotifier.isPresent() &&
475                 (oldBehavior == null || (oldBehavior.state() != currentBehavior.state()))) {
476             roleChangeNotifier.get().tell(new RoleChanged(getId(), oldBehaviorStateName ,
477                     currentBehavior.state().name()), getSelf());
478         }
479     }
480
481     /**
482      * When a derived RaftActor needs to persist something it must call
483      * persistData.
484      *
485      * @param clientActor
486      * @param identifier
487      * @param data
488      */
489     protected void persistData(final ActorRef clientActor, final String identifier,
490         final Payload data) {
491
492         ReplicatedLogEntry replicatedLogEntry = new ReplicatedLogImplEntry(
493             context.getReplicatedLog().lastIndex() + 1,
494             context.getTermInformation().getCurrentTerm(), data);
495
496         if(LOG.isDebugEnabled()) {
497             LOG.debug("{}: Persist data {}", persistenceId(), replicatedLogEntry);
498         }
499
500         final RaftActorContext raftContext = getRaftActorContext();
501
502         replicatedLog
503                 .appendAndPersist(replicatedLogEntry, new Procedure<ReplicatedLogEntry>() {
504                     @Override
505                     public void apply(ReplicatedLogEntry replicatedLogEntry) throws Exception {
506                         if(!hasFollowers()){
507                             // Increment the Commit Index and the Last Applied values
508                             raftContext.setCommitIndex(replicatedLogEntry.getIndex());
509                             raftContext.setLastApplied(replicatedLogEntry.getIndex());
510
511                             // Apply the state immediately
512                             applyState(clientActor, identifier, data);
513
514                             // Send a ApplyJournalEntries message so that we write the fact that we applied
515                             // the state to durable storage
516                             self().tell(new ApplyJournalEntries(replicatedLogEntry.getIndex()), self());
517
518                             // Check if the "real" snapshot capture has been initiated. If no then do the fake snapshot
519                             if(!context.isSnapshotCaptureInitiated()){
520                                 raftContext.getReplicatedLog().snapshotPreCommit(raftContext.getLastApplied(),
521                                         raftContext.getTermInformation().getCurrentTerm());
522                                 raftContext.getReplicatedLog().snapshotCommit();
523                             } else {
524                                 LOG.debug("{}: Skipping fake snapshotting for {} because real snapshotting is in progress",
525                                         persistenceId(), getId());
526                             }
527                         } else if (clientActor != null) {
528                             // Send message for replication
529                             currentBehavior.handleMessage(getSelf(),
530                                     new Replicate(clientActor, identifier,
531                                             replicatedLogEntry)
532                             );
533                         }
534
535                     }
536                 });    }
537
538     protected String getId() {
539         return context.getId();
540     }
541
542     /**
543      * Derived actors can call the isLeader method to check if the current
544      * RaftActor is the Leader or not
545      *
546      * @return true it this RaftActor is a Leader false otherwise
547      */
548     protected boolean isLeader() {
549         return context.getId().equals(currentBehavior.getLeaderId());
550     }
551
552     /**
553      * Derived actor can call getLeader if they need a reference to the Leader.
554      * This would be useful for example in forwarding a request to an actor
555      * which is the leader
556      *
557      * @return A reference to the leader if known, null otherwise
558      */
559     protected ActorSelection getLeader(){
560         String leaderAddress = getLeaderAddress();
561
562         if(leaderAddress == null){
563             return null;
564         }
565
566         return context.actorSelection(leaderAddress);
567     }
568
569     /**
570      *
571      * @return the current leader's id
572      */
573     protected String getLeaderId(){
574         return currentBehavior.getLeaderId();
575     }
576
577     protected RaftState getRaftState() {
578         return currentBehavior.state();
579     }
580
581     protected ReplicatedLogEntry getLastLogEntry() {
582         return replicatedLog.last();
583     }
584
585     protected Long getCurrentTerm(){
586         return context.getTermInformation().getCurrentTerm();
587     }
588
589     protected Long getCommitIndex(){
590         return context.getCommitIndex();
591     }
592
593     protected Long getLastApplied(){
594         return context.getLastApplied();
595     }
596
597     protected RaftActorContext getRaftActorContext() {
598         return context;
599     }
600
601     protected void updateConfigParams(ConfigParams configParams) {
602         context.setConfigParams(configParams);
603     }
604
605     /**
606      * setPeerAddress sets the address of a known peer at a later time.
607      * <p>
608      * This is to account for situations where a we know that a peer
609      * exists but we do not know an address up-front. This may also be used in
610      * situations where a known peer starts off in a different location and we
611      * need to change it's address
612      * <p>
613      * Note that if the peerId does not match the list of peers passed to
614      * this actor during construction an IllegalStateException will be thrown.
615      *
616      * @param peerId
617      * @param peerAddress
618      */
619     protected void setPeerAddress(String peerId, String peerAddress){
620         context.setPeerAddress(peerId, peerAddress);
621     }
622
623     protected void commitSnapshot(long sequenceNumber) {
624         context.getReplicatedLog().snapshotCommit();
625
626         // TODO: Not sure if we want to be this aggressive with trimming stuff
627         trimPersistentData(sequenceNumber);
628     }
629
630     /**
631      * The applyState method will be called by the RaftActor when some data
632      * needs to be applied to the actor's state
633      *
634      * @param clientActor A reference to the client who sent this message. This
635      *                    is the same reference that was passed to persistData
636      *                    by the derived actor. clientActor may be null when
637      *                    the RaftActor is behaving as a follower or during
638      *                    recovery.
639      * @param identifier  The identifier of the persisted data. This is also
640      *                    the same identifier that was passed to persistData by
641      *                    the derived actor. identifier may be null when
642      *                    the RaftActor is behaving as a follower or during
643      *                    recovery
644      * @param data        A piece of data that was persisted by the persistData call.
645      *                    This should NEVER be null.
646      */
647     protected abstract void applyState(ActorRef clientActor, String identifier,
648         Object data);
649
650     /**
651      * This method is called during recovery at the start of a batch of state entries. Derived
652      * classes should perform any initialization needed to start a batch.
653      */
654     protected abstract void startLogRecoveryBatch(int maxBatchSize);
655
656     /**
657      * This method is called during recovery to append state data to the current batch. This method
658      * is called 1 or more times after {@link #startLogRecoveryBatch}.
659      *
660      * @param data the state data
661      */
662     protected abstract void appendRecoveredLogEntry(Payload data);
663
664     /**
665      * This method is called during recovery to reconstruct the state of the actor.
666      *
667      * @param snapshotBytes A snapshot of the state of the actor
668      */
669     protected abstract void applyRecoverySnapshot(byte[] snapshotBytes);
670
671     /**
672      * This method is called during recovery at the end of a batch to apply the current batched
673      * log entries. This method is called after {@link #appendRecoveredLogEntry}.
674      */
675     protected abstract void applyCurrentLogRecoveryBatch();
676
677     /**
678      * This method is called when recovery is complete.
679      */
680     protected abstract void onRecoveryComplete();
681
682     /**
683      * This method will be called by the RaftActor when a snapshot needs to be
684      * created. The derived actor should respond with its current state.
685      * <p/>
686      * During recovery the state that is returned by the derived actor will
687      * be passed back to it by calling the applySnapshot  method
688      *
689      * @return The current state of the actor
690      */
691     protected abstract void createSnapshot();
692
693     /**
694      * This method can be called at any other point during normal
695      * operations when the derived actor is out of sync with it's peers
696      * and the only way to bring it in sync is by applying a snapshot
697      *
698      * @param snapshotBytes A snapshot of the state of the actor
699      */
700     protected abstract void applySnapshot(byte[] snapshotBytes);
701
702     /**
703      * This method will be called by the RaftActor when the state of the
704      * RaftActor changes. The derived actor can then use methods like
705      * isLeader or getLeader to do something useful
706      */
707     protected abstract void onStateChanged();
708
709     protected abstract DataPersistenceProvider persistence();
710
711     /**
712      * Notifier Actor for this RaftActor to notify when a role change happens
713      * @return ActorRef - ActorRef of the notifier or Optional.absent if none.
714      */
715     protected abstract Optional<ActorRef> getRoleChangeNotifier();
716
717     protected void onLeaderChanged(String oldLeader, String newLeader){};
718
719     private void trimPersistentData(long sequenceNumber) {
720         // Trim akka snapshots
721         // FIXME : Not sure how exactly the SnapshotSelectionCriteria is applied
722         // For now guessing that it is ANDed.
723         persistence().deleteSnapshots(new SnapshotSelectionCriteria(
724             sequenceNumber - context.getConfigParams().getSnapshotBatchCount(), 43200000));
725
726         // Trim akka journal
727         persistence().deleteMessages(sequenceNumber);
728     }
729
730     private String getLeaderAddress(){
731         if(isLeader()){
732             return getSelf().path().toString();
733         }
734         String leaderId = currentBehavior.getLeaderId();
735         if (leaderId == null) {
736             return null;
737         }
738         String peerAddress = context.getPeerAddress(leaderId);
739         if(LOG.isDebugEnabled()) {
740             LOG.debug("{}: getLeaderAddress leaderId = {} peerAddress = {}",
741                     persistenceId(), leaderId, peerAddress);
742         }
743
744         return peerAddress;
745     }
746
747     private void handleCaptureSnapshotReply(byte[] snapshotBytes) {
748         LOG.debug("{}: CaptureSnapshotReply received by actor: snapshot size {}", persistenceId(), snapshotBytes.length);
749
750         // create a snapshot object from the state provided and save it
751         // when snapshot is saved async, SaveSnapshotSuccess is raised.
752
753         Snapshot sn = Snapshot.create(snapshotBytes,
754             context.getReplicatedLog().getFrom(captureSnapshot.getLastAppliedIndex() + 1),
755             captureSnapshot.getLastIndex(), captureSnapshot.getLastTerm(),
756             captureSnapshot.getLastAppliedIndex(), captureSnapshot.getLastAppliedTerm());
757
758         persistence().saveSnapshot(sn);
759
760         LOG.info("{}: Persisting of snapshot done:{}", persistenceId(), sn.getLogMessage());
761
762         long dataThreshold = getTotalMemory() *
763                 getRaftActorContext().getConfigParams().getSnapshotDataThresholdPercentage() / 100;
764         if (context.getReplicatedLog().dataSize() > dataThreshold) {
765
766             if(LOG.isDebugEnabled()) {
767                 LOG.debug("{}: dataSize {} exceeds dataThreshold {} - doing snapshotPreCommit with index {}",
768                         persistenceId(), context.getReplicatedLog().dataSize(), dataThreshold,
769                         captureSnapshot.getLastAppliedIndex());
770             }
771
772             // if memory is less, clear the log based on lastApplied.
773             // this could/should only happen if one of the followers is down
774             // as normally we keep removing from the log when its replicated to all.
775             context.getReplicatedLog().snapshotPreCommit(captureSnapshot.getLastAppliedIndex(),
776                     captureSnapshot.getLastAppliedTerm());
777
778             // Don't reset replicatedToAllIndex to -1 as this may prevent us from trimming the log after an
779             // install snapshot to a follower.
780             if(captureSnapshot.getReplicatedToAllIndex() >= 0) {
781                 getCurrentBehavior().setReplicatedToAllIndex(captureSnapshot.getReplicatedToAllIndex());
782             }
783         } else if(captureSnapshot.getReplicatedToAllIndex() != -1){
784             // clear the log based on replicatedToAllIndex
785             context.getReplicatedLog().snapshotPreCommit(captureSnapshot.getReplicatedToAllIndex(),
786                     captureSnapshot.getReplicatedToAllTerm());
787
788             getCurrentBehavior().setReplicatedToAllIndex(captureSnapshot.getReplicatedToAllIndex());
789         } else {
790             // The replicatedToAllIndex was not found in the log
791             // This means that replicatedToAllIndex never moved beyond -1 or that it is already in the snapshot.
792             // In this scenario we may need to save the snapshot to the akka persistence
793             // snapshot for recovery but we do not need to do the replicated log trimming.
794             context.getReplicatedLog().snapshotPreCommit(replicatedLog.getSnapshotIndex(),
795                     replicatedLog.getSnapshotTerm());
796         }
797
798
799         LOG.info("{}: Removed in-memory snapshotted entries, adjusted snaphsotIndex: {} " +
800             "and term: {}", persistenceId(), replicatedLog.getSnapshotIndex(),
801             replicatedLog.getSnapshotTerm());
802
803         if (isLeader() && captureSnapshot.isInstallSnapshotInitiated()) {
804             // this would be call straight to the leader and won't initiate in serialization
805             currentBehavior.handleMessage(getSelf(), new SendInstallSnapshot(
806                     ByteString.copyFrom(snapshotBytes)));
807         }
808
809         captureSnapshot = null;
810         context.setSnapshotCaptureInitiated(false);
811     }
812
813     protected long getTotalMemory() {
814         return Runtime.getRuntime().totalMemory();
815     }
816
817     protected boolean hasFollowers(){
818         return getRaftActorContext().getPeerAddresses().keySet().size() > 0;
819     }
820
821     private class ReplicatedLogImpl extends AbstractReplicatedLogImpl {
822
823         private static final int DATA_SIZE_DIVIDER = 5;
824         private long dataSizeSinceLastSnapshot = 0;
825
826         public ReplicatedLogImpl(Snapshot snapshot) {
827             super(snapshot.getLastAppliedIndex(), snapshot.getLastAppliedTerm(),
828                 snapshot.getUnAppliedEntries());
829         }
830
831         public ReplicatedLogImpl() {
832             super();
833         }
834
835         @Override public void removeFromAndPersist(long logEntryIndex) {
836             int adjustedIndex = adjustedIndex(logEntryIndex);
837
838             if (adjustedIndex < 0) {
839                 return;
840             }
841
842             // FIXME: Maybe this should be done after the command is saved
843             journal.subList(adjustedIndex , journal.size()).clear();
844
845             persistence().persist(new DeleteEntries(adjustedIndex), new Procedure<DeleteEntries>() {
846
847                 @Override
848                 public void apply(DeleteEntries param)
849                         throws Exception {
850                     //FIXME : Doing nothing for now
851                     dataSize = 0;
852                     for (ReplicatedLogEntry entry : journal) {
853                         dataSize += entry.size();
854                     }
855                 }
856             });
857         }
858
859         @Override public void appendAndPersist(
860             final ReplicatedLogEntry replicatedLogEntry) {
861             appendAndPersist(replicatedLogEntry, null);
862         }
863
864         public void appendAndPersist(
865             final ReplicatedLogEntry replicatedLogEntry,
866             final Procedure<ReplicatedLogEntry> callback)  {
867
868             if(LOG.isDebugEnabled()) {
869                 LOG.debug("{}: Append log entry and persist {} ", persistenceId(), replicatedLogEntry);
870             }
871
872             // FIXME : By adding the replicated log entry to the in-memory journal we are not truly ensuring durability of the logs
873             journal.add(replicatedLogEntry);
874
875             // When persisting events with persist it is guaranteed that the
876             // persistent actor will not receive further commands between the
877             // persist call and the execution(s) of the associated event
878             // handler. This also holds for multiple persist calls in context
879             // of a single command.
880             persistence().persist(replicatedLogEntry,
881                 new Procedure<ReplicatedLogEntry>() {
882                     @Override
883                     public void apply(ReplicatedLogEntry evt) throws Exception {
884                         int logEntrySize = replicatedLogEntry.size();
885
886                         dataSize += logEntrySize;
887                         long dataSizeForCheck = dataSize;
888
889                         dataSizeSinceLastSnapshot += logEntrySize;
890                         long journalSize = lastIndex() + 1;
891
892                         if(!hasFollowers()) {
893                             // When we do not have followers we do not maintain an in-memory log
894                             // due to this the journalSize will never become anything close to the
895                             // snapshot batch count. In fact will mostly be 1.
896                             // Similarly since the journal's dataSize depends on the entries in the
897                             // journal the journal's dataSize will never reach a value close to the
898                             // memory threshold.
899                             // By maintaining the dataSize outside the journal we are tracking essentially
900                             // what we have written to the disk however since we no longer are in
901                             // need of doing a snapshot just for the sake of freeing up memory we adjust
902                             // the real size of data by the DATA_SIZE_DIVIDER so that we do not snapshot as often
903                             // as if we were maintaining a real snapshot
904                             dataSizeForCheck = dataSizeSinceLastSnapshot / DATA_SIZE_DIVIDER;
905                         }
906
907                         long dataThreshold = getTotalMemory() *
908                                 getRaftActorContext().getConfigParams().getSnapshotDataThresholdPercentage() / 100;
909
910                         // when a snaphsot is being taken, captureSnapshot != null
911                         if (!context.isSnapshotCaptureInitiated() &&
912                                 ( journalSize % context.getConfigParams().getSnapshotBatchCount() == 0 ||
913                                         dataSizeForCheck > dataThreshold)) {
914
915                             dataSizeSinceLastSnapshot = 0;
916
917                             LOG.info("{}: Initiating Snapshot Capture, journalSize = {}, dataSizeForCheck = {}," +
918                                 " dataThreshold = {}", persistenceId(), journalSize, dataSizeForCheck, dataThreshold);
919
920                             long lastAppliedIndex = -1;
921                             long lastAppliedTerm = -1;
922
923                             ReplicatedLogEntry lastAppliedEntry = get(context.getLastApplied());
924                             if (!hasFollowers()) {
925                                 lastAppliedIndex = replicatedLogEntry.getIndex();
926                                 lastAppliedTerm = replicatedLogEntry.getTerm();
927                             } else if (lastAppliedEntry != null) {
928                                 lastAppliedIndex = lastAppliedEntry.getIndex();
929                                 lastAppliedTerm = lastAppliedEntry.getTerm();
930                             }
931
932                             if(LOG.isDebugEnabled()) {
933                                 LOG.debug("{}: Snapshot Capture logSize: {}", persistenceId(), journal.size());
934                                 LOG.debug("{}: Snapshot Capture lastApplied:{} ",
935                                         persistenceId(), context.getLastApplied());
936                                 LOG.debug("{}: Snapshot Capture lastAppliedIndex:{}", persistenceId(),
937                                         lastAppliedIndex);
938                                 LOG.debug("{}: Snapshot Capture lastAppliedTerm:{}", persistenceId(),
939                                         lastAppliedTerm);
940                             }
941
942                             // send a CaptureSnapshot to self to make the expensive operation async.
943                             long replicatedToAllIndex = getCurrentBehavior().getReplicatedToAllIndex();
944                             ReplicatedLogEntry replicatedToAllEntry = context.getReplicatedLog().get(replicatedToAllIndex);
945                             getSelf().tell(new CaptureSnapshot(lastIndex(), lastTerm(), lastAppliedIndex, lastAppliedTerm,
946                                 (replicatedToAllEntry != null ? replicatedToAllEntry.getIndex() : -1),
947                                 (replicatedToAllEntry != null ? replicatedToAllEntry.getTerm() : -1)),
948                                 null);
949                             context.setSnapshotCaptureInitiated(true);
950                         }
951                         if (callback != null){
952                             callback.apply(replicatedLogEntry);
953                         }
954                     }
955                 }
956             );
957         }
958
959     }
960
961     static class DeleteEntries implements Serializable {
962         private static final long serialVersionUID = 1L;
963         private final int fromIndex;
964
965         public DeleteEntries(int fromIndex) {
966             this.fromIndex = fromIndex;
967         }
968
969         public int getFromIndex() {
970             return fromIndex;
971         }
972     }
973
974
975     private class ElectionTermImpl implements ElectionTerm {
976         /**
977          * Identifier of the actor whose election term information this is
978          */
979         private long currentTerm = 0;
980         private String votedFor = null;
981
982         @Override
983         public long getCurrentTerm() {
984             return currentTerm;
985         }
986
987         @Override
988         public String getVotedFor() {
989             return votedFor;
990         }
991
992         @Override public void update(long currentTerm, String votedFor) {
993             if(LOG.isDebugEnabled()) {
994                 LOG.debug("{}: Set currentTerm={}, votedFor={}", persistenceId(), currentTerm, votedFor);
995             }
996             this.currentTerm = currentTerm;
997             this.votedFor = votedFor;
998         }
999
1000         @Override
1001         public void updateAndPersist(long currentTerm, String votedFor){
1002             update(currentTerm, votedFor);
1003             // FIXME : Maybe first persist then update the state
1004             persistence().persist(new UpdateElectionTerm(this.currentTerm, this.votedFor), new Procedure<UpdateElectionTerm>(){
1005
1006                 @Override public void apply(UpdateElectionTerm param)
1007                     throws Exception {
1008
1009                 }
1010             });
1011         }
1012     }
1013
1014     static class UpdateElectionTerm implements Serializable {
1015         private static final long serialVersionUID = 1L;
1016         private final long currentTerm;
1017         private final String votedFor;
1018
1019         public UpdateElectionTerm(long currentTerm, String votedFor) {
1020             this.currentTerm = currentTerm;
1021             this.votedFor = votedFor;
1022         }
1023
1024         public long getCurrentTerm() {
1025             return currentTerm;
1026         }
1027
1028         public String getVotedFor() {
1029             return votedFor;
1030         }
1031     }
1032
1033     protected class NonPersistentRaftDataProvider extends NonPersistentDataProvider {
1034
1035         public NonPersistentRaftDataProvider(){
1036
1037         }
1038
1039         /**
1040          * The way snapshotting works is,
1041          * <ol>
1042          * <li> RaftActor calls createSnapshot on the Shard
1043          * <li> Shard sends a CaptureSnapshotReply and RaftActor then calls saveSnapshot
1044          * <li> When saveSnapshot is invoked on the akka-persistence API it uses the SnapshotStore to save the snapshot.
1045          * The SnapshotStore sends SaveSnapshotSuccess or SaveSnapshotFailure. When the RaftActor gets SaveSnapshot
1046          * success it commits the snapshot to the in-memory journal. This commitSnapshot is mimicking what is done
1047          * in SaveSnapshotSuccess.
1048          * </ol>
1049          * @param o
1050          */
1051         @Override
1052         public void saveSnapshot(Object o) {
1053             // Make saving Snapshot successful
1054             commitSnapshot(-1L);
1055         }
1056     }
1057
1058     @VisibleForTesting
1059     void setCurrentBehavior(AbstractRaftActorBehavior behavior) {
1060         currentBehavior = behavior;
1061     }
1062
1063     protected RaftActorBehavior getCurrentBehavior() {
1064         return currentBehavior;
1065     }
1066
1067     private static class BehaviorStateHolder {
1068         private RaftActorBehavior behavior;
1069         private String leaderId;
1070
1071         void init(RaftActorBehavior behavior) {
1072             this.behavior = behavior;
1073             this.leaderId = behavior != null ? behavior.getLeaderId() : null;
1074         }
1075
1076         RaftActorBehavior getBehavior() {
1077             return behavior;
1078         }
1079
1080         String getLeaderId() {
1081             return leaderId;
1082         }
1083     }
1084 }