Merge "Bug-2692:Real snapshots should use the replicatedToAllIndex for clearing in...
[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.japi.Procedure;
14 import akka.persistence.RecoveryCompleted;
15 import akka.persistence.SaveSnapshotFailure;
16 import akka.persistence.SaveSnapshotSuccess;
17 import akka.persistence.SnapshotOffer;
18 import akka.persistence.SnapshotSelectionCriteria;
19 import com.google.common.annotations.VisibleForTesting;
20 import com.google.common.base.Optional;
21 import com.google.common.base.Stopwatch;
22 import com.google.protobuf.ByteString;
23 import java.io.Serializable;
24 import java.util.Map;
25 import org.opendaylight.controller.cluster.DataPersistenceProvider;
26 import org.opendaylight.controller.cluster.common.actor.AbstractUntypedPersistentActor;
27 import org.opendaylight.controller.cluster.notifications.RoleChanged;
28 import org.opendaylight.controller.cluster.raft.base.messages.ApplyLogEntries;
29 import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
30 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
31 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshot;
32 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshotReply;
33 import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
34 import org.opendaylight.controller.cluster.raft.base.messages.SendHeartBeat;
35 import org.opendaylight.controller.cluster.raft.base.messages.SendInstallSnapshot;
36 import org.opendaylight.controller.cluster.raft.behaviors.AbstractRaftActorBehavior;
37 import org.opendaylight.controller.cluster.raft.behaviors.Follower;
38 import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior;
39 import org.opendaylight.controller.cluster.raft.client.messages.FindLeader;
40 import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply;
41 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
42 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
43 import org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages;
44 import org.slf4j.Logger;
45 import org.slf4j.LoggerFactory;
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 Logger LOG = LoggerFactory.getLogger(getClass());
89
90     /**
91      * The current state determines the current behavior of a RaftActor
92      * A Raft Actor always starts off in the Follower State
93      */
94     private RaftActorBehavior currentBehavior;
95
96     /**
97      * This context should NOT be passed directly to any other actor it is
98      * only to be consumed by the RaftActorBehaviors
99      */
100     private final RaftActorContext context;
101
102     /**
103      * The in-memory journal
104      */
105     private ReplicatedLogImpl replicatedLog = new ReplicatedLogImpl();
106
107     private CaptureSnapshot captureSnapshot = null;
108
109     private Stopwatch recoveryTimer;
110
111     private int currentRecoveryBatchCount;
112
113     public RaftActor(String id, Map<String, String> peerAddresses) {
114         this(id, peerAddresses, Optional.<ConfigParams>absent());
115     }
116
117     public RaftActor(String id, Map<String, String> peerAddresses,
118          Optional<ConfigParams> configParams) {
119
120         context = new RaftActorContextImpl(this.getSelf(),
121             this.getContext(), id, new ElectionTermImpl(),
122             -1, -1, replicatedLog, peerAddresses,
123             (configParams.isPresent() ? configParams.get(): new DefaultConfigParamsImpl()),
124             LOG);
125     }
126
127     private void initRecoveryTimer() {
128         if(recoveryTimer == null) {
129             recoveryTimer = Stopwatch.createStarted();
130         }
131     }
132
133     @Override
134     public void preStart() throws Exception {
135         LOG.info("Starting recovery for {} with journal batch size {}", persistenceId(),
136                 context.getConfigParams().getJournalRecoveryLogBatchSize());
137
138         super.preStart();
139     }
140
141     @Override
142     public void handleRecover(Object message) {
143         if(persistence().isRecoveryApplicable()) {
144             if (message instanceof SnapshotOffer) {
145                 onRecoveredSnapshot((SnapshotOffer) message);
146             } else if (message instanceof ReplicatedLogEntry) {
147                 onRecoveredJournalLogEntry((ReplicatedLogEntry) message);
148             } else if (message instanceof ApplyLogEntries) {
149                 onRecoveredApplyLogEntries((ApplyLogEntries) message);
150             } else if (message instanceof DeleteEntries) {
151                 replicatedLog.removeFrom(((DeleteEntries) message).getFromIndex());
152             } else if (message instanceof UpdateElectionTerm) {
153                 context.getTermInformation().update(((UpdateElectionTerm) message).getCurrentTerm(),
154                         ((UpdateElectionTerm) message).getVotedFor());
155             } else if (message instanceof RecoveryCompleted) {
156                 onRecoveryCompletedMessage();
157             }
158         } else {
159             if (message instanceof RecoveryCompleted) {
160                 // Delete all the messages from the akka journal so that we do not end up with consistency issues
161                 // Note I am not using the dataPersistenceProvider and directly using the akka api here
162                 deleteMessages(lastSequenceNr());
163
164                 // Delete all the akka snapshots as they will not be needed
165                 deleteSnapshots(new SnapshotSelectionCriteria(scala.Long.MaxValue(), scala.Long.MaxValue()));
166
167                 onRecoveryComplete();
168
169                 initializeBehavior();
170             }
171         }
172     }
173
174     private void onRecoveredSnapshot(SnapshotOffer offer) {
175         if(LOG.isDebugEnabled()) {
176             LOG.debug("{}: SnapshotOffer called..", persistenceId());
177         }
178
179         initRecoveryTimer();
180
181         Snapshot snapshot = (Snapshot) offer.snapshot();
182
183         // Create a replicated log with the snapshot information
184         // The replicated log can be used later on to retrieve this snapshot
185         // when we need to install it on a peer
186         replicatedLog = new ReplicatedLogImpl(snapshot);
187
188         context.setReplicatedLog(replicatedLog);
189         context.setLastApplied(snapshot.getLastAppliedIndex());
190         context.setCommitIndex(snapshot.getLastAppliedIndex());
191
192         Stopwatch timer = Stopwatch.createStarted();
193
194         // Apply the snapshot to the actors state
195         applyRecoverySnapshot(snapshot.getState());
196
197         timer.stop();
198         LOG.info("Recovery snapshot applied for {} in {}: snapshotIndex={}, snapshotTerm={}, journal-size=" +
199                 replicatedLog.size(), persistenceId(), timer.toString(),
200                 replicatedLog.getSnapshotIndex(), replicatedLog.getSnapshotTerm());
201     }
202
203     private void onRecoveredJournalLogEntry(ReplicatedLogEntry logEntry) {
204         if(LOG.isDebugEnabled()) {
205             LOG.debug("{}: Received ReplicatedLogEntry for recovery: {}", persistenceId(), logEntry.getIndex());
206         }
207
208         replicatedLog.append(logEntry);
209     }
210
211     private void onRecoveredApplyLogEntries(ApplyLogEntries ale) {
212         if(LOG.isDebugEnabled()) {
213             LOG.debug("{}: Received ApplyLogEntries for recovery, applying to state: {} to {}",
214                     persistenceId(), context.getLastApplied() + 1, ale.getToIndex());
215         }
216
217         for (long i = context.getLastApplied() + 1; i <= ale.getToIndex(); i++) {
218             batchRecoveredLogEntry(replicatedLog.get(i));
219         }
220
221         context.setLastApplied(ale.getToIndex());
222         context.setCommitIndex(ale.getToIndex());
223     }
224
225     private void batchRecoveredLogEntry(ReplicatedLogEntry logEntry) {
226         initRecoveryTimer();
227
228         int batchSize = context.getConfigParams().getJournalRecoveryLogBatchSize();
229         if(currentRecoveryBatchCount == 0) {
230             startLogRecoveryBatch(batchSize);
231         }
232
233         appendRecoveredLogEntry(logEntry.getData());
234
235         if(++currentRecoveryBatchCount >= batchSize) {
236             endCurrentLogRecoveryBatch();
237         }
238     }
239
240     private void endCurrentLogRecoveryBatch() {
241         applyCurrentLogRecoveryBatch();
242         currentRecoveryBatchCount = 0;
243     }
244
245     private void onRecoveryCompletedMessage() {
246         if(currentRecoveryBatchCount > 0) {
247             endCurrentLogRecoveryBatch();
248         }
249
250         onRecoveryComplete();
251
252         String recoveryTime = "";
253         if(recoveryTimer != null) {
254             recoveryTimer.stop();
255             recoveryTime = " in " + recoveryTimer.toString();
256             recoveryTimer = null;
257         }
258
259         LOG.info(
260             "Recovery completed" + recoveryTime + " - Switching actor to Follower - " +
261                 "Persistence Id =  " + persistenceId() +
262                 " Last index in log={}, snapshotIndex={}, snapshotTerm={}, " +
263                 "journal-size={}",
264             replicatedLog.lastIndex(), replicatedLog.getSnapshotIndex(),
265             replicatedLog.getSnapshotTerm(), replicatedLog.size());
266
267         initializeBehavior();
268     }
269
270     protected void initializeBehavior(){
271         changeCurrentBehavior(new Follower(context));
272     }
273
274     protected void changeCurrentBehavior(RaftActorBehavior newBehavior){
275         RaftActorBehavior oldBehavior = currentBehavior;
276         currentBehavior = newBehavior;
277         handleBehaviorChange(oldBehavior, currentBehavior);
278     }
279
280     @Override public void handleCommand(Object message) {
281         if (message instanceof ApplyState){
282             ApplyState applyState = (ApplyState) message;
283
284             if(LOG.isDebugEnabled()) {
285                 LOG.debug("{}: Applying state for log index {} data {}",
286                     persistenceId(), applyState.getReplicatedLogEntry().getIndex(),
287                     applyState.getReplicatedLogEntry().getData());
288             }
289
290             applyState(applyState.getClientActor(), applyState.getIdentifier(),
291                 applyState.getReplicatedLogEntry().getData());
292
293         } else if (message instanceof ApplyLogEntries){
294             ApplyLogEntries ale = (ApplyLogEntries) message;
295             if(LOG.isDebugEnabled()) {
296                 LOG.debug("{}: Persisting ApplyLogEntries with index={}", persistenceId(), ale.getToIndex());
297             }
298             persistence().persist(new ApplyLogEntries(ale.getToIndex()), new Procedure<ApplyLogEntries>() {
299                 @Override
300                 public void apply(ApplyLogEntries param) throws Exception {
301                 }
302             });
303
304         } else if(message instanceof ApplySnapshot ) {
305             Snapshot snapshot = ((ApplySnapshot) message).getSnapshot();
306
307             if(LOG.isDebugEnabled()) {
308                 LOG.debug("{}: ApplySnapshot called on Follower Actor " +
309                         "snapshotIndex:{}, snapshotTerm:{}", persistenceId(), snapshot.getLastAppliedIndex(),
310                     snapshot.getLastAppliedTerm()
311                 );
312             }
313
314             applySnapshot(snapshot.getState());
315
316             //clears the followers log, sets the snapshot index to ensure adjusted-index works
317             replicatedLog = new ReplicatedLogImpl(snapshot);
318             context.setReplicatedLog(replicatedLog);
319             context.setLastApplied(snapshot.getLastAppliedIndex());
320
321         } else if (message instanceof FindLeader) {
322             getSender().tell(
323                 new FindLeaderReply(getLeaderAddress()),
324                 getSelf()
325             );
326
327         } else if (message instanceof SaveSnapshotSuccess) {
328             SaveSnapshotSuccess success = (SaveSnapshotSuccess) message;
329             LOG.info("{}: SaveSnapshotSuccess received for snapshot", persistenceId());
330
331             long sequenceNumber = success.metadata().sequenceNr();
332
333             commitSnapshot(sequenceNumber);
334
335         } else if (message instanceof SaveSnapshotFailure) {
336             SaveSnapshotFailure saveSnapshotFailure = (SaveSnapshotFailure) message;
337
338             LOG.error("{}: SaveSnapshotFailure received for snapshot Cause:",
339                     persistenceId(), saveSnapshotFailure.cause());
340
341             context.getReplicatedLog().snapshotRollback();
342
343             LOG.info("{}: Replicated Log rollbacked. Snapshot will be attempted in the next cycle." +
344                 "snapshotIndex:{}, snapshotTerm:{}, log-size:{}", persistenceId(),
345                 context.getReplicatedLog().getSnapshotIndex(),
346                 context.getReplicatedLog().getSnapshotTerm(),
347                 context.getReplicatedLog().size());
348
349         } else if (message instanceof CaptureSnapshot) {
350             LOG.info("{}: CaptureSnapshot received by actor", persistenceId());
351
352             if(captureSnapshot == null) {
353                 captureSnapshot = (CaptureSnapshot)message;
354                 createSnapshot();
355             }
356
357         } else if (message instanceof CaptureSnapshotReply){
358             handleCaptureSnapshotReply(((CaptureSnapshotReply) message).getSnapshot());
359
360         } else {
361             if (!(message instanceof AppendEntriesMessages.AppendEntries)
362                 && !(message instanceof AppendEntriesReply) && !(message instanceof SendHeartBeat)) {
363                 if(LOG.isDebugEnabled()) {
364                     LOG.debug("{}: onReceiveCommand: message: {}", persistenceId(), message.getClass());
365                 }
366             }
367
368             RaftActorBehavior oldBehavior = currentBehavior;
369             currentBehavior = currentBehavior.handleMessage(getSender(), message);
370
371             handleBehaviorChange(oldBehavior, currentBehavior);
372         }
373     }
374
375     private void handleBehaviorChange(RaftActorBehavior oldBehavior, RaftActorBehavior currentBehavior) {
376         if (oldBehavior != currentBehavior){
377             onStateChanged();
378         }
379
380         String oldBehaviorLeaderId = oldBehavior == null? null : oldBehavior.getLeaderId();
381         String oldBehaviorState = oldBehavior == null? null : oldBehavior.state().name();
382
383         // it can happen that the state has not changed but the leader has changed.
384         onLeaderChanged(oldBehaviorLeaderId, currentBehavior.getLeaderId());
385
386         if (getRoleChangeNotifier().isPresent() &&
387                 (oldBehavior == null || (oldBehavior.state() != currentBehavior.state()))) {
388             getRoleChangeNotifier().get().tell(
389                     new RoleChanged(getId(), oldBehaviorState , currentBehavior.state().name()),
390                     getSelf());
391         }
392     }
393
394     /**
395      * When a derived RaftActor needs to persist something it must call
396      * persistData.
397      *
398      * @param clientActor
399      * @param identifier
400      * @param data
401      */
402     protected void persistData(final ActorRef clientActor, final String identifier,
403         final Payload data) {
404
405         ReplicatedLogEntry replicatedLogEntry = new ReplicatedLogImplEntry(
406             context.getReplicatedLog().lastIndex() + 1,
407             context.getTermInformation().getCurrentTerm(), data);
408
409         if(LOG.isDebugEnabled()) {
410             LOG.debug("{}: Persist data {}", persistenceId(), replicatedLogEntry);
411         }
412
413         final RaftActorContext raftContext = getRaftActorContext();
414
415         replicatedLog
416                 .appendAndPersist(replicatedLogEntry, new Procedure<ReplicatedLogEntry>() {
417                     @Override
418                     public void apply(ReplicatedLogEntry replicatedLogEntry) throws Exception {
419                         if(!hasFollowers()){
420                             // Increment the Commit Index and the Last Applied values
421                             raftContext.setCommitIndex(replicatedLogEntry.getIndex());
422                             raftContext.setLastApplied(replicatedLogEntry.getIndex());
423
424                             // Apply the state immediately
425                             applyState(clientActor, identifier, data);
426
427                             // Send a ApplyLogEntries message so that we write the fact that we applied
428                             // the state to durable storage
429                             self().tell(new ApplyLogEntries((int) replicatedLogEntry.getIndex()), self());
430
431                             // Check if the "real" snapshot capture has been initiated. If no then do the fake snapshot
432                             if(!context.isSnapshotCaptureInitiated()){
433                                 raftContext.getReplicatedLog().snapshotPreCommit(raftContext.getLastApplied(),
434                                         raftContext.getTermInformation().getCurrentTerm());
435                                 raftContext.getReplicatedLog().snapshotCommit();
436                             } else {
437                                 LOG.debug("{}: Skipping fake snapshotting for {} because real snapshotting is in progress",
438                                         persistenceId(), getId());
439                             }
440                         } else if (clientActor != null) {
441                             // Send message for replication
442                             currentBehavior.handleMessage(getSelf(),
443                                     new Replicate(clientActor, identifier,
444                                             replicatedLogEntry)
445                             );
446                         }
447
448                     }
449                 });    }
450
451     protected String getId() {
452         return context.getId();
453     }
454
455     /**
456      * Derived actors can call the isLeader method to check if the current
457      * RaftActor is the Leader or not
458      *
459      * @return true it this RaftActor is a Leader false otherwise
460      */
461     protected boolean isLeader() {
462         return context.getId().equals(currentBehavior.getLeaderId());
463     }
464
465     /**
466      * Derived actor can call getLeader if they need a reference to the Leader.
467      * This would be useful for example in forwarding a request to an actor
468      * which is the leader
469      *
470      * @return A reference to the leader if known, null otherwise
471      */
472     protected ActorSelection getLeader(){
473         String leaderAddress = getLeaderAddress();
474
475         if(leaderAddress == null){
476             return null;
477         }
478
479         return context.actorSelection(leaderAddress);
480     }
481
482     /**
483      *
484      * @return the current leader's id
485      */
486     protected String getLeaderId(){
487         return currentBehavior.getLeaderId();
488     }
489
490     protected RaftState getRaftState() {
491         return currentBehavior.state();
492     }
493
494     protected ReplicatedLogEntry getLastLogEntry() {
495         return replicatedLog.last();
496     }
497
498     protected Long getCurrentTerm(){
499         return context.getTermInformation().getCurrentTerm();
500     }
501
502     protected Long getCommitIndex(){
503         return context.getCommitIndex();
504     }
505
506     protected Long getLastApplied(){
507         return context.getLastApplied();
508     }
509
510     protected RaftActorContext getRaftActorContext() {
511         return context;
512     }
513
514     /**
515      * setPeerAddress sets the address of a known peer at a later time.
516      * <p>
517      * This is to account for situations where a we know that a peer
518      * exists but we do not know an address up-front. This may also be used in
519      * situations where a known peer starts off in a different location and we
520      * need to change it's address
521      * <p>
522      * Note that if the peerId does not match the list of peers passed to
523      * this actor during construction an IllegalStateException will be thrown.
524      *
525      * @param peerId
526      * @param peerAddress
527      */
528     protected void setPeerAddress(String peerId, String peerAddress){
529         context.setPeerAddress(peerId, peerAddress);
530     }
531
532     protected void commitSnapshot(long sequenceNumber) {
533         context.getReplicatedLog().snapshotCommit();
534
535         // TODO: Not sure if we want to be this aggressive with trimming stuff
536         trimPersistentData(sequenceNumber);
537     }
538
539     /**
540      * The applyState method will be called by the RaftActor when some data
541      * needs to be applied to the actor's state
542      *
543      * @param clientActor A reference to the client who sent this message. This
544      *                    is the same reference that was passed to persistData
545      *                    by the derived actor. clientActor may be null when
546      *                    the RaftActor is behaving as a follower or during
547      *                    recovery.
548      * @param identifier  The identifier of the persisted data. This is also
549      *                    the same identifier that was passed to persistData by
550      *                    the derived actor. identifier may be null when
551      *                    the RaftActor is behaving as a follower or during
552      *                    recovery
553      * @param data        A piece of data that was persisted by the persistData call.
554      *                    This should NEVER be null.
555      */
556     protected abstract void applyState(ActorRef clientActor, String identifier,
557         Object data);
558
559     /**
560      * This method is called during recovery at the start of a batch of state entries. Derived
561      * classes should perform any initialization needed to start a batch.
562      */
563     protected abstract void startLogRecoveryBatch(int maxBatchSize);
564
565     /**
566      * This method is called during recovery to append state data to the current batch. This method
567      * is called 1 or more times after {@link #startLogRecoveryBatch}.
568      *
569      * @param data the state data
570      */
571     protected abstract void appendRecoveredLogEntry(Payload data);
572
573     /**
574      * This method is called during recovery to reconstruct the state of the actor.
575      *
576      * @param snapshotBytes A snapshot of the state of the actor
577      */
578     protected abstract void applyRecoverySnapshot(byte[] snapshotBytes);
579
580     /**
581      * This method is called during recovery at the end of a batch to apply the current batched
582      * log entries. This method is called after {@link #appendRecoveredLogEntry}.
583      */
584     protected abstract void applyCurrentLogRecoveryBatch();
585
586     /**
587      * This method is called when recovery is complete.
588      */
589     protected abstract void onRecoveryComplete();
590
591     /**
592      * This method will be called by the RaftActor when a snapshot needs to be
593      * created. The derived actor should respond with its current state.
594      * <p/>
595      * During recovery the state that is returned by the derived actor will
596      * be passed back to it by calling the applySnapshot  method
597      *
598      * @return The current state of the actor
599      */
600     protected abstract void createSnapshot();
601
602     /**
603      * This method can be called at any other point during normal
604      * operations when the derived actor is out of sync with it's peers
605      * and the only way to bring it in sync is by applying a snapshot
606      *
607      * @param snapshotBytes A snapshot of the state of the actor
608      */
609     protected abstract void applySnapshot(byte[] snapshotBytes);
610
611     /**
612      * This method will be called by the RaftActor when the state of the
613      * RaftActor changes. The derived actor can then use methods like
614      * isLeader or getLeader to do something useful
615      */
616     protected abstract void onStateChanged();
617
618     protected abstract DataPersistenceProvider persistence();
619
620     /**
621      * Notifier Actor for this RaftActor to notify when a role change happens
622      * @return ActorRef - ActorRef of the notifier or Optional.absent if none.
623      */
624     protected abstract Optional<ActorRef> getRoleChangeNotifier();
625
626     protected void onLeaderChanged(String oldLeader, String newLeader){};
627
628     private void trimPersistentData(long sequenceNumber) {
629         // Trim akka snapshots
630         // FIXME : Not sure how exactly the SnapshotSelectionCriteria is applied
631         // For now guessing that it is ANDed.
632         persistence().deleteSnapshots(new SnapshotSelectionCriteria(
633             sequenceNumber - context.getConfigParams().getSnapshotBatchCount(), 43200000));
634
635         // Trim akka journal
636         persistence().deleteMessages(sequenceNumber);
637     }
638
639     private String getLeaderAddress(){
640         if(isLeader()){
641             return getSelf().path().toString();
642         }
643         String leaderId = currentBehavior.getLeaderId();
644         if (leaderId == null) {
645             return null;
646         }
647         String peerAddress = context.getPeerAddress(leaderId);
648         if(LOG.isDebugEnabled()) {
649             LOG.debug("{}: getLeaderAddress leaderId = {} peerAddress = {}",
650                     persistenceId(), leaderId, peerAddress);
651         }
652
653         return peerAddress;
654     }
655
656     private void handleCaptureSnapshotReply(byte[] snapshotBytes) {
657         LOG.info("{}: CaptureSnapshotReply received by actor: snapshot size {}", persistenceId(), snapshotBytes.length);
658
659         // create a snapshot object from the state provided and save it
660         // when snapshot is saved async, SaveSnapshotSuccess is raised.
661
662         Snapshot sn = Snapshot.create(snapshotBytes,
663             context.getReplicatedLog().getFrom(captureSnapshot.getLastAppliedIndex() + 1),
664             captureSnapshot.getLastIndex(), captureSnapshot.getLastTerm(),
665             captureSnapshot.getLastAppliedIndex(), captureSnapshot.getLastAppliedTerm());
666
667         persistence().saveSnapshot(sn);
668
669         LOG.info("{}: Persisting of snapshot done:{}", persistenceId(), sn.getLogMessage());
670
671         long dataThreshold = Runtime.getRuntime().totalMemory() *
672                 getRaftActorContext().getConfigParams().getSnapshotDataThresholdPercentage() / 100;
673         if (context.getReplicatedLog().dataSize() > dataThreshold) {
674             // if memory is less, clear the log based on lastApplied.
675             // this could/should only happen if one of the followers is down
676             // as normally we keep removing from the log when its replicated to all.
677             context.getReplicatedLog().snapshotPreCommit(captureSnapshot.getLastAppliedIndex(),
678                     captureSnapshot.getLastAppliedTerm());
679
680         } else {
681             // clear the log based on replicatedToAllIndex
682             context.getReplicatedLog().snapshotPreCommit(captureSnapshot.getReplicatedToAllIndex(),
683                     captureSnapshot.getReplicatedToAllTerm());
684         }
685         getCurrentBehavior().setReplicatedToAllIndex(captureSnapshot.getReplicatedToAllIndex());
686
687         LOG.info("{}: Removed in-memory snapshotted entries, adjusted snaphsotIndex:{} " +
688             "and term:{}", persistenceId(), captureSnapshot.getLastAppliedIndex(),
689             captureSnapshot.getLastAppliedTerm());
690
691         if (isLeader() && captureSnapshot.isInstallSnapshotInitiated()) {
692             // this would be call straight to the leader and won't initiate in serialization
693             currentBehavior.handleMessage(getSelf(), new SendInstallSnapshot(
694                     ByteString.copyFrom(snapshotBytes)));
695         }
696
697         captureSnapshot = null;
698         context.setSnapshotCaptureInitiated(false);
699     }
700
701     protected boolean hasFollowers(){
702         return getRaftActorContext().getPeerAddresses().keySet().size() > 0;
703     }
704
705     private class ReplicatedLogImpl extends AbstractReplicatedLogImpl {
706
707         private static final int DATA_SIZE_DIVIDER = 5;
708         private long dataSizeSinceLastSnapshot = 0;
709
710         public ReplicatedLogImpl(Snapshot snapshot) {
711             super(snapshot.getLastAppliedIndex(), snapshot.getLastAppliedTerm(),
712                 snapshot.getUnAppliedEntries());
713         }
714
715         public ReplicatedLogImpl() {
716             super();
717         }
718
719         @Override public void removeFromAndPersist(long logEntryIndex) {
720             int adjustedIndex = adjustedIndex(logEntryIndex);
721
722             if (adjustedIndex < 0) {
723                 return;
724             }
725
726             // FIXME: Maybe this should be done after the command is saved
727             journal.subList(adjustedIndex , journal.size()).clear();
728
729             persistence().persist(new DeleteEntries(adjustedIndex), new Procedure<DeleteEntries>() {
730
731                 @Override
732                 public void apply(DeleteEntries param)
733                         throws Exception {
734                     //FIXME : Doing nothing for now
735                     dataSize = 0;
736                     for (ReplicatedLogEntry entry : journal) {
737                         dataSize += entry.size();
738                     }
739                 }
740             });
741         }
742
743         @Override public void appendAndPersist(
744             final ReplicatedLogEntry replicatedLogEntry) {
745             appendAndPersist(replicatedLogEntry, null);
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 {} ", persistenceId(), 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 (!context.isSnapshotCaptureInitiated() &&
796                                 ( journalSize % context.getConfigParams().getSnapshotBatchCount() == 0 ||
797                                         dataSizeForCheck > dataThreshold)) {
798
799                             dataSizeSinceLastSnapshot = 0;
800
801                             LOG.info("{}: Initiating Snapshot Capture..", persistenceId());
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: {}", persistenceId(), journal.size());
816                                 LOG.debug("{}: Snapshot Capture lastApplied:{} ",
817                                         persistenceId(), context.getLastApplied());
818                                 LOG.debug("{}: Snapshot Capture lastAppliedIndex:{}", persistenceId(),
819                                         lastAppliedIndex);
820                                 LOG.debug("{}: Snapshot Capture lastAppliedTerm:{}", persistenceId(),
821                                         lastAppliedTerm);
822                             }
823
824                             // send a CaptureSnapshot to self to make the expensive operation async.
825                             long replicatedToAllIndex = getCurrentBehavior().getReplicatedToAllIndex();
826                             ReplicatedLogEntry replicatedToAllEntry = context.getReplicatedLog().get(replicatedToAllIndex);
827                             getSelf().tell(new CaptureSnapshot(lastIndex(), lastTerm(), lastAppliedIndex, lastAppliedTerm,
828                                 (replicatedToAllEntry != null ? replicatedToAllEntry.getIndex() : -1),
829                                 (replicatedToAllEntry != null ? replicatedToAllEntry.getTerm() : -1)),
830                                 null);
831                             context.setSnapshotCaptureInitiated(true);
832                         }
833                         if (callback != null){
834                             callback.apply(replicatedLogEntry);
835                         }
836                     }
837                 }
838             );
839         }
840
841     }
842
843     static class DeleteEntries implements Serializable {
844         private static final long serialVersionUID = 1L;
845         private final int fromIndex;
846
847         public DeleteEntries(int fromIndex) {
848             this.fromIndex = fromIndex;
849         }
850
851         public int getFromIndex() {
852             return fromIndex;
853         }
854     }
855
856
857     private class ElectionTermImpl implements ElectionTerm {
858         /**
859          * Identifier of the actor whose election term information this is
860          */
861         private long currentTerm = 0;
862         private String votedFor = null;
863
864         @Override
865         public long getCurrentTerm() {
866             return currentTerm;
867         }
868
869         @Override
870         public String getVotedFor() {
871             return votedFor;
872         }
873
874         @Override public void update(long currentTerm, String votedFor) {
875             if(LOG.isDebugEnabled()) {
876                 LOG.debug("{}: Set currentTerm={}, votedFor={}", persistenceId(), currentTerm, votedFor);
877             }
878             this.currentTerm = currentTerm;
879             this.votedFor = votedFor;
880         }
881
882         @Override
883         public void updateAndPersist(long currentTerm, String votedFor){
884             update(currentTerm, votedFor);
885             // FIXME : Maybe first persist then update the state
886             persistence().persist(new UpdateElectionTerm(this.currentTerm, this.votedFor), new Procedure<UpdateElectionTerm>(){
887
888                 @Override public void apply(UpdateElectionTerm param)
889                     throws Exception {
890
891                 }
892             });
893         }
894     }
895
896     static class UpdateElectionTerm implements Serializable {
897         private static final long serialVersionUID = 1L;
898         private final long currentTerm;
899         private final String votedFor;
900
901         public UpdateElectionTerm(long currentTerm, String votedFor) {
902             this.currentTerm = currentTerm;
903             this.votedFor = votedFor;
904         }
905
906         public long getCurrentTerm() {
907             return currentTerm;
908         }
909
910         public String getVotedFor() {
911             return votedFor;
912         }
913     }
914
915     protected class NonPersistentRaftDataProvider extends NonPersistentDataProvider {
916
917         public NonPersistentRaftDataProvider(){
918
919         }
920
921         /**
922          * The way snapshotting works is,
923          * <ol>
924          * <li> RaftActor calls createSnapshot on the Shard
925          * <li> Shard sends a CaptureSnapshotReply and RaftActor then calls saveSnapshot
926          * <li> When saveSnapshot is invoked on the akka-persistence API it uses the SnapshotStore to save the snapshot.
927          * The SnapshotStore sends SaveSnapshotSuccess or SaveSnapshotFailure. When the RaftActor gets SaveSnapshot
928          * success it commits the snapshot to the in-memory journal. This commitSnapshot is mimicking what is done
929          * in SaveSnapshotSuccess.
930          * </ol>
931          * @param o
932          */
933         @Override
934         public void saveSnapshot(Object o) {
935             // Make saving Snapshot successful
936             commitSnapshot(-1L);
937         }
938     }
939
940     @VisibleForTesting
941     void setCurrentBehavior(AbstractRaftActorBehavior behavior) {
942         currentBehavior = behavior;
943     }
944
945     protected RaftActorBehavior getCurrentBehavior() {
946         return currentBehavior;
947     }
948
949 }