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