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