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.collect.Lists;
20 import java.io.Serializable;
21 import java.util.Collection;
22 import java.util.HashMap;
23 import java.util.List;
25 import java.util.concurrent.TimeUnit;
26 import javax.annotation.Nonnull;
27 import org.apache.commons.lang3.time.DurationFormatUtils;
28 import org.opendaylight.controller.cluster.DataPersistenceProvider;
29 import org.opendaylight.controller.cluster.DelegatingPersistentDataProvider;
30 import org.opendaylight.controller.cluster.NonPersistentDataProvider;
31 import org.opendaylight.controller.cluster.PersistentDataProvider;
32 import org.opendaylight.controller.cluster.common.actor.AbstractUntypedPersistentActor;
33 import org.opendaylight.controller.cluster.notifications.LeaderStateChanged;
34 import org.opendaylight.controller.cluster.notifications.RoleChanged;
35 import org.opendaylight.controller.cluster.raft.base.messages.ApplyJournalEntries;
36 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
37 import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
38 import org.opendaylight.controller.cluster.raft.behaviors.AbstractLeader;
39 import org.opendaylight.controller.cluster.raft.behaviors.DelegatingRaftActorBehavior;
40 import org.opendaylight.controller.cluster.raft.behaviors.Follower;
41 import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior;
42 import org.opendaylight.controller.cluster.raft.client.messages.FindLeader;
43 import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply;
44 import org.opendaylight.controller.cluster.raft.client.messages.FollowerInfo;
45 import org.opendaylight.controller.cluster.raft.client.messages.GetOnDemandRaftState;
46 import org.opendaylight.controller.cluster.raft.client.messages.OnDemandRaftState;
47 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
48 import org.slf4j.Logger;
49 import org.slf4j.LoggerFactory;
52 * RaftActor encapsulates a state machine that needs to be kept synchronized
53 * in a cluster. It implements the RAFT algorithm as described in the paper
54 * <a href='https://ramcloud.stanford.edu/wiki/download/attachments/11370504/raft.pdf'>
55 * In Search of an Understandable Consensus Algorithm</a>
57 * RaftActor has 3 states and each state has a certain behavior associated
58 * with it. A Raft actor can behave as,
61 * <li> A Follower (or) </li>
62 * <li> A Candidate </li>
66 * A RaftActor MUST be a Leader in order to accept requests from clients to
67 * change the state of it's encapsulated state machine. Once a RaftActor becomes
68 * a Leader it is also responsible for ensuring that all followers ultimately
69 * have the same log and therefore the same state machine as itself.
72 * The current behavior of a RaftActor determines how election for leadership
73 * is initiated and how peer RaftActors react to request for votes.
76 * Each RaftActor also needs to know the current election term. It uses this
77 * information for a couple of things. One is to simply figure out who it
78 * voted for in the last election. Another is to figure out if the message
79 * it received to update it's state is stale.
82 * The RaftActor uses akka-persistence to store it's replicated log.
83 * Furthermore through it's behaviors a Raft Actor determines
86 * <li> when a log entry should be persisted </li>
87 * <li> when a log entry should be applied to the state machine (and) </li>
88 * <li> when a snapshot should be saved </li>
91 public abstract class RaftActor extends AbstractUntypedPersistentActor {
93 private static final long APPLY_STATE_DELAY_THRESHOLD_IN_NANOS = TimeUnit.MILLISECONDS.toNanos(50L); // 50 millis
95 protected final Logger LOG = LoggerFactory.getLogger(getClass());
98 * The current state determines the current behavior of a RaftActor
99 * A Raft Actor always starts off in the Follower State
101 private final DelegatingRaftActorBehavior currentBehavior = new DelegatingRaftActorBehavior();
104 * This context should NOT be passed directly to any other actor it is
105 * only to be consumed by the RaftActorBehaviors
107 private final RaftActorContextImpl context;
109 private final DelegatingPersistentDataProvider delegatingPersistenceProvider = new DelegatingPersistentDataProvider(null);
111 private RaftActorRecoverySupport raftRecovery;
113 private RaftActorSnapshotMessageSupport snapshotSupport;
115 private final BehaviorStateHolder reusableBehaviorStateHolder = new BehaviorStateHolder();
117 public RaftActor(String id, Map<String, String> peerAddresses) {
118 this(id, peerAddresses, Optional.<ConfigParams>absent());
121 public RaftActor(String id, Map<String, String> peerAddresses,
122 Optional<ConfigParams> configParams) {
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.setReplicatedLog(ReplicatedLogImpl.newInstance(context, currentBehavior));
134 public void preStart() throws Exception {
135 LOG.info("Starting recovery for {} with journal batch size {}", persistenceId(),
136 context.getConfigParams().getJournalRecoveryLogBatchSize());
140 snapshotSupport = newRaftActorSnapshotMessageSupport();
144 public void postStop() {
145 if(currentBehavior.getDelegate() != null) {
147 currentBehavior.close();
148 } catch (Exception e) {
149 LOG.debug("{}: Error closing behavior {}", persistenceId(), currentBehavior.state());
157 public void handleRecover(Object message) {
158 if(raftRecovery == null) {
159 raftRecovery = newRaftActorRecoverySupport();
162 boolean recoveryComplete = raftRecovery.handleRecoveryMessage(message);
163 if(recoveryComplete) {
164 if(!persistence().isRecoveryApplicable()) {
165 // Delete all the messages from the akka journal so that we do not end up with consistency issues
166 // Note I am not using the dataPersistenceProvider and directly using the akka api here
167 deleteMessages(lastSequenceNr());
169 // Delete all the akka snapshots as they will not be needed
170 deleteSnapshots(new SnapshotSelectionCriteria(scala.Long.MaxValue(), scala.Long.MaxValue()));
173 onRecoveryComplete();
175 initializeBehavior();
181 protected RaftActorRecoverySupport newRaftActorRecoverySupport() {
182 return new RaftActorRecoverySupport(context, currentBehavior, getRaftActorRecoveryCohort());
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());
196 public void handleCommand(Object message) {
197 if (message instanceof ApplyState){
198 ApplyState applyState = (ApplyState) message;
200 long elapsedTime = (System.nanoTime() - applyState.getStartTime());
201 if(elapsedTime >= APPLY_STATE_DELAY_THRESHOLD_IN_NANOS){
202 LOG.warn("ApplyState took more time than expected. Elapsed Time = {} ms ApplyState = {}",
203 TimeUnit.NANOSECONDS.toMillis(elapsedTime), applyState);
206 if(LOG.isDebugEnabled()) {
207 LOG.debug("{}: Applying state for log index {} data {}",
208 persistenceId(), applyState.getReplicatedLogEntry().getIndex(),
209 applyState.getReplicatedLogEntry().getData());
212 applyState(applyState.getClientActor(), applyState.getIdentifier(),
213 applyState.getReplicatedLogEntry().getData());
215 } else if (message instanceof ApplyJournalEntries){
216 ApplyJournalEntries applyEntries = (ApplyJournalEntries) message;
217 if(LOG.isDebugEnabled()) {
218 LOG.debug("{}: Persisting ApplyLogEntries with index={}", persistenceId(), applyEntries.getToIndex());
221 persistence().persist(applyEntries, NoopProcedure.instance());
223 } else if (message instanceof FindLeader) {
225 new FindLeaderReply(getLeaderAddress()),
228 } else if(message instanceof GetOnDemandRaftState) {
229 onGetOnDemandRaftStats();
230 } else if(!snapshotSupport.handleSnapshotMessage(message)) {
231 reusableBehaviorStateHolder.init(getCurrentBehavior());
233 setCurrentBehavior(currentBehavior.handleMessage(getSender(), message));
235 handleBehaviorChange(reusableBehaviorStateHolder, getCurrentBehavior());
239 protected RaftActorSnapshotMessageSupport newRaftActorSnapshotMessageSupport() {
240 return new RaftActorSnapshotMessageSupport(context, currentBehavior,
241 getRaftActorSnapshotCohort());
244 private void onGetOnDemandRaftStats() {
245 // Debugging message to retrieve raft stats.
247 OnDemandRaftState.Builder builder = OnDemandRaftState.builder()
248 .commitIndex(context.getCommitIndex())
249 .currentTerm(context.getTermInformation().getCurrentTerm())
250 .inMemoryJournalDataSize(replicatedLog().dataSize())
251 .inMemoryJournalLogSize(replicatedLog().size())
252 .isSnapshotCaptureInitiated(context.getSnapshotManager().isCapturing())
253 .lastApplied(context.getLastApplied())
254 .lastIndex(replicatedLog().lastIndex())
255 .lastTerm(replicatedLog().lastTerm())
256 .leader(getLeaderId())
257 .raftState(currentBehavior.state().toString())
258 .replicatedToAllIndex(currentBehavior.getReplicatedToAllIndex())
259 .snapshotIndex(replicatedLog().getSnapshotIndex())
260 .snapshotTerm(replicatedLog().getSnapshotTerm())
261 .votedFor(context.getTermInformation().getVotedFor())
262 .peerAddresses(new HashMap<>(context.getPeerAddresses()));
264 ReplicatedLogEntry lastLogEntry = getLastLogEntry();
265 if (lastLogEntry != null) {
266 builder.lastLogIndex(lastLogEntry.getIndex());
267 builder.lastLogTerm(lastLogEntry.getTerm());
270 if(getCurrentBehavior() instanceof AbstractLeader) {
271 AbstractLeader leader = (AbstractLeader)getCurrentBehavior();
272 Collection<String> followerIds = leader.getFollowerIds();
273 List<FollowerInfo> followerInfoList = Lists.newArrayListWithCapacity(followerIds.size());
274 for(String id: followerIds) {
275 final FollowerLogInformation info = leader.getFollower(id);
276 followerInfoList.add(new FollowerInfo(id, info.getNextIndex(), info.getMatchIndex(),
277 info.isFollowerActive(), DurationFormatUtils.formatDurationHMS(info.timeSinceLastActivity())));
280 builder.followerInfoList(followerInfoList);
283 sender().tell(builder.build(), self());
287 private void handleBehaviorChange(BehaviorStateHolder oldBehaviorState, RaftActorBehavior currentBehavior) {
288 RaftActorBehavior oldBehavior = oldBehaviorState.getBehavior();
290 if (oldBehavior != currentBehavior){
294 String oldBehaviorLeaderId = oldBehavior == null ? null : oldBehaviorState.getLeaderId();
295 String oldBehaviorStateName = oldBehavior == null ? null : oldBehavior.state().name();
297 // it can happen that the state has not changed but the leader has changed.
298 Optional<ActorRef> roleChangeNotifier = getRoleChangeNotifier();
299 if(!Objects.equal(oldBehaviorLeaderId, currentBehavior.getLeaderId())) {
300 if(roleChangeNotifier.isPresent()) {
301 roleChangeNotifier.get().tell(newLeaderStateChanged(getId(), currentBehavior.getLeaderId()), getSelf());
304 onLeaderChanged(oldBehaviorLeaderId, currentBehavior.getLeaderId());
307 if (roleChangeNotifier.isPresent() &&
308 (oldBehavior == null || (oldBehavior.state() != currentBehavior.state()))) {
309 roleChangeNotifier.get().tell(new RoleChanged(getId(), oldBehaviorStateName ,
310 currentBehavior.state().name()), getSelf());
314 protected LeaderStateChanged newLeaderStateChanged(String memberId, String leaderId) {
315 return new LeaderStateChanged(memberId, leaderId);
319 * When a derived RaftActor needs to persist something it must call
326 protected void persistData(final ActorRef clientActor, final String identifier,
327 final Payload data) {
329 ReplicatedLogEntry replicatedLogEntry = new ReplicatedLogImplEntry(
330 context.getReplicatedLog().lastIndex() + 1,
331 context.getTermInformation().getCurrentTerm(), data);
333 if(LOG.isDebugEnabled()) {
334 LOG.debug("{}: Persist data {}", persistenceId(), replicatedLogEntry);
337 final RaftActorContext raftContext = getRaftActorContext();
339 replicatedLog().appendAndPersist(replicatedLogEntry, new Procedure<ReplicatedLogEntry>() {
341 public void apply(ReplicatedLogEntry replicatedLogEntry) throws Exception {
343 // Increment the Commit Index and the Last Applied values
344 raftContext.setCommitIndex(replicatedLogEntry.getIndex());
345 raftContext.setLastApplied(replicatedLogEntry.getIndex());
347 // Apply the state immediately
348 applyState(clientActor, identifier, data);
350 // Send a ApplyJournalEntries message so that we write the fact that we applied
351 // the state to durable storage
352 self().tell(new ApplyJournalEntries(replicatedLogEntry.getIndex()), self());
354 context.getSnapshotManager().trimLog(context.getLastApplied(), currentBehavior);
356 } else if (clientActor != null) {
357 // Send message for replication
358 currentBehavior.handleMessage(getSelf(),
359 new Replicate(clientActor, identifier, replicatedLogEntry));
365 private ReplicatedLog replicatedLog() {
366 return context.getReplicatedLog();
369 protected String getId() {
370 return context.getId();
374 void setCurrentBehavior(RaftActorBehavior behavior) {
375 currentBehavior.setDelegate(behavior);
378 protected RaftActorBehavior getCurrentBehavior() {
379 return currentBehavior.getDelegate();
383 * Derived actors can call the isLeader method to check if the current
384 * RaftActor is the Leader or not
386 * @return true it this RaftActor is a Leader false otherwise
388 protected boolean isLeader() {
389 return context.getId().equals(currentBehavior.getLeaderId());
393 * Derived actor can call getLeader if they need a reference to the Leader.
394 * This would be useful for example in forwarding a request to an actor
395 * which is the leader
397 * @return A reference to the leader if known, null otherwise
399 protected ActorSelection getLeader(){
400 String leaderAddress = getLeaderAddress();
402 if(leaderAddress == null){
406 return context.actorSelection(leaderAddress);
411 * @return the current leader's id
413 protected String getLeaderId(){
414 return currentBehavior.getLeaderId();
417 protected RaftState getRaftState() {
418 return currentBehavior.state();
421 protected ReplicatedLogEntry getLastLogEntry() {
422 return replicatedLog().last();
425 protected Long getCurrentTerm(){
426 return context.getTermInformation().getCurrentTerm();
429 protected Long getCommitIndex(){
430 return context.getCommitIndex();
433 protected Long getLastApplied(){
434 return context.getLastApplied();
437 protected RaftActorContext getRaftActorContext() {
441 protected void updateConfigParams(ConfigParams configParams) {
442 context.setConfigParams(configParams);
445 public final DataPersistenceProvider persistence() {
446 return delegatingPersistenceProvider.getDelegate();
449 public void setPersistence(DataPersistenceProvider provider) {
450 delegatingPersistenceProvider.setDelegate(provider);
453 protected void setPersistence(boolean persistent) {
455 setPersistence(new PersistentDataProvider(this));
457 setPersistence(new NonPersistentDataProvider() {
459 * The way snapshotting works is,
461 * <li> RaftActor calls createSnapshot on the Shard
462 * <li> Shard sends a CaptureSnapshotReply and RaftActor then calls saveSnapshot
463 * <li> When saveSnapshot is invoked on the akka-persistence API it uses the SnapshotStore to save
464 * the snapshot. The SnapshotStore sends SaveSnapshotSuccess or SaveSnapshotFailure. When the
465 * RaftActor gets SaveSnapshot success it commits the snapshot to the in-memory journal. This
466 * commitSnapshot is mimicking what is done in SaveSnapshotSuccess.
470 public void saveSnapshot(Object o) {
471 // Make saving Snapshot successful
472 // Committing the snapshot here would end up calling commit in the creating state which would
473 // be a state violation. That's why now we send a message to commit the snapshot.
474 self().tell(RaftActorSnapshotMessageSupport.COMMIT_SNAPSHOT, self());
481 * setPeerAddress sets the address of a known peer at a later time.
483 * This is to account for situations where a we know that a peer
484 * exists but we do not know an address up-front. This may also be used in
485 * situations where a known peer starts off in a different location and we
486 * need to change it's address
488 * Note that if the peerId does not match the list of peers passed to
489 * this actor during construction an IllegalStateException will be thrown.
494 protected void setPeerAddress(String peerId, String peerAddress){
495 context.setPeerAddress(peerId, peerAddress);
499 * The applyState method will be called by the RaftActor when some data
500 * needs to be applied to the actor's state
502 * @param clientActor A reference to the client who sent this message. This
503 * is the same reference that was passed to persistData
504 * by the derived actor. clientActor may be null when
505 * the RaftActor is behaving as a follower or during
507 * @param identifier The identifier of the persisted data. This is also
508 * the same identifier that was passed to persistData by
509 * the derived actor. identifier may be null when
510 * the RaftActor is behaving as a follower or during
512 * @param data A piece of data that was persisted by the persistData call.
513 * This should NEVER be null.
515 protected abstract void applyState(ActorRef clientActor, String identifier,
519 * Returns the RaftActorRecoveryCohort to participate in persistence recovery.
522 protected abstract RaftActorRecoveryCohort getRaftActorRecoveryCohort();
525 * This method is called when recovery is complete.
527 protected abstract void onRecoveryComplete();
530 * Returns the RaftActorSnapshotCohort to participate in persistence recovery.
533 protected abstract RaftActorSnapshotCohort getRaftActorSnapshotCohort();
536 * This method will be called by the RaftActor when the state of the
537 * RaftActor changes. The derived actor can then use methods like
538 * isLeader or getLeader to do something useful
540 protected abstract void onStateChanged();
543 * Notifier Actor for this RaftActor to notify when a role change happens
544 * @return ActorRef - ActorRef of the notifier or Optional.absent if none.
546 protected abstract Optional<ActorRef> getRoleChangeNotifier();
548 protected void onLeaderChanged(String oldLeader, String newLeader){};
550 private String getLeaderAddress(){
552 return getSelf().path().toString();
554 String leaderId = currentBehavior.getLeaderId();
555 if (leaderId == null) {
558 String peerAddress = context.getPeerAddress(leaderId);
559 if(LOG.isDebugEnabled()) {
560 LOG.debug("{}: getLeaderAddress leaderId = {} peerAddress = {}",
561 persistenceId(), leaderId, peerAddress);
567 protected boolean hasFollowers(){
568 return getRaftActorContext().hasFollowers();
572 * @deprecated Deprecated in favor of {@link org.opendaylight.controller.cluster.raft.base.messages.DeleteEntriesTest}
573 * whose type for fromIndex is long instead of int. This class was kept for backwards
574 * compatibility with Helium.
576 // Suppressing this warning as we can't set serialVersionUID to maintain backwards compatibility.
577 @SuppressWarnings("serial")
579 static class DeleteEntries implements Serializable {
580 private final int fromIndex;
582 public DeleteEntries(int fromIndex) {
583 this.fromIndex = fromIndex;
586 public int getFromIndex() {
591 // Suppressing this warning as we can't set serialVersionUID to maintain backwards compatibility.
592 @SuppressWarnings("serial")
593 static class UpdateElectionTerm implements Serializable {
594 private final long currentTerm;
595 private final String votedFor;
597 public UpdateElectionTerm(long currentTerm, String votedFor) {
598 this.currentTerm = currentTerm;
599 this.votedFor = votedFor;
602 public long getCurrentTerm() {
606 public String getVotedFor() {
611 private static class BehaviorStateHolder {
612 private RaftActorBehavior behavior;
613 private String leaderId;
615 void init(RaftActorBehavior behavior) {
616 this.behavior = behavior;
617 this.leaderId = behavior != null ? behavior.getLeaderId() : null;
620 RaftActorBehavior getBehavior() {
624 String getLeaderId() {