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