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