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