Merge "Improve Shard logging output"
[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..", persistenceId());
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(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: {}", persistenceId(), 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                     persistenceId(), 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                     persistenceId(), 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={}", persistenceId(), 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:{}", persistenceId(), snapshot.getLastAppliedIndex(),
317                     snapshot.getLastAppliedTerm()
318                 );
319             }
320
321             applySnapshot(snapshot.getState());
322
323             //clears the followers log, sets the snapshot index to ensure adjusted-index works
324             replicatedLog = new ReplicatedLogImpl(snapshot);
325             context.setReplicatedLog(replicatedLog);
326             context.setLastApplied(snapshot.getLastAppliedIndex());
327
328         } else if (message instanceof FindLeader) {
329             getSender().tell(
330                 new FindLeaderReply(getLeaderAddress()),
331                 getSelf()
332             );
333
334         } else if (message instanceof SaveSnapshotSuccess) {
335             SaveSnapshotSuccess success = (SaveSnapshotSuccess) message;
336             LOG.info("{}: SaveSnapshotSuccess received for snapshot", persistenceId());
337
338             long sequenceNumber = success.metadata().sequenceNr();
339
340             commitSnapshot(sequenceNumber);
341
342         } else if (message instanceof SaveSnapshotFailure) {
343             SaveSnapshotFailure saveSnapshotFailure = (SaveSnapshotFailure) message;
344
345             LOG.error(saveSnapshotFailure.cause(), "{}: SaveSnapshotFailure received for snapshot Cause:",
346                     persistenceId());
347
348             context.getReplicatedLog().snapshotRollback();
349
350             LOG.info("{}: Replicated Log rollbacked. Snapshot will be attempted in the next cycle." +
351                 "snapshotIndex:{}, snapshotTerm:{}, log-size:{}", persistenceId(),
352                 context.getReplicatedLog().getSnapshotIndex(),
353                 context.getReplicatedLog().getSnapshotTerm(),
354                 context.getReplicatedLog().size());
355
356         } else if (message instanceof CaptureSnapshot) {
357             LOG.info("{}: CaptureSnapshot received by actor", persistenceId());
358
359             if(captureSnapshot == null) {
360                 captureSnapshot = (CaptureSnapshot)message;
361                 createSnapshot();
362             }
363
364         } else if (message instanceof CaptureSnapshotReply){
365             handleCaptureSnapshotReply(((CaptureSnapshotReply) message).getSnapshot());
366
367         } else {
368             if (!(message instanceof AppendEntriesMessages.AppendEntries)
369                 && !(message instanceof AppendEntriesReply) && !(message instanceof SendHeartBeat)) {
370                 if(LOG.isDebugEnabled()) {
371                     LOG.debug("{}: onReceiveCommand: message: {}", persistenceId(), message.getClass());
372                 }
373             }
374
375             RaftActorBehavior oldBehavior = currentBehavior;
376             currentBehavior = currentBehavior.handleMessage(getSender(), message);
377
378             handleBehaviorChange(oldBehavior, currentBehavior);
379         }
380     }
381
382     private void handleBehaviorChange(RaftActorBehavior oldBehavior, RaftActorBehavior currentBehavior) {
383         if (oldBehavior != currentBehavior){
384             onStateChanged();
385         }
386
387         String oldBehaviorLeaderId = oldBehavior == null? null : oldBehavior.getLeaderId();
388         String oldBehaviorState = oldBehavior == null? null : oldBehavior.state().name();
389
390         // it can happen that the state has not changed but the leader has changed.
391         onLeaderChanged(oldBehaviorLeaderId, currentBehavior.getLeaderId());
392
393         if (getRoleChangeNotifier().isPresent() &&
394                 (oldBehavior == null || (oldBehavior.state() != currentBehavior.state()))) {
395             getRoleChangeNotifier().get().tell(
396                     new RoleChanged(getId(), oldBehaviorState , currentBehavior.state().name()),
397                     getSelf());
398         }
399     }
400
401     /**
402      * When a derived RaftActor needs to persist something it must call
403      * persistData.
404      *
405      * @param clientActor
406      * @param identifier
407      * @param data
408      */
409     protected void persistData(final ActorRef clientActor, final String identifier,
410         final Payload data) {
411
412         ReplicatedLogEntry replicatedLogEntry = new ReplicatedLogImplEntry(
413             context.getReplicatedLog().lastIndex() + 1,
414             context.getTermInformation().getCurrentTerm(), data);
415
416         if(LOG.isDebugEnabled()) {
417             LOG.debug("{}: Persist data {}", persistenceId(), replicatedLogEntry);
418         }
419
420         final RaftActorContext raftContext = getRaftActorContext();
421
422         replicatedLog
423                 .appendAndPersist(replicatedLogEntry, new Procedure<ReplicatedLogEntry>() {
424                     @Override
425                     public void apply(ReplicatedLogEntry replicatedLogEntry) throws Exception {
426                         if(!hasFollowers()){
427                             // Increment the Commit Index and the Last Applied values
428                             raftContext.setCommitIndex(replicatedLogEntry.getIndex());
429                             raftContext.setLastApplied(replicatedLogEntry.getIndex());
430
431                             // Apply the state immediately
432                             applyState(clientActor, identifier, data);
433
434                             // Send a ApplyLogEntries message so that we write the fact that we applied
435                             // the state to durable storage
436                             self().tell(new ApplyLogEntries((int) replicatedLogEntry.getIndex()), self());
437
438                             // Check if the "real" snapshot capture has been initiated. If no then do the fake snapshot
439                             if(!hasSnapshotCaptureInitiated){
440                                 raftContext.getReplicatedLog().snapshotPreCommit(raftContext.getLastApplied(),
441                                         raftContext.getTermInformation().getCurrentTerm());
442                                 raftContext.getReplicatedLog().snapshotCommit();
443                             } else {
444                                 LOG.debug("{}: Skipping fake snapshotting for {} because real snapshotting is in progress",
445                                         persistenceId(), getId());
446                             }
447                         } else if (clientActor != null) {
448                             // Send message for replication
449                             currentBehavior.handleMessage(getSelf(),
450                                     new Replicate(clientActor, identifier,
451                                             replicatedLogEntry)
452                             );
453                         }
454
455                     }
456                 });    }
457
458     protected String getId() {
459         return context.getId();
460     }
461
462     /**
463      * Derived actors can call the isLeader method to check if the current
464      * RaftActor is the Leader or not
465      *
466      * @return true it this RaftActor is a Leader false otherwise
467      */
468     protected boolean isLeader() {
469         return context.getId().equals(currentBehavior.getLeaderId());
470     }
471
472     /**
473      * Derived actor can call getLeader if they need a reference to the Leader.
474      * This would be useful for example in forwarding a request to an actor
475      * which is the leader
476      *
477      * @return A reference to the leader if known, null otherwise
478      */
479     protected ActorSelection getLeader(){
480         String leaderAddress = getLeaderAddress();
481
482         if(leaderAddress == null){
483             return null;
484         }
485
486         return context.actorSelection(leaderAddress);
487     }
488
489     /**
490      *
491      * @return the current leader's id
492      */
493     protected String getLeaderId(){
494         return currentBehavior.getLeaderId();
495     }
496
497     protected RaftState getRaftState() {
498         return currentBehavior.state();
499     }
500
501     protected ReplicatedLogEntry getLastLogEntry() {
502         return replicatedLog.last();
503     }
504
505     protected Long getCurrentTerm(){
506         return context.getTermInformation().getCurrentTerm();
507     }
508
509     protected Long getCommitIndex(){
510         return context.getCommitIndex();
511     }
512
513     protected Long getLastApplied(){
514         return context.getLastApplied();
515     }
516
517     protected RaftActorContext getRaftActorContext() {
518         return context;
519     }
520
521     /**
522      * setPeerAddress sets the address of a known peer at a later time.
523      * <p>
524      * This is to account for situations where a we know that a peer
525      * exists but we do not know an address up-front. This may also be used in
526      * situations where a known peer starts off in a different location and we
527      * need to change it's address
528      * <p>
529      * Note that if the peerId does not match the list of peers passed to
530      * this actor during construction an IllegalStateException will be thrown.
531      *
532      * @param peerId
533      * @param peerAddress
534      */
535     protected void setPeerAddress(String peerId, String peerAddress){
536         context.setPeerAddress(peerId, peerAddress);
537     }
538
539     protected void commitSnapshot(long sequenceNumber) {
540         context.getReplicatedLog().snapshotCommit();
541
542         // TODO: Not sure if we want to be this aggressive with trimming stuff
543         trimPersistentData(sequenceNumber);
544     }
545
546     /**
547      * The applyState method will be called by the RaftActor when some data
548      * needs to be applied to the actor's state
549      *
550      * @param clientActor A reference to the client who sent this message. This
551      *                    is the same reference that was passed to persistData
552      *                    by the derived actor. clientActor may be null when
553      *                    the RaftActor is behaving as a follower or during
554      *                    recovery.
555      * @param identifier  The identifier of the persisted data. This is also
556      *                    the same identifier that was passed to persistData by
557      *                    the derived actor. identifier may be null when
558      *                    the RaftActor is behaving as a follower or during
559      *                    recovery
560      * @param data        A piece of data that was persisted by the persistData call.
561      *                    This should NEVER be null.
562      */
563     protected abstract void applyState(ActorRef clientActor, String identifier,
564         Object data);
565
566     /**
567      * This method is called during recovery at the start of a batch of state entries. Derived
568      * classes should perform any initialization needed to start a batch.
569      */
570     protected abstract void startLogRecoveryBatch(int maxBatchSize);
571
572     /**
573      * This method is called during recovery to append state data to the current batch. This method
574      * is called 1 or more times after {@link #startLogRecoveryBatch}.
575      *
576      * @param data the state data
577      */
578     protected abstract void appendRecoveredLogEntry(Payload data);
579
580     /**
581      * This method is called during recovery to reconstruct the state of the actor.
582      *
583      * @param snapshot A snapshot of the state of the actor
584      */
585     protected abstract void applyRecoverySnapshot(byte[] snapshotBytes);
586
587     /**
588      * This method is called during recovery at the end of a batch to apply the current batched
589      * log entries. This method is called after {@link #appendRecoveredLogEntry}.
590      */
591     protected abstract void applyCurrentLogRecoveryBatch();
592
593     /**
594      * This method is called when recovery is complete.
595      */
596     protected abstract void onRecoveryComplete();
597
598     /**
599      * This method will be called by the RaftActor when a snapshot needs to be
600      * created. The derived actor should respond with its current state.
601      * <p/>
602      * During recovery the state that is returned by the derived actor will
603      * be passed back to it by calling the applySnapshot  method
604      *
605      * @return The current state of the actor
606      */
607     protected abstract void createSnapshot();
608
609     /**
610      * This method can be called at any other point during normal
611      * operations when the derived actor is out of sync with it's peers
612      * and the only way to bring it in sync is by applying a snapshot
613      *
614      * @param snapshotBytes A snapshot of the state of the actor
615      */
616     protected abstract void applySnapshot(byte[] snapshotBytes);
617
618     /**
619      * This method will be called by the RaftActor when the state of the
620      * RaftActor changes. The derived actor can then use methods like
621      * isLeader or getLeader to do something useful
622      */
623     protected abstract void onStateChanged();
624
625     protected abstract DataPersistenceProvider persistence();
626
627     /**
628      * Notifier Actor for this RaftActor to notify when a role change happens
629      * @return ActorRef - ActorRef of the notifier or Optional.absent if none.
630      */
631     protected abstract Optional<ActorRef> getRoleChangeNotifier();
632
633     protected void onLeaderChanged(String oldLeader, String newLeader){};
634
635     private void trimPersistentData(long sequenceNumber) {
636         // Trim akka snapshots
637         // FIXME : Not sure how exactly the SnapshotSelectionCriteria is applied
638         // For now guessing that it is ANDed.
639         persistence().deleteSnapshots(new SnapshotSelectionCriteria(
640             sequenceNumber - context.getConfigParams().getSnapshotBatchCount(), 43200000));
641
642         // Trim akka journal
643         persistence().deleteMessages(sequenceNumber);
644     }
645
646     private String getLeaderAddress(){
647         if(isLeader()){
648             return getSelf().path().toString();
649         }
650         String leaderId = currentBehavior.getLeaderId();
651         if (leaderId == null) {
652             return null;
653         }
654         String peerAddress = context.getPeerAddress(leaderId);
655         if(LOG.isDebugEnabled()) {
656             LOG.debug("{}: getLeaderAddress leaderId = {} peerAddress = {}",
657                     persistenceId(), leaderId, peerAddress);
658         }
659
660         return peerAddress;
661     }
662
663     private void handleCaptureSnapshotReply(byte[] snapshotBytes) {
664         LOG.info("{}: CaptureSnapshotReply received by actor: snapshot size {}", persistenceId(), snapshotBytes.length);
665
666         // create a snapshot object from the state provided and save it
667         // when snapshot is saved async, SaveSnapshotSuccess is raised.
668
669         Snapshot sn = Snapshot.create(snapshotBytes,
670             context.getReplicatedLog().getFrom(captureSnapshot.getLastAppliedIndex() + 1),
671             captureSnapshot.getLastIndex(), captureSnapshot.getLastTerm(),
672             captureSnapshot.getLastAppliedIndex(), captureSnapshot.getLastAppliedTerm());
673
674         persistence().saveSnapshot(sn);
675
676         LOG.info("{}: Persisting of snapshot done:{}", persistenceId(), sn.getLogMessage());
677
678         //be greedy and remove entries from in-mem journal which are in the snapshot
679         // and update snapshotIndex and snapshotTerm without waiting for the success,
680
681         context.getReplicatedLog().snapshotPreCommit(
682             captureSnapshot.getLastAppliedIndex(),
683             captureSnapshot.getLastAppliedTerm());
684
685         LOG.info("{}: Removed in-memory snapshotted entries, adjusted snaphsotIndex:{} " +
686             "and term:{}", persistenceId(), captureSnapshot.getLastAppliedIndex(),
687             captureSnapshot.getLastAppliedTerm());
688
689         if (isLeader() && captureSnapshot.isInstallSnapshotInitiated()) {
690             // this would be call straight to the leader and won't initiate in serialization
691             currentBehavior.handleMessage(getSelf(), new SendInstallSnapshot(
692                     ByteString.copyFrom(snapshotBytes)));
693         }
694
695         captureSnapshot = null;
696         hasSnapshotCaptureInitiated = false;
697     }
698
699     protected boolean hasFollowers(){
700         return getRaftActorContext().getPeerAddresses().keySet().size() > 0;
701     }
702
703     private class ReplicatedLogImpl extends AbstractReplicatedLogImpl {
704
705         private static final int DATA_SIZE_DIVIDER = 5;
706         private long dataSizeSinceLastSnapshot = 0;
707
708         public ReplicatedLogImpl(Snapshot snapshot) {
709             super(snapshot.getLastAppliedIndex(), snapshot.getLastAppliedTerm(),
710                 snapshot.getUnAppliedEntries());
711         }
712
713         public ReplicatedLogImpl() {
714             super();
715         }
716
717         @Override public void removeFromAndPersist(long logEntryIndex) {
718             int adjustedIndex = adjustedIndex(logEntryIndex);
719
720             if (adjustedIndex < 0) {
721                 return;
722             }
723
724             // FIXME: Maybe this should be done after the command is saved
725             journal.subList(adjustedIndex , journal.size()).clear();
726
727             persistence().persist(new DeleteEntries(adjustedIndex), new Procedure<DeleteEntries>(){
728
729                 @Override public void apply(DeleteEntries param)
730                     throws Exception {
731                     //FIXME : Doing nothing for now
732                     dataSize = 0;
733                     for(ReplicatedLogEntry entry : journal){
734                         dataSize += entry.size();
735                     }
736                 }
737             });
738         }
739
740         @Override public void appendAndPersist(
741             final ReplicatedLogEntry replicatedLogEntry) {
742             appendAndPersist(replicatedLogEntry, null);
743         }
744
745         @Override
746         public int dataSize() {
747             return dataSize;
748         }
749
750         public void appendAndPersist(
751             final ReplicatedLogEntry replicatedLogEntry,
752             final Procedure<ReplicatedLogEntry> callback)  {
753
754             if(LOG.isDebugEnabled()) {
755                 LOG.debug("{}: Append log entry and persist {} ", persistenceId(), replicatedLogEntry);
756             }
757
758             // FIXME : By adding the replicated log entry to the in-memory journal we are not truly ensuring durability of the logs
759             journal.add(replicatedLogEntry);
760
761             // When persisting events with persist it is guaranteed that the
762             // persistent actor will not receive further commands between the
763             // persist call and the execution(s) of the associated event
764             // handler. This also holds for multiple persist calls in context
765             // of a single command.
766             persistence().persist(replicatedLogEntry,
767                 new Procedure<ReplicatedLogEntry>() {
768                     @Override
769                     public void apply(ReplicatedLogEntry evt) throws Exception {
770                         int logEntrySize = replicatedLogEntry.size();
771
772                         dataSize += logEntrySize;
773                         long dataSizeForCheck = dataSize;
774
775                         dataSizeSinceLastSnapshot += logEntrySize;
776                         long journalSize = lastIndex()+1;
777
778                         if(!hasFollowers()) {
779                             // When we do not have followers we do not maintain an in-memory log
780                             // due to this the journalSize will never become anything close to the
781                             // snapshot batch count. In fact will mostly be 1.
782                             // Similarly since the journal's dataSize depends on the entries in the
783                             // journal the journal's dataSize will never reach a value close to the
784                             // memory threshold.
785                             // By maintaining the dataSize outside the journal we are tracking essentially
786                             // what we have written to the disk however since we no longer are in
787                             // need of doing a snapshot just for the sake of freeing up memory we adjust
788                             // the real size of data by the DATA_SIZE_DIVIDER so that we do not snapshot as often
789                             // as if we were maintaining a real snapshot
790                             dataSizeForCheck = dataSizeSinceLastSnapshot / DATA_SIZE_DIVIDER;
791                         }
792
793                         long dataThreshold = Runtime.getRuntime().totalMemory() *
794                                 getRaftActorContext().getConfigParams().getSnapshotDataThresholdPercentage() / 100;
795
796                         // when a snaphsot is being taken, captureSnapshot != null
797                         if (hasSnapshotCaptureInitiated == false &&
798                                 ( journalSize % context.getConfigParams().getSnapshotBatchCount() == 0 ||
799                                         dataSizeForCheck > dataThreshold)) {
800
801                             dataSizeSinceLastSnapshot = 0;
802
803                             LOG.info("{}: Initiating Snapshot Capture..", persistenceId());
804                             long lastAppliedIndex = -1;
805                             long lastAppliedTerm = -1;
806
807                             ReplicatedLogEntry lastAppliedEntry = get(context.getLastApplied());
808                             if (!hasFollowers()) {
809                                 lastAppliedIndex = replicatedLogEntry.getIndex();
810                                 lastAppliedTerm = replicatedLogEntry.getTerm();
811                             } else if (lastAppliedEntry != null) {
812                                 lastAppliedIndex = lastAppliedEntry.getIndex();
813                                 lastAppliedTerm = lastAppliedEntry.getTerm();
814                             }
815
816                             if(LOG.isDebugEnabled()) {
817                                 LOG.debug("{}: Snapshot Capture logSize: {}", persistenceId(), journal.size());
818                                 LOG.debug("{}: Snapshot Capture lastApplied:{} ",
819                                         persistenceId(), context.getLastApplied());
820                                 LOG.debug("{}: Snapshot Capture lastAppliedIndex:{}", persistenceId(),
821                                         lastAppliedIndex);
822                                 LOG.debug("{}: Snapshot Capture lastAppliedTerm:{}", persistenceId(),
823                                         lastAppliedTerm);
824                             }
825
826                             // send a CaptureSnapshot to self to make the expensive operation async.
827                             getSelf().tell(new CaptureSnapshot(
828                                 lastIndex(), lastTerm(), lastAppliedIndex, lastAppliedTerm),
829                                 null);
830                             hasSnapshotCaptureInitiated = true;
831                         }
832                         if(callback != null){
833                             callback.apply(replicatedLogEntry);
834                         }
835                     }
836                 }
837             );
838         }
839
840     }
841
842     static class DeleteEntries implements Serializable {
843         private static final long serialVersionUID = 1L;
844         private final int fromIndex;
845
846         public DeleteEntries(int fromIndex) {
847             this.fromIndex = fromIndex;
848         }
849
850         public int getFromIndex() {
851             return fromIndex;
852         }
853     }
854
855
856     private class ElectionTermImpl implements ElectionTerm {
857         /**
858          * Identifier of the actor whose election term information this is
859          */
860         private long currentTerm = 0;
861         private String votedFor = null;
862
863         @Override
864         public long getCurrentTerm() {
865             return currentTerm;
866         }
867
868         @Override
869         public String getVotedFor() {
870             return votedFor;
871         }
872
873         @Override public void update(long currentTerm, String votedFor) {
874             if(LOG.isDebugEnabled()) {
875                 LOG.debug("{}: Set currentTerm={}, votedFor={}", persistenceId(), currentTerm, votedFor);
876             }
877             this.currentTerm = currentTerm;
878             this.votedFor = votedFor;
879         }
880
881         @Override
882         public void updateAndPersist(long currentTerm, String votedFor){
883             update(currentTerm, votedFor);
884             // FIXME : Maybe first persist then update the state
885             persistence().persist(new UpdateElectionTerm(this.currentTerm, this.votedFor), new Procedure<UpdateElectionTerm>(){
886
887                 @Override public void apply(UpdateElectionTerm param)
888                     throws Exception {
889
890                 }
891             });
892         }
893     }
894
895     static class UpdateElectionTerm implements Serializable {
896         private static final long serialVersionUID = 1L;
897         private final long currentTerm;
898         private final String votedFor;
899
900         public UpdateElectionTerm(long currentTerm, String votedFor) {
901             this.currentTerm = currentTerm;
902             this.votedFor = votedFor;
903         }
904
905         public long getCurrentTerm() {
906             return currentTerm;
907         }
908
909         public String getVotedFor() {
910             return votedFor;
911         }
912     }
913
914     protected class NonPersistentRaftDataProvider extends NonPersistentDataProvider {
915
916         public NonPersistentRaftDataProvider(){
917
918         }
919
920         /**
921          * The way snapshotting works is,
922          * <ol>
923          * <li> RaftActor calls createSnapshot on the Shard
924          * <li> Shard sends a CaptureSnapshotReply and RaftActor then calls saveSnapshot
925          * <li> When saveSnapshot is invoked on the akka-persistence API it uses the SnapshotStore to save the snapshot.
926          * The SnapshotStore sends SaveSnapshotSuccess or SaveSnapshotFailure. When the RaftActor gets SaveSnapshot
927          * success it commits the snapshot to the in-memory journal. This commitSnapshot is mimicking what is done
928          * in SaveSnapshotSuccess.
929          * </ol>
930          * @param o
931          */
932         @Override
933         public void saveSnapshot(Object o) {
934             // Make saving Snapshot successful
935             commitSnapshot(-1L);
936         }
937     }
938
939     @VisibleForTesting
940     void setCurrentBehavior(AbstractRaftActorBehavior behavior) {
941         currentBehavior = behavior;
942     }
943
944     protected RaftActorBehavior getCurrentBehavior() {
945         return currentBehavior;
946     }
947
948 }