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