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