X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?p=controller.git;a=blobdiff_plain;f=opendaylight%2Fmd-sal%2Fsal-akka-raft%2Fsrc%2Fmain%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Fraft%2FRaftActorServerConfigurationSupport.java;h=7012e0db86f8847b3660f78de8cefb3a1f8c7a6a;hp=db9088ebae7b78477f8a11ede661a49748dcf8d4;hb=24ace09aacc620fd9768e0a7004e802f9385bcfc;hpb=f1e4796a07ad925697db5e3b672e2b46980cd408 diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/RaftActorServerConfigurationSupport.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/RaftActorServerConfigurationSupport.java index db9088ebae..7012e0db86 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/RaftActorServerConfigurationSupport.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/RaftActorServerConfigurationSupport.java @@ -12,22 +12,32 @@ import akka.actor.ActorSelection; import akka.actor.Cancellable; import com.google.common.base.Preconditions; import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Map; import java.util.Queue; import java.util.UUID; import javax.annotation.Nullable; +import org.opendaylight.controller.cluster.raft.ServerConfigurationPayload.ServerInfo; import org.opendaylight.controller.cluster.raft.base.messages.ApplyState; +import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout; import org.opendaylight.controller.cluster.raft.base.messages.SnapshotComplete; import org.opendaylight.controller.cluster.raft.behaviors.AbstractLeader; import org.opendaylight.controller.cluster.raft.messages.AddServer; import org.opendaylight.controller.cluster.raft.messages.AddServerReply; +import org.opendaylight.controller.cluster.raft.messages.ChangeServersVotingStatus; import org.opendaylight.controller.cluster.raft.messages.RemoveServer; import org.opendaylight.controller.cluster.raft.messages.RemoveServerReply; +import org.opendaylight.controller.cluster.raft.messages.ServerChangeReply; import org.opendaylight.controller.cluster.raft.messages.ServerChangeStatus; import org.opendaylight.controller.cluster.raft.messages.ServerRemoved; import org.opendaylight.controller.cluster.raft.messages.UnInitializedFollowerSnapshotReply; import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import scala.concurrent.duration.FiniteDuration; /** * Handles server configuration related messages for a RaftActor. @@ -59,6 +69,9 @@ class RaftActorServerConfigurationSupport { } else if(message instanceof RemoveServer) { onRemoveServer((RemoveServer) message, sender); return true; + } else if(message instanceof ChangeServersVotingStatus) { + onChangeServersVotingStatus((ChangeServersVotingStatus) message, sender); + return true; } else if (message instanceof ServerOperationTimeout) { currentOperationState.onServerOperationTimeout((ServerOperationTimeout) message); return true; @@ -75,9 +88,39 @@ class RaftActorServerConfigurationSupport { } } + void onNewLeader(String leaderId) { + currentOperationState.onNewLeader(leaderId); + } + + private void onChangeServersVotingStatus(ChangeServersVotingStatus message, ActorRef sender) { + LOG.debug("{}: onChangeServersVotingStatus: {}, state: {}", raftContext.getId(), message, + currentOperationState); + + // The following check is a special case. Normally we fail an operation if there's no leader. + // Consider a scenario where one has 2 geographically-separated 3-node clusters, one a primary and + // the other a backup such that if the primary cluster is lost, the backup can take over. In this + // scenario, we have a logical 6-node cluster where the primary sub-cluster is configured as voting + // and the backup sub-cluster as non-voting such that the primary cluster can make progress without + // consensus from the backup cluster while still replicating to the backup. On fail-over to the backup, + // a request would be sent to a member of the backup cluster to flip the voting states, ie make the + // backup sub-cluster voting and the lost primary non-voting. However since the primary majority + // cluster is lost, there would be no leader to apply, persist and replicate the server config change. + // Therefore, if the local server is currently non-voting and is to be changed to voting and there is + // no current leader, we will try to elect a leader using the new server config in order to replicate + // the change and progress. + boolean localServerChangingToVoting = Boolean.TRUE.equals(message. + getServerVotingStatusMap().get(raftActor.getRaftActorContext().getId())); + boolean hasNoLeader = raftActor.getLeaderId() == null; + if(localServerChangingToVoting && !raftContext.isVotingMember() && hasNoLeader) { + currentOperationState.onNewOperation(new ChangeServersVotingStatusContext(message, sender, true)); + } else { + onNewOperation(new ChangeServersVotingStatusContext(message, sender, false)); + } + } + private void onRemoveServer(RemoveServer removeServer, ActorRef sender) { LOG.debug("{}: onRemoveServer: {}, state: {}", raftContext.getId(), removeServer, currentOperationState); - boolean isSelf = removeServer.getServerId().equals(raftActor.getId()); + boolean isSelf = removeServer.getServerId().equals(raftContext.getId()); if(isSelf && !raftContext.hasFollowers()) { sender.tell(new RemoveServerReply(ServerChangeStatus.NOT_SUPPORTED, raftActor.getLeaderId()), raftActor.getSelf()); @@ -135,7 +178,7 @@ class RaftActorServerConfigurationSupport { ActorSelection leader = raftActor.getLeader(); if (leader != null) { LOG.debug("{}: Not leader - forwarding to leader {}", raftContext.getId(), leader); - leader.forward(operationContext.getOperation(), raftActor.getContext()); + leader.tell(operationContext.getOperation(), operationContext.getClientRequestor()); } else { LOG.debug("{}: No leader - returning NO_LEADER reply", raftContext.getId()); operationContext.getClientRequestor().tell(operationContext.newReply( @@ -144,21 +187,6 @@ class RaftActorServerConfigurationSupport { } } - /** - * Interface for a server operation FSM state. - */ - private interface OperationState { - void onNewOperation(ServerOperationContext operationContext); - - void onServerOperationTimeout(ServerOperationTimeout timeout); - - void onUnInitializedFollowerSnapshotReply(UnInitializedFollowerSnapshotReply reply); - - void onApplyState(ApplyState applyState); - - void onSnapshotComplete(); - } - /** * Interface for the initial state for a server operation. */ @@ -169,9 +197,8 @@ class RaftActorServerConfigurationSupport { /** * Abstract base class for a server operation FSM state. Handles common behavior for all states. */ - private abstract class AbstractOperationState implements OperationState { - @Override - public void onNewOperation(ServerOperationContext operationContext) { + private abstract class OperationState { + void onNewOperation(ServerOperationContext operationContext) { // We're currently processing another operation so queue it to be processed later. LOG.debug("{}: Server operation already in progress - queueing {}", raftContext.getId(), @@ -180,35 +207,36 @@ class RaftActorServerConfigurationSupport { pendingOperationsQueue.add(operationContext); } - @Override - public void onServerOperationTimeout(ServerOperationTimeout timeout) { + void onServerOperationTimeout(ServerOperationTimeout timeout) { LOG.debug("onServerOperationTimeout should not be called in state {}", this); } - @Override - public void onUnInitializedFollowerSnapshotReply(UnInitializedFollowerSnapshotReply reply) { + void onUnInitializedFollowerSnapshotReply(UnInitializedFollowerSnapshotReply reply) { LOG.debug("onUnInitializedFollowerSnapshotReply was called in state {}", this); } - @Override - public void onApplyState(ApplyState applyState) { + void onApplyState(ApplyState applyState) { LOG.debug("onApplyState was called in state {}", this); } - @Override - public void onSnapshotComplete() { + void onSnapshotComplete() { + + } + + void onNewLeader(String newLeader) { } protected void persistNewServerConfiguration(ServerOperationContext operationContext){ raftContext.setDynamicServerConfigurationInUse(); - boolean includeSelf = !operationContext.getServerId().equals(raftActor.getId()); - ServerConfigurationPayload payload = raftContext.getPeerServerInfo(includeSelf); + ServerConfigurationPayload payload = raftContext.getPeerServerInfo( + operationContext.includeSelfInNewConfiguration(raftActor)); LOG.debug("{}: New server configuration : {}", raftContext.getId(), payload.getServerConfig()); raftActor.persistData(operationContext.getClientRequestor(), operationContext.getContextId(), payload); - currentOperationState = new Persisting(operationContext, newTimer(new ServerOperationTimeout(operationContext.getServerId()))); + currentOperationState = new Persisting(operationContext, newTimer(new ServerOperationTimeout( + operationContext.getLoggingContext()))); sendReply(operationContext, ServerChangeStatus.OK); } @@ -218,8 +246,12 @@ class RaftActorServerConfigurationSupport { sendReply(operationContext, replyStatus); } - operationContext.operationComplete(raftActor, replyStatus); + operationContext.operationComplete(raftActor, replyStatus == null || replyStatus == ServerChangeStatus.OK); + + changeToIdleState(); + } + protected void changeToIdleState() { currentOperationState = IDLE; ServerOperationContext nextOperation = pendingOperationsQueue.poll(); @@ -236,8 +268,12 @@ class RaftActorServerConfigurationSupport { } Cancellable newTimer(Object message) { + return newTimer(raftContext.getConfigParams().getElectionTimeOutInterval().$times(2), message); + } + + Cancellable newTimer(FiniteDuration timeout, Object message) { return raftContext.getActorSystem().scheduler().scheduleOnce( - raftContext.getConfigParams().getElectionTimeOutInterval().$times(2), raftContext.getActor(), message, + timeout, raftContext.getActor(), message, raftContext.getActorSystem().dispatcher(), raftContext.getActor()); } @@ -250,7 +286,7 @@ class RaftActorServerConfigurationSupport { /** * The state when no server operation is in progress. It immediately initiates new server operations. */ - private class Idle extends AbstractOperationState { + private final class Idle extends OperationState { @Override public void onNewOperation(ServerOperationContext operationContext) { operationContext.newInitialOperationState(RaftActorServerConfigurationSupport.this).initiate(); @@ -265,7 +301,7 @@ class RaftActorServerConfigurationSupport { /** * The state when a new server configuration is being persisted and replicated. */ - private class Persisting extends AbstractOperationState { + private final class Persisting extends OperationState { private final ServerOperationContext operationContext; private final Cancellable timer; private boolean timedOut = false; @@ -280,7 +316,7 @@ class RaftActorServerConfigurationSupport { // Sanity check - we could get an ApplyState from a previous operation that timed out so make // sure it's meant for us. if(operationContext.getContextId().equals(applyState.getIdentifier())) { - LOG.info("{}: {} has been successfully replicated to a majority of followers", raftActor.getId(), + LOG.info("{}: {} has been successfully replicated to a majority of followers", raftContext.getId(), applyState.getReplicatedLogEntry().getData()); timer.cancel(); @@ -291,7 +327,7 @@ class RaftActorServerConfigurationSupport { @Override public void onServerOperationTimeout(ServerOperationTimeout timeout) { LOG.warn("{}: Timeout occured while replicating the new server configuration for {}", raftContext.getId(), - timeout.getServerId()); + timeout.getLoggingContext()); timedOut = true; @@ -316,7 +352,7 @@ class RaftActorServerConfigurationSupport { /** * Abstract base class for an AddServer operation state. */ - private abstract class AddServerState extends AbstractOperationState { + private abstract class AddServerState extends OperationState { private final AddServerContext addServerContext; AddServerState(AddServerContext addServerContext) { @@ -332,7 +368,7 @@ class RaftActorServerConfigurationSupport { } void handleInstallSnapshotTimeout(ServerOperationTimeout timeout) { - String serverId = timeout.getServerId(); + String serverId = timeout.getLoggingContext(); LOG.debug("{}: handleInstallSnapshotTimeout for new server {}", raftContext.getId(), serverId); @@ -354,7 +390,7 @@ class RaftActorServerConfigurationSupport { * The initial state for the AddServer operation. It adds the new follower as a peer and initiates * snapshot capture, if necessary. */ - private class InitialAddServerState extends AddServerState implements InitialOperationState { + private final class InitialAddServerState extends AddServerState implements InitialOperationState { InitialAddServerState(AddServerContext addServerContext) { super(addServerContext); } @@ -404,7 +440,7 @@ class RaftActorServerConfigurationSupport { * The AddServer operation state for when the catch-up snapshot is being installed. It handles successful * reply or timeout. */ - private class InstallingSnapshot extends AddServerState { + private final class InstallingSnapshot extends AddServerState { private final Cancellable installSnapshotTimer; InstallingSnapshot(AddServerContext addServerContext, Cancellable installSnapshotTimer) { @@ -417,7 +453,7 @@ class RaftActorServerConfigurationSupport { handleInstallSnapshotTimeout(timeout); LOG.warn("{}: Timeout occured for new server {} while installing snapshot", raftContext.getId(), - timeout.getServerId()); + timeout.getLoggingContext()); } @Override @@ -448,7 +484,7 @@ class RaftActorServerConfigurationSupport { * The AddServer operation state for when there is a snapshot already in progress. When the current * snapshot completes, it initiates an install snapshot. */ - private class WaitingForPriorSnapshotComplete extends AddServerState { + private final class WaitingForPriorSnapshotComplete extends AddServerState { private final Cancellable snapshotTimer; WaitingForPriorSnapshotComplete(AddServerContext addServerContext, Cancellable snapshotTimer) { @@ -482,7 +518,7 @@ class RaftActorServerConfigurationSupport { handleInstallSnapshotTimeout(timeout); LOG.warn("{}: Timeout occured for new server {} while waiting for prior snapshot to complete", - raftContext.getId(), timeout.getServerId()); + raftContext.getId(), timeout.getLoggingContext()); } } @@ -514,13 +550,18 @@ class RaftActorServerConfigurationSupport { return clientRequestor; } + void operationComplete(RaftActor raftActor, boolean succeeded) { + } + + boolean includeSelfInNewConfiguration(RaftActor raftActor) { + return true; + } + abstract Object newReply(ServerChangeStatus status, String leaderId); abstract InitialOperationState newInitialOperationState(RaftActorServerConfigurationSupport support); - abstract void operationComplete(RaftActor raftActor, ServerChangeStatus serverChangeStatus); - - abstract String getServerId(); + abstract String getLoggingContext(); } /** @@ -542,20 +583,14 @@ class RaftActorServerConfigurationSupport { } @Override - void operationComplete(RaftActor raftActor, ServerChangeStatus serverChangeStatus) { - - } - - @Override - String getServerId() { + String getLoggingContext() { return getOperation().getNewServerId(); } } - private abstract class RemoveServerState extends AbstractOperationState { + private abstract class RemoveServerState extends OperationState { private final RemoveServerContext removeServerContext; - protected RemoveServerState(RemoveServerContext removeServerContext) { this.removeServerContext = Preconditions.checkNotNull(removeServerContext); @@ -566,7 +601,7 @@ class RaftActorServerConfigurationSupport { } } - private class InitialRemoveServerState extends RemoveServerState implements InitialOperationState{ + private final class InitialRemoveServerState extends RemoveServerState implements InitialOperationState{ protected InitialRemoveServerState(RemoveServerContext removeServerContext) { super(removeServerContext); @@ -601,27 +636,230 @@ class RaftActorServerConfigurationSupport { } @Override - void operationComplete(RaftActor raftActor, ServerChangeStatus serverChangeStatus) { + void operationComplete(RaftActor raftActor, boolean succeeded) { if(peerAddress != null) { raftActor.context().actorSelection(peerAddress).tell(new ServerRemoved(getOperation().getServerId()), raftActor.getSelf()); } } @Override - String getServerId() { + boolean includeSelfInNewConfiguration(RaftActor raftActor) { + return !getOperation().getServerId().equals(raftActor.getId()); + } + + @Override + String getLoggingContext() { return getOperation().getServerId(); } } + private static class ChangeServersVotingStatusContext extends ServerOperationContext { + private final boolean tryToElectLeader; + + ChangeServersVotingStatusContext(ChangeServersVotingStatus convertMessage, ActorRef clientRequestor, + boolean tryToElectLeader) { + super(convertMessage, clientRequestor); + this.tryToElectLeader = tryToElectLeader; + } + + @Override + InitialOperationState newInitialOperationState(RaftActorServerConfigurationSupport support) { + return support.new ChangeServersVotingStatusState(this, tryToElectLeader); + } + + @Override + Object newReply(ServerChangeStatus status, String leaderId) { + return new ServerChangeReply(status, leaderId); + } + + @Override + void operationComplete(final RaftActor raftActor, boolean succeeded) { + // If this leader changed to non-voting we need to step down as leader so we'll try to transfer + // leadership. + boolean localServerChangedToNonVoting = Boolean.FALSE.equals(getOperation(). + getServerVotingStatusMap().get(raftActor.getRaftActorContext().getId())); + if(succeeded && localServerChangedToNonVoting && raftActor.isLeader()) { + raftActor.initiateLeadershipTransfer(new RaftActorLeadershipTransferCohort.OnComplete() { + @Override + public void onSuccess(ActorRef raftActorRef, ActorRef replyTo) { + LOG.debug("{}: leader transfer succeeded after change to non-voting", raftActor.persistenceId()); + ensureFollowerState(raftActor); + } + + @Override + public void onFailure(ActorRef raftActorRef, ActorRef replyTo) { + LOG.debug("{}: leader transfer failed after change to non-voting", raftActor.persistenceId()); + ensureFollowerState(raftActor); + } + + private void ensureFollowerState(RaftActor raftActor) { + // Whether or not leadership transfer succeeded, we have to step down as leader and + // switch to Follower so ensure that. + if(raftActor.getRaftState() != RaftState.Follower) { + raftActor.initializeBehavior(); + } + } + }); + } + } + + @Override + String getLoggingContext() { + return getOperation().toString(); + } + } + + private class ChangeServersVotingStatusState extends OperationState implements InitialOperationState { + private final ChangeServersVotingStatusContext changeVotingStatusContext; + private final boolean tryToElectLeader; + + ChangeServersVotingStatusState(ChangeServersVotingStatusContext changeVotingStatusContext, + boolean tryToElectLeader) { + this.changeVotingStatusContext = changeVotingStatusContext; + this.tryToElectLeader = tryToElectLeader; + } + + @Override + public void initiate() { + LOG.debug("Initiating ChangeServersVotingStatusState"); + + if(tryToElectLeader) { + initiateLocalLeaderElection(); + } else { + updateLocalPeerInfo(); + + persistNewServerConfiguration(changeVotingStatusContext); + } + } + + private void initiateLocalLeaderElection() { + LOG.debug("{}: Sending local ElectionTimeout to start leader election", raftContext.getId()); + + ServerConfigurationPayload previousServerConfig = raftContext.getPeerServerInfo(true); + updateLocalPeerInfo(); + + raftContext.getActor().tell(ElectionTimeout.INSTANCE, raftContext.getActor()); + + currentOperationState = new WaitingForLeaderElected(changeVotingStatusContext, previousServerConfig); + } + + private void updateLocalPeerInfo() { + List newServerInfoList = newServerInfoList(); + + raftContext.updatePeerIds(new ServerConfigurationPayload(newServerInfoList)); + if(raftActor.getCurrentBehavior() instanceof AbstractLeader) { + AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior(); + leader.updateMinReplicaCount(); + } + } + + private List newServerInfoList() { + Map serverVotingStatusMap = changeVotingStatusContext.getOperation().getServerVotingStatusMap(); + List newServerInfoList = new ArrayList<>(); + for(String peerId: raftContext.getPeerIds()) { + newServerInfoList.add(new ServerInfo(peerId, serverVotingStatusMap.containsKey(peerId) ? + serverVotingStatusMap.get(peerId) : raftContext.getPeerInfo(peerId).isVoting())); + } + + newServerInfoList.add(new ServerInfo(raftContext.getId(), serverVotingStatusMap.containsKey( + raftContext.getId()) ? serverVotingStatusMap.get(raftContext.getId()) : raftContext.isVotingMember())); + + return newServerInfoList; + } + } + + private class WaitingForLeaderElected extends OperationState { + private final ServerConfigurationPayload previousServerConfig; + private final ChangeServersVotingStatusContext operationContext; + private final Cancellable timer; + + WaitingForLeaderElected(ChangeServersVotingStatusContext operationContext, + ServerConfigurationPayload previousServerConfig) { + this.operationContext = operationContext; + this.previousServerConfig = previousServerConfig; + + timer = newTimer(raftContext.getConfigParams().getElectionTimeOutInterval(), + new ServerOperationTimeout(operationContext.getLoggingContext())); + } + + @Override + void onNewLeader(String newLeader) { + LOG.debug("{}: New leader {} elected", raftContext.getId(), newLeader); + + timer.cancel(); + + if(raftActor.isLeader()) { + persistNewServerConfiguration(operationContext); + } else { + // Edge case - some other node became leader so forward the operation. + LOG.debug("{}: Forwarding {} to new leader", raftContext.getId(), operationContext.getOperation()); + + // Revert the local server config change. + raftContext.updatePeerIds(previousServerConfig); + + changeToIdleState(); + RaftActorServerConfigurationSupport.this.onNewOperation(operationContext); + } + } + + @Override + void onServerOperationTimeout(ServerOperationTimeout timeout) { + LOG.warn("{}: Leader election timed out - cannot apply operation {}", + raftContext.getId(), timeout.getLoggingContext()); + + // Revert the local server config change. + raftContext.updatePeerIds(previousServerConfig); + raftActor.initializeBehavior(); + + tryToForwardOperationToAnotherServer(); + } + + private void tryToForwardOperationToAnotherServer() { + Collection serversVisited = new HashSet<>(operationContext.getOperation().getServersVisited()); + + LOG.debug("{}: tryToForwardOperationToAnotherServer - servers already visited {}", raftContext.getId(), + serversVisited); + + serversVisited.add(raftContext.getId()); + + // Try to find another whose state is being changed from non-voting to voting and that we haven't + // tried yet. + Map serverVotingStatusMap = operationContext.getOperation().getServerVotingStatusMap(); + ActorSelection forwardToPeerActor = null; + for(Map.Entry e: serverVotingStatusMap.entrySet()) { + Boolean isVoting = e.getValue(); + String serverId = e.getKey(); + PeerInfo peerInfo = raftContext.getPeerInfo(serverId); + if(isVoting && peerInfo != null && !peerInfo.isVoting() && !serversVisited.contains(serverId)) { + ActorSelection actor = raftContext.getPeerActorSelection(serverId); + if(actor != null) { + forwardToPeerActor = actor; + break; + } + } + } + + if(forwardToPeerActor != null) { + LOG.debug("{}: Found server {} to forward to", raftContext.getId(), forwardToPeerActor); + + forwardToPeerActor.tell(new ChangeServersVotingStatus(serverVotingStatusMap, serversVisited), + operationContext.getClientRequestor()); + changeToIdleState(); + } else { + operationComplete(operationContext, ServerChangeStatus.NO_LEADER); + } + } + } + static class ServerOperationTimeout { - private final String serverId; + private final String loggingContext; - ServerOperationTimeout(String serverId){ - this.serverId = Preconditions.checkNotNull(serverId, "serverId should not be null"); + ServerOperationTimeout(String loggingContext){ + this.loggingContext = Preconditions.checkNotNull(loggingContext, "loggingContext should not be null"); } - String getServerId() { - return serverId; + String getLoggingContext() { + return loggingContext; } } }