dc65869f6df051b01f277676ebb810cff47e419b
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / RaftActor.java
1 /*
2  * Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
3  *
4  * This program and the accompanying materials are made available under the
5  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
6  * and is available at http://www.eclipse.org/legal/epl-v10.html
7  */
8
9 package org.opendaylight.controller.cluster.raft;
10
11 import akka.actor.ActorRef;
12 import akka.actor.ActorSelection;
13 import akka.event.Logging;
14 import akka.event.LoggingAdapter;
15 import akka.japi.Procedure;
16 import akka.persistence.RecoveryCompleted;
17 import akka.persistence.SaveSnapshotFailure;
18 import akka.persistence.SaveSnapshotSuccess;
19 import akka.persistence.SnapshotOffer;
20 import akka.persistence.SnapshotSelectionCriteria;
21 import com.google.common.annotations.VisibleForTesting;
22 import com.google.common.base.Optional;
23 import com.google.common.base.Stopwatch;
24 import com.google.protobuf.ByteString;
25 import java.io.Serializable;
26 import java.util.Map;
27 import org.opendaylight.controller.cluster.DataPersistenceProvider;
28 import org.opendaylight.controller.cluster.common.actor.AbstractUntypedPersistentActor;
29 import org.opendaylight.controller.cluster.notifications.RoleChanged;
30 import org.opendaylight.controller.cluster.raft.base.messages.ApplyLogEntries;
31 import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
32 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
33 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshot;
34 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshotReply;
35 import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
36 import org.opendaylight.controller.cluster.raft.base.messages.SendHeartBeat;
37 import org.opendaylight.controller.cluster.raft.base.messages.SendInstallSnapshot;
38 import org.opendaylight.controller.cluster.raft.behaviors.AbstractRaftActorBehavior;
39 import org.opendaylight.controller.cluster.raft.behaviors.Follower;
40 import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior;
41 import org.opendaylight.controller.cluster.raft.client.messages.FindLeader;
42 import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply;
43 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
44 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
45 import org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages;
46
47 /**
48  * RaftActor encapsulates a state machine that needs to be kept synchronized
49  * in a cluster. It implements the RAFT algorithm as described in the paper
50  * <a href='https://ramcloud.stanford.edu/wiki/download/attachments/11370504/raft.pdf'>
51  * In Search of an Understandable Consensus Algorithm</a>
52  * <p/>
53  * RaftActor has 3 states and each state has a certain behavior associated
54  * with it. A Raft actor can behave as,
55  * <ul>
56  * <li> A Leader </li>
57  * <li> A Follower (or) </li>
58  * <li> A Candidate </li>
59  * </ul>
60  * <p/>
61  * <p/>
62  * A RaftActor MUST be a Leader in order to accept requests from clients to
63  * change the state of it's encapsulated state machine. Once a RaftActor becomes
64  * a Leader it is also responsible for ensuring that all followers ultimately
65  * have the same log and therefore the same state machine as itself.
66  * <p/>
67  * <p/>
68  * The current behavior of a RaftActor determines how election for leadership
69  * is initiated and how peer RaftActors react to request for votes.
70  * <p/>
71  * <p/>
72  * Each RaftActor also needs to know the current election term. It uses this
73  * information for a couple of things. One is to simply figure out who it
74  * voted for in the last election. Another is to figure out if the message
75  * it received to update it's state is stale.
76  * <p/>
77  * <p/>
78  * The RaftActor uses akka-persistence to store it's replicated log.
79  * Furthermore through it's behaviors a Raft Actor determines
80  * <p/>
81  * <ul>
82  * <li> when a log entry should be persisted </li>
83  * <li> when a log entry should be applied to the state machine (and) </li>
84  * <li> when a snapshot should be saved </li>
85  * </ul>
86  */
87 public abstract class RaftActor extends AbstractUntypedPersistentActor {
88     protected final LoggingAdapter LOG =
89         Logging.getLogger(getContext().system(), this);
90
91     /**
92      * The current state determines the current behavior of a RaftActor
93      * A Raft Actor always starts off in the Follower State
94      */
95     private RaftActorBehavior currentBehavior;
96
97     /**
98      * This context should NOT be passed directly to any other actor it is
99      * only to be consumed by the RaftActorBehaviors
100      */
101     private final RaftActorContext context;
102
103     /**
104      * The in-memory journal
105      */
106     private ReplicatedLogImpl replicatedLog = new ReplicatedLogImpl();
107
108     private CaptureSnapshot captureSnapshot = null;
109
110     private volatile boolean hasSnapshotCaptureInitiated = false;
111
112     private Stopwatch recoveryTimer;
113
114     private int currentRecoveryBatchCount;
115
116     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
173                 RaftActorBehavior oldBehavior = currentBehavior;
174                 currentBehavior = new Follower(context);
175                 handleBehaviorChange(oldBehavior, currentBehavior);
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         RaftActorBehavior oldBehavior = currentBehavior;
275         currentBehavior = new Follower(context);
276         handleBehaviorChange(oldBehavior, currentBehavior);
277     }
278
279     @Override public void handleCommand(Object message) {
280         if (message instanceof ApplyState){
281             ApplyState applyState = (ApplyState) message;
282
283             if(LOG.isDebugEnabled()) {
284                 LOG.debug("Applying state for log index {} data {}",
285                     applyState.getReplicatedLogEntry().getIndex(),
286                     applyState.getReplicatedLogEntry().getData());
287             }
288
289             applyState(applyState.getClientActor(), applyState.getIdentifier(),
290                 applyState.getReplicatedLogEntry().getData());
291
292         } else if (message instanceof ApplyLogEntries){
293             ApplyLogEntries ale = (ApplyLogEntries) message;
294             if(LOG.isDebugEnabled()) {
295                 LOG.debug("Persisting ApplyLogEntries with index={}", ale.getToIndex());
296             }
297             persistence().persist(new ApplyLogEntries(ale.getToIndex()), new Procedure<ApplyLogEntries>() {
298                 @Override
299                 public void apply(ApplyLogEntries param) throws Exception {
300                 }
301             });
302
303         } else if(message instanceof ApplySnapshot ) {
304             Snapshot snapshot = ((ApplySnapshot) message).getSnapshot();
305
306             if(LOG.isDebugEnabled()) {
307                 LOG.debug("ApplySnapshot called on Follower Actor " +
308                         "snapshotIndex:{}, snapshotTerm:{}", snapshot.getLastAppliedIndex(),
309                     snapshot.getLastAppliedTerm()
310                 );
311             }
312             applySnapshot(ByteString.copyFrom(snapshot.getState()));
313
314             //clears the followers log, sets the snapshot index to ensure adjusted-index works
315             replicatedLog = new ReplicatedLogImpl(snapshot);
316             context.setReplicatedLog(replicatedLog);
317             context.setLastApplied(snapshot.getLastAppliedIndex());
318
319         } else if (message instanceof FindLeader) {
320             getSender().tell(
321                 new FindLeaderReply(getLeaderAddress()),
322                 getSelf()
323             );
324
325         } else if (message instanceof SaveSnapshotSuccess) {
326             SaveSnapshotSuccess success = (SaveSnapshotSuccess) message;
327             LOG.info("SaveSnapshotSuccess received for snapshot");
328
329             long sequenceNumber = success.metadata().sequenceNr();
330
331             commitSnapshot(sequenceNumber);
332
333         } else if (message instanceof SaveSnapshotFailure) {
334             SaveSnapshotFailure saveSnapshotFailure = (SaveSnapshotFailure) message;
335
336             LOG.info("saveSnapshotFailure.metadata():{}", saveSnapshotFailure.metadata().toString());
337             LOG.error(saveSnapshotFailure.cause(), "SaveSnapshotFailure received for snapshot Cause:");
338
339             context.getReplicatedLog().snapshotRollback();
340
341             LOG.info("Replicated Log rollbacked. Snapshot will be attempted in the next cycle." +
342                 "snapshotIndex:{}, snapshotTerm:{}, log-size:{}",
343                 context.getReplicatedLog().getSnapshotIndex(),
344                 context.getReplicatedLog().getSnapshotTerm(),
345                 context.getReplicatedLog().size());
346
347         } else if (message instanceof CaptureSnapshot) {
348             LOG.info("CaptureSnapshot received by actor");
349             CaptureSnapshot cs = (CaptureSnapshot)message;
350             captureSnapshot = cs;
351             createSnapshot();
352
353         } else if (message instanceof CaptureSnapshotReply){
354             LOG.info("CaptureSnapshotReply received by actor");
355             CaptureSnapshotReply csr = (CaptureSnapshotReply) message;
356
357             ByteString stateInBytes = csr.getSnapshot();
358             LOG.info("CaptureSnapshotReply stateInBytes size:{}", stateInBytes.size());
359             handleCaptureSnapshotReply(stateInBytes);
360
361         } else {
362             if (!(message instanceof AppendEntriesMessages.AppendEntries)
363                 && !(message instanceof AppendEntriesReply) && !(message instanceof SendHeartBeat)) {
364                 if(LOG.isDebugEnabled()) {
365                     LOG.debug("onReceiveCommand: message: {}", message.getClass());
366                 }
367             }
368
369             RaftActorBehavior oldBehavior = currentBehavior;
370             currentBehavior = currentBehavior.handleMessage(getSender(), message);
371
372             handleBehaviorChange(oldBehavior, currentBehavior);
373         }
374     }
375
376     private void handleBehaviorChange(RaftActorBehavior oldBehavior, RaftActorBehavior currentBehavior) {
377         if (oldBehavior != currentBehavior){
378             onStateChanged();
379         }
380
381         String oldBehaviorLeaderId = oldBehavior == null? null : oldBehavior.getLeaderId();
382         String oldBehaviorState = oldBehavior == null? null : oldBehavior.state().name();
383
384         // it can happen that the state has not changed but the leader has changed.
385         onLeaderChanged(oldBehaviorLeaderId, currentBehavior.getLeaderId());
386
387         if (getRoleChangeNotifier().isPresent() &&
388                 (oldBehavior == null || (oldBehavior.state() != currentBehavior.state()))) {
389             getRoleChangeNotifier().get().tell(
390                     new RoleChanged(getId(), oldBehaviorState , currentBehavior.state().name()),
391                     getSelf());
392         }
393     }
394
395     /**
396      * When a derived RaftActor needs to persist something it must call
397      * persistData.
398      *
399      * @param clientActor
400      * @param identifier
401      * @param data
402      */
403     protected void persistData(ActorRef clientActor, String identifier,
404         Payload data) {
405
406         ReplicatedLogEntry replicatedLogEntry = new ReplicatedLogImplEntry(
407             context.getReplicatedLog().lastIndex() + 1,
408             context.getTermInformation().getCurrentTerm(), data);
409
410         if(LOG.isDebugEnabled()) {
411             LOG.debug("Persist data {}", replicatedLogEntry);
412         }
413
414         replicatedLog
415             .appendAndPersist(clientActor, identifier, replicatedLogEntry);
416     }
417
418     protected String getId() {
419         return context.getId();
420     }
421
422     /**
423      * Derived actors can call the isLeader method to check if the current
424      * RaftActor is the Leader or not
425      *
426      * @return true it this RaftActor is a Leader false otherwise
427      */
428     protected boolean isLeader() {
429         return context.getId().equals(currentBehavior.getLeaderId());
430     }
431
432     /**
433      * Derived actor can call getLeader if they need a reference to the Leader.
434      * This would be useful for example in forwarding a request to an actor
435      * which is the leader
436      *
437      * @return A reference to the leader if known, null otherwise
438      */
439     protected ActorSelection getLeader(){
440         String leaderAddress = getLeaderAddress();
441
442         if(leaderAddress == null){
443             return null;
444         }
445
446         return context.actorSelection(leaderAddress);
447     }
448
449     /**
450      *
451      * @return the current leader's id
452      */
453     protected String getLeaderId(){
454         return currentBehavior.getLeaderId();
455     }
456
457     protected RaftState getRaftState() {
458         return currentBehavior.state();
459     }
460
461     protected ReplicatedLogEntry getLastLogEntry() {
462         return replicatedLog.last();
463     }
464
465     protected Long getCurrentTerm(){
466         return context.getTermInformation().getCurrentTerm();
467     }
468
469     protected Long getCommitIndex(){
470         return context.getCommitIndex();
471     }
472
473     protected Long getLastApplied(){
474         return context.getLastApplied();
475     }
476
477     protected RaftActorContext getRaftActorContext() {
478         return context;
479     }
480
481     /**
482      * setPeerAddress sets the address of a known peer at a later time.
483      * <p>
484      * This is to account for situations where a we know that a peer
485      * exists but we do not know an address up-front. This may also be used in
486      * situations where a known peer starts off in a different location and we
487      * need to change it's address
488      * <p>
489      * Note that if the peerId does not match the list of peers passed to
490      * this actor during construction an IllegalStateException will be thrown.
491      *
492      * @param peerId
493      * @param peerAddress
494      */
495     protected void setPeerAddress(String peerId, String peerAddress){
496         context.setPeerAddress(peerId, peerAddress);
497     }
498
499     protected void commitSnapshot(long sequenceNumber) {
500         context.getReplicatedLog().snapshotCommit();
501
502         // TODO: Not sure if we want to be this aggressive with trimming stuff
503         trimPersistentData(sequenceNumber);
504     }
505
506     /**
507      * The applyState method will be called by the RaftActor when some data
508      * needs to be applied to the actor's state
509      *
510      * @param clientActor A reference to the client who sent this message. This
511      *                    is the same reference that was passed to persistData
512      *                    by the derived actor. clientActor may be null when
513      *                    the RaftActor is behaving as a follower or during
514      *                    recovery.
515      * @param identifier  The identifier of the persisted data. This is also
516      *                    the same identifier that was passed to persistData by
517      *                    the derived actor. identifier may be null when
518      *                    the RaftActor is behaving as a follower or during
519      *                    recovery
520      * @param data        A piece of data that was persisted by the persistData call.
521      *                    This should NEVER be null.
522      */
523     protected abstract void applyState(ActorRef clientActor, String identifier,
524         Object data);
525
526     /**
527      * This method is called during recovery at the start of a batch of state entries. Derived
528      * classes should perform any initialization needed to start a batch.
529      */
530     protected abstract void startLogRecoveryBatch(int maxBatchSize);
531
532     /**
533      * This method is called during recovery to append state data to the current batch. This method
534      * is called 1 or more times after {@link #startLogRecoveryBatch}.
535      *
536      * @param data the state data
537      */
538     protected abstract void appendRecoveredLogEntry(Payload data);
539
540     /**
541      * This method is called during recovery to reconstruct the state of the actor.
542      *
543      * @param snapshot A snapshot of the state of the actor
544      */
545     protected abstract void applyRecoverySnapshot(ByteString snapshot);
546
547     /**
548      * This method is called during recovery at the end of a batch to apply the current batched
549      * log entries. This method is called after {@link #appendRecoveredLogEntry}.
550      */
551     protected abstract void applyCurrentLogRecoveryBatch();
552
553     /**
554      * This method is called when recovery is complete.
555      */
556     protected abstract void onRecoveryComplete();
557
558     /**
559      * This method will be called by the RaftActor when a snapshot needs to be
560      * created. The derived actor should respond with its current state.
561      * <p/>
562      * During recovery the state that is returned by the derived actor will
563      * be passed back to it by calling the applySnapshot  method
564      *
565      * @return The current state of the actor
566      */
567     protected abstract void createSnapshot();
568
569     /**
570      * This method can be called at any other point during normal
571      * operations when the derived actor is out of sync with it's peers
572      * and the only way to bring it in sync is by applying a snapshot
573      *
574      * @param snapshot A snapshot of the state of the actor
575      */
576     protected abstract void applySnapshot(ByteString snapshot);
577
578     /**
579      * This method will be called by the RaftActor when the state of the
580      * RaftActor changes. The derived actor can then use methods like
581      * isLeader or getLeader to do something useful
582      */
583     protected abstract void onStateChanged();
584
585     protected abstract DataPersistenceProvider persistence();
586
587     /**
588      * Notifier Actor for this RaftActor to notify when a role change happens
589      * @return ActorRef - ActorRef of the notifier or Optional.absent if none.
590      */
591     protected abstract Optional<ActorRef> getRoleChangeNotifier();
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(
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         if (isLeader() && captureSnapshot.isInstallSnapshotInitiated()) {
648             // this would be call straight to the leader and won't initiate in serialization
649             currentBehavior.handleMessage(getSelf(), new SendInstallSnapshot(stateInBytes));
650         }
651
652         captureSnapshot = null;
653         hasSnapshotCaptureInitiated = false;
654     }
655
656     private class ReplicatedLogImpl extends AbstractReplicatedLogImpl {
657
658         public ReplicatedLogImpl(Snapshot snapshot) {
659             super(snapshot.getLastAppliedIndex(), snapshot.getLastAppliedTerm(),
660                 snapshot.getUnAppliedEntries());
661         }
662
663         public ReplicatedLogImpl() {
664             super();
665         }
666
667         @Override public void removeFromAndPersist(long logEntryIndex) {
668             int adjustedIndex = adjustedIndex(logEntryIndex);
669
670             if (adjustedIndex < 0) {
671                 return;
672             }
673
674             // FIXME: Maybe this should be done after the command is saved
675             journal.subList(adjustedIndex , journal.size()).clear();
676
677             persistence().persist(new DeleteEntries(adjustedIndex), new Procedure<DeleteEntries>(){
678
679                 @Override public void apply(DeleteEntries param)
680                     throws Exception {
681                     //FIXME : Doing nothing for now
682                     dataSize = 0;
683                     for(ReplicatedLogEntry entry : journal){
684                         dataSize += entry.size();
685                     }
686                 }
687             });
688         }
689
690         @Override public void appendAndPersist(
691             final ReplicatedLogEntry replicatedLogEntry) {
692             appendAndPersist(null, null, replicatedLogEntry);
693         }
694
695         @Override
696         public int dataSize() {
697             return dataSize;
698         }
699
700         public void appendAndPersist(final ActorRef clientActor,
701             final String identifier,
702             final ReplicatedLogEntry replicatedLogEntry) {
703
704             if(LOG.isDebugEnabled()) {
705                 LOG.debug("Append log entry and persist {} ", replicatedLogEntry);
706             }
707
708             // FIXME : By adding the replicated log entry to the in-memory journal we are not truly ensuring durability of the logs
709             journal.add(replicatedLogEntry);
710
711             // When persisting events with persist it is guaranteed that the
712             // persistent actor will not receive further commands between the
713             // persist call and the execution(s) of the associated event
714             // handler. This also holds for multiple persist calls in context
715             // of a single command.
716             persistence().persist(replicatedLogEntry,
717                 new Procedure<ReplicatedLogEntry>() {
718                     @Override
719                     public void apply(ReplicatedLogEntry evt) throws Exception {
720                         dataSize += replicatedLogEntry.size();
721
722                         long dataThreshold = Runtime.getRuntime().totalMemory() *
723                                 getRaftActorContext().getConfigParams().getSnapshotDataThresholdPercentage() / 100;
724
725                         // when a snaphsot is being taken, captureSnapshot != null
726                         if (hasSnapshotCaptureInitiated == false &&
727                                 ( journal.size() % context.getConfigParams().getSnapshotBatchCount() == 0 ||
728                                         dataSize > dataThreshold)) {
729
730                             LOG.info("Initiating Snapshot Capture..");
731                             long lastAppliedIndex = -1;
732                             long lastAppliedTerm = -1;
733
734                             ReplicatedLogEntry lastAppliedEntry = get(context.getLastApplied());
735                             if (lastAppliedEntry != null) {
736                                 lastAppliedIndex = lastAppliedEntry.getIndex();
737                                 lastAppliedTerm = lastAppliedEntry.getTerm();
738                             }
739
740                             if(LOG.isDebugEnabled()) {
741                                 LOG.debug("Snapshot Capture logSize: {}", journal.size());
742                                 LOG.debug("Snapshot Capture lastApplied:{} ",
743                                     context.getLastApplied());
744                                 LOG.debug("Snapshot Capture lastAppliedIndex:{}", lastAppliedIndex);
745                                 LOG.debug("Snapshot Capture lastAppliedTerm:{}", lastAppliedTerm);
746                             }
747
748                             // send a CaptureSnapshot to self to make the expensive operation async.
749                             getSelf().tell(new CaptureSnapshot(
750                                 lastIndex(), lastTerm(), lastAppliedIndex, lastAppliedTerm),
751                                 null);
752                             hasSnapshotCaptureInitiated = true;
753                         }
754                         // Send message for replication
755                         if (clientActor != null) {
756                             currentBehavior.handleMessage(getSelf(),
757                                 new Replicate(clientActor, identifier,
758                                     replicatedLogEntry)
759                             );
760                         }
761                     }
762                 }
763             );
764         }
765
766     }
767
768     static class DeleteEntries implements Serializable {
769         private static final long serialVersionUID = 1L;
770         private final int fromIndex;
771
772         public DeleteEntries(int fromIndex) {
773             this.fromIndex = fromIndex;
774         }
775
776         public int getFromIndex() {
777             return fromIndex;
778         }
779     }
780
781
782     private class ElectionTermImpl implements ElectionTerm {
783         /**
784          * Identifier of the actor whose election term information this is
785          */
786         private long currentTerm = 0;
787         private String votedFor = null;
788
789         @Override
790         public long getCurrentTerm() {
791             return currentTerm;
792         }
793
794         @Override
795         public String getVotedFor() {
796             return votedFor;
797         }
798
799         @Override public void update(long currentTerm, String votedFor) {
800             if(LOG.isDebugEnabled()) {
801                 LOG.debug("Set currentTerm={}, votedFor={}", currentTerm, votedFor);
802             }
803             this.currentTerm = currentTerm;
804             this.votedFor = votedFor;
805         }
806
807         @Override
808         public void updateAndPersist(long currentTerm, String votedFor){
809             update(currentTerm, votedFor);
810             // FIXME : Maybe first persist then update the state
811             persistence().persist(new UpdateElectionTerm(this.currentTerm, this.votedFor), new Procedure<UpdateElectionTerm>(){
812
813                 @Override public void apply(UpdateElectionTerm param)
814                     throws Exception {
815
816                 }
817             });
818         }
819     }
820
821     static class UpdateElectionTerm implements Serializable {
822         private static final long serialVersionUID = 1L;
823         private final long currentTerm;
824         private final String votedFor;
825
826         public UpdateElectionTerm(long currentTerm, String votedFor) {
827             this.currentTerm = currentTerm;
828             this.votedFor = votedFor;
829         }
830
831         public long getCurrentTerm() {
832             return currentTerm;
833         }
834
835         public String getVotedFor() {
836             return votedFor;
837         }
838     }
839
840     protected class NonPersistentRaftDataProvider extends NonPersistentDataProvider {
841
842         public NonPersistentRaftDataProvider(){
843
844         }
845
846         /**
847          * The way snapshotting works is,
848          * <ol>
849          * <li> RaftActor calls createSnapshot on the Shard
850          * <li> Shard sends a CaptureSnapshotReply and RaftActor then calls saveSnapshot
851          * <li> When saveSnapshot is invoked on the akka-persistence API it uses the SnapshotStore to save the snapshot.
852          * The SnapshotStore sends SaveSnapshotSuccess or SaveSnapshotFailure. When the RaftActor gets SaveSnapshot
853          * success it commits the snapshot to the in-memory journal. This commitSnapshot is mimicking what is done
854          * in SaveSnapshotSuccess.
855          * </ol>
856          * @param o
857          */
858         @Override
859         public void saveSnapshot(Object o) {
860             // Make saving Snapshot successful
861             commitSnapshot(-1L);
862         }
863     }
864
865     @VisibleForTesting
866     void setCurrentBehavior(AbstractRaftActorBehavior behavior) {
867         currentBehavior = behavior;
868     }
869
870     protected RaftActorBehavior getCurrentBehavior() {
871         return currentBehavior;
872     }
873
874 }