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