Improve segmented journal actor metrics
[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 package org.opendaylight.controller.cluster.raft;
10
11 import static com.google.common.base.Verify.verify;
12 import static java.util.Objects.requireNonNull;
13
14 import akka.actor.ActorRef;
15 import akka.actor.ActorSelection;
16 import akka.actor.PoisonPill;
17 import akka.actor.Status;
18 import akka.persistence.JournalProtocol;
19 import akka.persistence.SnapshotProtocol;
20 import com.google.common.annotations.VisibleForTesting;
21 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
22 import java.util.ArrayList;
23 import java.util.Collection;
24 import java.util.HashMap;
25 import java.util.List;
26 import java.util.Map;
27 import java.util.Objects;
28 import java.util.Optional;
29 import java.util.concurrent.TimeUnit;
30 import org.apache.commons.lang3.time.DurationFormatUtils;
31 import org.eclipse.jdt.annotation.NonNull;
32 import org.eclipse.jdt.annotation.Nullable;
33 import org.opendaylight.controller.cluster.DataPersistenceProvider;
34 import org.opendaylight.controller.cluster.DelegatingPersistentDataProvider;
35 import org.opendaylight.controller.cluster.NonPersistentDataProvider;
36 import org.opendaylight.controller.cluster.PersistentDataProvider;
37 import org.opendaylight.controller.cluster.common.actor.AbstractUntypedPersistentActor;
38 import org.opendaylight.controller.cluster.mgmt.api.FollowerInfo;
39 import org.opendaylight.controller.cluster.notifications.LeaderStateChanged;
40 import org.opendaylight.controller.cluster.notifications.RoleChanged;
41 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
42 import org.opendaylight.controller.cluster.raft.base.messages.CheckConsensusReached;
43 import org.opendaylight.controller.cluster.raft.base.messages.InitiateCaptureSnapshot;
44 import org.opendaylight.controller.cluster.raft.base.messages.LeaderTransitioning;
45 import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
46 import org.opendaylight.controller.cluster.raft.base.messages.SwitchBehavior;
47 import org.opendaylight.controller.cluster.raft.behaviors.AbstractLeader;
48 import org.opendaylight.controller.cluster.raft.behaviors.AbstractRaftActorBehavior;
49 import org.opendaylight.controller.cluster.raft.behaviors.Follower;
50 import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior;
51 import org.opendaylight.controller.cluster.raft.client.messages.FindLeader;
52 import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply;
53 import org.opendaylight.controller.cluster.raft.client.messages.GetOnDemandRaftState;
54 import org.opendaylight.controller.cluster.raft.client.messages.OnDemandRaftState;
55 import org.opendaylight.controller.cluster.raft.client.messages.Shutdown;
56 import org.opendaylight.controller.cluster.raft.messages.Payload;
57 import org.opendaylight.controller.cluster.raft.messages.RequestLeadership;
58 import org.opendaylight.controller.cluster.raft.persisted.ApplyJournalEntries;
59 import org.opendaylight.controller.cluster.raft.persisted.NoopPayload;
60 import org.opendaylight.controller.cluster.raft.persisted.ServerConfigurationPayload;
61 import org.opendaylight.controller.cluster.raft.persisted.SimpleReplicatedLogEntry;
62 import org.opendaylight.yangtools.concepts.Identifier;
63 import org.opendaylight.yangtools.concepts.Immutable;
64
65 /**
66  * RaftActor encapsulates a state machine that needs to be kept synchronized
67  * in a cluster. It implements the RAFT algorithm as described in the paper
68  * <a href='https://ramcloud.stanford.edu/wiki/download/attachments/11370504/raft.pdf'>
69  * In Search of an Understandable Consensus Algorithm</a>
70  *
71  * <p>
72  * RaftActor has 3 states and each state has a certain behavior associated
73  * with it. A Raft actor can behave as,
74  * <ul>
75  * <li> A Leader </li>
76  * <li> A Follower (or) </li>
77  * <li> A Candidate </li>
78  * </ul>
79  *
80  * <p>
81  * A RaftActor MUST be a Leader in order to accept requests from clients to
82  * change the state of it's encapsulated state machine. Once a RaftActor becomes
83  * a Leader it is also responsible for ensuring that all followers ultimately
84  * have the same log and therefore the same state machine as itself.
85  *
86  * <p>
87  * The current behavior of a RaftActor determines how election for leadership
88  * is initiated and how peer RaftActors react to request for votes.
89  *
90  * <p>
91  * Each RaftActor also needs to know the current election term. It uses this
92  * information for a couple of things. One is to simply figure out who it
93  * voted for in the last election. Another is to figure out if the message
94  * it received to update it's state is stale.
95  *
96  * <p>
97  * The RaftActor uses akka-persistence to store it's replicated log.
98  * Furthermore through it's behaviors a Raft Actor determines
99  * <ul>
100  * <li> when a log entry should be persisted </li>
101  * <li> when a log entry should be applied to the state machine (and) </li>
102  * <li> when a snapshot should be saved </li>
103  * </ul>
104  */
105 public abstract class RaftActor extends AbstractUntypedPersistentActor {
106
107     private static final long APPLY_STATE_DELAY_THRESHOLD_IN_NANOS = TimeUnit.MILLISECONDS.toNanos(50L); // 50 millis
108
109     /**
110      * This context should NOT be passed directly to any other actor it is
111      * only to be consumed by the RaftActorBehaviors.
112      */
113     private final RaftActorContextImpl context;
114
115     private final DelegatingPersistentDataProvider delegatingPersistenceProvider;
116
117     private final PersistentDataProvider persistentProvider;
118
119     private final BehaviorStateTracker behaviorStateTracker = new BehaviorStateTracker();
120
121     private RaftActorRecoverySupport raftRecovery;
122
123     private RaftActorSnapshotMessageSupport snapshotSupport;
124
125     private RaftActorServerConfigurationSupport serverConfigurationSupport;
126
127     private boolean shuttingDown;
128
129     @SuppressFBWarnings(value = "MC_OVERRIDABLE_METHOD_CALL_IN_CONSTRUCTOR", justification = "Akka class design")
130     protected RaftActor(final String id, final Map<String, String> peerAddresses,
131          final Optional<ConfigParams> configParams, final short payloadVersion) {
132
133         persistentProvider = new PersistentDataProvider(this);
134         delegatingPersistenceProvider = new RaftActorDelegatingPersistentDataProvider(null, persistentProvider);
135
136         context = new RaftActorContextImpl(getSelf(), getContext(), id,
137             new ElectionTermImpl(persistentProvider, id, LOG), -1, -1, peerAddresses,
138             configParams.isPresent() ? configParams.get() : new DefaultConfigParamsImpl(),
139             delegatingPersistenceProvider, this::handleApplyState, LOG, this::executeInSelf);
140
141         context.setPayloadVersion(payloadVersion);
142         context.setReplicatedLog(ReplicatedLogImpl.newInstance(context));
143     }
144
145     @Override
146     public void preStart() throws Exception {
147         LOG.info("Starting recovery for {} with journal batch size {}", persistenceId(),
148                 context.getConfigParams().getJournalRecoveryLogBatchSize());
149
150         super.preStart();
151
152         snapshotSupport = newRaftActorSnapshotMessageSupport();
153         serverConfigurationSupport = new RaftActorServerConfigurationSupport(this);
154     }
155
156     @Override
157     public void postStop() throws Exception {
158         context.close();
159         super.postStop();
160     }
161
162     @Override
163     protected void handleRecover(final Object message) {
164         if (raftRecovery == null) {
165             raftRecovery = newRaftActorRecoverySupport();
166         }
167
168         boolean recoveryComplete = raftRecovery.handleRecoveryMessage(message, persistentProvider);
169         if (recoveryComplete) {
170             onRecoveryComplete();
171
172             initializeBehavior();
173
174             raftRecovery = null;
175         }
176     }
177
178     protected RaftActorRecoverySupport newRaftActorRecoverySupport() {
179         return new RaftActorRecoverySupport(context, getRaftActorRecoveryCohort());
180     }
181
182     @VisibleForTesting
183     void initializeBehavior() {
184         changeCurrentBehavior(new Follower(context));
185     }
186
187     @VisibleForTesting
188     @SuppressWarnings("checkstyle:IllegalCatch")
189     protected void changeCurrentBehavior(final RaftActorBehavior newBehavior) {
190         final RaftActorBehavior currentBehavior = getCurrentBehavior();
191         if (currentBehavior != null) {
192             try {
193                 currentBehavior.close();
194             } catch (Exception e) {
195                 LOG.warn("{}: Error closing behavior {}", persistence(), currentBehavior, e);
196             }
197         }
198
199         final BehaviorState state = behaviorStateTracker.capture(currentBehavior);
200         setCurrentBehavior(newBehavior);
201         handleBehaviorChange(state, newBehavior);
202     }
203
204     /**
205      * Method exposed for subclasses to plug-in their logic. This method is invoked by {@link #handleCommand(Object)}
206      * for messages which are not handled by this class. Subclasses overriding this class should fall back to this
207      * implementation for messages which they do not handle
208      *
209      * @param message Incoming command message
210      */
211     protected void handleNonRaftCommand(final Object message) {
212         unhandled(message);
213     }
214
215     /**
216      * Handles a message.
217      *
218      * @deprecated This method is not final for testing purposes. DO NOT OVERRIDE IT, override
219      *             {@link #handleNonRaftCommand(Object)} instead.
220      */
221     @Deprecated
222     @Override
223     // FIXME: make this method final once our unit tests do not need to override it
224     protected void handleCommand(final Object message) {
225         if (serverConfigurationSupport.handleMessage(message, getSender())) {
226             return;
227         }
228         if (snapshotSupport.handleSnapshotMessage(message, getSender())) {
229             return;
230         }
231         if (message instanceof ApplyState) {
232             ApplyState applyState = (ApplyState) message;
233
234             if (!hasFollowers()) {
235                 // for single node, the capture should happen after the apply state
236                 // as we delete messages from the persistent journal which have made it to the snapshot
237                 // capturing the snapshot before applying makes the persistent journal and snapshot out of sync
238                 // and recovery shows data missing
239                 context.getReplicatedLog().captureSnapshotIfReady(applyState.getReplicatedLogEntry());
240
241                 context.getSnapshotManager().trimLog(context.getLastApplied());
242             }
243
244             possiblyHandleBehaviorMessage(message);
245         } else if (message instanceof ApplyJournalEntries) {
246             ApplyJournalEntries applyEntries = (ApplyJournalEntries) message;
247             LOG.debug("{}: Persisting ApplyJournalEntries with index={}", persistenceId(), applyEntries.getToIndex());
248
249             persistence().persistAsync(applyEntries, NoopProcedure.instance());
250         } else if (message instanceof FindLeader) {
251             getSender().tell(new FindLeaderReply(getLeaderAddress()), getSelf());
252         } else if (message instanceof GetOnDemandRaftState) {
253             onGetOnDemandRaftStats();
254         } else if (message instanceof InitiateCaptureSnapshot) {
255             captureSnapshot();
256         } else if (message instanceof SwitchBehavior) {
257             switchBehavior((SwitchBehavior) message);
258         } else if (message instanceof LeaderTransitioning) {
259             onLeaderTransitioning((LeaderTransitioning)message);
260         } else if (message instanceof Shutdown) {
261             onShutDown();
262         } else if (message instanceof Runnable) {
263             ((Runnable)message).run();
264         } else if (message instanceof NoopPayload) {
265             persistData(null, null, (NoopPayload) message, false);
266         } else if (message instanceof RequestLeadership) {
267             onRequestLeadership((RequestLeadership) message);
268         } else if (!possiblyHandleBehaviorMessage(message)) {
269             if (message instanceof JournalProtocol.Response
270                 && delegatingPersistenceProvider.handleJournalResponse((JournalProtocol.Response) message)) {
271                 LOG.debug("{}: handled a journal response", persistenceId());
272             } else if (message instanceof SnapshotProtocol.Response
273                 && delegatingPersistenceProvider.handleSnapshotResponse((SnapshotProtocol.Response) message)) {
274                 LOG.debug("{}: handled a snapshot response", persistenceId());
275             } else {
276                 handleNonRaftCommand(message);
277             }
278         }
279     }
280
281     private void onRequestLeadership(final RequestLeadership message) {
282         LOG.debug("{}: onRequestLeadership {}", persistenceId(), message);
283         if (!isLeader()) {
284             // non-leader cannot satisfy leadership request
285             LOG.warn("{}: onRequestLeadership {} was sent to non-leader."
286                     + " Current behavior: {}. Sending failure response",
287                     persistenceId(), message, getCurrentBehavior().state());
288             message.getReplyTo().tell(new LeadershipTransferFailedException("Cannot transfer leader to "
289                     + message.getRequestedFollowerId()
290                     + ". RequestLeadership message was sent to non-leader " + persistenceId()), getSelf());
291             return;
292         }
293
294         final String requestedFollowerId = message.getRequestedFollowerId();
295         final ActorRef replyTo = message.getReplyTo();
296         initiateLeadershipTransfer(new RaftActorLeadershipTransferCohort.OnComplete() {
297             @Override
298             public void onSuccess(final ActorRef raftActorRef) {
299                 // sanity check
300                 if (!requestedFollowerId.equals(getLeaderId())) {
301                     onFailure(raftActorRef);
302                 }
303
304                 LOG.debug("{}: Leadership transferred successfully to {}", persistenceId(), requestedFollowerId);
305                 replyTo.tell(new Status.Success(null), getSelf());
306             }
307
308             @Override
309             public void onFailure(final ActorRef raftActorRef) {
310                 LOG.debug("{}: LeadershipTransfer request from {} failed", persistenceId(), requestedFollowerId);
311                 replyTo.tell(new Status.Failure(
312                         new LeadershipTransferFailedException(
313                                 "Failed to transfer leadership to " + requestedFollowerId
314                                         + ". Follower is not ready to become leader")),
315                         getSelf());
316             }
317         }, message.getRequestedFollowerId(), RaftActorLeadershipTransferCohort.USE_DEFAULT_LEADER_TIMEOUT);
318     }
319
320     private boolean possiblyHandleBehaviorMessage(final Object message) {
321         final RaftActorBehavior currentBehavior = getCurrentBehavior();
322         final BehaviorState state = behaviorStateTracker.capture(currentBehavior);
323
324         // A behavior indicates that it processed the change by returning a reference to the next behavior
325         // to be used. A null return indicates it has not processed the message and we should be passing it to
326         // the subclass for handling.
327         final RaftActorBehavior nextBehavior = currentBehavior.handleMessage(getSender(), message);
328         if (nextBehavior != null) {
329             switchBehavior(state, nextBehavior);
330             return true;
331         }
332
333         return false;
334     }
335
336     private void initiateLeadershipTransfer(final RaftActorLeadershipTransferCohort.OnComplete onComplete,
337             final @Nullable String followerId, final long newLeaderTimeoutInMillis) {
338         LOG.debug("{}: Initiating leader transfer", persistenceId());
339
340         RaftActorLeadershipTransferCohort leadershipTransferInProgress = context.getRaftActorLeadershipTransferCohort();
341         if (leadershipTransferInProgress == null) {
342             leadershipTransferInProgress = new RaftActorLeadershipTransferCohort(this, followerId);
343             leadershipTransferInProgress.setNewLeaderTimeoutInMillis(newLeaderTimeoutInMillis);
344             leadershipTransferInProgress.addOnComplete(new RaftActorLeadershipTransferCohort.OnComplete() {
345                 @Override
346                 public void onSuccess(final ActorRef raftActorRef) {
347                     context.setRaftActorLeadershipTransferCohort(null);
348                 }
349
350                 @Override
351                 public void onFailure(final ActorRef raftActorRef) {
352                     context.setRaftActorLeadershipTransferCohort(null);
353                 }
354             });
355
356             leadershipTransferInProgress.addOnComplete(onComplete);
357
358             context.setRaftActorLeadershipTransferCohort(leadershipTransferInProgress);
359             leadershipTransferInProgress.init();
360
361         } else {
362             LOG.debug("{}: prior leader transfer in progress - adding callback", persistenceId());
363             leadershipTransferInProgress.addOnComplete(onComplete);
364         }
365     }
366
367     private void onShutDown() {
368         LOG.debug("{}: onShutDown", persistenceId());
369
370         if (shuttingDown) {
371             return;
372         }
373
374         shuttingDown = true;
375
376         final RaftActorBehavior currentBehavior = context.getCurrentBehavior();
377         switch (currentBehavior.state()) {
378             case Leader:
379             case PreLeader:
380                 // Fall-through to more work
381                 break;
382             default:
383                 // For non-leaders shutdown is a no-op
384                 self().tell(PoisonPill.getInstance(), self());
385                 return;
386         }
387
388         if (context.hasFollowers()) {
389             initiateLeadershipTransfer(new RaftActorLeadershipTransferCohort.OnComplete() {
390                 @Override
391                 public void onSuccess(final ActorRef raftActorRef) {
392                     LOG.debug("{}: leader transfer succeeded - sending PoisonPill", persistenceId());
393                     raftActorRef.tell(PoisonPill.getInstance(), raftActorRef);
394                 }
395
396                 @Override
397                 public void onFailure(final ActorRef raftActorRef) {
398                     LOG.debug("{}: leader transfer failed - sending PoisonPill", persistenceId());
399                     raftActorRef.tell(PoisonPill.getInstance(), raftActorRef);
400                 }
401             }, null, TimeUnit.MILLISECONDS.convert(2, TimeUnit.SECONDS));
402         } else {
403             pauseLeader(new TimedRunnable(context.getConfigParams().getElectionTimeOutInterval(), this) {
404                 @Override
405                 protected void doRun() {
406                     self().tell(PoisonPill.getInstance(), self());
407                 }
408
409                 @Override
410                 protected void doCancel() {
411                     self().tell(PoisonPill.getInstance(), self());
412                 }
413             });
414         }
415     }
416
417     private void onLeaderTransitioning(final LeaderTransitioning leaderTransitioning) {
418         LOG.debug("{}: onLeaderTransitioning: {}", persistenceId(), leaderTransitioning);
419         Optional<ActorRef> roleChangeNotifier = getRoleChangeNotifier();
420         if (getRaftState() == RaftState.Follower && roleChangeNotifier.isPresent()
421                 && leaderTransitioning.getLeaderId().equals(getCurrentBehavior().getLeaderId())) {
422             roleChangeNotifier.get().tell(newLeaderStateChanged(getId(), null,
423                 getCurrentBehavior().getLeaderPayloadVersion()), getSelf());
424         }
425     }
426
427     private void switchBehavior(final SwitchBehavior message) {
428         if (!getRaftActorContext().getRaftPolicy().automaticElectionsEnabled()) {
429             RaftState newState = message.getNewState();
430             if (newState == RaftState.Leader || newState == RaftState.Follower) {
431                 getRaftActorContext().getTermInformation().updateAndPersist(message.getNewTerm(), "");
432                 switchBehavior(behaviorStateTracker.capture(getCurrentBehavior()),
433                     AbstractRaftActorBehavior.createBehavior(context, message.getNewState()));
434             } else {
435                 LOG.warn("Switching to behavior : {} - not supported", newState);
436             }
437         }
438     }
439
440     private void switchBehavior(final BehaviorState oldBehaviorState, final RaftActorBehavior nextBehavior) {
441         setCurrentBehavior(nextBehavior);
442         handleBehaviorChange(oldBehaviorState, nextBehavior);
443     }
444
445     @VisibleForTesting
446     RaftActorSnapshotMessageSupport newRaftActorSnapshotMessageSupport() {
447         return new RaftActorSnapshotMessageSupport(context, getRaftActorSnapshotCohort());
448     }
449
450     private void onGetOnDemandRaftStats() {
451         // Debugging message to retrieve raft stats.
452
453         Map<String, String> peerAddresses = new HashMap<>();
454         Map<String, Boolean> peerVotingStates = new HashMap<>();
455         for (PeerInfo info: context.getPeers()) {
456             peerVotingStates.put(info.getId(), info.isVoting());
457             peerAddresses.put(info.getId(), info.getAddress() != null ? info.getAddress() : "");
458         }
459
460         final RaftActorBehavior currentBehavior = context.getCurrentBehavior();
461         OnDemandRaftState.AbstractBuilder<?, ?> builder = newOnDemandRaftStateBuilder()
462                 .commitIndex(context.getCommitIndex())
463                 .currentTerm(context.getTermInformation().getCurrentTerm())
464                 .inMemoryJournalDataSize(replicatedLog().dataSize())
465                 .inMemoryJournalLogSize(replicatedLog().size())
466                 .isSnapshotCaptureInitiated(context.getSnapshotManager().isCapturing())
467                 .lastApplied(context.getLastApplied())
468                 .lastIndex(replicatedLog().lastIndex())
469                 .lastTerm(replicatedLog().lastTerm())
470                 .leader(getLeaderId())
471                 .raftState(currentBehavior.state().toString())
472                 .replicatedToAllIndex(currentBehavior.getReplicatedToAllIndex())
473                 .snapshotIndex(replicatedLog().getSnapshotIndex())
474                 .snapshotTerm(replicatedLog().getSnapshotTerm())
475                 .votedFor(context.getTermInformation().getVotedFor())
476                 .isVoting(context.isVotingMember())
477                 .peerAddresses(peerAddresses)
478                 .peerVotingStates(peerVotingStates)
479                 .customRaftPolicyClassName(context.getConfigParams().getCustomRaftPolicyImplementationClass());
480
481         ReplicatedLogEntry lastLogEntry = replicatedLog().last();
482         if (lastLogEntry != null) {
483             builder.lastLogIndex(lastLogEntry.getIndex());
484             builder.lastLogTerm(lastLogEntry.getTerm());
485         }
486
487         if (getCurrentBehavior() instanceof AbstractLeader) {
488             AbstractLeader leader = (AbstractLeader)getCurrentBehavior();
489             Collection<String> followerIds = leader.getFollowerIds();
490             List<FollowerInfo> followerInfoList = new ArrayList<>(followerIds.size());
491             for (String id: followerIds) {
492                 final FollowerLogInformation info = leader.getFollower(id);
493                 followerInfoList.add(new FollowerInfo(id, info.getNextIndex(), info.getMatchIndex(),
494                         info.isFollowerActive(), DurationFormatUtils.formatDurationHMS(
495                             TimeUnit.NANOSECONDS.toMillis(info.nanosSinceLastActivity())),
496                         context.getPeerInfo(info.getId()).isVoting()));
497             }
498
499             builder.followerInfoList(followerInfoList);
500         }
501
502         sender().tell(builder.build(), self());
503
504     }
505
506     protected OnDemandRaftState.AbstractBuilder<?, ?> newOnDemandRaftStateBuilder() {
507         return OnDemandRaftState.builder();
508     }
509
510     private void handleBehaviorChange(final BehaviorState oldBehaviorState, final RaftActorBehavior currentBehavior) {
511         RaftActorBehavior oldBehavior = oldBehaviorState.getBehavior();
512
513         if (oldBehavior != currentBehavior) {
514             onStateChanged();
515         }
516
517         String lastLeaderId = oldBehavior == null ? null : oldBehaviorState.getLastLeaderId();
518         String lastValidLeaderId = oldBehavior == null ? null : oldBehaviorState.getLastValidLeaderId();
519         String oldBehaviorStateName = oldBehavior == null ? null : oldBehavior.state().name();
520
521         // it can happen that the state has not changed but the leader has changed.
522         Optional<ActorRef> roleChangeNotifier = getRoleChangeNotifier();
523         if (!Objects.equals(lastLeaderId, currentBehavior.getLeaderId())
524                 || oldBehaviorState.getLeaderPayloadVersion() != currentBehavior.getLeaderPayloadVersion()) {
525             if (roleChangeNotifier.isPresent()) {
526                 roleChangeNotifier.get().tell(newLeaderStateChanged(getId(), currentBehavior.getLeaderId(),
527                         currentBehavior.getLeaderPayloadVersion()), getSelf());
528             }
529
530             onLeaderChanged(lastValidLeaderId, currentBehavior.getLeaderId());
531
532             RaftActorLeadershipTransferCohort leadershipTransferInProgress =
533                     context.getRaftActorLeadershipTransferCohort();
534             if (leadershipTransferInProgress != null) {
535                 leadershipTransferInProgress.onNewLeader(currentBehavior.getLeaderId());
536             }
537
538             serverConfigurationSupport.onNewLeader(currentBehavior.getLeaderId());
539         }
540
541         if (roleChangeNotifier.isPresent()
542                 && (oldBehavior == null || oldBehavior.state() != currentBehavior.state())) {
543             roleChangeNotifier.get().tell(new RoleChanged(getId(), oldBehaviorStateName ,
544                     currentBehavior.state().name()), getSelf());
545         }
546     }
547
548     private void handleApplyState(final ApplyState applyState) {
549         long startTime = System.nanoTime();
550
551         Payload payload = applyState.getReplicatedLogEntry().getData();
552         if (LOG.isDebugEnabled()) {
553             LOG.debug("{}: Applying state for log index {} data {}",
554                 persistenceId(), applyState.getReplicatedLogEntry().getIndex(), payload);
555         }
556
557         if (!(payload instanceof NoopPayload) && !(payload instanceof ServerConfigurationPayload)) {
558             applyState(applyState.getClientActor(), applyState.getIdentifier(), payload);
559         }
560
561         long elapsedTime = System.nanoTime() - startTime;
562         if (elapsedTime >= APPLY_STATE_DELAY_THRESHOLD_IN_NANOS) {
563             LOG.debug("ApplyState took more time than expected. Elapsed Time = {} ms ApplyState = {}",
564                     TimeUnit.NANOSECONDS.toMillis(elapsedTime), applyState);
565         }
566
567         // Send the ApplyState message back to self to handle further processing asynchronously.
568         self().tell(applyState, self());
569     }
570
571     protected LeaderStateChanged newLeaderStateChanged(final String memberId, final String leaderId,
572             final short leaderPayloadVersion) {
573         return new LeaderStateChanged(memberId, leaderId, leaderPayloadVersion);
574     }
575
576     @Override
577     public long snapshotSequenceNr() {
578         // When we do a snapshot capture, we also capture and save the sequence-number of the persistent journal,
579         // so that we can delete the persistent journal based on the saved sequence-number
580         // However , when akka replays the journal during recovery, it replays it from the sequence number when the
581         // snapshot was saved and not the number we saved. We would want to override it , by asking akka to use the
582         // last-sequence number known to us.
583         return context.getSnapshotManager().getLastSequenceNumber();
584     }
585
586     /**
587      * Persists the given Payload in the journal and replicates to any followers. After successful completion,
588      * {@link #applyState(ActorRef, Identifier, Object)} is notified.
589      *
590      * @param clientActor optional ActorRef that is provided via the applyState callback
591      * @param identifier the payload identifier
592      * @param data the payload data to persist
593      * @param batchHint if true, an attempt is made to delay immediate replication and batch the payload with
594      *        subsequent payloads for efficiency. Otherwise the payload is immediately replicated.
595      */
596     protected final void persistData(final ActorRef clientActor, final Identifier identifier, final Payload data,
597             final boolean batchHint) {
598         ReplicatedLogEntry replicatedLogEntry = new SimpleReplicatedLogEntry(
599             context.getReplicatedLog().lastIndex() + 1,
600             context.getTermInformation().getCurrentTerm(), data);
601         replicatedLogEntry.setPersistencePending(true);
602
603         LOG.debug("{}: Persist data {}", persistenceId(), replicatedLogEntry);
604
605         final RaftActorContext raftContext = getRaftActorContext();
606
607         boolean wasAppended = replicatedLog().appendAndPersist(replicatedLogEntry, persistedLogEntry -> {
608             // Clear the persistence pending flag in the log entry.
609             persistedLogEntry.setPersistencePending(false);
610
611             if (!hasFollowers()) {
612                 // Increment the Commit Index and the Last Applied values
613                 raftContext.setCommitIndex(persistedLogEntry.getIndex());
614                 raftContext.setLastApplied(persistedLogEntry.getIndex());
615
616                 // Apply the state immediately.
617                 handleApplyState(new ApplyState(clientActor, identifier, persistedLogEntry));
618
619                 // Send a ApplyJournalEntries message so that we write the fact that we applied
620                 // the state to durable storage
621                 self().tell(new ApplyJournalEntries(persistedLogEntry.getIndex()), self());
622
623             } else {
624                 context.getReplicatedLog().captureSnapshotIfReady(replicatedLogEntry);
625
626                 // Local persistence is complete so send the CheckConsensusReached message to the behavior (which
627                 // normally should still be the leader) to check if consensus has now been reached in conjunction with
628                 // follower replication.
629                 getCurrentBehavior().handleMessage(getSelf(), CheckConsensusReached.INSTANCE);
630             }
631         }, true);
632
633         if (wasAppended && hasFollowers()) {
634             // Send log entry for replication.
635             getCurrentBehavior().handleMessage(getSelf(), new Replicate(clientActor, identifier, replicatedLogEntry,
636                     !batchHint));
637         }
638     }
639
640     private ReplicatedLog replicatedLog() {
641         return context.getReplicatedLog();
642     }
643
644     protected String getId() {
645         return context.getId();
646     }
647
648     @VisibleForTesting
649     void setCurrentBehavior(final RaftActorBehavior behavior) {
650         context.setCurrentBehavior(behavior);
651     }
652
653     protected RaftActorBehavior getCurrentBehavior() {
654         return context.getCurrentBehavior();
655     }
656
657     /**
658      * Derived actors can call the isLeader method to check if the current
659      * RaftActor is the Leader or not.
660      *
661      * @return true it this RaftActor is a Leader false otherwise
662      */
663     protected boolean isLeader() {
664         return context.getId().equals(getCurrentBehavior().getLeaderId());
665     }
666
667     protected final boolean isLeaderActive() {
668         return getRaftState() != RaftState.IsolatedLeader && getRaftState() != RaftState.PreLeader
669                 && !shuttingDown && !isLeadershipTransferInProgress();
670     }
671
672     protected boolean isLeadershipTransferInProgress() {
673         RaftActorLeadershipTransferCohort leadershipTransferInProgress = context.getRaftActorLeadershipTransferCohort();
674         return leadershipTransferInProgress != null && leadershipTransferInProgress.isTransferring();
675     }
676
677     /**
678      * Derived actor can call getLeader if they need a reference to the Leader.
679      * This would be useful for example in forwarding a request to an actor
680      * which is the leader
681      *
682      * @return A reference to the leader if known, null otherwise
683      */
684     public ActorSelection getLeader() {
685         String leaderAddress = getLeaderAddress();
686
687         if (leaderAddress == null) {
688             return null;
689         }
690
691         return context.actorSelection(leaderAddress);
692     }
693
694     /**
695      * Returns the id of the current leader.
696      *
697      * @return the current leader's id
698      */
699     protected final String getLeaderId() {
700         return getCurrentBehavior().getLeaderId();
701     }
702
703     @VisibleForTesting
704     protected final RaftState getRaftState() {
705         return getCurrentBehavior().state();
706     }
707
708     protected Long getCurrentTerm() {
709         return context.getTermInformation().getCurrentTerm();
710     }
711
712     protected RaftActorContext getRaftActorContext() {
713         return context;
714     }
715
716     protected void updateConfigParams(final ConfigParams configParams) {
717
718         // obtain the RaftPolicy for oldConfigParams and the updated one.
719         String oldRaftPolicy = context.getConfigParams().getCustomRaftPolicyImplementationClass();
720         String newRaftPolicy = configParams.getCustomRaftPolicyImplementationClass();
721
722         LOG.debug("{}: RaftPolicy used with prev.config {}, RaftPolicy used with newConfig {}", persistenceId(),
723             oldRaftPolicy, newRaftPolicy);
724         context.setConfigParams(configParams);
725         if (!Objects.equals(oldRaftPolicy, newRaftPolicy)) {
726             // The RaftPolicy was modified. If the current behavior is Follower then re-initialize to Follower
727             // but transfer the previous leaderId so it doesn't immediately try to schedule an election. This
728             // avoids potential disruption. Otherwise, switch to Follower normally.
729             RaftActorBehavior behavior = getCurrentBehavior();
730             if (behavior != null && behavior.state() == RaftState.Follower) {
731                 String previousLeaderId = behavior.getLeaderId();
732                 short previousLeaderPayloadVersion = behavior.getLeaderPayloadVersion();
733
734                 LOG.debug("{}: Re-initializing to Follower with previous leaderId {}", persistenceId(),
735                         previousLeaderId);
736
737                 changeCurrentBehavior(new Follower(context, previousLeaderId, previousLeaderPayloadVersion));
738             } else {
739                 initializeBehavior();
740             }
741         }
742     }
743
744     public final DataPersistenceProvider persistence() {
745         return delegatingPersistenceProvider.getDelegate();
746     }
747
748     public void setPersistence(final DataPersistenceProvider provider) {
749         delegatingPersistenceProvider.setDelegate(provider);
750     }
751
752     protected void setPersistence(final boolean persistent) {
753         DataPersistenceProvider currentPersistence = persistence();
754         if (persistent && (currentPersistence == null || !currentPersistence.isRecoveryApplicable())) {
755             setPersistence(new PersistentDataProvider(this));
756
757             if (getCurrentBehavior() != null) {
758                 LOG.info("{}: Persistence has been enabled - capturing snapshot", persistenceId());
759                 captureSnapshot();
760             }
761         } else if (!persistent && (currentPersistence == null || currentPersistence.isRecoveryApplicable())) {
762             setPersistence(new NonPersistentDataProvider(this) {
763                 /*
764                  * The way snapshotting works is,
765                  * <ol>
766                  * <li> RaftActor calls createSnapshot on the Shard
767                  * <li> Shard sends a CaptureSnapshotReply and RaftActor then calls saveSnapshot
768                  * <li> When saveSnapshot is invoked on the akka-persistence API it uses the SnapshotStore to save
769                  * the snapshot. The SnapshotStore sends SaveSnapshotSuccess or SaveSnapshotFailure. When the
770                  * RaftActor gets SaveSnapshot success it commits the snapshot to the in-memory journal. This
771                  * commitSnapshot is mimicking what is done in SaveSnapshotSuccess.
772                  * </ol>
773                  */
774                 @Override
775                 public void saveSnapshot(final Object object) {
776                     // Make saving Snapshot successful
777                     // Committing the snapshot here would end up calling commit in the creating state which would
778                     // be a state violation. That's why now we send a message to commit the snapshot.
779                     self().tell(RaftActorSnapshotMessageSupport.COMMIT_SNAPSHOT, self());
780                 }
781             });
782         }
783     }
784
785     /**
786      * setPeerAddress sets the address of a known peer at a later time.
787      *
788      * <p>
789      * This is to account for situations where a we know that a peer
790      * exists but we do not know an address up-front. This may also be used in
791      * situations where a known peer starts off in a different location and we
792      * need to change it's address
793      *
794      * <p>
795      * Note that if the peerId does not match the list of peers passed to
796      * this actor during construction an IllegalStateException will be thrown.
797      */
798     protected void setPeerAddress(final String peerId, final String peerAddress) {
799         context.setPeerAddress(peerId, peerAddress);
800     }
801
802     /**
803      * The applyState method will be called by the RaftActor when some data
804      * needs to be applied to the actor's state.
805      *
806      * @param clientActor A reference to the client who sent this message. This
807      *                    is the same reference that was passed to persistData
808      *                    by the derived actor. clientActor may be null when
809      *                    the RaftActor is behaving as a follower or during
810      *                    recovery.
811      * @param identifier  The identifier of the persisted data. This is also
812      *                    the same identifier that was passed to persistData by
813      *                    the derived actor. identifier may be null when
814      *                    the RaftActor is behaving as a follower or during
815      *                    recovery
816      * @param data        A piece of data that was persisted by the persistData call.
817      *                    This should NEVER be null.
818      */
819     protected abstract void applyState(ActorRef clientActor, Identifier identifier, Object data);
820
821     /**
822      * Returns the RaftActorRecoveryCohort to participate in persistence recovery.
823      */
824     protected abstract @NonNull RaftActorRecoveryCohort getRaftActorRecoveryCohort();
825
826     /**
827      * This method is called when recovery is complete.
828      */
829     protected abstract void onRecoveryComplete();
830
831     /**
832      * Returns the RaftActorSnapshotCohort to participate in snapshot captures.
833      */
834     protected abstract @NonNull RaftActorSnapshotCohort getRaftActorSnapshotCohort();
835
836     /**
837      * This method will be called by the RaftActor when the state of the
838      * RaftActor changes. The derived actor can then use methods like
839      * isLeader or getLeader to do something useful
840      */
841     protected abstract void onStateChanged();
842
843     /**
844      * Notifier Actor for this RaftActor to notify when a role change happens.
845      *
846      * @return ActorRef - ActorRef of the notifier or Optional.absent if none.
847      */
848     protected abstract Optional<ActorRef> getRoleChangeNotifier();
849
850     /**
851      * This method is called on the leader when a voting change operation completes.
852      */
853     protected void onVotingStateChangeComplete() {
854     }
855
856     /**
857      * This method is called prior to operations such as leadership transfer and actor shutdown when the leader
858      * must pause or stop its duties. This method allows derived classes to gracefully pause or finish current
859      * work prior to performing the operation. On completion of any work, the run method must be called on the
860      * given Runnable to proceed with the given operation. <b>Important:</b> the run method must be called on
861      * this actor's thread dispatcher as as it modifies internal state.
862      *
863      * <p>
864      * The default implementation immediately runs the operation.
865      *
866      * @param operation the operation to run
867      */
868     protected void pauseLeader(final Runnable operation) {
869         operation.run();
870     }
871
872     /**
873      * This method is invoked when the actions hooked to the leader becoming paused failed to execute and the leader
874      * should resume normal operations.
875      *
876      * <p>
877      * Note this method can be invoked even before the operation supplied to {@link #pauseLeader(Runnable)} is invoked.
878      */
879     protected void unpauseLeader() {
880
881     }
882
883     protected void onLeaderChanged(final String oldLeader, final String newLeader) {
884     }
885
886     private String getLeaderAddress() {
887         if (isLeader()) {
888             return getSelf().path().toString();
889         }
890         String leaderId = getLeaderId();
891         if (leaderId == null) {
892             return null;
893         }
894         String peerAddress = context.getPeerAddress(leaderId);
895         LOG.debug("{}: getLeaderAddress leaderId = {} peerAddress = {}", persistenceId(), leaderId, peerAddress);
896
897         return peerAddress;
898     }
899
900     protected boolean hasFollowers() {
901         return getRaftActorContext().hasFollowers();
902     }
903
904     private void captureSnapshot() {
905         SnapshotManager snapshotManager = context.getSnapshotManager();
906
907         if (!snapshotManager.isCapturing()) {
908             final long idx = getCurrentBehavior().getReplicatedToAllIndex();
909             LOG.debug("Take a snapshot of current state. lastReplicatedLog is {} and replicatedToAllIndex is {}",
910                 replicatedLog().last(), idx);
911
912             snapshotManager.captureWithForcedTrim(replicatedLog().last(), idx);
913         }
914     }
915
916     /**
917      * Switch this member to non-voting status. This is a no-op for all behaviors except when we are the leader,
918      * in which case we need to step down.
919      */
920     void becomeNonVoting() {
921         if (isLeader()) {
922             initiateLeadershipTransfer(new RaftActorLeadershipTransferCohort.OnComplete() {
923                 @Override
924                 public void onSuccess(final ActorRef raftActorRef) {
925                     LOG.debug("{}: leader transfer succeeded after change to non-voting", persistenceId());
926                     ensureFollowerState();
927                 }
928
929                 @Override
930                 public void onFailure(final ActorRef raftActorRef) {
931                     LOG.debug("{}: leader transfer failed after change to non-voting", persistenceId());
932                     ensureFollowerState();
933                 }
934
935                 private void ensureFollowerState() {
936                     // Whether or not leadership transfer succeeded, we have to step down as leader and
937                     // switch to Follower so ensure that.
938                     if (getRaftState() != RaftState.Follower) {
939                         initializeBehavior();
940                     }
941                 }
942             }, null, RaftActorLeadershipTransferCohort.USE_DEFAULT_LEADER_TIMEOUT);
943         }
944     }
945
946     /**
947      * A point-in-time capture of {@link RaftActorBehavior} state critical for transitioning between behaviors.
948      */
949     private abstract static class BehaviorState implements Immutable {
950         @Nullable abstract RaftActorBehavior getBehavior();
951
952         @Nullable abstract String getLastValidLeaderId();
953
954         @Nullable abstract String getLastLeaderId();
955
956         abstract short getLeaderPayloadVersion();
957     }
958
959     /**
960      * A {@link BehaviorState} corresponding to non-null {@link RaftActorBehavior} state.
961      */
962     private static final class SimpleBehaviorState extends BehaviorState {
963         private final RaftActorBehavior behavior;
964         private final String lastValidLeaderId;
965         private final String lastLeaderId;
966         private final short leaderPayloadVersion;
967
968         SimpleBehaviorState(final String lastValidLeaderId, final String lastLeaderId,
969                 final RaftActorBehavior behavior) {
970             this.lastValidLeaderId = lastValidLeaderId;
971             this.lastLeaderId = lastLeaderId;
972             this.behavior = requireNonNull(behavior);
973             leaderPayloadVersion = behavior.getLeaderPayloadVersion();
974         }
975
976         @Override
977         RaftActorBehavior getBehavior() {
978             return behavior;
979         }
980
981         @Override
982         String getLastValidLeaderId() {
983             return lastValidLeaderId;
984         }
985
986         @Override
987         short getLeaderPayloadVersion() {
988             return leaderPayloadVersion;
989         }
990
991         @Override
992         String getLastLeaderId() {
993             return lastLeaderId;
994         }
995     }
996
997     /**
998      * Class tracking behavior-related information, which we need to keep around and pass across behavior switches.
999      * An instance is created for each RaftActor. It has two functions:
1000      * - it keeps track of the last leader ID we have encountered since we have been created
1001      * - it creates state capture needed to transition from one behavior to the next
1002      */
1003     private static final class BehaviorStateTracker {
1004         /**
1005          * A {@link BehaviorState} corresponding to null {@link RaftActorBehavior} state. Since null behavior is only
1006          * allowed before we receive the first message, we know the leader ID to be null.
1007          */
1008         private static final BehaviorState NULL_BEHAVIOR_STATE = new BehaviorState() {
1009             @Override
1010             RaftActorBehavior getBehavior() {
1011                 return null;
1012             }
1013
1014             @Override
1015             String getLastValidLeaderId() {
1016                 return null;
1017             }
1018
1019             @Override
1020             short getLeaderPayloadVersion() {
1021                 return -1;
1022             }
1023
1024             @Override
1025             String getLastLeaderId() {
1026                 return null;
1027             }
1028         };
1029
1030         private String lastValidLeaderId;
1031         private String lastLeaderId;
1032
1033         BehaviorState capture(final RaftActorBehavior behavior) {
1034             if (behavior == null) {
1035                 verify(lastValidLeaderId == null, "Null behavior with non-null last leader");
1036                 return NULL_BEHAVIOR_STATE;
1037             }
1038
1039             lastLeaderId = behavior.getLeaderId();
1040             if (lastLeaderId != null) {
1041                 lastValidLeaderId = lastLeaderId;
1042             }
1043
1044             return new SimpleBehaviorState(lastValidLeaderId, lastLeaderId, behavior);
1045         }
1046     }
1047 }