Merge "BUG 2799: SPI for EventSources"
[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.Objects;
21 import com.google.common.base.Optional;
22 import com.google.common.base.Stopwatch;
23 import com.google.common.collect.ImmutableMap;
24 import com.google.common.collect.Lists;
25 import java.io.Serializable;
26 import java.util.Collection;
27 import java.util.List;
28 import java.util.Map;
29 import java.util.concurrent.TimeUnit;
30 import org.apache.commons.lang3.time.DurationFormatUtils;
31 import org.opendaylight.controller.cluster.DataPersistenceProvider;
32 import org.opendaylight.controller.cluster.DelegatingPersistentDataProvider;
33 import org.opendaylight.controller.cluster.NonPersistentDataProvider;
34 import org.opendaylight.controller.cluster.PersistentDataProvider;
35 import org.opendaylight.controller.cluster.common.actor.AbstractUntypedPersistentActor;
36 import org.opendaylight.controller.cluster.notifications.LeaderStateChanged;
37 import org.opendaylight.controller.cluster.notifications.RoleChanged;
38 import org.opendaylight.controller.cluster.raft.base.messages.ApplyJournalEntries;
39 import org.opendaylight.controller.cluster.raft.base.messages.ApplyLogEntries;
40 import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
41 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
42 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshot;
43 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshotReply;
44 import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
45 import org.opendaylight.controller.cluster.raft.behaviors.AbstractLeader;
46 import org.opendaylight.controller.cluster.raft.behaviors.DelegatingRaftActorBehavior;
47 import org.opendaylight.controller.cluster.raft.behaviors.Follower;
48 import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior;
49 import org.opendaylight.controller.cluster.raft.client.messages.FindLeader;
50 import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply;
51 import org.opendaylight.controller.cluster.raft.client.messages.FollowerInfo;
52 import org.opendaylight.controller.cluster.raft.client.messages.GetOnDemandRaftState;
53 import org.opendaylight.controller.cluster.raft.client.messages.OnDemandRaftState;
54 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
55 import org.slf4j.Logger;
56 import org.slf4j.LoggerFactory;
57
58 /**
59  * RaftActor encapsulates a state machine that needs to be kept synchronized
60  * in a cluster. It implements the RAFT algorithm as described in the paper
61  * <a href='https://ramcloud.stanford.edu/wiki/download/attachments/11370504/raft.pdf'>
62  * In Search of an Understandable Consensus Algorithm</a>
63  * <p/>
64  * RaftActor has 3 states and each state has a certain behavior associated
65  * with it. A Raft actor can behave as,
66  * <ul>
67  * <li> A Leader </li>
68  * <li> A Follower (or) </li>
69  * <li> A Candidate </li>
70  * </ul>
71  * <p/>
72  * <p/>
73  * A RaftActor MUST be a Leader in order to accept requests from clients to
74  * change the state of it's encapsulated state machine. Once a RaftActor becomes
75  * a Leader it is also responsible for ensuring that all followers ultimately
76  * have the same log and therefore the same state machine as itself.
77  * <p/>
78  * <p/>
79  * The current behavior of a RaftActor determines how election for leadership
80  * is initiated and how peer RaftActors react to request for votes.
81  * <p/>
82  * <p/>
83  * Each RaftActor also needs to know the current election term. It uses this
84  * information for a couple of things. One is to simply figure out who it
85  * voted for in the last election. Another is to figure out if the message
86  * it received to update it's state is stale.
87  * <p/>
88  * <p/>
89  * The RaftActor uses akka-persistence to store it's replicated log.
90  * Furthermore through it's behaviors a Raft Actor determines
91  * <p/>
92  * <ul>
93  * <li> when a log entry should be persisted </li>
94  * <li> when a log entry should be applied to the state machine (and) </li>
95  * <li> when a snapshot should be saved </li>
96  * </ul>
97  */
98 public abstract class RaftActor extends AbstractUntypedPersistentActor {
99
100     private static final long APPLY_STATE_DELAY_THRESHOLD_IN_NANOS = TimeUnit.MILLISECONDS.toNanos(50L); // 50 millis
101
102     private static final String COMMIT_SNAPSHOT = "commit_snapshot";
103
104     protected final Logger LOG = LoggerFactory.getLogger(getClass());
105
106     /**
107      * The current state determines the current behavior of a RaftActor
108      * A Raft Actor always starts off in the Follower State
109      */
110     private final DelegatingRaftActorBehavior currentBehavior = new DelegatingRaftActorBehavior();
111
112     /**
113      * This context should NOT be passed directly to any other actor it is
114      * only to be consumed by the RaftActorBehaviors
115      */
116     private final RaftActorContextImpl context;
117
118     private final DelegatingPersistentDataProvider delegatingPersistenceProvider = new DelegatingPersistentDataProvider(null);
119
120     private final Procedure<Void> createSnapshotProcedure = new CreateSnapshotProcedure();
121
122     private Stopwatch recoveryTimer;
123
124     private int currentRecoveryBatchCount;
125
126     private final BehaviorStateHolder reusableBehaviorStateHolder = new BehaviorStateHolder();
127
128     public RaftActor(String id, Map<String, String> peerAddresses) {
129         this(id, peerAddresses, Optional.<ConfigParams>absent());
130     }
131
132     public RaftActor(String id, Map<String, String> peerAddresses,
133          Optional<ConfigParams> configParams) {
134
135         context = new RaftActorContextImpl(this.getSelf(),
136             this.getContext(), id, new ElectionTermImpl(delegatingPersistenceProvider, id, LOG),
137             -1, -1, peerAddresses,
138             (configParams.isPresent() ? configParams.get(): new DefaultConfigParamsImpl()), LOG);
139
140         context.setReplicatedLog(ReplicatedLogImpl.newInstance(context, delegatingPersistenceProvider, currentBehavior));
141     }
142
143     private void initRecoveryTimer() {
144         if(recoveryTimer == null) {
145             recoveryTimer = Stopwatch.createStarted();
146         }
147     }
148
149     @Override
150     public void preStart() throws Exception {
151         LOG.info("Starting recovery for {} with journal batch size {}", persistenceId(),
152                 context.getConfigParams().getJournalRecoveryLogBatchSize());
153
154         super.preStart();
155     }
156
157     @Override
158     public void postStop() {
159         if(currentBehavior != null) {
160             try {
161                 currentBehavior.close();
162             } catch (Exception e) {
163                 LOG.debug("{}: Error closing behavior {}", persistenceId(), currentBehavior.state());
164             }
165         }
166
167         super.postStop();
168     }
169
170     @Override
171     public void handleRecover(Object message) {
172         if(persistence().isRecoveryApplicable()) {
173             if (message instanceof SnapshotOffer) {
174                 onRecoveredSnapshot((SnapshotOffer) message);
175             } else if (message instanceof ReplicatedLogEntry) {
176                 onRecoveredJournalLogEntry((ReplicatedLogEntry) message);
177             } else if (message instanceof ApplyLogEntries) {
178                 // Handle this message for backwards compatibility with pre-Lithium versions.
179                 onRecoveredApplyLogEntries(((ApplyLogEntries) message).getToIndex());
180             } else if (message instanceof ApplyJournalEntries) {
181                 onRecoveredApplyLogEntries(((ApplyJournalEntries) message).getToIndex());
182             } else if (message instanceof DeleteEntries) {
183                 replicatedLog().removeFrom(((DeleteEntries) message).getFromIndex());
184             } else if (message instanceof UpdateElectionTerm) {
185                 context.getTermInformation().update(((UpdateElectionTerm) message).getCurrentTerm(),
186                         ((UpdateElectionTerm) message).getVotedFor());
187             } else if (message instanceof RecoveryCompleted) {
188                 onRecoveryCompletedMessage();
189             }
190         } else {
191             if (message instanceof RecoveryCompleted) {
192                 // Delete all the messages from the akka journal so that we do not end up with consistency issues
193                 // Note I am not using the dataPersistenceProvider and directly using the akka api here
194                 deleteMessages(lastSequenceNr());
195
196                 // Delete all the akka snapshots as they will not be needed
197                 deleteSnapshots(new SnapshotSelectionCriteria(scala.Long.MaxValue(), scala.Long.MaxValue()));
198
199                 onRecoveryComplete();
200
201                 initializeBehavior();
202             }
203         }
204     }
205
206     private void onRecoveredSnapshot(SnapshotOffer offer) {
207         if(LOG.isDebugEnabled()) {
208             LOG.debug("{}: SnapshotOffer called..", persistenceId());
209         }
210
211         initRecoveryTimer();
212
213         Snapshot snapshot = (Snapshot) offer.snapshot();
214
215         // Create a replicated log with the snapshot information
216         // The replicated log can be used later on to retrieve this snapshot
217         // when we need to install it on a peer
218
219         context.setReplicatedLog(ReplicatedLogImpl.newInstance(snapshot, context, delegatingPersistenceProvider,
220                 currentBehavior));
221         context.setLastApplied(snapshot.getLastAppliedIndex());
222         context.setCommitIndex(snapshot.getLastAppliedIndex());
223
224         Stopwatch timer = Stopwatch.createStarted();
225
226         // Apply the snapshot to the actors state
227         applyRecoverySnapshot(snapshot.getState());
228
229         timer.stop();
230         LOG.info("Recovery snapshot applied for {} in {}: snapshotIndex={}, snapshotTerm={}, journal-size=" +
231                 replicatedLog().size(), persistenceId(), timer.toString(),
232                 replicatedLog().getSnapshotIndex(), replicatedLog().getSnapshotTerm());
233     }
234
235     private void onRecoveredJournalLogEntry(ReplicatedLogEntry logEntry) {
236         if(LOG.isDebugEnabled()) {
237             LOG.debug("{}: Received ReplicatedLogEntry for recovery: {}", persistenceId(), logEntry.getIndex());
238         }
239
240         replicatedLog().append(logEntry);
241     }
242
243     private void onRecoveredApplyLogEntries(long toIndex) {
244         if(LOG.isDebugEnabled()) {
245             LOG.debug("{}: Received ApplyLogEntries for recovery, applying to state: {} to {}",
246                     persistenceId(), context.getLastApplied() + 1, toIndex);
247         }
248
249         for (long i = context.getLastApplied() + 1; i <= toIndex; i++) {
250             batchRecoveredLogEntry(replicatedLog().get(i));
251         }
252
253         context.setLastApplied(toIndex);
254         context.setCommitIndex(toIndex);
255     }
256
257     private void batchRecoveredLogEntry(ReplicatedLogEntry logEntry) {
258         initRecoveryTimer();
259
260         int batchSize = context.getConfigParams().getJournalRecoveryLogBatchSize();
261         if(currentRecoveryBatchCount == 0) {
262             startLogRecoveryBatch(batchSize);
263         }
264
265         appendRecoveredLogEntry(logEntry.getData());
266
267         if(++currentRecoveryBatchCount >= batchSize) {
268             endCurrentLogRecoveryBatch();
269         }
270     }
271
272     private void endCurrentLogRecoveryBatch() {
273         applyCurrentLogRecoveryBatch();
274         currentRecoveryBatchCount = 0;
275     }
276
277     private void onRecoveryCompletedMessage() {
278         if(currentRecoveryBatchCount > 0) {
279             endCurrentLogRecoveryBatch();
280         }
281
282         onRecoveryComplete();
283
284         String recoveryTime = "";
285         if(recoveryTimer != null) {
286             recoveryTimer.stop();
287             recoveryTime = " in " + recoveryTimer.toString();
288             recoveryTimer = null;
289         }
290
291         LOG.info(
292             "Recovery completed" + recoveryTime + " - Switching actor to Follower - " +
293                 "Persistence Id =  " + persistenceId() +
294                 " Last index in log={}, snapshotIndex={}, snapshotTerm={}, " +
295                 "journal-size={}",
296             replicatedLog().lastIndex(), replicatedLog().getSnapshotIndex(),
297             replicatedLog().getSnapshotTerm(), replicatedLog().size());
298
299         initializeBehavior();
300     }
301
302     protected void initializeBehavior(){
303         changeCurrentBehavior(new Follower(context));
304     }
305
306     protected void changeCurrentBehavior(RaftActorBehavior newBehavior){
307         reusableBehaviorStateHolder.init(getCurrentBehavior());
308         setCurrentBehavior(newBehavior);
309         handleBehaviorChange(reusableBehaviorStateHolder, getCurrentBehavior());
310     }
311
312     @Override public void handleCommand(Object message) {
313         if (message instanceof ApplyState){
314             ApplyState applyState = (ApplyState) message;
315
316             long elapsedTime = (System.nanoTime() - applyState.getStartTime());
317             if(elapsedTime >= APPLY_STATE_DELAY_THRESHOLD_IN_NANOS){
318                 LOG.warn("ApplyState took more time than expected. Elapsed Time = {} ms ApplyState = {}",
319                         TimeUnit.NANOSECONDS.toMillis(elapsedTime), applyState);
320             }
321
322             if(LOG.isDebugEnabled()) {
323                 LOG.debug("{}: Applying state for log index {} data {}",
324                     persistenceId(), applyState.getReplicatedLogEntry().getIndex(),
325                     applyState.getReplicatedLogEntry().getData());
326             }
327
328             applyState(applyState.getClientActor(), applyState.getIdentifier(),
329                 applyState.getReplicatedLogEntry().getData());
330
331         } else if (message instanceof ApplyJournalEntries){
332             ApplyJournalEntries applyEntries = (ApplyJournalEntries) message;
333             if(LOG.isDebugEnabled()) {
334                 LOG.debug("{}: Persisting ApplyLogEntries with index={}", persistenceId(), applyEntries.getToIndex());
335             }
336
337             persistence().persist(applyEntries, NoopProcedure.instance());
338
339         } else if(message instanceof ApplySnapshot ) {
340             Snapshot snapshot = ((ApplySnapshot) message).getSnapshot();
341
342             if(LOG.isDebugEnabled()) {
343                 LOG.debug("{}: ApplySnapshot called on Follower Actor " +
344                         "snapshotIndex:{}, snapshotTerm:{}", persistenceId(), snapshot.getLastAppliedIndex(),
345                     snapshot.getLastAppliedTerm()
346                 );
347             }
348
349             applySnapshot(snapshot.getState());
350
351             //clears the followers log, sets the snapshot index to ensure adjusted-index works
352             context.setReplicatedLog(ReplicatedLogImpl.newInstance(snapshot, context, delegatingPersistenceProvider,
353                     currentBehavior));
354             context.setLastApplied(snapshot.getLastAppliedIndex());
355
356         } else if (message instanceof FindLeader) {
357             getSender().tell(
358                 new FindLeaderReply(getLeaderAddress()),
359                 getSelf()
360             );
361
362         } else if (message instanceof SaveSnapshotSuccess) {
363             SaveSnapshotSuccess success = (SaveSnapshotSuccess) message;
364             LOG.info("{}: SaveSnapshotSuccess received for snapshot", persistenceId());
365
366             long sequenceNumber = success.metadata().sequenceNr();
367
368             commitSnapshot(sequenceNumber);
369
370         } else if (message instanceof SaveSnapshotFailure) {
371             SaveSnapshotFailure saveSnapshotFailure = (SaveSnapshotFailure) message;
372
373             LOG.error("{}: SaveSnapshotFailure received for snapshot Cause:",
374                     persistenceId(), saveSnapshotFailure.cause());
375
376             context.getSnapshotManager().rollback();
377
378         } else if (message instanceof CaptureSnapshot) {
379             LOG.debug("{}: CaptureSnapshot received by actor: {}", persistenceId(), message);
380
381             context.getSnapshotManager().create(createSnapshotProcedure);
382
383         } else if (message instanceof CaptureSnapshotReply) {
384             handleCaptureSnapshotReply(((CaptureSnapshotReply) message).getSnapshot());
385         } else if(message instanceof GetOnDemandRaftState) {
386             onGetOnDemandRaftStats();
387         } else if (message.equals(COMMIT_SNAPSHOT)) {
388             commitSnapshot(-1);
389         } else {
390             reusableBehaviorStateHolder.init(getCurrentBehavior());
391
392             setCurrentBehavior(currentBehavior.handleMessage(getSender(), message));
393
394             handleBehaviorChange(reusableBehaviorStateHolder, getCurrentBehavior());
395         }
396     }
397
398     private void onGetOnDemandRaftStats() {
399         // Debugging message to retrieve raft stats.
400
401         OnDemandRaftState.Builder builder = OnDemandRaftState.builder()
402                 .commitIndex(context.getCommitIndex())
403                 .currentTerm(context.getTermInformation().getCurrentTerm())
404                 .inMemoryJournalDataSize(replicatedLog().dataSize())
405                 .inMemoryJournalLogSize(replicatedLog().size())
406                 .isSnapshotCaptureInitiated(context.getSnapshotManager().isCapturing())
407                 .lastApplied(context.getLastApplied())
408                 .lastIndex(replicatedLog().lastIndex())
409                 .lastTerm(replicatedLog().lastTerm())
410                 .leader(getLeaderId())
411                 .raftState(currentBehavior.state().toString())
412                 .replicatedToAllIndex(currentBehavior.getReplicatedToAllIndex())
413                 .snapshotIndex(replicatedLog().getSnapshotIndex())
414                 .snapshotTerm(replicatedLog().getSnapshotTerm())
415                 .votedFor(context.getTermInformation().getVotedFor())
416                 .peerAddresses(ImmutableMap.copyOf(context.getPeerAddresses()));
417
418         ReplicatedLogEntry lastLogEntry = getLastLogEntry();
419         if (lastLogEntry != null) {
420             builder.lastLogIndex(lastLogEntry.getIndex());
421             builder.lastLogTerm(lastLogEntry.getTerm());
422         }
423
424         if(getCurrentBehavior() instanceof AbstractLeader) {
425             AbstractLeader leader = (AbstractLeader)getCurrentBehavior();
426             Collection<String> followerIds = leader.getFollowerIds();
427             List<FollowerInfo> followerInfoList = Lists.newArrayListWithCapacity(followerIds.size());
428             for(String id: followerIds) {
429                 final FollowerLogInformation info = leader.getFollower(id);
430                 followerInfoList.add(new FollowerInfo(id, info.getNextIndex(), info.getMatchIndex(),
431                         info.isFollowerActive(), DurationFormatUtils.formatDurationHMS(info.timeSinceLastActivity())));
432             }
433
434             builder.followerInfoList(followerInfoList);
435         }
436
437         sender().tell(builder.build(), self());
438
439     }
440
441     private void handleBehaviorChange(BehaviorStateHolder oldBehaviorState, RaftActorBehavior currentBehavior) {
442         RaftActorBehavior oldBehavior = oldBehaviorState.getBehavior();
443
444         if (oldBehavior != currentBehavior){
445             onStateChanged();
446         }
447
448         String oldBehaviorLeaderId = oldBehavior == null ? null : oldBehaviorState.getLeaderId();
449         String oldBehaviorStateName = oldBehavior == null ? null : oldBehavior.state().name();
450
451         // it can happen that the state has not changed but the leader has changed.
452         Optional<ActorRef> roleChangeNotifier = getRoleChangeNotifier();
453         if(!Objects.equal(oldBehaviorLeaderId, currentBehavior.getLeaderId())) {
454             if(roleChangeNotifier.isPresent()) {
455                 roleChangeNotifier.get().tell(new LeaderStateChanged(getId(), currentBehavior.getLeaderId()), getSelf());
456             }
457
458             onLeaderChanged(oldBehaviorLeaderId, currentBehavior.getLeaderId());
459         }
460
461         if (roleChangeNotifier.isPresent() &&
462                 (oldBehavior == null || (oldBehavior.state() != currentBehavior.state()))) {
463             roleChangeNotifier.get().tell(new RoleChanged(getId(), oldBehaviorStateName ,
464                     currentBehavior.state().name()), getSelf());
465         }
466     }
467
468     /**
469      * When a derived RaftActor needs to persist something it must call
470      * persistData.
471      *
472      * @param clientActor
473      * @param identifier
474      * @param data
475      */
476     protected void persistData(final ActorRef clientActor, final String identifier,
477         final Payload data) {
478
479         ReplicatedLogEntry replicatedLogEntry = new ReplicatedLogImplEntry(
480             context.getReplicatedLog().lastIndex() + 1,
481             context.getTermInformation().getCurrentTerm(), data);
482
483         if(LOG.isDebugEnabled()) {
484             LOG.debug("{}: Persist data {}", persistenceId(), replicatedLogEntry);
485         }
486
487         final RaftActorContext raftContext = getRaftActorContext();
488
489         replicatedLog().appendAndPersist(replicatedLogEntry, new Procedure<ReplicatedLogEntry>() {
490             @Override
491             public void apply(ReplicatedLogEntry replicatedLogEntry) throws Exception {
492                 if(!hasFollowers()){
493                     // Increment the Commit Index and the Last Applied values
494                     raftContext.setCommitIndex(replicatedLogEntry.getIndex());
495                     raftContext.setLastApplied(replicatedLogEntry.getIndex());
496
497                     // Apply the state immediately
498                     applyState(clientActor, identifier, data);
499
500                     // Send a ApplyJournalEntries message so that we write the fact that we applied
501                     // the state to durable storage
502                     self().tell(new ApplyJournalEntries(replicatedLogEntry.getIndex()), self());
503
504                     context.getSnapshotManager().trimLog(context.getLastApplied(), currentBehavior);
505
506                 } else if (clientActor != null) {
507                     // Send message for replication
508                     currentBehavior.handleMessage(getSelf(),
509                             new Replicate(clientActor, identifier, replicatedLogEntry));
510                 }
511             }
512         });
513     }
514
515     private ReplicatedLog replicatedLog() {
516         return context.getReplicatedLog();
517     }
518
519     protected String getId() {
520         return context.getId();
521     }
522
523     @VisibleForTesting
524     void setCurrentBehavior(RaftActorBehavior behavior) {
525         currentBehavior.setDelegate(behavior);
526     }
527
528     protected RaftActorBehavior getCurrentBehavior() {
529         return currentBehavior.getDelegate();
530     }
531
532     /**
533      * Derived actors can call the isLeader method to check if the current
534      * RaftActor is the Leader or not
535      *
536      * @return true it this RaftActor is a Leader false otherwise
537      */
538     protected boolean isLeader() {
539         return context.getId().equals(currentBehavior.getLeaderId());
540     }
541
542     /**
543      * Derived actor can call getLeader if they need a reference to the Leader.
544      * This would be useful for example in forwarding a request to an actor
545      * which is the leader
546      *
547      * @return A reference to the leader if known, null otherwise
548      */
549     protected ActorSelection getLeader(){
550         String leaderAddress = getLeaderAddress();
551
552         if(leaderAddress == null){
553             return null;
554         }
555
556         return context.actorSelection(leaderAddress);
557     }
558
559     /**
560      *
561      * @return the current leader's id
562      */
563     protected String getLeaderId(){
564         return currentBehavior.getLeaderId();
565     }
566
567     protected RaftState getRaftState() {
568         return currentBehavior.state();
569     }
570
571     protected ReplicatedLogEntry getLastLogEntry() {
572         return replicatedLog().last();
573     }
574
575     protected Long getCurrentTerm(){
576         return context.getTermInformation().getCurrentTerm();
577     }
578
579     protected Long getCommitIndex(){
580         return context.getCommitIndex();
581     }
582
583     protected Long getLastApplied(){
584         return context.getLastApplied();
585     }
586
587     protected RaftActorContext getRaftActorContext() {
588         return context;
589     }
590
591     protected void updateConfigParams(ConfigParams configParams) {
592         context.setConfigParams(configParams);
593     }
594
595     public final DataPersistenceProvider persistence() {
596         return delegatingPersistenceProvider.getDelegate();
597     }
598
599     public void setPersistence(DataPersistenceProvider provider) {
600         delegatingPersistenceProvider.setDelegate(provider);
601     }
602
603     protected void setPersistence(boolean persistent) {
604         if(persistent) {
605             setPersistence(new PersistentDataProvider(this));
606         } else {
607             setPersistence(new NonPersistentDataProvider() {
608                 /**
609                  * The way snapshotting works is,
610                  * <ol>
611                  * <li> RaftActor calls createSnapshot on the Shard
612                  * <li> Shard sends a CaptureSnapshotReply and RaftActor then calls saveSnapshot
613                  * <li> When saveSnapshot is invoked on the akka-persistence API it uses the SnapshotStore to save
614                  * the snapshot. The SnapshotStore sends SaveSnapshotSuccess or SaveSnapshotFailure. When the
615                  * RaftActor gets SaveSnapshot success it commits the snapshot to the in-memory journal. This
616                  * commitSnapshot is mimicking what is done in SaveSnapshotSuccess.
617                  * </ol>
618                  */
619                 @Override
620                 public void saveSnapshot(Object o) {
621                     // Make saving Snapshot successful
622                     // Committing the snapshot here would end up calling commit in the creating state which would
623                     // be a state violation. That's why now we send a message to commit the snapshot.
624                     self().tell(COMMIT_SNAPSHOT, self());
625                 }
626             });
627         }
628     }
629
630     /**
631      * setPeerAddress sets the address of a known peer at a later time.
632      * <p>
633      * This is to account for situations where a we know that a peer
634      * exists but we do not know an address up-front. This may also be used in
635      * situations where a known peer starts off in a different location and we
636      * need to change it's address
637      * <p>
638      * Note that if the peerId does not match the list of peers passed to
639      * this actor during construction an IllegalStateException will be thrown.
640      *
641      * @param peerId
642      * @param peerAddress
643      */
644     protected void setPeerAddress(String peerId, String peerAddress){
645         context.setPeerAddress(peerId, peerAddress);
646     }
647
648     protected void commitSnapshot(long sequenceNumber) {
649         context.getSnapshotManager().commit(persistence(), sequenceNumber);
650     }
651
652     /**
653      * The applyState method will be called by the RaftActor when some data
654      * needs to be applied to the actor's state
655      *
656      * @param clientActor A reference to the client who sent this message. This
657      *                    is the same reference that was passed to persistData
658      *                    by the derived actor. clientActor may be null when
659      *                    the RaftActor is behaving as a follower or during
660      *                    recovery.
661      * @param identifier  The identifier of the persisted data. This is also
662      *                    the same identifier that was passed to persistData by
663      *                    the derived actor. identifier may be null when
664      *                    the RaftActor is behaving as a follower or during
665      *                    recovery
666      * @param data        A piece of data that was persisted by the persistData call.
667      *                    This should NEVER be null.
668      */
669     protected abstract void applyState(ActorRef clientActor, String identifier,
670         Object data);
671
672     /**
673      * This method is called during recovery at the start of a batch of state entries. Derived
674      * classes should perform any initialization needed to start a batch.
675      */
676     protected abstract void startLogRecoveryBatch(int maxBatchSize);
677
678     /**
679      * This method is called during recovery to append state data to the current batch. This method
680      * is called 1 or more times after {@link #startLogRecoveryBatch}.
681      *
682      * @param data the state data
683      */
684     protected abstract void appendRecoveredLogEntry(Payload data);
685
686     /**
687      * This method is called during recovery to reconstruct the state of the actor.
688      *
689      * @param snapshotBytes A snapshot of the state of the actor
690      */
691     protected abstract void applyRecoverySnapshot(byte[] snapshotBytes);
692
693     /**
694      * This method is called during recovery at the end of a batch to apply the current batched
695      * log entries. This method is called after {@link #appendRecoveredLogEntry}.
696      */
697     protected abstract void applyCurrentLogRecoveryBatch();
698
699     /**
700      * This method is called when recovery is complete.
701      */
702     protected abstract void onRecoveryComplete();
703
704     /**
705      * This method will be called by the RaftActor when a snapshot needs to be
706      * created. The derived actor should respond with its current state.
707      * <p/>
708      * During recovery the state that is returned by the derived actor will
709      * be passed back to it by calling the applySnapshot  method
710      *
711      * @return The current state of the actor
712      */
713     protected abstract void createSnapshot();
714
715     /**
716      * This method can be called at any other point during normal
717      * operations when the derived actor is out of sync with it's peers
718      * and the only way to bring it in sync is by applying a snapshot
719      *
720      * @param snapshotBytes A snapshot of the state of the actor
721      */
722     protected abstract void applySnapshot(byte[] snapshotBytes);
723
724     /**
725      * This method will be called by the RaftActor when the state of the
726      * RaftActor changes. The derived actor can then use methods like
727      * isLeader or getLeader to do something useful
728      */
729     protected abstract void onStateChanged();
730
731     /**
732      * Notifier Actor for this RaftActor to notify when a role change happens
733      * @return ActorRef - ActorRef of the notifier or Optional.absent if none.
734      */
735     protected abstract Optional<ActorRef> getRoleChangeNotifier();
736
737     protected void onLeaderChanged(String oldLeader, String newLeader){};
738
739     private String getLeaderAddress(){
740         if(isLeader()){
741             return getSelf().path().toString();
742         }
743         String leaderId = currentBehavior.getLeaderId();
744         if (leaderId == null) {
745             return null;
746         }
747         String peerAddress = context.getPeerAddress(leaderId);
748         if(LOG.isDebugEnabled()) {
749             LOG.debug("{}: getLeaderAddress leaderId = {} peerAddress = {}",
750                     persistenceId(), leaderId, peerAddress);
751         }
752
753         return peerAddress;
754     }
755
756     private void handleCaptureSnapshotReply(byte[] snapshotBytes) {
757         LOG.debug("{}: CaptureSnapshotReply received by actor: snapshot size {}", persistenceId(), snapshotBytes.length);
758
759         context.getSnapshotManager().persist(persistence(), snapshotBytes, currentBehavior, context.getTotalMemory());
760     }
761
762     protected boolean hasFollowers(){
763         return getRaftActorContext().hasFollowers();
764     }
765
766     static class DeleteEntries implements Serializable {
767         private static final long serialVersionUID = 1L;
768         private final int fromIndex;
769
770         public DeleteEntries(int fromIndex) {
771             this.fromIndex = fromIndex;
772         }
773
774         public int getFromIndex() {
775             return fromIndex;
776         }
777     }
778
779     static class UpdateElectionTerm implements Serializable {
780         private static final long serialVersionUID = 1L;
781         private final long currentTerm;
782         private final String votedFor;
783
784         public UpdateElectionTerm(long currentTerm, String votedFor) {
785             this.currentTerm = currentTerm;
786             this.votedFor = votedFor;
787         }
788
789         public long getCurrentTerm() {
790             return currentTerm;
791         }
792
793         public String getVotedFor() {
794             return votedFor;
795         }
796     }
797
798     private class CreateSnapshotProcedure implements Procedure<Void> {
799
800         @Override
801         public void apply(Void aVoid) throws Exception {
802             createSnapshot();
803         }
804     }
805
806     private static class BehaviorStateHolder {
807         private RaftActorBehavior behavior;
808         private String leaderId;
809
810         void init(RaftActorBehavior behavior) {
811             this.behavior = behavior;
812             this.leaderId = behavior != null ? behavior.getLeaderId() : null;
813         }
814
815         RaftActorBehavior getBehavior() {
816             return behavior;
817         }
818
819         String getLeaderId() {
820             return leaderId;
821         }
822     }
823 }