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