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