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.ImmutableMap;
20 import com.google.common.collect.Lists;
21 import java.io.Serializable;
22 import java.util.Collection;
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, delegatingPersistenceProvider, currentBehavior));
134 public void preStart() throws Exception {
135 LOG.info("Starting recovery for {} with journal batch size {}", persistenceId(),
136 context.getConfigParams().getJournalRecoveryLogBatchSize());
142 public void postStop() {
143 if(currentBehavior.getDelegate() != null) {
145 currentBehavior.close();
146 } catch (Exception e) {
147 LOG.debug("{}: Error closing behavior {}", persistenceId(), currentBehavior.state());
155 public void handleRecover(Object message) {
156 if(raftRecovery == null) {
157 raftRecovery = newRaftActorRecoverySupport();
160 boolean recoveryComplete = raftRecovery.handleRecoveryMessage(message);
161 if(recoveryComplete) {
162 if(!persistence().isRecoveryApplicable()) {
163 // Delete all the messages from the akka journal so that we do not end up with consistency issues
164 // Note I am not using the dataPersistenceProvider and directly using the akka api here
165 deleteMessages(lastSequenceNr());
167 // Delete all the akka snapshots as they will not be needed
168 deleteSnapshots(new SnapshotSelectionCriteria(scala.Long.MaxValue(), scala.Long.MaxValue()));
171 onRecoveryComplete();
173 initializeBehavior();
179 protected RaftActorRecoverySupport newRaftActorRecoverySupport() {
180 return new RaftActorRecoverySupport(delegatingPersistenceProvider, context, currentBehavior,
181 getRaftActorRecoveryCohort());
184 protected void initializeBehavior(){
185 changeCurrentBehavior(new Follower(context));
188 protected void changeCurrentBehavior(RaftActorBehavior newBehavior){
189 reusableBehaviorStateHolder.init(getCurrentBehavior());
190 setCurrentBehavior(newBehavior);
191 handleBehaviorChange(reusableBehaviorStateHolder, getCurrentBehavior());
195 public void handleCommand(Object message) {
196 if(snapshotSupport == null) {
197 snapshotSupport = newRaftActorSnapshotMessageSupport();
200 boolean handled = snapshotSupport.handleSnapshotMessage(message);
205 if (message instanceof ApplyState){
206 ApplyState applyState = (ApplyState) message;
208 long elapsedTime = (System.nanoTime() - applyState.getStartTime());
209 if(elapsedTime >= APPLY_STATE_DELAY_THRESHOLD_IN_NANOS){
210 LOG.warn("ApplyState took more time than expected. Elapsed Time = {} ms ApplyState = {}",
211 TimeUnit.NANOSECONDS.toMillis(elapsedTime), applyState);
214 if(LOG.isDebugEnabled()) {
215 LOG.debug("{}: Applying state for log index {} data {}",
216 persistenceId(), applyState.getReplicatedLogEntry().getIndex(),
217 applyState.getReplicatedLogEntry().getData());
220 applyState(applyState.getClientActor(), applyState.getIdentifier(),
221 applyState.getReplicatedLogEntry().getData());
223 } else if (message instanceof ApplyJournalEntries){
224 ApplyJournalEntries applyEntries = (ApplyJournalEntries) message;
225 if(LOG.isDebugEnabled()) {
226 LOG.debug("{}: Persisting ApplyLogEntries with index={}", persistenceId(), applyEntries.getToIndex());
229 persistence().persist(applyEntries, NoopProcedure.instance());
231 } else if (message instanceof FindLeader) {
233 new FindLeaderReply(getLeaderAddress()),
236 } else if(message instanceof GetOnDemandRaftState) {
237 onGetOnDemandRaftStats();
239 reusableBehaviorStateHolder.init(getCurrentBehavior());
241 setCurrentBehavior(currentBehavior.handleMessage(getSender(), message));
243 handleBehaviorChange(reusableBehaviorStateHolder, getCurrentBehavior());
247 protected RaftActorSnapshotMessageSupport newRaftActorSnapshotMessageSupport() {
248 return new RaftActorSnapshotMessageSupport(delegatingPersistenceProvider, context,
249 currentBehavior, getRaftActorSnapshotCohort());
252 private void onGetOnDemandRaftStats() {
253 // Debugging message to retrieve raft stats.
255 OnDemandRaftState.Builder builder = OnDemandRaftState.builder()
256 .commitIndex(context.getCommitIndex())
257 .currentTerm(context.getTermInformation().getCurrentTerm())
258 .inMemoryJournalDataSize(replicatedLog().dataSize())
259 .inMemoryJournalLogSize(replicatedLog().size())
260 .isSnapshotCaptureInitiated(context.getSnapshotManager().isCapturing())
261 .lastApplied(context.getLastApplied())
262 .lastIndex(replicatedLog().lastIndex())
263 .lastTerm(replicatedLog().lastTerm())
264 .leader(getLeaderId())
265 .raftState(currentBehavior.state().toString())
266 .replicatedToAllIndex(currentBehavior.getReplicatedToAllIndex())
267 .snapshotIndex(replicatedLog().getSnapshotIndex())
268 .snapshotTerm(replicatedLog().getSnapshotTerm())
269 .votedFor(context.getTermInformation().getVotedFor())
270 .peerAddresses(ImmutableMap.copyOf(context.getPeerAddresses()));
272 ReplicatedLogEntry lastLogEntry = getLastLogEntry();
273 if (lastLogEntry != null) {
274 builder.lastLogIndex(lastLogEntry.getIndex());
275 builder.lastLogTerm(lastLogEntry.getTerm());
278 if(getCurrentBehavior() instanceof AbstractLeader) {
279 AbstractLeader leader = (AbstractLeader)getCurrentBehavior();
280 Collection<String> followerIds = leader.getFollowerIds();
281 List<FollowerInfo> followerInfoList = Lists.newArrayListWithCapacity(followerIds.size());
282 for(String id: followerIds) {
283 final FollowerLogInformation info = leader.getFollower(id);
284 followerInfoList.add(new FollowerInfo(id, info.getNextIndex(), info.getMatchIndex(),
285 info.isFollowerActive(), DurationFormatUtils.formatDurationHMS(info.timeSinceLastActivity())));
288 builder.followerInfoList(followerInfoList);
291 sender().tell(builder.build(), self());
295 private void handleBehaviorChange(BehaviorStateHolder oldBehaviorState, RaftActorBehavior currentBehavior) {
296 RaftActorBehavior oldBehavior = oldBehaviorState.getBehavior();
298 if (oldBehavior != currentBehavior){
302 String oldBehaviorLeaderId = oldBehavior == null ? null : oldBehaviorState.getLeaderId();
303 String oldBehaviorStateName = oldBehavior == null ? null : oldBehavior.state().name();
305 // it can happen that the state has not changed but the leader has changed.
306 Optional<ActorRef> roleChangeNotifier = getRoleChangeNotifier();
307 if(!Objects.equal(oldBehaviorLeaderId, currentBehavior.getLeaderId())) {
308 if(roleChangeNotifier.isPresent()) {
309 roleChangeNotifier.get().tell(new LeaderStateChanged(getId(), currentBehavior.getLeaderId()), getSelf());
312 onLeaderChanged(oldBehaviorLeaderId, currentBehavior.getLeaderId());
315 if (roleChangeNotifier.isPresent() &&
316 (oldBehavior == null || (oldBehavior.state() != currentBehavior.state()))) {
317 roleChangeNotifier.get().tell(new RoleChanged(getId(), oldBehaviorStateName ,
318 currentBehavior.state().name()), getSelf());
323 * When a derived RaftActor needs to persist something it must call
330 protected void persistData(final ActorRef clientActor, final String identifier,
331 final Payload data) {
333 ReplicatedLogEntry replicatedLogEntry = new ReplicatedLogImplEntry(
334 context.getReplicatedLog().lastIndex() + 1,
335 context.getTermInformation().getCurrentTerm(), data);
337 if(LOG.isDebugEnabled()) {
338 LOG.debug("{}: Persist data {}", persistenceId(), replicatedLogEntry);
341 final RaftActorContext raftContext = getRaftActorContext();
343 replicatedLog().appendAndPersist(replicatedLogEntry, new Procedure<ReplicatedLogEntry>() {
345 public void apply(ReplicatedLogEntry replicatedLogEntry) throws Exception {
347 // Increment the Commit Index and the Last Applied values
348 raftContext.setCommitIndex(replicatedLogEntry.getIndex());
349 raftContext.setLastApplied(replicatedLogEntry.getIndex());
351 // Apply the state immediately
352 applyState(clientActor, identifier, data);
354 // Send a ApplyJournalEntries message so that we write the fact that we applied
355 // the state to durable storage
356 self().tell(new ApplyJournalEntries(replicatedLogEntry.getIndex()), self());
358 context.getSnapshotManager().trimLog(context.getLastApplied(), currentBehavior);
360 } else if (clientActor != null) {
361 // Send message for replication
362 currentBehavior.handleMessage(getSelf(),
363 new Replicate(clientActor, identifier, replicatedLogEntry));
369 private ReplicatedLog replicatedLog() {
370 return context.getReplicatedLog();
373 protected String getId() {
374 return context.getId();
378 void setCurrentBehavior(RaftActorBehavior behavior) {
379 currentBehavior.setDelegate(behavior);
382 protected RaftActorBehavior getCurrentBehavior() {
383 return currentBehavior.getDelegate();
387 * Derived actors can call the isLeader method to check if the current
388 * RaftActor is the Leader or not
390 * @return true it this RaftActor is a Leader false otherwise
392 protected boolean isLeader() {
393 return context.getId().equals(currentBehavior.getLeaderId());
397 * Derived actor can call getLeader if they need a reference to the Leader.
398 * This would be useful for example in forwarding a request to an actor
399 * which is the leader
401 * @return A reference to the leader if known, null otherwise
403 protected ActorSelection getLeader(){
404 String leaderAddress = getLeaderAddress();
406 if(leaderAddress == null){
410 return context.actorSelection(leaderAddress);
415 * @return the current leader's id
417 protected String getLeaderId(){
418 return currentBehavior.getLeaderId();
421 protected RaftState getRaftState() {
422 return currentBehavior.state();
425 protected ReplicatedLogEntry getLastLogEntry() {
426 return replicatedLog().last();
429 protected Long getCurrentTerm(){
430 return context.getTermInformation().getCurrentTerm();
433 protected Long getCommitIndex(){
434 return context.getCommitIndex();
437 protected Long getLastApplied(){
438 return context.getLastApplied();
441 protected RaftActorContext getRaftActorContext() {
445 protected void updateConfigParams(ConfigParams configParams) {
446 context.setConfigParams(configParams);
449 public final DataPersistenceProvider persistence() {
450 return delegatingPersistenceProvider.getDelegate();
453 public void setPersistence(DataPersistenceProvider provider) {
454 delegatingPersistenceProvider.setDelegate(provider);
457 protected void setPersistence(boolean persistent) {
459 setPersistence(new PersistentDataProvider(this));
461 setPersistence(new NonPersistentDataProvider() {
463 * The way snapshotting works is,
465 * <li> RaftActor calls createSnapshot on the Shard
466 * <li> Shard sends a CaptureSnapshotReply and RaftActor then calls saveSnapshot
467 * <li> When saveSnapshot is invoked on the akka-persistence API it uses the SnapshotStore to save
468 * the snapshot. The SnapshotStore sends SaveSnapshotSuccess or SaveSnapshotFailure. When the
469 * RaftActor gets SaveSnapshot success it commits the snapshot to the in-memory journal. This
470 * commitSnapshot is mimicking what is done in SaveSnapshotSuccess.
474 public void saveSnapshot(Object o) {
475 // Make saving Snapshot successful
476 // Committing the snapshot here would end up calling commit in the creating state which would
477 // be a state violation. That's why now we send a message to commit the snapshot.
478 self().tell(RaftActorSnapshotMessageSupport.COMMIT_SNAPSHOT, self());
485 * setPeerAddress sets the address of a known peer at a later time.
487 * This is to account for situations where a we know that a peer
488 * exists but we do not know an address up-front. This may also be used in
489 * situations where a known peer starts off in a different location and we
490 * need to change it's address
492 * Note that if the peerId does not match the list of peers passed to
493 * this actor during construction an IllegalStateException will be thrown.
498 protected void setPeerAddress(String peerId, String peerAddress){
499 context.setPeerAddress(peerId, peerAddress);
503 * The applyState method will be called by the RaftActor when some data
504 * needs to be applied to the actor's state
506 * @param clientActor A reference to the client who sent this message. This
507 * is the same reference that was passed to persistData
508 * by the derived actor. clientActor may be null when
509 * the RaftActor is behaving as a follower or during
511 * @param identifier The identifier of the persisted data. This is also
512 * the same identifier that was passed to persistData by
513 * the derived actor. identifier may be null when
514 * the RaftActor is behaving as a follower or during
516 * @param data A piece of data that was persisted by the persistData call.
517 * This should NEVER be null.
519 protected abstract void applyState(ActorRef clientActor, String identifier,
523 * Returns the RaftActorRecoveryCohort to participate in persistence recovery.
526 protected abstract RaftActorRecoveryCohort getRaftActorRecoveryCohort();
529 * This method is called when recovery is complete.
531 protected abstract void onRecoveryComplete();
534 * Returns the RaftActorSnapshotCohort to participate in persistence recovery.
537 protected abstract RaftActorSnapshotCohort getRaftActorSnapshotCohort();
540 * This method will be called by the RaftActor when the state of the
541 * RaftActor changes. The derived actor can then use methods like
542 * isLeader or getLeader to do something useful
544 protected abstract void onStateChanged();
547 * Notifier Actor for this RaftActor to notify when a role change happens
548 * @return ActorRef - ActorRef of the notifier or Optional.absent if none.
550 protected abstract Optional<ActorRef> getRoleChangeNotifier();
552 protected void onLeaderChanged(String oldLeader, String newLeader){};
554 private String getLeaderAddress(){
556 return getSelf().path().toString();
558 String leaderId = currentBehavior.getLeaderId();
559 if (leaderId == null) {
562 String peerAddress = context.getPeerAddress(leaderId);
563 if(LOG.isDebugEnabled()) {
564 LOG.debug("{}: getLeaderAddress leaderId = {} peerAddress = {}",
565 persistenceId(), leaderId, peerAddress);
571 protected boolean hasFollowers(){
572 return getRaftActorContext().hasFollowers();
576 * @deprecated Deprecated in favor of {@link org.opendaylight.controller.cluster.raft.base.messages.DeleteEntriesTest}
577 * whose type for fromIndex is long instead of int. This class was kept for backwards
578 * compatibility with Helium.
581 static class DeleteEntries implements Serializable {
582 private static final long serialVersionUID = 1L;
583 private final int fromIndex;
585 public DeleteEntries(int fromIndex) {
586 this.fromIndex = fromIndex;
589 public int getFromIndex() {
594 static class UpdateElectionTerm implements Serializable {
595 private static final long serialVersionUID = 1L;
596 private final long currentTerm;
597 private final String votedFor;
599 public UpdateElectionTerm(long currentTerm, String votedFor) {
600 this.currentTerm = currentTerm;
601 this.votedFor = votedFor;
604 public long getCurrentTerm() {
608 public String getVotedFor() {
613 private static class BehaviorStateHolder {
614 private RaftActorBehavior behavior;
615 private String leaderId;
617 void init(RaftActorBehavior behavior) {
618 this.behavior = behavior;
619 this.leaderId = behavior != null ? behavior.getLeaderId() : null;
622 RaftActorBehavior getBehavior() {
626 String getLeaderId() {