2 * Copyright (c) 2015 Brocade Communications 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
8 package org.opendaylight.controller.cluster.raft;
10 import akka.actor.ActorRef;
11 import akka.actor.ActorSelection;
12 import akka.actor.Cancellable;
13 import com.google.common.base.Preconditions;
14 import java.util.ArrayDeque;
15 import java.util.Queue;
16 import java.util.UUID;
17 import javax.annotation.Nullable;
18 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
19 import org.opendaylight.controller.cluster.raft.base.messages.SnapshotComplete;
20 import org.opendaylight.controller.cluster.raft.behaviors.AbstractLeader;
21 import org.opendaylight.controller.cluster.raft.messages.AddServer;
22 import org.opendaylight.controller.cluster.raft.messages.AddServerReply;
23 import org.opendaylight.controller.cluster.raft.messages.RemoveServer;
24 import org.opendaylight.controller.cluster.raft.messages.RemoveServerReply;
25 import org.opendaylight.controller.cluster.raft.messages.ServerChangeStatus;
26 import org.opendaylight.controller.cluster.raft.messages.ServerRemoved;
27 import org.opendaylight.controller.cluster.raft.messages.UnInitializedFollowerSnapshotReply;
28 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
29 import org.slf4j.Logger;
30 import org.slf4j.LoggerFactory;
33 * Handles server configuration related messages for a RaftActor.
35 * @author Thomas Pantelis
37 class RaftActorServerConfigurationSupport {
38 private static final Logger LOG = LoggerFactory.getLogger(RaftActorServerConfigurationSupport.class);
40 private final OperationState IDLE = new Idle();
42 private final RaftActor raftActor;
44 private final RaftActorContext raftContext;
46 private final Queue<ServerOperationContext<?>> pendingOperationsQueue = new ArrayDeque<>();
48 private OperationState currentOperationState = IDLE;
50 RaftActorServerConfigurationSupport(RaftActor raftActor) {
51 this.raftActor = raftActor;
52 this.raftContext = raftActor.getRaftActorContext();
55 boolean handleMessage(Object message, ActorRef sender) {
56 if(message instanceof AddServer) {
57 onAddServer((AddServer) message, sender);
59 } else if(message instanceof RemoveServer) {
60 onRemoveServer((RemoveServer) message, sender);
62 } else if (message instanceof ServerOperationTimeout) {
63 currentOperationState.onServerOperationTimeout((ServerOperationTimeout) message);
65 } else if (message instanceof UnInitializedFollowerSnapshotReply) {
66 currentOperationState.onUnInitializedFollowerSnapshotReply((UnInitializedFollowerSnapshotReply) message);
68 } else if(message instanceof ApplyState) {
69 return onApplyState((ApplyState) message);
70 } else if(message instanceof SnapshotComplete) {
71 currentOperationState.onSnapshotComplete();
78 private void onRemoveServer(RemoveServer removeServer, ActorRef sender) {
79 LOG.debug("{}: onRemoveServer: {}, state: {}", raftContext.getId(), removeServer, currentOperationState);
80 boolean isSelf = removeServer.getServerId().equals(raftActor.getId());
81 if(!isSelf && !raftContext.getPeerIds().contains(removeServer.getServerId())) {
82 sender.tell(new RemoveServerReply(ServerChangeStatus.DOES_NOT_EXIST, raftActor.getLeaderId()),
85 String serverAddress = isSelf ? raftActor.self().path().toString() :
86 raftContext.getPeerAddress(removeServer.getServerId());
87 onNewOperation(new RemoveServerContext(removeServer, serverAddress, sender));
91 private boolean onApplyState(ApplyState applyState) {
92 Payload data = applyState.getReplicatedLogEntry().getData();
93 if(data instanceof ServerConfigurationPayload) {
94 currentOperationState.onApplyState(applyState);
102 * The algorithm for AddServer is as follows:
104 * <li>Add the new server as a peer.</li>
105 * <li>Add the new follower to the leader.</li>
106 * <li>If new server should be voting member</li>
108 * <li>Initialize FollowerState to VOTING_NOT_INITIALIZED.</li>
109 * <li>Initiate install snapshot to the new follower.</li>
110 * <li>When install snapshot complete, mark the follower as VOTING and re-calculate majority vote count.</li>
112 * <li>Persist and replicate ServerConfigurationPayload with the new server list.</li>
113 * <li>On replication consensus, respond to caller with OK.</li>
115 * If the install snapshot times out after a period of 2 * election time out
117 * <li>Remove the new server as a peer.</li>
118 * <li>Remove the new follower from the leader.</li>
119 * <li>Respond to caller with TIMEOUT.</li>
122 private void onAddServer(AddServer addServer, ActorRef sender) {
123 LOG.debug("{}: onAddServer: {}, state: {}", raftContext.getId(), addServer, currentOperationState);
125 onNewOperation(new AddServerContext(addServer, sender));
128 private void onNewOperation(ServerOperationContext<?> operationContext) {
129 if (raftActor.isLeader()) {
130 currentOperationState.onNewOperation(operationContext);
132 ActorSelection leader = raftActor.getLeader();
133 if (leader != null) {
134 LOG.debug("{}: Not leader - forwarding to leader {}", raftContext.getId(), leader);
135 leader.forward(operationContext.getOperation(), raftActor.getContext());
137 LOG.debug("{}: No leader - returning NO_LEADER reply", raftContext.getId());
138 operationContext.getClientRequestor().tell(operationContext.newReply(
139 ServerChangeStatus.NO_LEADER, null), raftActor.self());
145 * Interface for a server operation FSM state.
147 private interface OperationState {
148 void onNewOperation(ServerOperationContext<?> operationContext);
150 void onServerOperationTimeout(ServerOperationTimeout timeout);
152 void onUnInitializedFollowerSnapshotReply(UnInitializedFollowerSnapshotReply reply);
154 void onApplyState(ApplyState applyState);
156 void onSnapshotComplete();
160 * Interface for the initial state for a server operation.
162 private interface InitialOperationState {
167 * Abstract base class for a server operation FSM state. Handles common behavior for all states.
169 private abstract class AbstractOperationState implements OperationState {
171 public void onNewOperation(ServerOperationContext<?> operationContext) {
172 // We're currently processing another operation so queue it to be processed later.
174 LOG.debug("{}: Server operation already in progress - queueing {}", raftContext.getId(),
175 operationContext.getOperation());
177 pendingOperationsQueue.add(operationContext);
181 public void onServerOperationTimeout(ServerOperationTimeout timeout) {
182 LOG.debug("onServerOperationTimeout should not be called in state {}", this);
186 public void onUnInitializedFollowerSnapshotReply(UnInitializedFollowerSnapshotReply reply) {
187 LOG.debug("onUnInitializedFollowerSnapshotReply was called in state {}", this);
191 public void onApplyState(ApplyState applyState) {
192 LOG.debug("onApplyState was called in state {}", this);
196 public void onSnapshotComplete() {
199 protected void persistNewServerConfiguration(ServerOperationContext<?> operationContext){
200 raftContext.setDynamicServerConfigurationInUse();
202 boolean includeSelf = !operationContext.getServerId().equals(raftActor.getId());
203 ServerConfigurationPayload payload = raftContext.getPeerServerInfo(includeSelf);
204 LOG.debug("{}: New server configuration : {}", raftContext.getId(), payload.getServerConfig());
206 raftActor.persistData(operationContext.getClientRequestor(), operationContext.getContextId(), payload);
208 currentOperationState = new Persisting(operationContext, newTimer(new ServerOperationTimeout(operationContext.getServerId())));
210 sendReply(operationContext, ServerChangeStatus.OK);
213 protected void operationComplete(ServerOperationContext<?> operationContext, @Nullable ServerChangeStatus replyStatus) {
214 if(replyStatus != null) {
215 sendReply(operationContext, replyStatus);
218 operationContext.operationComplete(raftActor, replyStatus);
220 currentOperationState = IDLE;
222 ServerOperationContext<?> nextOperation = pendingOperationsQueue.poll();
223 if(nextOperation != null) {
224 RaftActorServerConfigurationSupport.this.onNewOperation(nextOperation);
228 protected void sendReply(ServerOperationContext<?> operationContext, ServerChangeStatus status) {
229 LOG.debug("{}: Returning {} for operation {}", raftContext.getId(), status, operationContext.getOperation());
231 operationContext.getClientRequestor().tell(operationContext.newReply(status, raftActor.getLeaderId()),
235 Cancellable newTimer(Object message) {
236 return raftContext.getActorSystem().scheduler().scheduleOnce(
237 raftContext.getConfigParams().getElectionTimeOutInterval().$times(2), raftContext.getActor(), message,
238 raftContext.getActorSystem().dispatcher(), raftContext.getActor());
242 public String toString() {
243 return getClass().getSimpleName();
248 * The state when no server operation is in progress. It immediately initiates new server operations.
250 private class Idle extends AbstractOperationState {
252 public void onNewOperation(ServerOperationContext<?> operationContext) {
253 operationContext.newInitialOperationState(RaftActorServerConfigurationSupport.this).initiate();
257 public void onApplyState(ApplyState applyState) {
258 // Noop - we override b/c ApplyState is called normally for followers in the idle state.
263 * The state when a new server configuration is being persisted and replicated.
265 private class Persisting extends AbstractOperationState {
266 private final ServerOperationContext<?> operationContext;
267 private final Cancellable timer;
268 private boolean timedOut = false;
270 Persisting(ServerOperationContext<?> operationContext, Cancellable timer) {
271 this.operationContext = operationContext;
276 public void onApplyState(ApplyState applyState) {
277 // Sanity check - we could get an ApplyState from a previous operation that timed out so make
278 // sure it's meant for us.
279 if(operationContext.getContextId().equals(applyState.getIdentifier())) {
280 LOG.info("{}: {} has been successfully replicated to a majority of followers", raftActor.getId(),
281 applyState.getReplicatedLogEntry().getData());
284 operationComplete(operationContext, null);
289 public void onServerOperationTimeout(ServerOperationTimeout timeout) {
290 LOG.warn("{}: Timeout occured while replicating the new server configuration for {}", raftContext.getId(),
291 timeout.getServerId());
295 // Fail any pending operations
296 ServerOperationContext<?> nextOperation = pendingOperationsQueue.poll();
297 while(nextOperation != null) {
298 sendReply(nextOperation, ServerChangeStatus.PRIOR_REQUEST_CONSENSUS_TIMEOUT);
299 nextOperation = pendingOperationsQueue.poll();
304 public void onNewOperation(ServerOperationContext<?> operationContext) {
306 sendReply(operationContext, ServerChangeStatus.PRIOR_REQUEST_CONSENSUS_TIMEOUT);
308 super.onNewOperation(operationContext);
314 * Abstract base class for an AddServer operation state.
316 private abstract class AddServerState extends AbstractOperationState {
317 private final AddServerContext addServerContext;
319 AddServerState(AddServerContext addServerContext) {
320 this.addServerContext = addServerContext;
323 AddServerContext getAddServerContext() {
324 return addServerContext;
327 Cancellable newInstallSnapshotTimer() {
328 return newTimer(new ServerOperationTimeout(addServerContext.getOperation().getNewServerId()));
331 void handleInstallSnapshotTimeout(ServerOperationTimeout timeout) {
332 String serverId = timeout.getServerId();
334 LOG.debug("{}: handleInstallSnapshotTimeout for new server {}", raftContext.getId(), serverId);
337 raftContext.removePeer(serverId);
339 boolean isLeader = raftActor.isLeader();
341 AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior();
342 leader.removeFollower(serverId);
345 operationComplete(getAddServerContext(), isLeader ? ServerChangeStatus.TIMEOUT : ServerChangeStatus.NO_LEADER);
351 * The initial state for the AddServer operation. It adds the new follower as a peer and initiates
352 * snapshot capture, if necessary.
354 private class InitialAddServerState extends AddServerState implements InitialOperationState {
355 InitialAddServerState(AddServerContext addServerContext) {
356 super(addServerContext);
360 public void initiate() {
361 AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior();
362 AddServer addServer = getAddServerContext().getOperation();
364 LOG.debug("{}: Initiating {}", raftContext.getId(), addServer);
366 if(raftContext.getPeerInfo(addServer.getNewServerId()) != null) {
367 operationComplete(getAddServerContext(), ServerChangeStatus.ALREADY_EXISTS);
371 VotingState votingState = addServer.isVotingMember() ? VotingState.VOTING_NOT_INITIALIZED :
372 VotingState.NON_VOTING;
373 raftContext.addToPeers(addServer.getNewServerId(), addServer.getNewServerAddress(), votingState);
375 leader.addFollower(addServer.getNewServerId());
377 if(votingState == VotingState.VOTING_NOT_INITIALIZED){
378 // schedule the install snapshot timeout timer
379 Cancellable installSnapshotTimer = newInstallSnapshotTimer();
380 if(leader.initiateCaptureSnapshot(addServer.getNewServerId())) {
381 LOG.debug("{}: Initiating capture snapshot for new server {}", raftContext.getId(),
382 addServer.getNewServerId());
384 currentOperationState = new InstallingSnapshot(getAddServerContext(), installSnapshotTimer);
386 LOG.debug("{}: Snapshot already in progress - waiting for completion", raftContext.getId());
388 currentOperationState = new WaitingForPriorSnapshotComplete(getAddServerContext(),
389 installSnapshotTimer);
392 LOG.debug("{}: New follower is non-voting - directly persisting new server configuration",
393 raftContext.getId());
395 persistNewServerConfiguration(getAddServerContext());
401 * The AddServer operation state for when the catch-up snapshot is being installed. It handles successful
404 private class InstallingSnapshot extends AddServerState {
405 private final Cancellable installSnapshotTimer;
407 InstallingSnapshot(AddServerContext addServerContext, Cancellable installSnapshotTimer) {
408 super(addServerContext);
409 this.installSnapshotTimer = Preconditions.checkNotNull(installSnapshotTimer);
413 public void onServerOperationTimeout(ServerOperationTimeout timeout) {
414 handleInstallSnapshotTimeout(timeout);
416 LOG.warn("{}: Timeout occured for new server {} while installing snapshot", raftContext.getId(),
417 timeout.getServerId());
421 public void onUnInitializedFollowerSnapshotReply(UnInitializedFollowerSnapshotReply reply) {
422 LOG.debug("{}: onUnInitializedFollowerSnapshotReply: {}", raftContext.getId(), reply);
424 String followerId = reply.getFollowerId();
426 // Sanity check to guard against receiving an UnInitializedFollowerSnapshotReply from a prior
427 // add server operation that timed out.
428 if(getAddServerContext().getOperation().getNewServerId().equals(followerId) && raftActor.isLeader()) {
429 AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior();
430 raftContext.getPeerInfo(followerId).setVotingState(VotingState.VOTING);
431 leader.updateMinReplicaCount();
433 persistNewServerConfiguration(getAddServerContext());
435 installSnapshotTimer.cancel();
437 LOG.debug("{}: Dropping UnInitializedFollowerSnapshotReply for server {}: {}",
438 raftContext.getId(), followerId,
439 !raftActor.isLeader() ? "not leader" : "server Id doesn't match");
445 * The AddServer operation state for when there is a snapshot already in progress. When the current
446 * snapshot completes, it initiates an install snapshot.
448 private class WaitingForPriorSnapshotComplete extends AddServerState {
449 private final Cancellable snapshotTimer;
451 WaitingForPriorSnapshotComplete(AddServerContext addServerContext, Cancellable snapshotTimer) {
452 super(addServerContext);
453 this.snapshotTimer = Preconditions.checkNotNull(snapshotTimer);
457 public void onSnapshotComplete() {
458 LOG.debug("{}: onSnapshotComplete", raftContext.getId());
460 if(!raftActor.isLeader()) {
461 LOG.debug("{}: No longer the leader", raftContext.getId());
465 AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior();
466 if(leader.initiateCaptureSnapshot(getAddServerContext().getOperation().getNewServerId())) {
467 LOG.debug("{}: Initiating capture snapshot for new server {}", raftContext.getId(),
468 getAddServerContext().getOperation().getNewServerId());
470 currentOperationState = new InstallingSnapshot(getAddServerContext(),
471 newInstallSnapshotTimer());
473 snapshotTimer.cancel();
478 public void onServerOperationTimeout(ServerOperationTimeout timeout) {
479 handleInstallSnapshotTimeout(timeout);
481 LOG.warn("{}: Timeout occured for new server {} while waiting for prior snapshot to complete",
482 raftContext.getId(), timeout.getServerId());
487 * Stores context information for a server operation.
489 * @param <T> the operation type
491 private static abstract class ServerOperationContext<T> {
492 private final T operation;
493 private final ActorRef clientRequestor;
494 private final String contextId;
496 ServerOperationContext(T operation, ActorRef clientRequestor){
497 this.operation = operation;
498 this.clientRequestor = clientRequestor;
499 contextId = UUID.randomUUID().toString();
502 String getContextId() {
510 ActorRef getClientRequestor() {
511 return clientRequestor;
514 abstract Object newReply(ServerChangeStatus status, String leaderId);
516 abstract InitialOperationState newInitialOperationState(RaftActorServerConfigurationSupport support);
518 abstract void operationComplete(RaftActor raftActor, ServerChangeStatus serverChangeStatus);
520 abstract String getServerId();
524 * Stores context information for an AddServer operation.
526 private static class AddServerContext extends ServerOperationContext<AddServer> {
527 AddServerContext(AddServer addServer, ActorRef clientRequestor) {
528 super(addServer, clientRequestor);
532 Object newReply(ServerChangeStatus status, String leaderId) {
533 return new AddServerReply(status, leaderId);
537 InitialOperationState newInitialOperationState(RaftActorServerConfigurationSupport support) {
538 return support.new InitialAddServerState(this);
542 void operationComplete(RaftActor raftActor, ServerChangeStatus serverChangeStatus) {
547 String getServerId() {
548 return getOperation().getNewServerId();
552 private abstract class RemoveServerState extends AbstractOperationState {
553 private final RemoveServerContext removeServerContext;
556 protected RemoveServerState(RemoveServerContext removeServerContext) {
557 this.removeServerContext = Preconditions.checkNotNull(removeServerContext);
561 public RemoveServerContext getRemoveServerContext() {
562 return removeServerContext;
566 private class InitialRemoveServerState extends RemoveServerState implements InitialOperationState{
568 protected InitialRemoveServerState(RemoveServerContext removeServerContext) {
569 super(removeServerContext);
573 public void initiate() {
574 raftContext.removePeer(getRemoveServerContext().getOperation().getServerId());
575 persistNewServerConfiguration(getRemoveServerContext());
579 private static class RemoveServerContext extends ServerOperationContext<RemoveServer> {
580 private final String peerAddress;
582 RemoveServerContext(RemoveServer operation, String peerAddress, ActorRef clientRequestor) {
583 super(operation, clientRequestor);
584 this.peerAddress = peerAddress;
588 Object newReply(ServerChangeStatus status, String leaderId) {
589 return new RemoveServerReply(status, leaderId);
593 InitialOperationState newInitialOperationState(RaftActorServerConfigurationSupport support) {
594 return support.new InitialRemoveServerState(this);
598 void operationComplete(RaftActor raftActor, ServerChangeStatus serverChangeStatus) {
599 if(peerAddress != null) {
600 raftActor.context().actorSelection(peerAddress).tell(new ServerRemoved(getOperation().getServerId()), raftActor.getSelf());
605 String getServerId() {
606 return getOperation().getServerId();
610 static class ServerOperationTimeout {
611 private final String serverId;
613 ServerOperationTimeout(String serverId){
614 this.serverId = Preconditions.checkNotNull(serverId, "serverId should not be null");
617 String getServerId() {