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.
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
10 package org.opendaylight.controller.cluster.raft;
12 import akka.actor.ActorRef;
13 import akka.actor.ActorSelection;
14 import akka.japi.Procedure;
15 import akka.persistence.SnapshotSelectionCriteria;
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;
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.Replicate;
39 import org.opendaylight.controller.cluster.raft.base.messages.SwitchBehavior;
40 import org.opendaylight.controller.cluster.raft.behaviors.AbstractLeader;
41 import org.opendaylight.controller.cluster.raft.behaviors.DelegatingRaftActorBehavior;
42 import org.opendaylight.controller.cluster.raft.behaviors.Follower;
43 import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior;
44 import org.opendaylight.controller.cluster.raft.client.messages.FindLeader;
45 import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply;
46 import org.opendaylight.controller.cluster.raft.client.messages.FollowerInfo;
47 import org.opendaylight.controller.cluster.raft.client.messages.GetOnDemandRaftState;
48 import org.opendaylight.controller.cluster.raft.client.messages.OnDemandRaftState;
49 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
50 import org.slf4j.Logger;
51 import org.slf4j.LoggerFactory;
54 * RaftActor encapsulates a state machine that needs to be kept synchronized
55 * in a cluster. It implements the RAFT algorithm as described in the paper
56 * <a href='https://ramcloud.stanford.edu/wiki/download/attachments/11370504/raft.pdf'>
57 * In Search of an Understandable Consensus Algorithm</a>
59 * RaftActor has 3 states and each state has a certain behavior associated
60 * with it. A Raft actor can behave as,
63 * <li> A Follower (or) </li>
64 * <li> A Candidate </li>
68 * A RaftActor MUST be a Leader in order to accept requests from clients to
69 * change the state of it's encapsulated state machine. Once a RaftActor becomes
70 * a Leader it is also responsible for ensuring that all followers ultimately
71 * have the same log and therefore the same state machine as itself.
74 * The current behavior of a RaftActor determines how election for leadership
75 * is initiated and how peer RaftActors react to request for votes.
78 * Each RaftActor also needs to know the current election term. It uses this
79 * information for a couple of things. One is to simply figure out who it
80 * voted for in the last election. Another is to figure out if the message
81 * it received to update it's state is stale.
84 * The RaftActor uses akka-persistence to store it's replicated log.
85 * Furthermore through it's behaviors a Raft Actor determines
88 * <li> when a log entry should be persisted </li>
89 * <li> when a log entry should be applied to the state machine (and) </li>
90 * <li> when a snapshot should be saved </li>
93 public abstract class RaftActor extends AbstractUntypedPersistentActor {
95 private static final long APPLY_STATE_DELAY_THRESHOLD_IN_NANOS = TimeUnit.MILLISECONDS.toNanos(50L); // 50 millis
97 protected final Logger LOG = LoggerFactory.getLogger(getClass());
100 * The current state determines the current behavior of a RaftActor
101 * A Raft Actor always starts off in the Follower State
103 private final DelegatingRaftActorBehavior currentBehavior = new DelegatingRaftActorBehavior();
106 * This context should NOT be passed directly to any other actor it is
107 * only to be consumed by the RaftActorBehaviors
109 private final RaftActorContextImpl context;
111 private final DelegatingPersistentDataProvider delegatingPersistenceProvider = new DelegatingPersistentDataProvider(null);
113 private RaftActorRecoverySupport raftRecovery;
115 private RaftActorSnapshotMessageSupport snapshotSupport;
117 private final BehaviorStateHolder reusableBehaviorStateHolder = new BehaviorStateHolder();
119 private final SwitchBehaviorSupplier reusableSwitchBehaviorSupplier = new SwitchBehaviorSupplier();
121 public RaftActor(String id, Map<String, String> peerAddresses,
122 Optional<ConfigParams> configParams, short payloadVersion) {
124 context = new RaftActorContextImpl(this.getSelf(),
125 this.getContext(), id, new ElectionTermImpl(delegatingPersistenceProvider, id, LOG),
126 -1, -1, peerAddresses,
127 (configParams.isPresent() ? configParams.get(): new DefaultConfigParamsImpl()),
128 delegatingPersistenceProvider, LOG);
130 context.setPayloadVersion(payloadVersion);
131 context.setReplicatedLog(ReplicatedLogImpl.newInstance(context, currentBehavior));
135 public void preStart() throws Exception {
136 LOG.info("Starting recovery for {} with journal batch size {}", persistenceId(),
137 context.getConfigParams().getJournalRecoveryLogBatchSize());
141 snapshotSupport = newRaftActorSnapshotMessageSupport();
145 public void postStop() {
146 if(currentBehavior.getDelegate() != null) {
148 currentBehavior.close();
149 } catch (Exception e) {
150 LOG.debug("{}: Error closing behavior {}", persistenceId(), currentBehavior.state());
158 public void handleRecover(Object message) {
159 if(raftRecovery == null) {
160 raftRecovery = newRaftActorRecoverySupport();
163 boolean recoveryComplete = raftRecovery.handleRecoveryMessage(message);
164 if(recoveryComplete) {
165 if(!persistence().isRecoveryApplicable()) {
166 // Delete all the messages from the akka journal so that we do not end up with consistency issues
167 // Note I am not using the dataPersistenceProvider and directly using the akka api here
168 deleteMessages(lastSequenceNr());
170 // Delete all the akka snapshots as they will not be needed
171 deleteSnapshots(new SnapshotSelectionCriteria(scala.Long.MaxValue(), scala.Long.MaxValue()));
174 onRecoveryComplete();
176 initializeBehavior();
182 protected RaftActorRecoverySupport newRaftActorRecoverySupport() {
183 return new RaftActorRecoverySupport(context, currentBehavior, getRaftActorRecoveryCohort());
186 protected void initializeBehavior(){
187 changeCurrentBehavior(new Follower(context));
190 protected void changeCurrentBehavior(RaftActorBehavior newBehavior){
191 reusableBehaviorStateHolder.init(getCurrentBehavior());
192 setCurrentBehavior(newBehavior);
193 handleBehaviorChange(reusableBehaviorStateHolder, getCurrentBehavior());
197 public void handleCommand(final Object message) {
198 if (message instanceof ApplyState){
199 ApplyState applyState = (ApplyState) message;
201 long elapsedTime = (System.nanoTime() - applyState.getStartTime());
202 if(elapsedTime >= APPLY_STATE_DELAY_THRESHOLD_IN_NANOS){
203 LOG.warn("ApplyState took more time than expected. Elapsed Time = {} ms ApplyState = {}",
204 TimeUnit.NANOSECONDS.toMillis(elapsedTime), applyState);
207 if(LOG.isDebugEnabled()) {
208 LOG.debug("{}: Applying state for log index {} data {}",
209 persistenceId(), applyState.getReplicatedLogEntry().getIndex(),
210 applyState.getReplicatedLogEntry().getData());
213 applyState(applyState.getClientActor(), applyState.getIdentifier(),
214 applyState.getReplicatedLogEntry().getData());
216 if (!hasFollowers()) {
217 // for single node, the capture should happen after the apply state
218 // as we delete messages from the persistent journal which have made it to the snapshot
219 // capturing the snapshot before applying makes the persistent journal and snapshot out of sync
220 // and recovery shows data missing
221 context.getReplicatedLog().captureSnapshotIfReady(applyState.getReplicatedLogEntry());
223 context.getSnapshotManager().trimLog(context.getLastApplied(), currentBehavior);
226 } else if (message instanceof ApplyJournalEntries){
227 ApplyJournalEntries applyEntries = (ApplyJournalEntries) message;
228 if(LOG.isDebugEnabled()) {
229 LOG.debug("{}: Persisting ApplyLogEntries with index={}", persistenceId(), applyEntries.getToIndex());
232 persistence().persist(applyEntries, NoopProcedure.instance());
234 } else if (message instanceof FindLeader) {
236 new FindLeaderReply(getLeaderAddress()),
239 } else if(message instanceof GetOnDemandRaftState) {
240 onGetOnDemandRaftStats();
241 } else if(message instanceof SwitchBehavior){
242 switchBehavior(((SwitchBehavior) message));
243 } else if(!snapshotSupport.handleSnapshotMessage(message)) {
244 switchBehavior(reusableSwitchBehaviorSupplier.handleMessage(getSender(), message));
248 private void switchBehavior(SwitchBehavior message) {
249 if(!getRaftActorContext().getRaftPolicy().automaticElectionsEnabled()) {
250 RaftState newState = message.getNewState();
251 if( newState == RaftState.Leader || newState == RaftState.Follower) {
252 switchBehavior(reusableSwitchBehaviorSupplier.handleMessage(getSender(), message));
253 getRaftActorContext().getTermInformation().updateAndPersist(message.getNewTerm(), "");
255 LOG.warn("Switching to behavior : {} - not supported", newState);
260 private void switchBehavior(Supplier<RaftActorBehavior> supplier){
261 reusableBehaviorStateHolder.init(getCurrentBehavior());
263 setCurrentBehavior(supplier.get());
265 handleBehaviorChange(reusableBehaviorStateHolder, getCurrentBehavior());
268 protected RaftActorSnapshotMessageSupport newRaftActorSnapshotMessageSupport() {
269 return new RaftActorSnapshotMessageSupport(context, currentBehavior,
270 getRaftActorSnapshotCohort());
273 private void onGetOnDemandRaftStats() {
274 // Debugging message to retrieve raft stats.
276 OnDemandRaftState.Builder builder = OnDemandRaftState.builder()
277 .commitIndex(context.getCommitIndex())
278 .currentTerm(context.getTermInformation().getCurrentTerm())
279 .inMemoryJournalDataSize(replicatedLog().dataSize())
280 .inMemoryJournalLogSize(replicatedLog().size())
281 .isSnapshotCaptureInitiated(context.getSnapshotManager().isCapturing())
282 .lastApplied(context.getLastApplied())
283 .lastIndex(replicatedLog().lastIndex())
284 .lastTerm(replicatedLog().lastTerm())
285 .leader(getLeaderId())
286 .raftState(currentBehavior.state().toString())
287 .replicatedToAllIndex(currentBehavior.getReplicatedToAllIndex())
288 .snapshotIndex(replicatedLog().getSnapshotIndex())
289 .snapshotTerm(replicatedLog().getSnapshotTerm())
290 .votedFor(context.getTermInformation().getVotedFor())
291 .peerAddresses(new HashMap<>(context.getPeerAddresses()));
293 ReplicatedLogEntry lastLogEntry = getLastLogEntry();
294 if (lastLogEntry != null) {
295 builder.lastLogIndex(lastLogEntry.getIndex());
296 builder.lastLogTerm(lastLogEntry.getTerm());
299 if(getCurrentBehavior() instanceof AbstractLeader) {
300 AbstractLeader leader = (AbstractLeader)getCurrentBehavior();
301 Collection<String> followerIds = leader.getFollowerIds();
302 List<FollowerInfo> followerInfoList = Lists.newArrayListWithCapacity(followerIds.size());
303 for(String id: followerIds) {
304 final FollowerLogInformation info = leader.getFollower(id);
305 followerInfoList.add(new FollowerInfo(id, info.getNextIndex(), info.getMatchIndex(),
306 info.isFollowerActive(), DurationFormatUtils.formatDurationHMS(info.timeSinceLastActivity())));
309 builder.followerInfoList(followerInfoList);
312 sender().tell(builder.build(), self());
316 private void handleBehaviorChange(BehaviorStateHolder oldBehaviorState, RaftActorBehavior currentBehavior) {
317 RaftActorBehavior oldBehavior = oldBehaviorState.getBehavior();
319 if (oldBehavior != currentBehavior){
323 String lastValidLeaderId = oldBehavior == null ? null : oldBehaviorState.getLastValidLeaderId();
324 String oldBehaviorStateName = oldBehavior == null ? null : oldBehavior.state().name();
326 // it can happen that the state has not changed but the leader has changed.
327 Optional<ActorRef> roleChangeNotifier = getRoleChangeNotifier();
328 if(!Objects.equal(lastValidLeaderId, currentBehavior.getLeaderId()) ||
329 oldBehaviorState.getLeaderPayloadVersion() != currentBehavior.getLeaderPayloadVersion()) {
330 if(roleChangeNotifier.isPresent()) {
331 roleChangeNotifier.get().tell(newLeaderStateChanged(getId(), currentBehavior.getLeaderId(),
332 currentBehavior.getLeaderPayloadVersion()), getSelf());
335 onLeaderChanged(lastValidLeaderId, currentBehavior.getLeaderId());
338 if (roleChangeNotifier.isPresent() &&
339 (oldBehavior == null || (oldBehavior.state() != currentBehavior.state()))) {
340 roleChangeNotifier.get().tell(new RoleChanged(getId(), oldBehaviorStateName ,
341 currentBehavior.state().name()), getSelf());
345 protected LeaderStateChanged newLeaderStateChanged(String memberId, String leaderId, short leaderPayloadVersion) {
346 return new LeaderStateChanged(memberId, leaderId, leaderPayloadVersion);
350 public long snapshotSequenceNr() {
351 // When we do a snapshot capture, we also capture and save the sequence-number of the persistent journal,
352 // so that we can delete the persistent journal based on the saved sequence-number
353 // However , when akka replays the journal during recovery, it replays it from the sequence number when the snapshot
354 // was saved and not the number we saved.
355 // We would want to override it , by asking akka to use the last-sequence number known to us.
356 return context.getSnapshotManager().getLastSequenceNumber();
360 * When a derived RaftActor needs to persist something it must call
367 protected void persistData(final ActorRef clientActor, final String identifier,
368 final Payload data) {
370 ReplicatedLogEntry replicatedLogEntry = new ReplicatedLogImplEntry(
371 context.getReplicatedLog().lastIndex() + 1,
372 context.getTermInformation().getCurrentTerm(), data);
374 if(LOG.isDebugEnabled()) {
375 LOG.debug("{}: Persist data {}", persistenceId(), replicatedLogEntry);
378 final RaftActorContext raftContext = getRaftActorContext();
380 replicatedLog().appendAndPersist(replicatedLogEntry, new Procedure<ReplicatedLogEntry>() {
382 public void apply(ReplicatedLogEntry replicatedLogEntry) throws Exception {
383 if (!hasFollowers()){
384 // Increment the Commit Index and the Last Applied values
385 raftContext.setCommitIndex(replicatedLogEntry.getIndex());
386 raftContext.setLastApplied(replicatedLogEntry.getIndex());
388 // Apply the state immediately.
389 self().tell(new ApplyState(clientActor, identifier, replicatedLogEntry), self());
391 // Send a ApplyJournalEntries message so that we write the fact that we applied
392 // the state to durable storage
393 self().tell(new ApplyJournalEntries(replicatedLogEntry.getIndex()), self());
395 } else if (clientActor != null) {
396 context.getReplicatedLog().captureSnapshotIfReady(replicatedLogEntry);
398 // Send message for replication
399 currentBehavior.handleMessage(getSelf(),
400 new Replicate(clientActor, identifier, replicatedLogEntry));
406 private ReplicatedLog replicatedLog() {
407 return context.getReplicatedLog();
410 protected String getId() {
411 return context.getId();
415 void setCurrentBehavior(RaftActorBehavior behavior) {
416 currentBehavior.setDelegate(behavior);
419 protected RaftActorBehavior getCurrentBehavior() {
420 return currentBehavior.getDelegate();
424 * Derived actors can call the isLeader method to check if the current
425 * RaftActor is the Leader or not
427 * @return true it this RaftActor is a Leader false otherwise
429 protected boolean isLeader() {
430 return context.getId().equals(currentBehavior.getLeaderId());
434 * Derived actor can call getLeader if they need a reference to the Leader.
435 * This would be useful for example in forwarding a request to an actor
436 * which is the leader
438 * @return A reference to the leader if known, null otherwise
440 protected ActorSelection getLeader(){
441 String leaderAddress = getLeaderAddress();
443 if(leaderAddress == null){
447 return context.actorSelection(leaderAddress);
452 * @return the current leader's id
454 protected String getLeaderId(){
455 return currentBehavior.getLeaderId();
458 protected RaftState getRaftState() {
459 return currentBehavior.state();
462 protected ReplicatedLogEntry getLastLogEntry() {
463 return replicatedLog().last();
466 protected Long getCurrentTerm(){
467 return context.getTermInformation().getCurrentTerm();
470 protected Long getCommitIndex(){
471 return context.getCommitIndex();
474 protected Long getLastApplied(){
475 return context.getLastApplied();
478 protected RaftActorContext getRaftActorContext() {
482 protected void updateConfigParams(ConfigParams configParams) {
483 context.setConfigParams(configParams);
486 public final DataPersistenceProvider persistence() {
487 return delegatingPersistenceProvider.getDelegate();
490 public void setPersistence(DataPersistenceProvider provider) {
491 delegatingPersistenceProvider.setDelegate(provider);
494 protected void setPersistence(boolean persistent) {
496 setPersistence(new PersistentDataProvider(this));
498 setPersistence(new NonPersistentDataProvider() {
500 * The way snapshotting works is,
502 * <li> RaftActor calls createSnapshot on the Shard
503 * <li> Shard sends a CaptureSnapshotReply and RaftActor then calls saveSnapshot
504 * <li> When saveSnapshot is invoked on the akka-persistence API it uses the SnapshotStore to save
505 * the snapshot. The SnapshotStore sends SaveSnapshotSuccess or SaveSnapshotFailure. When the
506 * RaftActor gets SaveSnapshot success it commits the snapshot to the in-memory journal. This
507 * commitSnapshot is mimicking what is done in SaveSnapshotSuccess.
511 public void saveSnapshot(Object o) {
512 // Make saving Snapshot successful
513 // Committing the snapshot here would end up calling commit in the creating state which would
514 // be a state violation. That's why now we send a message to commit the snapshot.
515 self().tell(RaftActorSnapshotMessageSupport.COMMIT_SNAPSHOT, self());
522 * setPeerAddress sets the address of a known peer at a later time.
524 * This is to account for situations where a we know that a peer
525 * exists but we do not know an address up-front. This may also be used in
526 * situations where a known peer starts off in a different location and we
527 * need to change it's address
529 * Note that if the peerId does not match the list of peers passed to
530 * this actor during construction an IllegalStateException will be thrown.
535 protected void setPeerAddress(String peerId, String peerAddress){
536 context.setPeerAddress(peerId, peerAddress);
540 * The applyState method will be called by the RaftActor when some data
541 * needs to be applied to the actor's state
543 * @param clientActor A reference to the client who sent this message. This
544 * is the same reference that was passed to persistData
545 * by the derived actor. clientActor may be null when
546 * the RaftActor is behaving as a follower or during
548 * @param identifier The identifier of the persisted data. This is also
549 * the same identifier that was passed to persistData by
550 * the derived actor. identifier may be null when
551 * the RaftActor is behaving as a follower or during
553 * @param data A piece of data that was persisted by the persistData call.
554 * This should NEVER be null.
556 protected abstract void applyState(ActorRef clientActor, String identifier,
560 * Returns the RaftActorRecoveryCohort to participate in persistence recovery.
563 protected abstract RaftActorRecoveryCohort getRaftActorRecoveryCohort();
566 * This method is called when recovery is complete.
568 protected abstract void onRecoveryComplete();
571 * Returns the RaftActorSnapshotCohort to participate in persistence recovery.
574 protected abstract RaftActorSnapshotCohort getRaftActorSnapshotCohort();
577 * This method will be called by the RaftActor when the state of the
578 * RaftActor changes. The derived actor can then use methods like
579 * isLeader or getLeader to do something useful
581 protected abstract void onStateChanged();
584 * Notifier Actor for this RaftActor to notify when a role change happens
585 * @return ActorRef - ActorRef of the notifier or Optional.absent if none.
587 protected abstract Optional<ActorRef> getRoleChangeNotifier();
589 protected void onLeaderChanged(String oldLeader, String newLeader){};
591 private String getLeaderAddress(){
593 return getSelf().path().toString();
595 String leaderId = currentBehavior.getLeaderId();
596 if (leaderId == null) {
599 String peerAddress = context.getPeerAddress(leaderId);
600 if(LOG.isDebugEnabled()) {
601 LOG.debug("{}: getLeaderAddress leaderId = {} peerAddress = {}",
602 persistenceId(), leaderId, peerAddress);
608 protected boolean hasFollowers(){
609 return getRaftActorContext().hasFollowers();
613 * @deprecated Deprecated in favor of {@link org.opendaylight.controller.cluster.raft.base.messages.DeleteEntries}
614 * whose type for fromIndex is long instead of int. This class was kept for backwards
615 * compatibility with Helium.
617 // Suppressing this warning as we can't set serialVersionUID to maintain backwards compatibility.
618 @SuppressWarnings("serial")
620 static class DeleteEntries implements Serializable {
621 private final int fromIndex;
623 public DeleteEntries(int fromIndex) {
624 this.fromIndex = fromIndex;
627 public int getFromIndex() {
633 * @deprecated Deprecated in favor of non-inner class {@link org.opendaylight.controller.cluster.raft.base.messages.UpdateElectionTerm}
634 * which has serialVersionUID set. This class was kept for backwards compatibility with Helium.
636 // Suppressing this warning as we can't set serialVersionUID to maintain backwards compatibility.
637 @SuppressWarnings("serial")
639 static class UpdateElectionTerm implements Serializable {
640 private final long currentTerm;
641 private final String votedFor;
643 public UpdateElectionTerm(long currentTerm, String votedFor) {
644 this.currentTerm = currentTerm;
645 this.votedFor = votedFor;
648 public long getCurrentTerm() {
652 public String getVotedFor() {
657 private static class BehaviorStateHolder {
658 private RaftActorBehavior behavior;
659 private String lastValidLeaderId;
660 private short leaderPayloadVersion;
662 void init(RaftActorBehavior behavior) {
663 this.behavior = behavior;
664 this.leaderPayloadVersion = behavior != null ? behavior.getLeaderPayloadVersion() : -1;
666 String behaviorLeaderId = behavior != null ? behavior.getLeaderId() : null;
667 if(behaviorLeaderId != null) {
668 this.lastValidLeaderId = behaviorLeaderId;
672 RaftActorBehavior getBehavior() {
676 String getLastValidLeaderId() {
677 return lastValidLeaderId;
680 short getLeaderPayloadVersion() {
681 return leaderPayloadVersion;
685 private class SwitchBehaviorSupplier implements Supplier<RaftActorBehavior> {
686 private Object message;
687 private ActorRef sender;
689 public SwitchBehaviorSupplier handleMessage(ActorRef sender, Object message){
690 this.sender = sender;
691 this.message = message;
696 public RaftActorBehavior get() {
697 if(this.message instanceof SwitchBehavior){
698 return ((SwitchBehavior) message).getNewState().createBehavior(getRaftActorContext());
700 return currentBehavior.handleMessage(sender, message);