2 * Copyright (c) 2014 Cisco Systems, Inc. and others. All rights reserved.
4 * This program and the accompanying materials are made available under the
5 * terms of the Eclipse Public License v1.0 which accompanies this distribution,
6 * and is available at http://www.eclipse.org/legal/epl-v10.html
9 package org.opendaylight.controller.cluster.raft;
11 import akka.actor.ActorRef;
12 import akka.actor.ActorSelection;
13 import akka.japi.Procedure;
14 import akka.persistence.SaveSnapshotFailure;
15 import akka.persistence.SaveSnapshotSuccess;
16 import akka.persistence.SnapshotSelectionCriteria;
17 import com.google.common.annotations.VisibleForTesting;
18 import com.google.common.base.Objects;
19 import com.google.common.base.Optional;
20 import com.google.common.collect.ImmutableMap;
21 import com.google.common.collect.Lists;
22 import java.io.Serializable;
23 import java.util.Collection;
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.ApplySnapshot;
38 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
39 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshot;
40 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshotReply;
41 import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
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.protobuff.client.messages.Payload;
52 import org.slf4j.Logger;
53 import org.slf4j.LoggerFactory;
56 * RaftActor encapsulates a state machine that needs to be kept synchronized
57 * in a cluster. It implements the RAFT algorithm as described in the paper
58 * <a href='https://ramcloud.stanford.edu/wiki/download/attachments/11370504/raft.pdf'>
59 * In Search of an Understandable Consensus Algorithm</a>
61 * RaftActor has 3 states and each state has a certain behavior associated
62 * with it. A Raft actor can behave as,
65 * <li> A Follower (or) </li>
66 * <li> A Candidate </li>
70 * A RaftActor MUST be a Leader in order to accept requests from clients to
71 * change the state of it's encapsulated state machine. Once a RaftActor becomes
72 * a Leader it is also responsible for ensuring that all followers ultimately
73 * have the same log and therefore the same state machine as itself.
76 * The current behavior of a RaftActor determines how election for leadership
77 * is initiated and how peer RaftActors react to request for votes.
80 * Each RaftActor also needs to know the current election term. It uses this
81 * information for a couple of things. One is to simply figure out who it
82 * voted for in the last election. Another is to figure out if the message
83 * it received to update it's state is stale.
86 * The RaftActor uses akka-persistence to store it's replicated log.
87 * Furthermore through it's behaviors a Raft Actor determines
90 * <li> when a log entry should be persisted </li>
91 * <li> when a log entry should be applied to the state machine (and) </li>
92 * <li> when a snapshot should be saved </li>
95 public abstract class RaftActor extends AbstractUntypedPersistentActor {
97 private static final long APPLY_STATE_DELAY_THRESHOLD_IN_NANOS = TimeUnit.MILLISECONDS.toNanos(50L); // 50 millis
99 private static final String COMMIT_SNAPSHOT = "commit_snapshot";
101 protected final Logger LOG = LoggerFactory.getLogger(getClass());
104 * The current state determines the current behavior of a RaftActor
105 * A Raft Actor always starts off in the Follower State
107 private final DelegatingRaftActorBehavior currentBehavior = new DelegatingRaftActorBehavior();
110 * This context should NOT be passed directly to any other actor it is
111 * only to be consumed by the RaftActorBehaviors
113 private final RaftActorContextImpl context;
115 private final DelegatingPersistentDataProvider delegatingPersistenceProvider = new DelegatingPersistentDataProvider(null);
117 private final Procedure<Void> createSnapshotProcedure = new CreateSnapshotProcedure();
119 private RaftActorRecoverySupport raftRecovery;
121 private final BehaviorStateHolder reusableBehaviorStateHolder = new BehaviorStateHolder();
123 public RaftActor(String id, Map<String, String> peerAddresses) {
124 this(id, peerAddresses, Optional.<ConfigParams>absent());
127 public RaftActor(String id, Map<String, String> peerAddresses,
128 Optional<ConfigParams> configParams) {
130 context = new RaftActorContextImpl(this.getSelf(),
131 this.getContext(), id, new ElectionTermImpl(delegatingPersistenceProvider, id, LOG),
132 -1, -1, peerAddresses,
133 (configParams.isPresent() ? configParams.get(): new DefaultConfigParamsImpl()), LOG);
135 context.setReplicatedLog(ReplicatedLogImpl.newInstance(context, delegatingPersistenceProvider, currentBehavior));
139 public void preStart() throws Exception {
140 LOG.info("Starting recovery for {} with journal batch size {}", persistenceId(),
141 context.getConfigParams().getJournalRecoveryLogBatchSize());
147 public void postStop() {
148 if(currentBehavior != null) {
150 currentBehavior.close();
151 } catch (Exception e) {
152 LOG.debug("{}: Error closing behavior {}", persistenceId(), currentBehavior.state());
160 public void handleRecover(Object message) {
161 if(raftRecovery == null) {
162 raftRecovery = new RaftActorRecoverySupport(delegatingPersistenceProvider, context, currentBehavior,
163 getRaftActorRecoveryCohort());
166 boolean recoveryComplete = raftRecovery.handleRecoveryMessage(message);
167 if(recoveryComplete) {
168 if(!persistence().isRecoveryApplicable()) {
169 // Delete all the messages from the akka journal so that we do not end up with consistency issues
170 // Note I am not using the dataPersistenceProvider and directly using the akka api here
171 deleteMessages(lastSequenceNr());
173 // Delete all the akka snapshots as they will not be needed
174 deleteSnapshots(new SnapshotSelectionCriteria(scala.Long.MaxValue(), scala.Long.MaxValue()));
177 onRecoveryComplete();
179 initializeBehavior();
185 protected void initializeBehavior(){
186 changeCurrentBehavior(new Follower(context));
189 protected void changeCurrentBehavior(RaftActorBehavior newBehavior){
190 reusableBehaviorStateHolder.init(getCurrentBehavior());
191 setCurrentBehavior(newBehavior);
192 handleBehaviorChange(reusableBehaviorStateHolder, getCurrentBehavior());
195 @Override public void handleCommand(Object message) {
196 if (message instanceof ApplyState){
197 ApplyState applyState = (ApplyState) message;
199 long elapsedTime = (System.nanoTime() - applyState.getStartTime());
200 if(elapsedTime >= APPLY_STATE_DELAY_THRESHOLD_IN_NANOS){
201 LOG.warn("ApplyState took more time than expected. Elapsed Time = {} ms ApplyState = {}",
202 TimeUnit.NANOSECONDS.toMillis(elapsedTime), applyState);
205 if(LOG.isDebugEnabled()) {
206 LOG.debug("{}: Applying state for log index {} data {}",
207 persistenceId(), applyState.getReplicatedLogEntry().getIndex(),
208 applyState.getReplicatedLogEntry().getData());
211 applyState(applyState.getClientActor(), applyState.getIdentifier(),
212 applyState.getReplicatedLogEntry().getData());
214 } else if (message instanceof ApplyJournalEntries){
215 ApplyJournalEntries applyEntries = (ApplyJournalEntries) message;
216 if(LOG.isDebugEnabled()) {
217 LOG.debug("{}: Persisting ApplyLogEntries with index={}", persistenceId(), applyEntries.getToIndex());
220 persistence().persist(applyEntries, NoopProcedure.instance());
222 } else if(message instanceof ApplySnapshot ) {
223 Snapshot snapshot = ((ApplySnapshot) message).getSnapshot();
225 if(LOG.isDebugEnabled()) {
226 LOG.debug("{}: ApplySnapshot called on Follower Actor " +
227 "snapshotIndex:{}, snapshotTerm:{}", persistenceId(), snapshot.getLastAppliedIndex(),
228 snapshot.getLastAppliedTerm()
232 applySnapshot(snapshot.getState());
234 //clears the followers log, sets the snapshot index to ensure adjusted-index works
235 context.setReplicatedLog(ReplicatedLogImpl.newInstance(snapshot, context, delegatingPersistenceProvider,
237 context.setLastApplied(snapshot.getLastAppliedIndex());
239 } else if (message instanceof FindLeader) {
241 new FindLeaderReply(getLeaderAddress()),
245 } else if (message instanceof SaveSnapshotSuccess) {
246 SaveSnapshotSuccess success = (SaveSnapshotSuccess) message;
247 LOG.info("{}: SaveSnapshotSuccess received for snapshot", persistenceId());
249 long sequenceNumber = success.metadata().sequenceNr();
251 commitSnapshot(sequenceNumber);
253 } else if (message instanceof SaveSnapshotFailure) {
254 SaveSnapshotFailure saveSnapshotFailure = (SaveSnapshotFailure) message;
256 LOG.error("{}: SaveSnapshotFailure received for snapshot Cause:",
257 persistenceId(), saveSnapshotFailure.cause());
259 context.getSnapshotManager().rollback();
261 } else if (message instanceof CaptureSnapshot) {
262 LOG.debug("{}: CaptureSnapshot received by actor: {}", persistenceId(), message);
264 context.getSnapshotManager().create(createSnapshotProcedure);
266 } else if (message instanceof CaptureSnapshotReply) {
267 handleCaptureSnapshotReply(((CaptureSnapshotReply) message).getSnapshot());
268 } else if(message instanceof GetOnDemandRaftState) {
269 onGetOnDemandRaftStats();
270 } else if (message.equals(COMMIT_SNAPSHOT)) {
273 reusableBehaviorStateHolder.init(getCurrentBehavior());
275 setCurrentBehavior(currentBehavior.handleMessage(getSender(), message));
277 handleBehaviorChange(reusableBehaviorStateHolder, getCurrentBehavior());
281 private void onGetOnDemandRaftStats() {
282 // Debugging message to retrieve raft stats.
284 OnDemandRaftState.Builder builder = OnDemandRaftState.builder()
285 .commitIndex(context.getCommitIndex())
286 .currentTerm(context.getTermInformation().getCurrentTerm())
287 .inMemoryJournalDataSize(replicatedLog().dataSize())
288 .inMemoryJournalLogSize(replicatedLog().size())
289 .isSnapshotCaptureInitiated(context.getSnapshotManager().isCapturing())
290 .lastApplied(context.getLastApplied())
291 .lastIndex(replicatedLog().lastIndex())
292 .lastTerm(replicatedLog().lastTerm())
293 .leader(getLeaderId())
294 .raftState(currentBehavior.state().toString())
295 .replicatedToAllIndex(currentBehavior.getReplicatedToAllIndex())
296 .snapshotIndex(replicatedLog().getSnapshotIndex())
297 .snapshotTerm(replicatedLog().getSnapshotTerm())
298 .votedFor(context.getTermInformation().getVotedFor())
299 .peerAddresses(ImmutableMap.copyOf(context.getPeerAddresses()));
301 ReplicatedLogEntry lastLogEntry = getLastLogEntry();
302 if (lastLogEntry != null) {
303 builder.lastLogIndex(lastLogEntry.getIndex());
304 builder.lastLogTerm(lastLogEntry.getTerm());
307 if(getCurrentBehavior() instanceof AbstractLeader) {
308 AbstractLeader leader = (AbstractLeader)getCurrentBehavior();
309 Collection<String> followerIds = leader.getFollowerIds();
310 List<FollowerInfo> followerInfoList = Lists.newArrayListWithCapacity(followerIds.size());
311 for(String id: followerIds) {
312 final FollowerLogInformation info = leader.getFollower(id);
313 followerInfoList.add(new FollowerInfo(id, info.getNextIndex(), info.getMatchIndex(),
314 info.isFollowerActive(), DurationFormatUtils.formatDurationHMS(info.timeSinceLastActivity())));
317 builder.followerInfoList(followerInfoList);
320 sender().tell(builder.build(), self());
324 private void handleBehaviorChange(BehaviorStateHolder oldBehaviorState, RaftActorBehavior currentBehavior) {
325 RaftActorBehavior oldBehavior = oldBehaviorState.getBehavior();
327 if (oldBehavior != currentBehavior){
331 String oldBehaviorLeaderId = oldBehavior == null ? null : oldBehaviorState.getLeaderId();
332 String oldBehaviorStateName = oldBehavior == null ? null : oldBehavior.state().name();
334 // it can happen that the state has not changed but the leader has changed.
335 Optional<ActorRef> roleChangeNotifier = getRoleChangeNotifier();
336 if(!Objects.equal(oldBehaviorLeaderId, currentBehavior.getLeaderId())) {
337 if(roleChangeNotifier.isPresent()) {
338 roleChangeNotifier.get().tell(new LeaderStateChanged(getId(), currentBehavior.getLeaderId()), getSelf());
341 onLeaderChanged(oldBehaviorLeaderId, currentBehavior.getLeaderId());
344 if (roleChangeNotifier.isPresent() &&
345 (oldBehavior == null || (oldBehavior.state() != currentBehavior.state()))) {
346 roleChangeNotifier.get().tell(new RoleChanged(getId(), oldBehaviorStateName ,
347 currentBehavior.state().name()), getSelf());
352 * When a derived RaftActor needs to persist something it must call
359 protected void persistData(final ActorRef clientActor, final String identifier,
360 final Payload data) {
362 ReplicatedLogEntry replicatedLogEntry = new ReplicatedLogImplEntry(
363 context.getReplicatedLog().lastIndex() + 1,
364 context.getTermInformation().getCurrentTerm(), data);
366 if(LOG.isDebugEnabled()) {
367 LOG.debug("{}: Persist data {}", persistenceId(), replicatedLogEntry);
370 final RaftActorContext raftContext = getRaftActorContext();
372 replicatedLog().appendAndPersist(replicatedLogEntry, new Procedure<ReplicatedLogEntry>() {
374 public void apply(ReplicatedLogEntry replicatedLogEntry) throws Exception {
376 // Increment the Commit Index and the Last Applied values
377 raftContext.setCommitIndex(replicatedLogEntry.getIndex());
378 raftContext.setLastApplied(replicatedLogEntry.getIndex());
380 // Apply the state immediately
381 applyState(clientActor, identifier, data);
383 // Send a ApplyJournalEntries message so that we write the fact that we applied
384 // the state to durable storage
385 self().tell(new ApplyJournalEntries(replicatedLogEntry.getIndex()), self());
387 context.getSnapshotManager().trimLog(context.getLastApplied(), currentBehavior);
389 } else if (clientActor != null) {
390 // Send message for replication
391 currentBehavior.handleMessage(getSelf(),
392 new Replicate(clientActor, identifier, replicatedLogEntry));
398 private ReplicatedLog replicatedLog() {
399 return context.getReplicatedLog();
402 protected String getId() {
403 return context.getId();
407 void setCurrentBehavior(RaftActorBehavior behavior) {
408 currentBehavior.setDelegate(behavior);
411 protected RaftActorBehavior getCurrentBehavior() {
412 return currentBehavior.getDelegate();
416 * Derived actors can call the isLeader method to check if the current
417 * RaftActor is the Leader or not
419 * @return true it this RaftActor is a Leader false otherwise
421 protected boolean isLeader() {
422 return context.getId().equals(currentBehavior.getLeaderId());
426 * Derived actor can call getLeader if they need a reference to the Leader.
427 * This would be useful for example in forwarding a request to an actor
428 * which is the leader
430 * @return A reference to the leader if known, null otherwise
432 protected ActorSelection getLeader(){
433 String leaderAddress = getLeaderAddress();
435 if(leaderAddress == null){
439 return context.actorSelection(leaderAddress);
444 * @return the current leader's id
446 protected String getLeaderId(){
447 return currentBehavior.getLeaderId();
450 protected RaftState getRaftState() {
451 return currentBehavior.state();
454 protected ReplicatedLogEntry getLastLogEntry() {
455 return replicatedLog().last();
458 protected Long getCurrentTerm(){
459 return context.getTermInformation().getCurrentTerm();
462 protected Long getCommitIndex(){
463 return context.getCommitIndex();
466 protected Long getLastApplied(){
467 return context.getLastApplied();
470 protected RaftActorContext getRaftActorContext() {
474 protected void updateConfigParams(ConfigParams configParams) {
475 context.setConfigParams(configParams);
478 public final DataPersistenceProvider persistence() {
479 return delegatingPersistenceProvider.getDelegate();
482 public void setPersistence(DataPersistenceProvider provider) {
483 delegatingPersistenceProvider.setDelegate(provider);
486 protected void setPersistence(boolean persistent) {
488 setPersistence(new PersistentDataProvider(this));
490 setPersistence(new NonPersistentDataProvider() {
492 * The way snapshotting works is,
494 * <li> RaftActor calls createSnapshot on the Shard
495 * <li> Shard sends a CaptureSnapshotReply and RaftActor then calls saveSnapshot
496 * <li> When saveSnapshot is invoked on the akka-persistence API it uses the SnapshotStore to save
497 * the snapshot. The SnapshotStore sends SaveSnapshotSuccess or SaveSnapshotFailure. When the
498 * RaftActor gets SaveSnapshot success it commits the snapshot to the in-memory journal. This
499 * commitSnapshot is mimicking what is done in SaveSnapshotSuccess.
503 public void saveSnapshot(Object o) {
504 // Make saving Snapshot successful
505 // Committing the snapshot here would end up calling commit in the creating state which would
506 // be a state violation. That's why now we send a message to commit the snapshot.
507 self().tell(COMMIT_SNAPSHOT, self());
514 * setPeerAddress sets the address of a known peer at a later time.
516 * This is to account for situations where a we know that a peer
517 * exists but we do not know an address up-front. This may also be used in
518 * situations where a known peer starts off in a different location and we
519 * need to change it's address
521 * Note that if the peerId does not match the list of peers passed to
522 * this actor during construction an IllegalStateException will be thrown.
527 protected void setPeerAddress(String peerId, String peerAddress){
528 context.setPeerAddress(peerId, peerAddress);
531 protected void commitSnapshot(long sequenceNumber) {
532 context.getSnapshotManager().commit(persistence(), sequenceNumber);
536 * The applyState method will be called by the RaftActor when some data
537 * needs to be applied to the actor's state
539 * @param clientActor A reference to the client who sent this message. This
540 * is the same reference that was passed to persistData
541 * by the derived actor. clientActor may be null when
542 * the RaftActor is behaving as a follower or during
544 * @param identifier The identifier of the persisted data. This is also
545 * the same identifier that was passed to persistData by
546 * the derived actor. identifier may be null when
547 * the RaftActor is behaving as a follower or during
549 * @param data A piece of data that was persisted by the persistData call.
550 * This should NEVER be null.
552 protected abstract void applyState(ActorRef clientActor, String identifier,
556 * Returns the RaftActorRecoveryCohort to participate in persistence recovery.
559 protected abstract RaftActorRecoveryCohort getRaftActorRecoveryCohort();
562 * This method is called when recovery is complete.
564 protected abstract void onRecoveryComplete();
567 * This method will be called by the RaftActor when a snapshot needs to be
568 * created. The derived actor should respond with its current state.
570 * During recovery the state that is returned by the derived actor will
571 * be passed back to it by calling the applySnapshot method
573 * @return The current state of the actor
575 protected abstract void createSnapshot();
578 * This method can be called at any other point during normal
579 * operations when the derived actor is out of sync with it's peers
580 * and the only way to bring it in sync is by applying a snapshot
582 * @param snapshotBytes A snapshot of the state of the actor
584 protected abstract void applySnapshot(byte[] snapshotBytes);
587 * This method will be called by the RaftActor when the state of the
588 * RaftActor changes. The derived actor can then use methods like
589 * isLeader or getLeader to do something useful
591 protected abstract void onStateChanged();
594 * Notifier Actor for this RaftActor to notify when a role change happens
595 * @return ActorRef - ActorRef of the notifier or Optional.absent if none.
597 protected abstract Optional<ActorRef> getRoleChangeNotifier();
599 protected void onLeaderChanged(String oldLeader, String newLeader){};
601 private String getLeaderAddress(){
603 return getSelf().path().toString();
605 String leaderId = currentBehavior.getLeaderId();
606 if (leaderId == null) {
609 String peerAddress = context.getPeerAddress(leaderId);
610 if(LOG.isDebugEnabled()) {
611 LOG.debug("{}: getLeaderAddress leaderId = {} peerAddress = {}",
612 persistenceId(), leaderId, peerAddress);
618 private void handleCaptureSnapshotReply(byte[] snapshotBytes) {
619 LOG.debug("{}: CaptureSnapshotReply received by actor: snapshot size {}", persistenceId(), snapshotBytes.length);
621 context.getSnapshotManager().persist(persistence(), snapshotBytes, currentBehavior, context.getTotalMemory());
624 protected boolean hasFollowers(){
625 return getRaftActorContext().hasFollowers();
628 static class DeleteEntries implements Serializable {
629 private static final long serialVersionUID = 1L;
630 private final int fromIndex;
632 public DeleteEntries(int fromIndex) {
633 this.fromIndex = fromIndex;
636 public int getFromIndex() {
641 static class UpdateElectionTerm implements Serializable {
642 private static final long serialVersionUID = 1L;
643 private final long currentTerm;
644 private final String votedFor;
646 public UpdateElectionTerm(long currentTerm, String votedFor) {
647 this.currentTerm = currentTerm;
648 this.votedFor = votedFor;
651 public long getCurrentTerm() {
655 public String getVotedFor() {
660 private class CreateSnapshotProcedure implements Procedure<Void> {
663 public void apply(Void aVoid) throws Exception {
668 private static class BehaviorStateHolder {
669 private RaftActorBehavior behavior;
670 private String leaderId;
672 void init(RaftActorBehavior behavior) {
673 this.behavior = behavior;
674 this.leaderId = behavior != null ? behavior.getLeaderId() : null;
677 RaftActorBehavior getBehavior() {
681 String getLeaderId() {