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=004272192c393b2e9267f6175312bb59bb71fdb4;hp=e78f39cdb1be0db11a39b1a91255effa5c3dc0aa;hb=HEAD;hpb=f2b5692224570e7ecccb139594ed55237efeec03 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 e78f39cdb1..fd2cd419d7 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 @@ -7,34 +7,40 @@ */ package org.opendaylight.controller.cluster.raft; +import static java.util.Objects.requireNonNull; + import akka.actor.ActorRef; import akka.actor.ActorSelection; import akka.actor.Cancellable; -import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; import java.util.ArrayDeque; -import java.util.ArrayList; -import java.util.List; +import java.util.Collection; +import java.util.HashSet; import java.util.Map; import java.util.Queue; import java.util.UUID; -import javax.annotation.Nullable; -import org.opendaylight.controller.cluster.raft.RaftActorLeadershipTransferCohort.OnComplete; -import org.opendaylight.controller.cluster.raft.ServerConfigurationPayload.ServerInfo; +import org.eclipse.jdt.annotation.Nullable; import org.opendaylight.controller.cluster.raft.base.messages.ApplyState; import org.opendaylight.controller.cluster.raft.base.messages.SnapshotComplete; +import org.opendaylight.controller.cluster.raft.base.messages.TimeoutNow; 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.Payload; 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.opendaylight.controller.cluster.raft.persisted.ServerConfigurationPayload; +import org.opendaylight.controller.cluster.raft.persisted.ServerInfo; +import org.opendaylight.yangtools.concepts.Identifier; +import org.opendaylight.yangtools.util.AbstractUUIDIdentifier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import scala.concurrent.duration.FiniteDuration; /** * Handles server configuration related messages for a RaftActor. @@ -44,6 +50,7 @@ import org.slf4j.LoggerFactory; class RaftActorServerConfigurationSupport { private static final Logger LOG = LoggerFactory.getLogger(RaftActorServerConfigurationSupport.class); + @SuppressWarnings("checkstyle:MemberName") private final OperationState IDLE = new Idle(); private final RaftActor raftActor; @@ -54,30 +61,30 @@ class RaftActorServerConfigurationSupport { private OperationState currentOperationState = IDLE; - RaftActorServerConfigurationSupport(RaftActor raftActor) { + RaftActorServerConfigurationSupport(final RaftActor raftActor) { this.raftActor = raftActor; - this.raftContext = raftActor.getRaftActorContext(); + raftContext = raftActor.getRaftActorContext(); } - boolean handleMessage(Object message, ActorRef sender) { - if(message instanceof AddServer) { - onAddServer((AddServer) message, sender); + boolean handleMessage(final Object message, final ActorRef sender) { + if (message instanceof AddServer addServer) { + onAddServer(addServer, sender); return true; - } else if(message instanceof RemoveServer) { - onRemoveServer((RemoveServer) message, sender); + } else if (message instanceof RemoveServer removeServer) { + onRemoveServer(removeServer, sender); return true; - } else if(message instanceof ChangeServersVotingStatus) { - onChangeServersVotingStatus((ChangeServersVotingStatus) message, sender); + } else if (message instanceof ChangeServersVotingStatus changeServersVotingStatus) { + onChangeServersVotingStatus(changeServersVotingStatus, sender); return true; - } else if (message instanceof ServerOperationTimeout) { - currentOperationState.onServerOperationTimeout((ServerOperationTimeout) message); + } else if (message instanceof ServerOperationTimeout serverOperationTimeout) { + currentOperationState.onServerOperationTimeout(serverOperationTimeout); return true; - } else if (message instanceof UnInitializedFollowerSnapshotReply) { - currentOperationState.onUnInitializedFollowerSnapshotReply((UnInitializedFollowerSnapshotReply) message); + } else if (message instanceof UnInitializedFollowerSnapshotReply uninitFollowerSnapshotReply) { + currentOperationState.onUnInitializedFollowerSnapshotReply(uninitFollowerSnapshotReply); return true; - } else if(message instanceof ApplyState) { - return onApplyState((ApplyState) message); - } else if(message instanceof SnapshotComplete) { + } else if (message instanceof ApplyState applyState) { + return onApplyState(applyState); + } else if (message instanceof SnapshotComplete) { currentOperationState.onSnapshotComplete(); return false; } else { @@ -85,20 +92,43 @@ class RaftActorServerConfigurationSupport { } } - private void onChangeServersVotingStatus(ChangeServersVotingStatus message, ActorRef sender) { + void onNewLeader(final String leaderId) { + currentOperationState.onNewLeader(leaderId); + } + + private void onChangeServersVotingStatus(final ChangeServersVotingStatus message, final ActorRef sender) { LOG.debug("{}: onChangeServersVotingStatus: {}, state: {}", raftContext.getId(), message, currentOperationState); - onNewOperation(new ChangeServersVotingStatusContext(message, sender)); + // 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) { + private void onRemoveServer(final RemoveServer removeServer, final ActorRef sender) { LOG.debug("{}: onRemoveServer: {}, state: {}", raftContext.getId(), removeServer, currentOperationState); - boolean isSelf = removeServer.getServerId().equals(raftActor.getId()); - if(isSelf && !raftContext.hasFollowers()) { + boolean isSelf = removeServer.getServerId().equals(raftContext.getId()); + if (isSelf && !raftContext.hasFollowers()) { sender.tell(new RemoveServerReply(ServerChangeStatus.NOT_SUPPORTED, raftActor.getLeaderId()), raftActor.getSelf()); - } else if(!isSelf && !raftContext.getPeerIds().contains(removeServer.getServerId())) { + } else if (!isSelf && !raftContext.getPeerIds().contains(removeServer.getServerId())) { sender.tell(new RemoveServerReply(ServerChangeStatus.DOES_NOT_EXIST, raftActor.getLeaderId()), raftActor.getSelf()); } else { @@ -108,9 +138,9 @@ class RaftActorServerConfigurationSupport { } } - private boolean onApplyState(ApplyState applyState) { + private boolean onApplyState(final ApplyState applyState) { Payload data = applyState.getReplicatedLogEntry().getData(); - if(data instanceof ServerConfigurationPayload) { + if (data instanceof ServerConfigurationPayload) { currentOperationState.onApplyState(applyState); return true; } @@ -119,7 +149,7 @@ class RaftActorServerConfigurationSupport { } /** - * The algorithm for AddServer is as follows: + * Add a server. The algorithm for AddServer is as follows: * */ - private void onAddServer(AddServer addServer, ActorRef sender) { + private void onAddServer(final AddServer addServer, final ActorRef sender) { LOG.debug("{}: onAddServer: {}, state: {}", raftContext.getId(), addServer, currentOperationState); onNewOperation(new AddServerContext(addServer, sender)); } - private void onNewOperation(ServerOperationContext operationContext) { + private void onNewOperation(final ServerOperationContext operationContext) { if (raftActor.isLeader()) { currentOperationState.onNewOperation(operationContext); } else { 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( @@ -172,7 +202,7 @@ class RaftActorServerConfigurationSupport { * Abstract base class for a server operation FSM state. Handles common behavior for all states. */ private abstract class OperationState { - void onNewOperation(ServerOperationContext operationContext) { + void onNewOperation(final 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(), @@ -181,15 +211,15 @@ class RaftActorServerConfigurationSupport { pendingOperationsQueue.add(operationContext); } - void onServerOperationTimeout(ServerOperationTimeout timeout) { + void onServerOperationTimeout(final ServerOperationTimeout timeout) { LOG.debug("onServerOperationTimeout should not be called in state {}", this); } - void onUnInitializedFollowerSnapshotReply(UnInitializedFollowerSnapshotReply reply) { + void onUnInitializedFollowerSnapshotReply(final UnInitializedFollowerSnapshotReply reply) { LOG.debug("onUnInitializedFollowerSnapshotReply was called in state {}", this); } - void onApplyState(ApplyState applyState) { + void onApplyState(final ApplyState applyState) { LOG.debug("onApplyState was called in state {}", this); } @@ -197,14 +227,18 @@ class RaftActorServerConfigurationSupport { } - protected void persistNewServerConfiguration(ServerOperationContext operationContext){ + void onNewLeader(final String newLeader) { + } + + protected void persistNewServerConfiguration(final ServerOperationContext operationContext) { raftContext.setDynamicServerConfigurationInUse(); ServerConfigurationPayload payload = raftContext.getPeerServerInfo( operationContext.includeSelfInNewConfiguration(raftActor)); LOG.debug("{}: New server configuration : {}", raftContext.getId(), payload.getServerConfig()); - raftActor.persistData(operationContext.getClientRequestor(), operationContext.getContextId(), payload); + raftActor.persistData(operationContext.getClientRequestor(), operationContext.getContextId(), + payload, false); currentOperationState = new Persisting(operationContext, newTimer(new ServerOperationTimeout( operationContext.getLoggingContext()))); @@ -212,31 +246,41 @@ class RaftActorServerConfigurationSupport { sendReply(operationContext, ServerChangeStatus.OK); } - protected void operationComplete(ServerOperationContext operationContext, @Nullable ServerChangeStatus replyStatus) { - if(replyStatus != null) { + protected void operationComplete(final ServerOperationContext operationContext, + final @Nullable ServerChangeStatus replyStatus) { + if (replyStatus != null) { sendReply(operationContext, replyStatus); } operationContext.operationComplete(raftActor, replyStatus == null || replyStatus == ServerChangeStatus.OK); + changeToIdleState(); + } + + protected void changeToIdleState() { currentOperationState = IDLE; ServerOperationContext nextOperation = pendingOperationsQueue.poll(); - if(nextOperation != null) { + if (nextOperation != null) { RaftActorServerConfigurationSupport.this.onNewOperation(nextOperation); } } - protected void sendReply(ServerOperationContext operationContext, ServerChangeStatus status) { - LOG.debug("{}: Returning {} for operation {}", raftContext.getId(), status, operationContext.getOperation()); + protected void sendReply(final ServerOperationContext operationContext, final ServerChangeStatus status) { + LOG.debug("{}: Returning {} for operation {}", raftContext.getId(), status, + operationContext.getOperation()); operationContext.getClientRequestor().tell(operationContext.newReply(status, raftActor.getLeaderId()), raftActor.self()); } - Cancellable newTimer(Object message) { + Cancellable newTimer(final Object message) { + return newTimer(raftContext.getConfigParams().getElectionTimeOutInterval().$times(2), message); + } + + Cancellable newTimer(final FiniteDuration timeout, final Object message) { return raftContext.getActorSystem().scheduler().scheduleOnce( - raftContext.getConfigParams().getElectionTimeOutInterval().$times(2), raftContext.getActor(), message, + timeout, raftContext.getActor(), message, raftContext.getActorSystem().dispatcher(), raftContext.getActor()); } @@ -251,12 +295,12 @@ class RaftActorServerConfigurationSupport { */ private final class Idle extends OperationState { @Override - public void onNewOperation(ServerOperationContext operationContext) { + public void onNewOperation(final ServerOperationContext operationContext) { operationContext.newInitialOperationState(RaftActorServerConfigurationSupport.this).initiate(); } @Override - public void onApplyState(ApplyState applyState) { + public void onApplyState(final ApplyState applyState) { // Noop - we override b/c ApplyState is called normally for followers in the idle state. } } @@ -269,17 +313,17 @@ class RaftActorServerConfigurationSupport { private final Cancellable timer; private boolean timedOut = false; - Persisting(ServerOperationContext operationContext, Cancellable timer) { + Persisting(final ServerOperationContext operationContext, final Cancellable timer) { this.operationContext = operationContext; this.timer = timer; } @Override - public void onApplyState(ApplyState applyState) { + public void onApplyState(final ApplyState applyState) { // 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(), + if (operationContext.getContextId().equals(applyState.getIdentifier())) { + LOG.info("{}: {} has been successfully replicated to a majority of followers", raftContext.getId(), applyState.getReplicatedLogEntry().getData()); timer.cancel(); @@ -288,7 +332,7 @@ class RaftActorServerConfigurationSupport { } @Override - public void onServerOperationTimeout(ServerOperationTimeout timeout) { + public void onServerOperationTimeout(final ServerOperationTimeout timeout) { LOG.warn("{}: Timeout occured while replicating the new server configuration for {}", raftContext.getId(), timeout.getLoggingContext()); @@ -296,18 +340,18 @@ class RaftActorServerConfigurationSupport { // Fail any pending operations ServerOperationContext nextOperation = pendingOperationsQueue.poll(); - while(nextOperation != null) { + while (nextOperation != null) { sendReply(nextOperation, ServerChangeStatus.PRIOR_REQUEST_CONSENSUS_TIMEOUT); nextOperation = pendingOperationsQueue.poll(); } } @Override - public void onNewOperation(ServerOperationContext operationContext) { - if(timedOut) { - sendReply(operationContext, ServerChangeStatus.PRIOR_REQUEST_CONSENSUS_TIMEOUT); + public void onNewOperation(final ServerOperationContext newOperationContext) { + if (timedOut) { + sendReply(newOperationContext, ServerChangeStatus.PRIOR_REQUEST_CONSENSUS_TIMEOUT); } else { - super.onNewOperation(operationContext); + super.onNewOperation(newOperationContext); } } } @@ -318,7 +362,7 @@ class RaftActorServerConfigurationSupport { private abstract class AddServerState extends OperationState { private final AddServerContext addServerContext; - AddServerState(AddServerContext addServerContext) { + AddServerState(final AddServerContext addServerContext) { this.addServerContext = addServerContext; } @@ -330,7 +374,7 @@ class RaftActorServerConfigurationSupport { return newTimer(new ServerOperationTimeout(addServerContext.getOperation().getNewServerId())); } - void handleInstallSnapshotTimeout(ServerOperationTimeout timeout) { + void handleInstallSnapshotTimeout(final ServerOperationTimeout timeout) { String serverId = timeout.getLoggingContext(); LOG.debug("{}: handleInstallSnapshotTimeout for new server {}", raftContext.getId(), serverId); @@ -339,12 +383,13 @@ class RaftActorServerConfigurationSupport { raftContext.removePeer(serverId); boolean isLeader = raftActor.isLeader(); - if(isLeader) { + if (isLeader) { AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior(); leader.removeFollower(serverId); } - operationComplete(getAddServerContext(), isLeader ? ServerChangeStatus.TIMEOUT : ServerChangeStatus.NO_LEADER); + operationComplete(getAddServerContext(), isLeader ? ServerChangeStatus.TIMEOUT + : ServerChangeStatus.NO_LEADER); } } @@ -354,18 +399,18 @@ class RaftActorServerConfigurationSupport { * snapshot capture, if necessary. */ private final class InitialAddServerState extends AddServerState implements InitialOperationState { - InitialAddServerState(AddServerContext addServerContext) { + InitialAddServerState(final AddServerContext addServerContext) { super(addServerContext); } @Override public void initiate() { - AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior(); + final AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior(); AddServer addServer = getAddServerContext().getOperation(); LOG.debug("{}: Initiating {}", raftContext.getId(), addServer); - if(raftContext.getPeerInfo(addServer.getNewServerId()) != null) { + if (raftContext.getPeerInfo(addServer.getNewServerId()) != null) { operationComplete(getAddServerContext(), ServerChangeStatus.ALREADY_EXISTS); return; } @@ -376,10 +421,10 @@ class RaftActorServerConfigurationSupport { leader.addFollower(addServer.getNewServerId()); - if(votingState == VotingState.VOTING_NOT_INITIALIZED){ + if (votingState == VotingState.VOTING_NOT_INITIALIZED) { // schedule the install snapshot timeout timer Cancellable installSnapshotTimer = newInstallSnapshotTimer(); - if(leader.initiateCaptureSnapshot(addServer.getNewServerId())) { + if (leader.initiateCaptureSnapshot(addServer.getNewServerId())) { LOG.debug("{}: Initiating capture snapshot for new server {}", raftContext.getId(), addServer.getNewServerId()); @@ -406,13 +451,13 @@ class RaftActorServerConfigurationSupport { private final class InstallingSnapshot extends AddServerState { private final Cancellable installSnapshotTimer; - InstallingSnapshot(AddServerContext addServerContext, Cancellable installSnapshotTimer) { + InstallingSnapshot(final AddServerContext addServerContext, final Cancellable installSnapshotTimer) { super(addServerContext); - this.installSnapshotTimer = Preconditions.checkNotNull(installSnapshotTimer); + this.installSnapshotTimer = requireNonNull(installSnapshotTimer); } @Override - public void onServerOperationTimeout(ServerOperationTimeout timeout) { + public void onServerOperationTimeout(final ServerOperationTimeout timeout) { handleInstallSnapshotTimeout(timeout); LOG.warn("{}: Timeout occured for new server {} while installing snapshot", raftContext.getId(), @@ -420,14 +465,14 @@ class RaftActorServerConfigurationSupport { } @Override - public void onUnInitializedFollowerSnapshotReply(UnInitializedFollowerSnapshotReply reply) { + public void onUnInitializedFollowerSnapshotReply(final UnInitializedFollowerSnapshotReply reply) { LOG.debug("{}: onUnInitializedFollowerSnapshotReply: {}", raftContext.getId(), reply); String followerId = reply.getFollowerId(); // Sanity check to guard against receiving an UnInitializedFollowerSnapshotReply from a prior // add server operation that timed out. - if(getAddServerContext().getOperation().getNewServerId().equals(followerId) && raftActor.isLeader()) { + if (getAddServerContext().getOperation().getNewServerId().equals(followerId) && raftActor.isLeader()) { AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior(); raftContext.getPeerInfo(followerId).setVotingState(VotingState.VOTING); leader.updateMinReplicaCount(); @@ -450,22 +495,22 @@ class RaftActorServerConfigurationSupport { private final class WaitingForPriorSnapshotComplete extends AddServerState { private final Cancellable snapshotTimer; - WaitingForPriorSnapshotComplete(AddServerContext addServerContext, Cancellable snapshotTimer) { + WaitingForPriorSnapshotComplete(final AddServerContext addServerContext, final Cancellable snapshotTimer) { super(addServerContext); - this.snapshotTimer = Preconditions.checkNotNull(snapshotTimer); + this.snapshotTimer = requireNonNull(snapshotTimer); } @Override public void onSnapshotComplete() { LOG.debug("{}: onSnapshotComplete", raftContext.getId()); - if(!raftActor.isLeader()) { + if (!raftActor.isLeader()) { LOG.debug("{}: No longer the leader", raftContext.getId()); return; } AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior(); - if(leader.initiateCaptureSnapshot(getAddServerContext().getOperation().getNewServerId())) { + if (leader.initiateCaptureSnapshot(getAddServerContext().getOperation().getNewServerId())) { LOG.debug("{}: Initiating capture snapshot for new server {}", raftContext.getId(), getAddServerContext().getOperation().getNewServerId()); @@ -477,7 +522,7 @@ class RaftActorServerConfigurationSupport { } @Override - public void onServerOperationTimeout(ServerOperationTimeout timeout) { + public void onServerOperationTimeout(final ServerOperationTimeout timeout) { handleInstallSnapshotTimeout(timeout); LOG.warn("{}: Timeout occured for new server {} while waiting for prior snapshot to complete", @@ -485,23 +530,32 @@ class RaftActorServerConfigurationSupport { } } + private static final class ServerOperationContextIdentifier + extends AbstractUUIDIdentifier { + private static final long serialVersionUID = 1L; + + ServerOperationContextIdentifier() { + super(UUID.randomUUID()); + } + } + /** * Stores context information for a server operation. * * @param the operation type */ - private static abstract class ServerOperationContext { + private abstract static class ServerOperationContext { private final T operation; private final ActorRef clientRequestor; - private final String contextId; + private final Identifier contextId; - ServerOperationContext(T operation, ActorRef clientRequestor){ + ServerOperationContext(final T operation, final ActorRef clientRequestor) { this.operation = operation; this.clientRequestor = clientRequestor; - contextId = UUID.randomUUID().toString(); + contextId = new ServerOperationContextIdentifier(); } - String getContextId() { + Identifier getContextId() { return contextId; } @@ -513,10 +567,10 @@ class RaftActorServerConfigurationSupport { return clientRequestor; } - void operationComplete(RaftActor raftActor, boolean succeeded) { + void operationComplete(final RaftActor raftActor, final boolean succeeded) { } - boolean includeSelfInNewConfiguration(RaftActor raftActor) { + boolean includeSelfInNewConfiguration(final RaftActor raftActor) { return true; } @@ -531,17 +585,17 @@ class RaftActorServerConfigurationSupport { * Stores context information for an AddServer operation. */ private static class AddServerContext extends ServerOperationContext { - AddServerContext(AddServer addServer, ActorRef clientRequestor) { + AddServerContext(final AddServer addServer, final ActorRef clientRequestor) { super(addServer, clientRequestor); } @Override - Object newReply(ServerChangeStatus status, String leaderId) { + Object newReply(final ServerChangeStatus status, final String leaderId) { return new AddServerReply(status, leaderId); } @Override - InitialOperationState newInitialOperationState(RaftActorServerConfigurationSupport support) { + InitialOperationState newInitialOperationState(final RaftActorServerConfigurationSupport support) { return support.new InitialAddServerState(this); } @@ -554,8 +608,8 @@ class RaftActorServerConfigurationSupport { private abstract class RemoveServerState extends OperationState { private final RemoveServerContext removeServerContext; - protected RemoveServerState(RemoveServerContext removeServerContext) { - this.removeServerContext = Preconditions.checkNotNull(removeServerContext); + protected RemoveServerState(final RemoveServerContext removeServerContext) { + this.removeServerContext = requireNonNull(removeServerContext); } @@ -564,9 +618,9 @@ class RaftActorServerConfigurationSupport { } } - private final class InitialRemoveServerState extends RemoveServerState implements InitialOperationState{ + private final class InitialRemoveServerState extends RemoveServerState implements InitialOperationState { - protected InitialRemoveServerState(RemoveServerContext removeServerContext) { + protected InitialRemoveServerState(final RemoveServerContext removeServerContext) { super(removeServerContext); } @@ -574,7 +628,9 @@ class RaftActorServerConfigurationSupport { public void initiate() { String serverId = getRemoveServerContext().getOperation().getServerId(); raftContext.removePeer(serverId); - ((AbstractLeader)raftActor.getCurrentBehavior()).removeFollower(serverId); + AbstractLeader leader = (AbstractLeader)raftActor.getCurrentBehavior(); + leader.removeFollower(serverId); + leader.updateMinReplicaCount(); persistNewServerConfiguration(getRemoveServerContext()); } @@ -583,30 +639,31 @@ class RaftActorServerConfigurationSupport { private static class RemoveServerContext extends ServerOperationContext { private final String peerAddress; - RemoveServerContext(RemoveServer operation, String peerAddress, ActorRef clientRequestor) { + RemoveServerContext(final RemoveServer operation, final String peerAddress, final ActorRef clientRequestor) { super(operation, clientRequestor); this.peerAddress = peerAddress; } @Override - Object newReply(ServerChangeStatus status, String leaderId) { + Object newReply(final ServerChangeStatus status, final String leaderId) { return new RemoveServerReply(status, leaderId); } @Override - InitialOperationState newInitialOperationState(RaftActorServerConfigurationSupport support) { + InitialOperationState newInitialOperationState(final RaftActorServerConfigurationSupport support) { return support.new InitialRemoveServerState(this); } @Override - void operationComplete(RaftActor raftActor, boolean succeeded) { - if(peerAddress != null) { - raftActor.context().actorSelection(peerAddress).tell(new ServerRemoved(getOperation().getServerId()), raftActor.getSelf()); + void operationComplete(final RaftActor raftActor, final boolean succeeded) { + if (peerAddress != null) { + raftActor.context().actorSelection(peerAddress).tell( + new ServerRemoved(getOperation().getServerId()), raftActor.getSelf()); } } @Override - boolean includeSelfInNewConfiguration(RaftActor raftActor) { + boolean includeSelfInNewConfiguration(final RaftActor raftActor) { return !getOperation().getServerId().equals(raftActor.getId()); } @@ -617,91 +674,213 @@ class RaftActorServerConfigurationSupport { } private static class ChangeServersVotingStatusContext extends ServerOperationContext { - ChangeServersVotingStatusContext(ChangeServersVotingStatus convertMessage, ActorRef clientRequestor) { + private final boolean tryToElectLeader; + + ChangeServersVotingStatusContext(final ChangeServersVotingStatus convertMessage, final ActorRef clientRequestor, + final boolean tryToElectLeader) { super(convertMessage, clientRequestor); + this.tryToElectLeader = tryToElectLeader; } @Override - InitialOperationState newInitialOperationState(RaftActorServerConfigurationSupport support) { - return support.new ChangeServersVotingStatusState(this); + InitialOperationState newInitialOperationState(final RaftActorServerConfigurationSupport support) { + return support.new ChangeServersVotingStatusState(this, tryToElectLeader); } @Override - Object newReply(ServerChangeStatus status, String leaderId) { + Object newReply(final ServerChangeStatus status, final String leaderId) { return new ServerChangeReply(status, leaderId); } @Override - void operationComplete(final RaftActor raftActor, boolean succeeded) { + void operationComplete(final RaftActor raftActor, final 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 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(); - } - } - }); + boolean localServerChangedToNonVoting = Boolean.FALSE.equals(getOperation() + .getServerVotingStatusMap().get(raftActor.getRaftActorContext().getId())); + if (succeeded && localServerChangedToNonVoting) { + LOG.debug("Leader changed to non-voting - trying leadership transfer"); + raftActor.becomeNonVoting(); + } else if (raftActor.isLeader()) { + raftActor.onVotingStateChangeComplete(); } } @Override String getLoggingContext() { - return getOperation().getServerVotingStatusMap().toString(); + return getOperation().toString(); } } private class ChangeServersVotingStatusState extends OperationState implements InitialOperationState { private final ChangeServersVotingStatusContext changeVotingStatusContext; + private final boolean tryToElectLeader; - ChangeServersVotingStatusState(ChangeServersVotingStatusContext changeVotingStatusContext) { + ChangeServersVotingStatusState(final ChangeServersVotingStatusContext changeVotingStatusContext, + final boolean tryToElectLeader) { this.changeVotingStatusContext = changeVotingStatusContext; + this.tryToElectLeader = tryToElectLeader; } @Override public void initiate() { LOG.debug("Initiating ChangeServersVotingStatusState"); - 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())); + if (tryToElectLeader) { + initiateLocalLeaderElection(); + } else if (updateLocalPeerInfo()) { + persistNewServerConfiguration(changeVotingStatusContext); } + } + + private void initiateLocalLeaderElection() { + LOG.debug("{}: Sending local ElectionTimeout to start leader election", raftContext.getId()); + + ServerConfigurationPayload previousServerConfig = raftContext.getPeerServerInfo(true); + if (!updateLocalPeerInfo()) { + return; + } + + raftContext.getActor().tell(TimeoutNow.INSTANCE, raftContext.getActor()); + + currentOperationState = new WaitingForLeaderElected(changeVotingStatusContext, previousServerConfig); + } - newServerInfoList.add(new ServerInfo(raftContext.getId(), serverVotingStatusMap.containsKey( - raftContext.getId()) ? serverVotingStatusMap.get(raftContext.getId()) : raftContext.isVotingMember())); + private boolean updateLocalPeerInfo() { + final var newServerInfoList = newServerInfoList(); + + // Check if new voting state would leave us with no voting members. + boolean atLeastOneVoting = false; + for (ServerInfo info: newServerInfoList) { + if (info.isVoting()) { + atLeastOneVoting = true; + break; + } + } + + if (!atLeastOneVoting) { + operationComplete(changeVotingStatusContext, ServerChangeStatus.INVALID_REQUEST); + return false; + } raftContext.updatePeerIds(new ServerConfigurationPayload(newServerInfoList)); - AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior(); - leader.updateMinReplicaCount(); + if (raftActor.getCurrentBehavior() instanceof AbstractLeader leader) { + leader.updateMinReplicaCount(); + } + + return true; + } + + private ImmutableList newServerInfoList() { + final var serverVotingStatusMap = changeVotingStatusContext.getOperation().getServerVotingStatusMap(); + final var peerInfos = raftContext.getPeers(); + final var newServerInfoList = ImmutableList.builderWithExpectedSize(peerInfos.size() + 1); + for (var peerInfo : peerInfos) { + final var peerId = peerInfo.getId(); + final var voting = serverVotingStatusMap.get(peerId); + newServerInfoList.add(new ServerInfo(peerId, voting != null ? voting : peerInfo.isVoting())); + } + + final var myId = raftContext.getId(); + final var myVoting = serverVotingStatusMap.get(myId); + newServerInfoList.add(new ServerInfo(myId, myVoting != null ? myVoting : raftContext.isVotingMember())); + + return newServerInfoList.build(); + } + } + + private class WaitingForLeaderElected extends OperationState { + private final ServerConfigurationPayload previousServerConfig; + private final ChangeServersVotingStatusContext operationContext; + private final Cancellable timer; + + WaitingForLeaderElected(final ChangeServersVotingStatusContext operationContext, + final ServerConfigurationPayload previousServerConfig) { + this.operationContext = operationContext; + this.previousServerConfig = previousServerConfig; + + timer = newTimer(raftContext.getConfigParams().getElectionTimeOutInterval(), + new ServerOperationTimeout(operationContext.getLoggingContext())); + } + + @Override + void onNewLeader(final String newLeader) { + if (newLeader == null) { + return; + } + + 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(final 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(); + } - persistNewServerConfiguration(changeVotingStatusContext); + 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 loggingContext; - ServerOperationTimeout(String loggingContext){ - this.loggingContext = Preconditions.checkNotNull(loggingContext, "loggingContext should not be null"); + ServerOperationTimeout(final String loggingContext) { + this.loggingContext = requireNonNull(loggingContext, "loggingContext should not be null"); } String getLoggingContext() {