Bug 4564: Implement GetSnapshot message in RaftActor
[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  * Copyright (c) 2015 Brocade Communications Systems, Inc. and others.  All rights reserved.
4  *
5  * This program and the accompanying materials are made available under the
6  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
7  * and is available at http://www.eclipse.org/legal/epl-v10.html
8  */
9
10 package org.opendaylight.controller.cluster.raft;
11
12 import akka.actor.ActorRef;
13 import akka.actor.ActorSelection;
14 import akka.japi.Procedure;
15 import com.google.common.annotations.VisibleForTesting;
16 import com.google.common.base.Objects;
17 import com.google.common.base.Optional;
18 import com.google.common.base.Supplier;
19 import com.google.common.collect.Lists;
20 import java.io.Serializable;
21 import java.util.Collection;
22 import java.util.HashMap;
23 import java.util.List;
24 import java.util.Map;
25 import java.util.concurrent.TimeUnit;
26 import javax.annotation.Nonnull;
27 import org.apache.commons.lang3.time.DurationFormatUtils;
28 import org.opendaylight.controller.cluster.DataPersistenceProvider;
29 import org.opendaylight.controller.cluster.DelegatingPersistentDataProvider;
30 import org.opendaylight.controller.cluster.NonPersistentDataProvider;
31 import org.opendaylight.controller.cluster.PersistentDataProvider;
32 import org.opendaylight.controller.cluster.common.actor.AbstractUntypedPersistentActor;
33 import org.opendaylight.controller.cluster.notifications.LeaderStateChanged;
34 import org.opendaylight.controller.cluster.notifications.RoleChanged;
35 import org.opendaylight.controller.cluster.raft.base.messages.ApplyJournalEntries;
36 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
37 import org.opendaylight.controller.cluster.raft.base.messages.InitiateCaptureSnapshot;
38 import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
39 import org.opendaylight.controller.cluster.raft.base.messages.SwitchBehavior;
40 import org.opendaylight.controller.cluster.raft.behaviors.AbstractLeader;
41 import org.opendaylight.controller.cluster.raft.behaviors.DelegatingRaftActorBehavior;
42 import org.opendaylight.controller.cluster.raft.behaviors.Follower;
43 import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior;
44 import org.opendaylight.controller.cluster.raft.client.messages.FindLeader;
45 import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply;
46 import org.opendaylight.controller.cluster.raft.client.messages.FollowerInfo;
47 import org.opendaylight.controller.cluster.raft.client.messages.GetOnDemandRaftState;
48 import org.opendaylight.controller.cluster.raft.client.messages.OnDemandRaftState;
49 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
50 import org.slf4j.Logger;
51 import org.slf4j.LoggerFactory;
52
53 /**
54  * RaftActor encapsulates a state machine that needs to be kept synchronized
55  * in a cluster. It implements the RAFT algorithm as described in the paper
56  * <a href='https://ramcloud.stanford.edu/wiki/download/attachments/11370504/raft.pdf'>
57  * In Search of an Understandable Consensus Algorithm</a>
58  * <p/>
59  * RaftActor has 3 states and each state has a certain behavior associated
60  * with it. A Raft actor can behave as,
61  * <ul>
62  * <li> A Leader </li>
63  * <li> A Follower (or) </li>
64  * <li> A Candidate </li>
65  * </ul>
66  * <p/>
67  * <p/>
68  * A RaftActor MUST be a Leader in order to accept requests from clients to
69  * change the state of it's encapsulated state machine. Once a RaftActor becomes
70  * a Leader it is also responsible for ensuring that all followers ultimately
71  * have the same log and therefore the same state machine as itself.
72  * <p/>
73  * <p/>
74  * The current behavior of a RaftActor determines how election for leadership
75  * is initiated and how peer RaftActors react to request for votes.
76  * <p/>
77  * <p/>
78  * Each RaftActor also needs to know the current election term. It uses this
79  * information for a couple of things. One is to simply figure out who it
80  * voted for in the last election. Another is to figure out if the message
81  * it received to update it's state is stale.
82  * <p/>
83  * <p/>
84  * The RaftActor uses akka-persistence to store it's replicated log.
85  * Furthermore through it's behaviors a Raft Actor determines
86  * <p/>
87  * <ul>
88  * <li> when a log entry should be persisted </li>
89  * <li> when a log entry should be applied to the state machine (and) </li>
90  * <li> when a snapshot should be saved </li>
91  * </ul>
92  */
93 public abstract class RaftActor extends AbstractUntypedPersistentActor {
94
95     private static final long APPLY_STATE_DELAY_THRESHOLD_IN_NANOS = TimeUnit.MILLISECONDS.toNanos(50L); // 50 millis
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 final DelegatingRaftActorBehavior currentBehavior = new DelegatingRaftActorBehavior();
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     private final DelegatingPersistentDataProvider delegatingPersistenceProvider;
112
113     private final PersistentDataProvider persistentProvider;
114
115     private RaftActorRecoverySupport raftRecovery;
116
117     private RaftActorSnapshotMessageSupport snapshotSupport;
118
119     private final BehaviorStateHolder reusableBehaviorStateHolder = new BehaviorStateHolder();
120
121     private final SwitchBehaviorSupplier reusableSwitchBehaviorSupplier = new SwitchBehaviorSupplier();
122
123     private RaftActorServerConfigurationSupport serverConfigurationSupport;
124
125     public RaftActor(String id, Map<String, String> peerAddresses,
126          Optional<ConfigParams> configParams, short payloadVersion) {
127
128         persistentProvider = new PersistentDataProvider(this);
129         delegatingPersistenceProvider = new RaftActorDelegatingPersistentDataProvider(null, persistentProvider);
130
131         context = new RaftActorContextImpl(this.getSelf(),
132             this.getContext(), id, new ElectionTermImpl(persistentProvider, id, LOG),
133             -1, -1, peerAddresses,
134             (configParams.isPresent() ? configParams.get(): new DefaultConfigParamsImpl()),
135             delegatingPersistenceProvider, LOG);
136
137         context.setPayloadVersion(payloadVersion);
138         context.setReplicatedLog(ReplicatedLogImpl.newInstance(context, currentBehavior));
139     }
140
141     @Override
142     public void preStart() throws Exception {
143         LOG.info("Starting recovery for {} with journal batch size {}", persistenceId(),
144                 context.getConfigParams().getJournalRecoveryLogBatchSize());
145
146         super.preStart();
147
148         snapshotSupport = newRaftActorSnapshotMessageSupport();
149         serverConfigurationSupport = new RaftActorServerConfigurationSupport(getRaftActorContext());
150     }
151
152     @Override
153     public void postStop() {
154         if(currentBehavior.getDelegate() != null) {
155             try {
156                 currentBehavior.close();
157             } catch (Exception e) {
158                 LOG.debug("{}: Error closing behavior {}", persistenceId(), currentBehavior.state());
159             }
160         }
161
162         super.postStop();
163     }
164
165     @Override
166     public void handleRecover(Object message) {
167         if(raftRecovery == null) {
168             raftRecovery = newRaftActorRecoverySupport();
169         }
170
171         boolean recoveryComplete = raftRecovery.handleRecoveryMessage(message, persistentProvider);
172         if(recoveryComplete) {
173             onRecoveryComplete();
174
175             initializeBehavior();
176
177             raftRecovery = null;
178
179             if (context.getReplicatedLog().size() > 0) {
180                 self().tell(new InitiateCaptureSnapshot(), self());
181                 LOG.info("Snapshot capture initiated after recovery");
182             } else {
183                 LOG.info("Snapshot capture NOT initiated after recovery, journal empty");
184             }
185         }
186     }
187
188     protected RaftActorRecoverySupport newRaftActorRecoverySupport() {
189         return new RaftActorRecoverySupport(context, currentBehavior, getRaftActorRecoveryCohort());
190     }
191
192     protected void initializeBehavior(){
193         changeCurrentBehavior(new Follower(context));
194     }
195
196     protected void changeCurrentBehavior(RaftActorBehavior newBehavior){
197         reusableBehaviorStateHolder.init(getCurrentBehavior());
198         setCurrentBehavior(newBehavior);
199         handleBehaviorChange(reusableBehaviorStateHolder, getCurrentBehavior());
200     }
201
202     @Override
203     public void handleCommand(final Object message) {
204         if(serverConfigurationSupport.handleMessage(message, this, getSender())) {
205             return;
206         } else if (message instanceof ApplyState){
207             ApplyState applyState = (ApplyState) message;
208
209             long elapsedTime = (System.nanoTime() - applyState.getStartTime());
210             if(elapsedTime >= APPLY_STATE_DELAY_THRESHOLD_IN_NANOS){
211                 LOG.warn("ApplyState took more time than expected. Elapsed Time = {} ms ApplyState = {}",
212                         TimeUnit.NANOSECONDS.toMillis(elapsedTime), applyState);
213             }
214
215             if(LOG.isDebugEnabled()) {
216                 LOG.debug("{}: Applying state for log index {} data {}",
217                     persistenceId(), applyState.getReplicatedLogEntry().getIndex(),
218                     applyState.getReplicatedLogEntry().getData());
219             }
220
221             applyState(applyState.getClientActor(), applyState.getIdentifier(),
222                 applyState.getReplicatedLogEntry().getData());
223
224             if (!hasFollowers()) {
225                 // for single node, the capture should happen after the apply state
226                 // as we delete messages from the persistent journal which have made it to the snapshot
227                 // capturing the snapshot before applying makes the persistent journal and snapshot out of sync
228                 // and recovery shows data missing
229                 context.getReplicatedLog().captureSnapshotIfReady(applyState.getReplicatedLogEntry());
230
231                 context.getSnapshotManager().trimLog(context.getLastApplied(), currentBehavior);
232             }
233
234         } else if (message instanceof ApplyJournalEntries){
235             ApplyJournalEntries applyEntries = (ApplyJournalEntries) message;
236             if(LOG.isDebugEnabled()) {
237                 LOG.debug("{}: Persisting ApplyLogEntries with index={}", persistenceId(), applyEntries.getToIndex());
238             }
239
240             persistence().persist(applyEntries, NoopProcedure.instance());
241
242         } else if (message instanceof FindLeader) {
243             getSender().tell(
244                 new FindLeaderReply(getLeaderAddress()),
245                 getSelf()
246             );
247         } else if(message instanceof GetOnDemandRaftState) {
248             onGetOnDemandRaftStats();
249         } else if(message instanceof InitiateCaptureSnapshot) {
250             captureSnapshot();
251         } else if(message instanceof SwitchBehavior){
252             switchBehavior(((SwitchBehavior) message));
253         } else if(!snapshotSupport.handleSnapshotMessage(message, getSender())) {
254             switchBehavior(reusableSwitchBehaviorSupplier.handleMessage(getSender(), message));
255         }
256     }
257
258     private void switchBehavior(SwitchBehavior message) {
259         if(!getRaftActorContext().getRaftPolicy().automaticElectionsEnabled()) {
260             RaftState newState = message.getNewState();
261             if( newState == RaftState.Leader || newState == RaftState.Follower) {
262                 switchBehavior(reusableSwitchBehaviorSupplier.handleMessage(getSender(), message));
263                 getRaftActorContext().getTermInformation().updateAndPersist(message.getNewTerm(), "");
264             } else {
265                 LOG.warn("Switching to behavior : {} - not supported", newState);
266             }
267         }
268     }
269
270     private void switchBehavior(Supplier<RaftActorBehavior> supplier){
271         reusableBehaviorStateHolder.init(getCurrentBehavior());
272
273         setCurrentBehavior(supplier.get());
274
275         handleBehaviorChange(reusableBehaviorStateHolder, getCurrentBehavior());
276     }
277
278     protected RaftActorSnapshotMessageSupport newRaftActorSnapshotMessageSupport() {
279         return new RaftActorSnapshotMessageSupport(context, currentBehavior,
280                 getRaftActorSnapshotCohort());
281     }
282
283     private void onGetOnDemandRaftStats() {
284         // Debugging message to retrieve raft stats.
285
286         Map<String, String> peerAddresses = new HashMap<>();
287         for(String peerId: context.getPeerIds()) {
288             peerAddresses.put(peerId, context.getPeerAddress(peerId));
289         }
290
291         OnDemandRaftState.Builder builder = OnDemandRaftState.builder()
292                 .commitIndex(context.getCommitIndex())
293                 .currentTerm(context.getTermInformation().getCurrentTerm())
294                 .inMemoryJournalDataSize(replicatedLog().dataSize())
295                 .inMemoryJournalLogSize(replicatedLog().size())
296                 .isSnapshotCaptureInitiated(context.getSnapshotManager().isCapturing())
297                 .lastApplied(context.getLastApplied())
298                 .lastIndex(replicatedLog().lastIndex())
299                 .lastTerm(replicatedLog().lastTerm())
300                 .leader(getLeaderId())
301                 .raftState(currentBehavior.state().toString())
302                 .replicatedToAllIndex(currentBehavior.getReplicatedToAllIndex())
303                 .snapshotIndex(replicatedLog().getSnapshotIndex())
304                 .snapshotTerm(replicatedLog().getSnapshotTerm())
305                 .votedFor(context.getTermInformation().getVotedFor())
306                 .peerAddresses(peerAddresses);
307
308         ReplicatedLogEntry lastLogEntry = getLastLogEntry();
309         if (lastLogEntry != null) {
310             builder.lastLogIndex(lastLogEntry.getIndex());
311             builder.lastLogTerm(lastLogEntry.getTerm());
312         }
313
314         if(getCurrentBehavior() instanceof AbstractLeader) {
315             AbstractLeader leader = (AbstractLeader)getCurrentBehavior();
316             Collection<String> followerIds = leader.getFollowerIds();
317             List<FollowerInfo> followerInfoList = Lists.newArrayListWithCapacity(followerIds.size());
318             for(String id: followerIds) {
319                 final FollowerLogInformation info = leader.getFollower(id);
320                 followerInfoList.add(new FollowerInfo(id, info.getNextIndex(), info.getMatchIndex(),
321                         info.isFollowerActive(), DurationFormatUtils.formatDurationHMS(info.timeSinceLastActivity())));
322             }
323
324             builder.followerInfoList(followerInfoList);
325         }
326
327         sender().tell(builder.build(), self());
328
329     }
330
331     private void handleBehaviorChange(BehaviorStateHolder oldBehaviorState, RaftActorBehavior currentBehavior) {
332         RaftActorBehavior oldBehavior = oldBehaviorState.getBehavior();
333
334         if (oldBehavior != currentBehavior){
335             onStateChanged();
336         }
337
338         String lastValidLeaderId = oldBehavior == null ? null : oldBehaviorState.getLastValidLeaderId();
339         String oldBehaviorStateName = oldBehavior == null ? null : oldBehavior.state().name();
340
341         // it can happen that the state has not changed but the leader has changed.
342         Optional<ActorRef> roleChangeNotifier = getRoleChangeNotifier();
343         if(!Objects.equal(lastValidLeaderId, currentBehavior.getLeaderId()) ||
344            oldBehaviorState.getLeaderPayloadVersion() != currentBehavior.getLeaderPayloadVersion()) {
345             if(roleChangeNotifier.isPresent()) {
346                 roleChangeNotifier.get().tell(newLeaderStateChanged(getId(), currentBehavior.getLeaderId(),
347                         currentBehavior.getLeaderPayloadVersion()), getSelf());
348             }
349
350             onLeaderChanged(lastValidLeaderId, currentBehavior.getLeaderId());
351         }
352
353         if (roleChangeNotifier.isPresent() &&
354                 (oldBehavior == null || (oldBehavior.state() != currentBehavior.state()))) {
355             roleChangeNotifier.get().tell(new RoleChanged(getId(), oldBehaviorStateName ,
356                     currentBehavior.state().name()), getSelf());
357         }
358     }
359
360     protected LeaderStateChanged newLeaderStateChanged(String memberId, String leaderId, short leaderPayloadVersion) {
361         return new LeaderStateChanged(memberId, leaderId, leaderPayloadVersion);
362     }
363
364     @Override
365     public long snapshotSequenceNr() {
366         // When we do a snapshot capture, we also capture and save the sequence-number of the persistent journal,
367         // so that we can delete the persistent journal based on the saved sequence-number
368         // However , when akka replays the journal during recovery, it replays it from the sequence number when the snapshot
369         // was saved and not the number we saved.
370         // We would want to override it , by asking akka to use the last-sequence number known to us.
371         return context.getSnapshotManager().getLastSequenceNumber();
372     }
373
374     /**
375      * When a derived RaftActor needs to persist something it must call
376      * persistData.
377      *
378      * @param clientActor
379      * @param identifier
380      * @param data
381      */
382     protected void persistData(final ActorRef clientActor, final String identifier,
383         final Payload data) {
384
385         ReplicatedLogEntry replicatedLogEntry = new ReplicatedLogImplEntry(
386             context.getReplicatedLog().lastIndex() + 1,
387             context.getTermInformation().getCurrentTerm(), data);
388
389         if(LOG.isDebugEnabled()) {
390             LOG.debug("{}: Persist data {}", persistenceId(), replicatedLogEntry);
391         }
392
393         final RaftActorContext raftContext = getRaftActorContext();
394
395         replicatedLog().appendAndPersist(replicatedLogEntry, new Procedure<ReplicatedLogEntry>() {
396             @Override
397             public void apply(ReplicatedLogEntry replicatedLogEntry) {
398                 if (!hasFollowers()){
399                     // Increment the Commit Index and the Last Applied values
400                     raftContext.setCommitIndex(replicatedLogEntry.getIndex());
401                     raftContext.setLastApplied(replicatedLogEntry.getIndex());
402
403                     // Apply the state immediately.
404                     self().tell(new ApplyState(clientActor, identifier, replicatedLogEntry), self());
405
406                     // Send a ApplyJournalEntries message so that we write the fact that we applied
407                     // the state to durable storage
408                     self().tell(new ApplyJournalEntries(replicatedLogEntry.getIndex()), self());
409
410                 } else if (clientActor != null) {
411                     context.getReplicatedLog().captureSnapshotIfReady(replicatedLogEntry);
412
413                     // Send message for replication
414                     currentBehavior.handleMessage(getSelf(),
415                             new Replicate(clientActor, identifier, replicatedLogEntry));
416                 }
417             }
418         });
419     }
420
421     private ReplicatedLog replicatedLog() {
422         return context.getReplicatedLog();
423     }
424
425     protected String getId() {
426         return context.getId();
427     }
428
429     @VisibleForTesting
430     void setCurrentBehavior(RaftActorBehavior behavior) {
431         currentBehavior.setDelegate(behavior);
432     }
433
434     protected RaftActorBehavior getCurrentBehavior() {
435         return currentBehavior.getDelegate();
436     }
437
438     /**
439      * Derived actors can call the isLeader method to check if the current
440      * RaftActor is the Leader or not
441      *
442      * @return true it this RaftActor is a Leader false otherwise
443      */
444     protected boolean isLeader() {
445         return context.getId().equals(currentBehavior.getLeaderId());
446     }
447
448     /**
449      * Derived actor can call getLeader if they need a reference to the Leader.
450      * This would be useful for example in forwarding a request to an actor
451      * which is the leader
452      *
453      * @return A reference to the leader if known, null otherwise
454      */
455     protected ActorSelection getLeader(){
456         String leaderAddress = getLeaderAddress();
457
458         if(leaderAddress == null){
459             return null;
460         }
461
462         return context.actorSelection(leaderAddress);
463     }
464
465     /**
466      *
467      * @return the current leader's id
468      */
469     protected String getLeaderId(){
470         return currentBehavior.getLeaderId();
471     }
472
473     protected RaftState getRaftState() {
474         return currentBehavior.state();
475     }
476
477     protected ReplicatedLogEntry getLastLogEntry() {
478         return replicatedLog().last();
479     }
480
481     protected Long getCurrentTerm(){
482         return context.getTermInformation().getCurrentTerm();
483     }
484
485     protected Long getCommitIndex(){
486         return context.getCommitIndex();
487     }
488
489     protected Long getLastApplied(){
490         return context.getLastApplied();
491     }
492
493     protected RaftActorContext getRaftActorContext() {
494         return context;
495     }
496
497     protected void updateConfigParams(ConfigParams configParams) {
498
499         // obtain the RaftPolicy for oldConfigParams and the updated one.
500         String oldRaftPolicy = context.getConfigParams().
501             getCustomRaftPolicyImplementationClass();
502         String newRaftPolicy = configParams.
503             getCustomRaftPolicyImplementationClass();
504
505         LOG.debug ("RaftPolicy used with prev.config {}, RaftPolicy used with newConfig {}",
506             oldRaftPolicy, newRaftPolicy);
507         context.setConfigParams(configParams);
508         if (!Objects.equal(oldRaftPolicy, newRaftPolicy)) {
509             //RaftPolicy is modifed for the Actor. Re-initialize its current behaviour
510             initializeBehavior();
511         }
512     }
513
514     public final DataPersistenceProvider persistence() {
515         return delegatingPersistenceProvider.getDelegate();
516     }
517
518     public void setPersistence(DataPersistenceProvider provider) {
519         delegatingPersistenceProvider.setDelegate(provider);
520     }
521
522     protected void setPersistence(boolean persistent) {
523         if(persistent) {
524             setPersistence(new PersistentDataProvider(this));
525         } else {
526             setPersistence(new NonPersistentDataProvider() {
527                 /**
528                  * The way snapshotting works is,
529                  * <ol>
530                  * <li> RaftActor calls createSnapshot on the Shard
531                  * <li> Shard sends a CaptureSnapshotReply and RaftActor then calls saveSnapshot
532                  * <li> When saveSnapshot is invoked on the akka-persistence API it uses the SnapshotStore to save
533                  * the snapshot. The SnapshotStore sends SaveSnapshotSuccess or SaveSnapshotFailure. When the
534                  * RaftActor gets SaveSnapshot success it commits the snapshot to the in-memory journal. This
535                  * commitSnapshot is mimicking what is done in SaveSnapshotSuccess.
536                  * </ol>
537                  */
538                 @Override
539                 public void saveSnapshot(Object o) {
540                     // Make saving Snapshot successful
541                     // Committing the snapshot here would end up calling commit in the creating state which would
542                     // be a state violation. That's why now we send a message to commit the snapshot.
543                     self().tell(RaftActorSnapshotMessageSupport.COMMIT_SNAPSHOT, self());
544                 }
545             });
546         }
547     }
548
549     /**
550      * setPeerAddress sets the address of a known peer at a later time.
551      * <p>
552      * This is to account for situations where a we know that a peer
553      * exists but we do not know an address up-front. This may also be used in
554      * situations where a known peer starts off in a different location and we
555      * need to change it's address
556      * <p>
557      * Note that if the peerId does not match the list of peers passed to
558      * this actor during construction an IllegalStateException will be thrown.
559      *
560      * @param peerId
561      * @param peerAddress
562      */
563     protected void setPeerAddress(String peerId, String peerAddress){
564         context.setPeerAddress(peerId, peerAddress);
565     }
566
567     /**
568      * The applyState method will be called by the RaftActor when some data
569      * needs to be applied to the actor's state
570      *
571      * @param clientActor A reference to the client who sent this message. This
572      *                    is the same reference that was passed to persistData
573      *                    by the derived actor. clientActor may be null when
574      *                    the RaftActor is behaving as a follower or during
575      *                    recovery.
576      * @param identifier  The identifier of the persisted data. This is also
577      *                    the same identifier that was passed to persistData by
578      *                    the derived actor. identifier may be null when
579      *                    the RaftActor is behaving as a follower or during
580      *                    recovery
581      * @param data        A piece of data that was persisted by the persistData call.
582      *                    This should NEVER be null.
583      */
584     protected abstract void applyState(ActorRef clientActor, String identifier,
585         Object data);
586
587     /**
588      * Returns the RaftActorRecoveryCohort to participate in persistence recovery.
589      */
590     @Nonnull
591     protected abstract RaftActorRecoveryCohort getRaftActorRecoveryCohort();
592
593     /**
594      * This method is called when recovery is complete.
595      */
596     protected abstract void onRecoveryComplete();
597
598     /**
599      * Returns the RaftActorSnapshotCohort to participate in persistence recovery.
600      */
601     @Nonnull
602     protected abstract RaftActorSnapshotCohort getRaftActorSnapshotCohort();
603
604     /**
605      * This method will be called by the RaftActor when the state of the
606      * RaftActor changes. The derived actor can then use methods like
607      * isLeader or getLeader to do something useful
608      */
609     protected abstract void onStateChanged();
610
611     /**
612      * Notifier Actor for this RaftActor to notify when a role change happens
613      * @return ActorRef - ActorRef of the notifier or Optional.absent if none.
614      */
615     protected abstract Optional<ActorRef> getRoleChangeNotifier();
616
617     protected void onLeaderChanged(String oldLeader, String newLeader){};
618
619     private String getLeaderAddress(){
620         if(isLeader()){
621             return getSelf().path().toString();
622         }
623         String leaderId = currentBehavior.getLeaderId();
624         if (leaderId == null) {
625             return null;
626         }
627         String peerAddress = context.getPeerAddress(leaderId);
628         if(LOG.isDebugEnabled()) {
629             LOG.debug("{}: getLeaderAddress leaderId = {} peerAddress = {}",
630                     persistenceId(), leaderId, peerAddress);
631         }
632
633         return peerAddress;
634     }
635
636     protected boolean hasFollowers(){
637         return getRaftActorContext().hasFollowers();
638     }
639
640     private void captureSnapshot() {
641         SnapshotManager snapshotManager = context.getSnapshotManager();
642
643         if(!snapshotManager.isCapturing()) {
644             LOG.debug("Take a snapshot of current state. lastReplicatedLog is {} and replicatedToAllIndex is {}",
645                 replicatedLog().last(), currentBehavior.getReplicatedToAllIndex());
646
647             snapshotManager.capture(replicatedLog().last(), currentBehavior.getReplicatedToAllIndex());
648         }
649     }
650
651     /**
652      * @deprecated Deprecated in favor of {@link org.opendaylight.controller.cluster.raft.base.messages.DeleteEntries}
653      *             whose type for fromIndex is long instead of int. This class was kept for backwards
654      *             compatibility with Helium.
655      */
656     // Suppressing this warning as we can't set serialVersionUID to maintain backwards compatibility.
657     @SuppressWarnings("serial")
658     @Deprecated
659     static class DeleteEntries implements Serializable {
660         private final int fromIndex;
661
662         public DeleteEntries(int fromIndex) {
663             this.fromIndex = fromIndex;
664         }
665
666         public int getFromIndex() {
667             return fromIndex;
668         }
669     }
670
671     /**
672      * @deprecated Deprecated in favor of non-inner class {@link org.opendaylight.controller.cluster.raft.base.messages.UpdateElectionTerm}
673      *             which has serialVersionUID set. This class was kept for backwards compatibility with Helium.
674      */
675     // Suppressing this warning as we can't set serialVersionUID to maintain backwards compatibility.
676     @SuppressWarnings("serial")
677     @Deprecated
678     static class UpdateElectionTerm implements Serializable {
679         private final long currentTerm;
680         private final String votedFor;
681
682         public UpdateElectionTerm(long currentTerm, String votedFor) {
683             this.currentTerm = currentTerm;
684             this.votedFor = votedFor;
685         }
686
687         public long getCurrentTerm() {
688             return currentTerm;
689         }
690
691         public String getVotedFor() {
692             return votedFor;
693         }
694     }
695
696     private static class BehaviorStateHolder {
697         private RaftActorBehavior behavior;
698         private String lastValidLeaderId;
699         private short leaderPayloadVersion;
700
701         void init(RaftActorBehavior behavior) {
702             this.behavior = behavior;
703             this.leaderPayloadVersion = behavior != null ? behavior.getLeaderPayloadVersion() : -1;
704
705             String behaviorLeaderId = behavior != null ? behavior.getLeaderId() : null;
706             if(behaviorLeaderId != null) {
707                 this.lastValidLeaderId = behaviorLeaderId;
708             }
709         }
710
711         RaftActorBehavior getBehavior() {
712             return behavior;
713         }
714
715         String getLastValidLeaderId() {
716             return lastValidLeaderId;
717         }
718
719         short getLeaderPayloadVersion() {
720             return leaderPayloadVersion;
721         }
722     }
723
724     private class SwitchBehaviorSupplier implements Supplier<RaftActorBehavior> {
725         private Object message;
726         private ActorRef sender;
727
728         public SwitchBehaviorSupplier handleMessage(ActorRef sender, Object message){
729             this.sender = sender;
730             this.message = message;
731             return this;
732         }
733
734         @Override
735         public RaftActorBehavior get() {
736             if(this.message instanceof SwitchBehavior){
737                 return ((SwitchBehavior) message).getNewState().createBehavior(getRaftActorContext());
738             }
739             return currentBehavior.handleMessage(sender, message);
740         }
741     }
742 }