BUG 2134 : Make persistence configurable at the datastore level
[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.base.Optional;
22 import com.google.common.base.Stopwatch;
23 import com.google.protobuf.ByteString;
24 import org.opendaylight.controller.cluster.DataPersistenceProvider;
25 import org.opendaylight.controller.cluster.common.actor.AbstractUntypedPersistentActor;
26 import org.opendaylight.controller.cluster.raft.base.messages.ApplyLogEntries;
27 import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
28 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
29 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshot;
30 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshotReply;
31 import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
32 import org.opendaylight.controller.cluster.raft.base.messages.SendHeartBeat;
33 import org.opendaylight.controller.cluster.raft.behaviors.Follower;
34 import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior;
35 import org.opendaylight.controller.cluster.raft.client.messages.AddRaftPeer;
36 import org.opendaylight.controller.cluster.raft.client.messages.FindLeader;
37 import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply;
38 import org.opendaylight.controller.cluster.raft.client.messages.RemoveRaftPeer;
39 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
40 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
41 import org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages;
42
43 import java.io.Serializable;
44 import java.util.Map;
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     protected final LoggingAdapter LOG =
88         Logging.getLogger(getContext().system(), this);
89
90     /**
91      * The current state determines the current behavior of a RaftActor
92      * A Raft Actor always starts off in the Follower State
93      */
94     private RaftActorBehavior currentBehavior;
95
96     /**
97      * This context should NOT be passed directly to any other actor it is
98      * only to be consumed by the RaftActorBehaviors
99      */
100     private final RaftActorContext context;
101
102     /**
103      * The in-memory journal
104      */
105     private ReplicatedLogImpl replicatedLog = new ReplicatedLogImpl();
106
107     private CaptureSnapshot captureSnapshot = null;
108
109     private volatile boolean hasSnapshotCaptureInitiated = false;
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 = new Stopwatch();
132             recoveryTimer.start();
133         }
134     }
135
136     @Override
137     public void preStart() throws Exception {
138         LOG.info("Starting recovery for {} with journal batch size {}", persistenceId(),
139                 context.getConfigParams().getJournalRecoveryLogBatchSize());
140
141         super.preStart();
142     }
143
144     @Override
145     public void handleRecover(Object message) {
146         if(persistence().isRecoveryApplicable()) {
147             if (message instanceof SnapshotOffer) {
148                 onRecoveredSnapshot((SnapshotOffer) message);
149             } else if (message instanceof ReplicatedLogEntry) {
150                 onRecoveredJournalLogEntry((ReplicatedLogEntry) message);
151             } else if (message instanceof ApplyLogEntries) {
152                 onRecoveredApplyLogEntries((ApplyLogEntries) message);
153             } else if (message instanceof DeleteEntries) {
154                 replicatedLog.removeFrom(((DeleteEntries) message).getFromIndex());
155             } else if (message instanceof UpdateElectionTerm) {
156                 context.getTermInformation().update(((UpdateElectionTerm) message).getCurrentTerm(),
157                         ((UpdateElectionTerm) message).getVotedFor());
158             } else if (message instanceof RecoveryCompleted) {
159                 onRecoveryCompletedMessage();
160             }
161         } else {
162             if (message instanceof RecoveryCompleted) {
163                 // Delete all the messages from the akka journal so that we do not end up with consistency issues
164                 // Note I am not using the dataPersistenceProvider and directly using the akka api here
165                 deleteMessages(lastSequenceNr());
166
167                 // Delete all the akka snapshots as they will not be needed
168                 deleteSnapshots(new SnapshotSelectionCriteria(scala.Long.MaxValue(), scala.Long.MaxValue()));
169
170                 onRecoveryComplete();
171                 currentBehavior = new Follower(context);
172                 onStateChanged();
173             }
174         }
175     }
176
177     private void onRecoveredSnapshot(SnapshotOffer offer) {
178         if(LOG.isDebugEnabled()) {
179             LOG.debug("SnapshotOffer called..");
180         }
181
182         initRecoveryTimer();
183
184         Snapshot snapshot = (Snapshot) offer.snapshot();
185
186         // Create a replicated log with the snapshot information
187         // The replicated log can be used later on to retrieve this snapshot
188         // when we need to install it on a peer
189         replicatedLog = new ReplicatedLogImpl(snapshot);
190
191         context.setReplicatedLog(replicatedLog);
192         context.setLastApplied(snapshot.getLastAppliedIndex());
193         context.setCommitIndex(snapshot.getLastAppliedIndex());
194
195         Stopwatch timer = new Stopwatch();
196         timer.start();
197
198         // Apply the snapshot to the actors state
199         applyRecoverySnapshot(ByteString.copyFrom(snapshot.getState()));
200
201         timer.stop();
202         LOG.info("Recovery snapshot applied for {} in {}: snapshotIndex={}, snapshotTerm={}, journal-size=" +
203                 replicatedLog.size(), persistenceId(), timer.toString(),
204                 replicatedLog.snapshotIndex, replicatedLog.snapshotTerm);
205     }
206
207     private void onRecoveredJournalLogEntry(ReplicatedLogEntry logEntry) {
208         if(LOG.isDebugEnabled()) {
209             LOG.debug("Received ReplicatedLogEntry for recovery: {}", logEntry.getIndex());
210         }
211
212         replicatedLog.append(logEntry);
213     }
214
215     private void onRecoveredApplyLogEntries(ApplyLogEntries ale) {
216         if(LOG.isDebugEnabled()) {
217             LOG.debug("Received ApplyLogEntries for recovery, applying to state: {} to {}",
218                     context.getLastApplied() + 1, ale.getToIndex());
219         }
220
221         for (long i = context.getLastApplied() + 1; i <= ale.getToIndex(); i++) {
222             batchRecoveredLogEntry(replicatedLog.get(i));
223         }
224
225         context.setLastApplied(ale.getToIndex());
226         context.setCommitIndex(ale.getToIndex());
227     }
228
229     private void batchRecoveredLogEntry(ReplicatedLogEntry logEntry) {
230         initRecoveryTimer();
231
232         int batchSize = context.getConfigParams().getJournalRecoveryLogBatchSize();
233         if(currentRecoveryBatchCount == 0) {
234             startLogRecoveryBatch(batchSize);
235         }
236
237         appendRecoveredLogEntry(logEntry.getData());
238
239         if(++currentRecoveryBatchCount >= batchSize) {
240             endCurrentLogRecoveryBatch();
241         }
242     }
243
244     private void endCurrentLogRecoveryBatch() {
245         applyCurrentLogRecoveryBatch();
246         currentRecoveryBatchCount = 0;
247     }
248
249     private void onRecoveryCompletedMessage() {
250         if(currentRecoveryBatchCount > 0) {
251             endCurrentLogRecoveryBatch();
252         }
253
254         onRecoveryComplete();
255
256         String recoveryTime = "";
257         if(recoveryTimer != null) {
258             recoveryTimer.stop();
259             recoveryTime = " in " + recoveryTimer.toString();
260             recoveryTimer = null;
261         }
262
263         LOG.info(
264             "Recovery completed" + recoveryTime + " - Switching actor to Follower - " +
265                 "Persistence Id =  " + persistenceId() +
266                 " Last index in log={}, snapshotIndex={}, snapshotTerm={}, " +
267                 "journal-size={}",
268             replicatedLog.lastIndex(), replicatedLog.snapshotIndex,
269             replicatedLog.snapshotTerm, replicatedLog.size());
270
271         currentBehavior = new Follower(context);
272         onStateChanged();
273     }
274
275     @Override public void handleCommand(Object message) {
276         if (message instanceof ApplyState){
277             ApplyState applyState = (ApplyState) message;
278
279             if(LOG.isDebugEnabled()) {
280                 LOG.debug("Applying state for log index {} data {}",
281                     applyState.getReplicatedLogEntry().getIndex(),
282                     applyState.getReplicatedLogEntry().getData());
283             }
284
285             applyState(applyState.getClientActor(), applyState.getIdentifier(),
286                 applyState.getReplicatedLogEntry().getData());
287
288         } else if (message instanceof ApplyLogEntries){
289             ApplyLogEntries ale = (ApplyLogEntries) message;
290             if(LOG.isDebugEnabled()) {
291                 LOG.debug("Persisting ApplyLogEntries with index={}", ale.getToIndex());
292             }
293             persistence().persist(new ApplyLogEntries(ale.getToIndex()), new Procedure<ApplyLogEntries>() {
294                 @Override
295                 public void apply(ApplyLogEntries param) throws Exception {
296                 }
297             });
298
299         } else if(message instanceof ApplySnapshot ) {
300             Snapshot snapshot = ((ApplySnapshot) message).getSnapshot();
301
302             if(LOG.isDebugEnabled()) {
303                 LOG.debug("ApplySnapshot called on Follower Actor " +
304                         "snapshotIndex:{}, snapshotTerm:{}", snapshot.getLastAppliedIndex(),
305                     snapshot.getLastAppliedTerm()
306                 );
307             }
308             applySnapshot(ByteString.copyFrom(snapshot.getState()));
309
310             //clears the followers log, sets the snapshot index to ensure adjusted-index works
311             replicatedLog = new ReplicatedLogImpl(snapshot);
312             context.setReplicatedLog(replicatedLog);
313             context.setLastApplied(snapshot.getLastAppliedIndex());
314
315         } else if (message instanceof FindLeader) {
316             getSender().tell(
317                 new FindLeaderReply(getLeaderAddress()),
318                 getSelf()
319             );
320
321         } else if (message instanceof SaveSnapshotSuccess) {
322             SaveSnapshotSuccess success = (SaveSnapshotSuccess) message;
323             LOG.info("SaveSnapshotSuccess received for snapshot");
324
325             long sequenceNumber = success.metadata().sequenceNr();
326
327             commitSnapshot(sequenceNumber);
328
329         } else if (message instanceof SaveSnapshotFailure) {
330             SaveSnapshotFailure saveSnapshotFailure = (SaveSnapshotFailure) message;
331
332             LOG.info("saveSnapshotFailure.metadata():{}", saveSnapshotFailure.metadata().toString());
333             LOG.error(saveSnapshotFailure.cause(), "SaveSnapshotFailure received for snapshot Cause:");
334
335             context.getReplicatedLog().snapshotRollback();
336
337             LOG.info("Replicated Log rollbacked. Snapshot will be attempted in the next cycle." +
338                 "snapshotIndex:{}, snapshotTerm:{}, log-size:{}",
339                 context.getReplicatedLog().getSnapshotIndex(),
340                 context.getReplicatedLog().getSnapshotTerm(),
341                 context.getReplicatedLog().size());
342
343         } else if (message instanceof AddRaftPeer){
344
345             // FIXME : Do not add raft peers like this.
346             // When adding a new Peer we have to ensure that the a majority of
347             // the peers know about the new Peer. Doing it this way may cause
348             // a situation where multiple Leaders may emerge
349             AddRaftPeer arp = (AddRaftPeer)message;
350            context.addToPeers(arp.getName(), arp.getAddress());
351
352         } else if (message instanceof RemoveRaftPeer){
353
354             RemoveRaftPeer rrp = (RemoveRaftPeer)message;
355             context.removePeer(rrp.getName());
356
357         } else if (message instanceof CaptureSnapshot) {
358             LOG.info("CaptureSnapshot received by actor");
359             CaptureSnapshot cs = (CaptureSnapshot)message;
360             captureSnapshot = cs;
361             createSnapshot();
362
363         } else if (message instanceof CaptureSnapshotReply){
364             LOG.info("CaptureSnapshotReply received by actor");
365             CaptureSnapshotReply csr = (CaptureSnapshotReply) message;
366
367             ByteString stateInBytes = csr.getSnapshot();
368             LOG.info("CaptureSnapshotReply stateInBytes size:{}", stateInBytes.size());
369             handleCaptureSnapshotReply(stateInBytes);
370
371         } else {
372             if (!(message instanceof AppendEntriesMessages.AppendEntries)
373                 && !(message instanceof AppendEntriesReply) && !(message instanceof SendHeartBeat)) {
374                 if(LOG.isDebugEnabled()) {
375                     LOG.debug("onReceiveCommand: message: {}", message.getClass());
376                 }
377             }
378
379             RaftActorBehavior oldBehavior = currentBehavior;
380             currentBehavior = currentBehavior.handleMessage(getSender(), message);
381
382             if(oldBehavior != currentBehavior){
383                 onStateChanged();
384             }
385
386             onLeaderChanged(oldBehavior.getLeaderId(), currentBehavior.getLeaderId());
387         }
388     }
389
390     public java.util.Set<String> getPeers() {
391         return context.getPeerAddresses().keySet();
392     }
393
394     protected String getReplicatedLogState() {
395         return "snapshotIndex=" + context.getReplicatedLog().getSnapshotIndex()
396             + ", snapshotTerm=" + context.getReplicatedLog().getSnapshotTerm()
397             + ", im-mem journal size=" + context.getReplicatedLog().size();
398     }
399
400
401     /**
402      * When a derived RaftActor needs to persist something it must call
403      * persistData.
404      *
405      * @param clientActor
406      * @param identifier
407      * @param data
408      */
409     protected void persistData(ActorRef clientActor, String identifier,
410         Payload data) {
411
412         ReplicatedLogEntry replicatedLogEntry = new ReplicatedLogImplEntry(
413             context.getReplicatedLog().lastIndex() + 1,
414             context.getTermInformation().getCurrentTerm(), data);
415
416         if(LOG.isDebugEnabled()) {
417             LOG.debug("Persist data {}", replicatedLogEntry);
418         }
419
420         replicatedLog
421             .appendAndPersist(clientActor, identifier, replicatedLogEntry);
422     }
423
424     protected String getId() {
425         return context.getId();
426     }
427
428     /**
429      * Derived actors can call the isLeader method to check if the current
430      * RaftActor is the Leader or not
431      *
432      * @return true it this RaftActor is a Leader false otherwise
433      */
434     protected boolean isLeader() {
435         return context.getId().equals(currentBehavior.getLeaderId());
436     }
437
438     /**
439      * Derived actor can call getLeader if they need a reference to the Leader.
440      * This would be useful for example in forwarding a request to an actor
441      * which is the leader
442      *
443      * @return A reference to the leader if known, null otherwise
444      */
445     protected ActorSelection getLeader(){
446         String leaderAddress = getLeaderAddress();
447
448         if(leaderAddress == null){
449             return null;
450         }
451
452         return context.actorSelection(leaderAddress);
453     }
454
455     /**
456      *
457      * @return the current leader's id
458      */
459     protected String getLeaderId(){
460         return currentBehavior.getLeaderId();
461     }
462
463     protected RaftState getRaftState() {
464         return currentBehavior.state();
465     }
466
467     protected ReplicatedLogEntry getLastLogEntry() {
468         return replicatedLog.last();
469     }
470
471     protected Long getCurrentTerm(){
472         return context.getTermInformation().getCurrentTerm();
473     }
474
475     protected Long getCommitIndex(){
476         return context.getCommitIndex();
477     }
478
479     protected Long getLastApplied(){
480         return context.getLastApplied();
481     }
482
483     protected RaftActorContext getRaftActorContext() {
484         return context;
485     }
486
487     /**
488      * setPeerAddress sets the address of a known peer at a later time.
489      * <p>
490      * This is to account for situations where a we know that a peer
491      * exists but we do not know an address up-front. This may also be used in
492      * situations where a known peer starts off in a different location and we
493      * need to change it's address
494      * <p>
495      * Note that if the peerId does not match the list of peers passed to
496      * this actor during construction an IllegalStateException will be thrown.
497      *
498      * @param peerId
499      * @param peerAddress
500      */
501     protected void setPeerAddress(String peerId, String peerAddress){
502         context.setPeerAddress(peerId, peerAddress);
503     }
504
505     protected void commitSnapshot(long sequenceNumber) {
506         context.getReplicatedLog().snapshotCommit();
507
508         // TODO: Not sure if we want to be this aggressive with trimming stuff
509         trimPersistentData(sequenceNumber);
510     }
511
512     /**
513      * The applyState method will be called by the RaftActor when some data
514      * needs to be applied to the actor's state
515      *
516      * @param clientActor A reference to the client who sent this message. This
517      *                    is the same reference that was passed to persistData
518      *                    by the derived actor. clientActor may be null when
519      *                    the RaftActor is behaving as a follower or during
520      *                    recovery.
521      * @param identifier  The identifier of the persisted data. This is also
522      *                    the same identifier that was passed to persistData by
523      *                    the derived actor. identifier may be null when
524      *                    the RaftActor is behaving as a follower or during
525      *                    recovery
526      * @param data        A piece of data that was persisted by the persistData call.
527      *                    This should NEVER be null.
528      */
529     protected abstract void applyState(ActorRef clientActor, String identifier,
530         Object data);
531
532     /**
533      * This method is called during recovery at the start of a batch of state entries. Derived
534      * classes should perform any initialization needed to start a batch.
535      */
536     protected abstract void startLogRecoveryBatch(int maxBatchSize);
537
538     /**
539      * This method is called during recovery to append state data to the current batch. This method
540      * is called 1 or more times after {@link #startLogRecoveryBatch}.
541      *
542      * @param data the state data
543      */
544     protected abstract void appendRecoveredLogEntry(Payload data);
545
546     /**
547      * This method is called during recovery to reconstruct the state of the actor.
548      *
549      * @param snapshot A snapshot of the state of the actor
550      */
551     protected abstract void applyRecoverySnapshot(ByteString snapshot);
552
553     /**
554      * This method is called during recovery at the end of a batch to apply the current batched
555      * log entries. This method is called after {@link #appendRecoveredLogEntry}.
556      */
557     protected abstract void applyCurrentLogRecoveryBatch();
558
559     /**
560      * This method is called when recovery is complete.
561      */
562     protected abstract void onRecoveryComplete();
563
564     /**
565      * This method will be called by the RaftActor when a snapshot needs to be
566      * created. The derived actor should respond with its current state.
567      * <p/>
568      * During recovery the state that is returned by the derived actor will
569      * be passed back to it by calling the applySnapshot  method
570      *
571      * @return The current state of the actor
572      */
573     protected abstract void createSnapshot();
574
575     /**
576      * This method can be called at any other point during normal
577      * operations when the derived actor is out of sync with it's peers
578      * and the only way to bring it in sync is by applying a snapshot
579      *
580      * @param snapshot A snapshot of the state of the actor
581      */
582     protected abstract void applySnapshot(ByteString snapshot);
583
584     /**
585      * This method will be called by the RaftActor when the state of the
586      * RaftActor changes. The derived actor can then use methods like
587      * isLeader or getLeader to do something useful
588      */
589     protected abstract void onStateChanged();
590
591     protected abstract DataPersistenceProvider persistence();
592
593     protected void onLeaderChanged(String oldLeader, String newLeader){};
594
595     private void trimPersistentData(long sequenceNumber) {
596         // Trim akka snapshots
597         // FIXME : Not sure how exactly the SnapshotSelectionCriteria is applied
598         // For now guessing that it is ANDed.
599         persistence().deleteSnapshots(new SnapshotSelectionCriteria(
600             sequenceNumber - context.getConfigParams().getSnapshotBatchCount(), 43200000));
601
602         // Trim akka journal
603         persistence().deleteMessages(sequenceNumber);
604     }
605
606     private String getLeaderAddress(){
607         if(isLeader()){
608             return getSelf().path().toString();
609         }
610         String leaderId = currentBehavior.getLeaderId();
611         if (leaderId == null) {
612             return null;
613         }
614         String peerAddress = context.getPeerAddress(leaderId);
615         if(LOG.isDebugEnabled()) {
616             LOG.debug("getLeaderAddress leaderId = {} peerAddress = {}",
617                     leaderId, peerAddress);
618         }
619
620         return peerAddress;
621     }
622
623     private void handleCaptureSnapshotReply(ByteString stateInBytes) {
624         // create a snapshot object from the state provided and save it
625         // when snapshot is saved async, SaveSnapshotSuccess is raised.
626
627         Snapshot sn = Snapshot.create(stateInBytes.toByteArray(),
628             context.getReplicatedLog().getFrom(captureSnapshot.getLastAppliedIndex() + 1),
629             captureSnapshot.getLastIndex(), captureSnapshot.getLastTerm(),
630             captureSnapshot.getLastAppliedIndex(), captureSnapshot.getLastAppliedTerm());
631
632         persistence().saveSnapshot(sn);
633
634         LOG.info("Persisting of snapshot done:{}", sn.getLogMessage());
635
636         //be greedy and remove entries from in-mem journal which are in the snapshot
637         // and update snapshotIndex and snapshotTerm without waiting for the success,
638
639         context.getReplicatedLog().snapshotPreCommit(stateInBytes,
640             captureSnapshot.getLastAppliedIndex(),
641             captureSnapshot.getLastAppliedTerm());
642
643         LOG.info("Removed in-memory snapshotted entries, adjusted snaphsotIndex:{} " +
644             "and term:{}", captureSnapshot.getLastAppliedIndex(),
645             captureSnapshot.getLastAppliedTerm());
646
647         captureSnapshot = null;
648         hasSnapshotCaptureInitiated = false;
649     }
650
651
652     private class ReplicatedLogImpl extends AbstractReplicatedLogImpl {
653
654         public ReplicatedLogImpl(Snapshot snapshot) {
655             super(ByteString.copyFrom(snapshot.getState()),
656                 snapshot.getLastAppliedIndex(), snapshot.getLastAppliedTerm(),
657                 snapshot.getUnAppliedEntries());
658         }
659
660         public ReplicatedLogImpl() {
661             super();
662         }
663
664         @Override public void removeFromAndPersist(long logEntryIndex) {
665             int adjustedIndex = adjustedIndex(logEntryIndex);
666
667             if (adjustedIndex < 0) {
668                 return;
669             }
670
671             // FIXME: Maybe this should be done after the command is saved
672             journal.subList(adjustedIndex , journal.size()).clear();
673
674             persistence().persist(new DeleteEntries(adjustedIndex), new Procedure<DeleteEntries>(){
675
676                 @Override public void apply(DeleteEntries param)
677                     throws Exception {
678                     //FIXME : Doing nothing for now
679                 }
680             });
681         }
682
683         @Override public void appendAndPersist(
684             final ReplicatedLogEntry replicatedLogEntry) {
685             appendAndPersist(null, null, replicatedLogEntry);
686         }
687
688         public void appendAndPersist(final ActorRef clientActor,
689             final String identifier,
690             final ReplicatedLogEntry replicatedLogEntry) {
691
692             if(LOG.isDebugEnabled()) {
693                 LOG.debug("Append log entry and persist {} ", replicatedLogEntry);
694             }
695
696             // FIXME : By adding the replicated log entry to the in-memory journal we are not truly ensuring durability of the logs
697             journal.add(replicatedLogEntry);
698
699             // When persisting events with persist it is guaranteed that the
700             // persistent actor will not receive further commands between the
701             // persist call and the execution(s) of the associated event
702             // handler. This also holds for multiple persist calls in context
703             // of a single command.
704             persistence().persist(replicatedLogEntry,
705                 new Procedure<ReplicatedLogEntry>() {
706                     @Override
707                     public void apply(ReplicatedLogEntry evt) throws Exception {
708                         // when a snaphsot is being taken, captureSnapshot != null
709                         if (hasSnapshotCaptureInitiated == false &&
710                             journal.size() % context.getConfigParams().getSnapshotBatchCount() == 0) {
711
712                             LOG.info("Initiating Snapshot Capture..");
713                             long lastAppliedIndex = -1;
714                             long lastAppliedTerm = -1;
715
716                             ReplicatedLogEntry lastAppliedEntry = get(context.getLastApplied());
717                             if (lastAppliedEntry != null) {
718                                 lastAppliedIndex = lastAppliedEntry.getIndex();
719                                 lastAppliedTerm = lastAppliedEntry.getTerm();
720                             }
721
722                             if(LOG.isDebugEnabled()) {
723                                 LOG.debug("Snapshot Capture logSize: {}", journal.size());
724                                 LOG.debug("Snapshot Capture lastApplied:{} ",
725                                     context.getLastApplied());
726                                 LOG.debug("Snapshot Capture lastAppliedIndex:{}", lastAppliedIndex);
727                                 LOG.debug("Snapshot Capture lastAppliedTerm:{}", lastAppliedTerm);
728                             }
729
730                             // send a CaptureSnapshot to self to make the expensive operation async.
731                             getSelf().tell(new CaptureSnapshot(
732                                 lastIndex(), lastTerm(), lastAppliedIndex, lastAppliedTerm),
733                                 null);
734                             hasSnapshotCaptureInitiated = true;
735                         }
736                         // Send message for replication
737                         if (clientActor != null) {
738                             currentBehavior.handleMessage(getSelf(),
739                                 new Replicate(clientActor, identifier,
740                                     replicatedLogEntry)
741                             );
742                         }
743                     }
744                 }
745             );
746         }
747
748     }
749
750     static class DeleteEntries implements Serializable {
751         private final int fromIndex;
752
753
754         public DeleteEntries(int fromIndex) {
755             this.fromIndex = fromIndex;
756         }
757
758         public int getFromIndex() {
759             return fromIndex;
760         }
761     }
762
763
764     private class ElectionTermImpl implements ElectionTerm {
765         /**
766          * Identifier of the actor whose election term information this is
767          */
768         private long currentTerm = 0;
769         private String votedFor = null;
770
771         @Override
772         public long getCurrentTerm() {
773             return currentTerm;
774         }
775
776         @Override
777         public String getVotedFor() {
778             return votedFor;
779         }
780
781         @Override public void update(long currentTerm, String votedFor) {
782             if(LOG.isDebugEnabled()) {
783                 LOG.debug("Set currentTerm={}, votedFor={}", currentTerm, votedFor);
784             }
785             this.currentTerm = currentTerm;
786             this.votedFor = votedFor;
787         }
788
789         @Override
790         public void updateAndPersist(long currentTerm, String votedFor){
791             update(currentTerm, votedFor);
792             // FIXME : Maybe first persist then update the state
793             persistence().persist(new UpdateElectionTerm(this.currentTerm, this.votedFor), new Procedure<UpdateElectionTerm>(){
794
795                 @Override public void apply(UpdateElectionTerm param)
796                     throws Exception {
797
798                 }
799             });
800         }
801     }
802
803     static class UpdateElectionTerm implements Serializable {
804         private final long currentTerm;
805         private final String votedFor;
806
807         public UpdateElectionTerm(long currentTerm, String votedFor) {
808             this.currentTerm = currentTerm;
809             this.votedFor = votedFor;
810         }
811
812         public long getCurrentTerm() {
813             return currentTerm;
814         }
815
816         public String getVotedFor() {
817             return votedFor;
818         }
819     }
820
821     protected class NonPersistentRaftDataProvider extends NonPersistentDataProvider {
822
823         public NonPersistentRaftDataProvider(){
824
825         }
826
827         /**
828          * The way snapshotting works is,
829          * <ol>
830          * <li> RaftActor calls createSnapshot on the Shard
831          * <li> Shard sends a CaptureSnapshotReply and RaftActor then calls saveSnapshot
832          * <li> When saveSnapshot is invoked on the akka-persistence API it uses the SnapshotStore to save the snapshot.
833          * The SnapshotStore sends SaveSnapshotSuccess or SaveSnapshotFailure. When the RaftActor gets SaveSnapshot
834          * success it commits the snapshot to the in-memory journal. This commitSnapshot is mimicking what is done
835          * in SaveSnapshotSuccess.
836          * </ol>
837          * @param o
838          */
839         @Override
840         public void saveSnapshot(Object o) {
841             // Make saving Snapshot successful
842             commitSnapshot(-1L);
843         }
844     }
845
846 }