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=207642a7213e637fc8af6539ea9e989c0a75d43e;hp=ae23140114ed7d2ca8e73584ecd997fefd437723;hb=250f3f77c80284536cc32e96739f713d21844103;hpb=3fda1a923defdbf18849c6080c3aa19f1ebf2c5f 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 ae23140114..207642a721 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 @@ -10,20 +10,22 @@ package org.opendaylight.controller.cluster.raft; import akka.actor.ActorRef; import akka.actor.ActorSelection; import akka.actor.Cancellable; -import java.util.ArrayList; -import java.util.Collections; +import com.google.common.base.Preconditions; import java.util.LinkedList; -import java.util.List; import java.util.Queue; import java.util.UUID; import java.util.concurrent.TimeUnit; -import org.opendaylight.controller.cluster.raft.FollowerLogInformation.FollowerState; +import javax.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.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.FollowerCatchUpTimeout; +import org.opendaylight.controller.cluster.raft.messages.RemoveServer; +import org.opendaylight.controller.cluster.raft.messages.RemoveServerReply; 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; @@ -37,67 +39,66 @@ import scala.concurrent.duration.FiniteDuration; */ class RaftActorServerConfigurationSupport { private static final Logger LOG = LoggerFactory.getLogger(RaftActorServerConfigurationSupport.class); - private final RaftActorContext context; - // client follower queue - private final Queue followerInfoQueue = new LinkedList(); - // timeout handle - private Cancellable followerTimeout = null; + + private final OperationState IDLE = new Idle(); + + private final RaftActorContext raftContext; + + private final Queue> pendingOperationsQueue = new LinkedList<>(); + + private OperationState currentOperationState = IDLE; RaftActorServerConfigurationSupport(RaftActorContext context) { - this.context = context; + this.raftContext = context; } boolean handleMessage(Object message, RaftActor raftActor, ActorRef sender) { if(message instanceof AddServer) { - onAddServer((AddServer)message, raftActor, sender); + onAddServer((AddServer) message, raftActor, sender); + return true; + } else if(message instanceof RemoveServer) { + onRemoveServer((RemoveServer) message, raftActor, sender); return true; - } else if (message instanceof FollowerCatchUpTimeout){ - FollowerCatchUpTimeout followerTimeout = (FollowerCatchUpTimeout)message; - // abort follower catchup on timeout - onFollowerCatchupTimeout(raftActor, sender, followerTimeout.getNewServerId()); + } else if (message instanceof FollowerCatchUpTimeout) { + currentOperationState.onFollowerCatchupTimeout(raftActor, (FollowerCatchUpTimeout) message); return true; - } else if (message instanceof UnInitializedFollowerSnapshotReply){ - // snapshot installation is successful - onUnInitializedFollowerSnapshotReply((UnInitializedFollowerSnapshotReply)message, raftActor,sender); + } else if (message instanceof UnInitializedFollowerSnapshotReply) { + currentOperationState.onUnInitializedFollowerSnapshotReply(raftActor, + (UnInitializedFollowerSnapshotReply) message); return true; } else if(message instanceof ApplyState) { return onApplyState((ApplyState) message, raftActor); + } else if(message instanceof SnapshotComplete) { + currentOperationState.onSnapshotComplete(raftActor); + return false; } else { return false; } } + private void onRemoveServer(RemoveServer removeServer, RaftActor raftActor, ActorRef sender) { + if(removeServer.getServerId().equals(raftActor.getLeaderId())){ + // Removing current leader is not supported yet + // TODO: To properly support current leader removal we need to first implement transfer of leadership + LOG.debug("Cannot remove {} replica because it is the Leader", removeServer.getServerId()); + sender.tell(new RemoveServerReply(ServerChangeStatus.NOT_SUPPORTED, raftActor.getLeaderId()), raftActor.getSelf()); + } else if(!raftContext.getPeerIds().contains(removeServer.getServerId())) { + sender.tell(new RemoveServerReply(ServerChangeStatus.DOES_NOT_EXIST, raftActor.getLeaderId()), raftActor.getSelf()); + } else { + onNewOperation(raftActor, new RemoveServerContext(removeServer, raftContext.getPeerAddress(removeServer.getServerId()), sender)); + } + } + private boolean onApplyState(ApplyState applyState, RaftActor raftActor) { Payload data = applyState.getReplicatedLogEntry().getData(); if(data instanceof ServerConfigurationPayload) { - CatchupFollowerInfo followerInfo = followerInfoQueue.peek(); - if(followerInfo != null && followerInfo.getContextId().equals(applyState.getIdentifier())) { - LOG.info("{} has been successfully replicated to a majority of followers", data); - - // respond ok to follower - respondToClient(raftActor, ServerChangeStatus.OK); - } - + currentOperationState.onApplyState(raftActor, applyState); return true; } return false; } - private void onAddServer(AddServer addServer, RaftActor raftActor, ActorRef sender) { - LOG.debug("{}: onAddServer: {}", context.getId(), addServer); - if(noLeaderOrForwardedToLeader(addServer, raftActor, sender)) { - return; - } - - CatchupFollowerInfo followerInfo = new CatchupFollowerInfo(addServer,sender); - boolean process = followerInfoQueue.isEmpty(); - followerInfoQueue.add(followerInfo); - if(process) { - processAddServer(raftActor); - } - } - /** * The algorithm for AddServer is as follows: *
    @@ -119,119 +120,354 @@ class RaftActorServerConfigurationSupport { *
  • Respond to caller with TIMEOUT.
  • *
*/ - private void processAddServer(RaftActor raftActor){ - LOG.debug("{}: In processAddServer", context.getId()); - - AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior(); - CatchupFollowerInfo followerInfo = followerInfoQueue.peek(); - AddServer addSrv = followerInfo.getAddServer(); - context.addToPeers(addSrv.getNewServerId(), addSrv.getNewServerAddress()); - - // if voting member - initialize to VOTING_NOT_INITIALIZED - FollowerState initialState = addSrv.isVotingMember() ? FollowerState.VOTING_NOT_INITIALIZED : - FollowerState.NON_VOTING; - leader.addFollower(addSrv.getNewServerId(), initialState); - - if(initialState == FollowerState.VOTING_NOT_INITIALIZED){ - LOG.debug("Leader sending initiate capture snapshot to follower : {}", addSrv.getNewServerId()); - leader.initiateCaptureSnapshot(addSrv.getNewServerId()); - // schedule the catchup timeout timer - followerTimeout = context.getActorSystem().scheduler() - .scheduleOnce(new FiniteDuration(((context.getConfigParams().getElectionTimeOutInterval().toMillis()) * 2), - TimeUnit.MILLISECONDS), - context.getActor(), new FollowerCatchUpTimeout(addSrv.getNewServerId()), - context.getActorSystem().dispatcher(), context.getActor()); + private void onAddServer(AddServer addServer, RaftActor raftActor, ActorRef sender) { + LOG.debug("{}: onAddServer: {}", raftContext.getId(), addServer); + + onNewOperation(raftActor, new AddServerContext(addServer, sender)); + } + + private void onNewOperation(RaftActor raftActor, ServerOperationContext operationContext) { + if (raftActor.isLeader()) { + currentOperationState.onNewOperation(raftActor, operationContext); } else { - LOG.debug("Directly persisting the new server configuration : {}", addSrv.getNewServerId()); - persistNewServerConfiguration(raftActor, followerInfo); + ActorSelection leader = raftActor.getLeader(); + if (leader != null) { + LOG.debug("{}: Not leader - forwarding to leader {}", raftContext.getId(), leader); + leader.forward(operationContext.getOperation(), raftActor.getContext()); + } else { + LOG.debug("{}: No leader - returning NO_LEADER reply", raftContext.getId()); + operationContext.getClientRequestor().tell(operationContext.newReply( + ServerChangeStatus.NO_LEADER, null), raftActor.self()); + } } } - private boolean noLeaderOrForwardedToLeader(Object message, RaftActor raftActor, ActorRef sender) { - if (raftActor.isLeader()) { - return false; + /** + * Interface for a server operation FSM state. + */ + private interface OperationState { + void onNewOperation(RaftActor raftActor, ServerOperationContext operationContext); + + void onFollowerCatchupTimeout(RaftActor raftActor, FollowerCatchUpTimeout followerTimeout); + + void onUnInitializedFollowerSnapshotReply(RaftActor raftActor, UnInitializedFollowerSnapshotReply reply); + + void onApplyState(RaftActor raftActor, ApplyState applyState); + + void onSnapshotComplete(RaftActor raftActor); + } + + /** + * Interface for the initial state for a server operation. + */ + private interface InitialOperationState { + void initiate(RaftActor raftActor); + } + + /** + * Abstract base class for server operation FSM state. Handles common behavior for all states. + */ + private abstract class AbstractOperationState implements OperationState { + @Override + public void onNewOperation(RaftActor raftActor, 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(), + operationContext.getOperation()); + + pendingOperationsQueue.add(operationContext); } - ActorSelection leader = raftActor.getLeader(); - if (leader != null) { - LOG.debug("Not leader - forwarding to leader {}", leader); - leader.forward(message, raftActor.getContext()); - } else { - LOG.debug("No leader - returning NO_LEADER AddServerReply"); - sender.tell(new AddServerReply(ServerChangeStatus.NO_LEADER, null), raftActor.self()); + @Override + public void onFollowerCatchupTimeout(RaftActor raftActor, FollowerCatchUpTimeout followerTimeout) { + LOG.debug("onFollowerCatchupTimeout should not be called in state {}", this); + } + + @Override + public void onUnInitializedFollowerSnapshotReply(RaftActor raftActor, UnInitializedFollowerSnapshotReply reply) { + LOG.debug("onUnInitializedFollowerSnapshotReply was called in state {}", this); + } + + @Override + public void onApplyState(RaftActor raftActor, ApplyState applyState) { + LOG.debug("onApplyState was called in state {}", this); + } + + @Override + public void onSnapshotComplete(RaftActor raftActor) { + } + + protected void persistNewServerConfiguration(RaftActor raftActor, ServerOperationContext operationContext){ + raftContext.setDynamicServerConfigurationInUse(); + ServerConfigurationPayload payload = raftContext.getPeerServerInfo(); + LOG.debug("{}: New server configuration : {}", raftContext.getId(), payload.getServerConfig()); + + raftActor.persistData(operationContext.getClientRequestor(), operationContext.getContextId(), payload); + + currentOperationState = new Persisting(operationContext); + + sendReply(raftActor, operationContext, ServerChangeStatus.OK); } - return true; + protected void operationComplete(RaftActor raftActor, ServerOperationContext operationContext, + @Nullable ServerChangeStatus replyStatus) { + if(replyStatus != null) { + sendReply(raftActor, operationContext, replyStatus); + } + + operationContext.operationComplete(raftActor, replyStatus); + + currentOperationState = IDLE; + + ServerOperationContext nextOperation = pendingOperationsQueue.poll(); + if(nextOperation != null) { + RaftActorServerConfigurationSupport.this.onNewOperation(raftActor, nextOperation); + } + } + + private void sendReply(RaftActor raftActor, ServerOperationContext operationContext, + ServerChangeStatus status) { + LOG.debug("{}: Returning {} for operation {}", raftContext.getId(), status, operationContext.getOperation()); + + operationContext.getClientRequestor().tell(operationContext.newReply(status, raftActor.getLeaderId()), + raftActor.self()); + } + + @Override + public String toString() { + return getClass().getSimpleName(); + } } - private void onUnInitializedFollowerSnapshotReply(UnInitializedFollowerSnapshotReply reply, - RaftActor raftActor, ActorRef sender){ - CatchupFollowerInfo followerInfo = followerInfoQueue.peek(); - // Sanity check - it's possible we get a reply after it timed out. - if(followerInfo == null) { - return; + /** + * The state when no server operation is in progress. It immediately initiates new server operations. + */ + private class Idle extends AbstractOperationState { + @Override + public void onNewOperation(RaftActor raftActor, ServerOperationContext operationContext) { + operationContext.newInitialOperationState(RaftActorServerConfigurationSupport.this).initiate(raftActor); } - String followerId = reply.getFollowerId(); - AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior(); - FollowerLogInformation followerLogInformation = leader.getFollower(followerId); - stopFollowerTimer(); - followerLogInformation.setFollowerState(FollowerState.VOTING); - leader.updateMinReplicaCountAndMinIsolatedLeaderPeerCount(); + @Override + public void onApplyState(RaftActor raftActor, ApplyState applyState) { + // Noop - we override b/c ApplyState is called normally for followers in the idle state. + } + } + + /** + * The state when a new server configuration is being persisted and replicated. + */ + private class Persisting extends AbstractOperationState { + private final ServerOperationContext operationContext; - persistNewServerConfiguration(raftActor, followerInfo); + Persisting(ServerOperationContext operationContext) { + this.operationContext = operationContext; + } + + @Override + public void onApplyState(RaftActor raftActor, 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(), + applyState.getReplicatedLogEntry().getData()); + + operationComplete(raftActor, operationContext, null); + } + } } - private void persistNewServerConfiguration(RaftActor raftActor, CatchupFollowerInfo followerInfo){ - List cNew = new ArrayList(context.getPeerAddresses().keySet()); - cNew.add(context.getId()); + /** + * Abstract base class for an AddServer operation state. + */ + private abstract class AddServerState extends AbstractOperationState { + private final AddServerContext addServerContext; + + AddServerState(AddServerContext addServerContext) { + this.addServerContext = addServerContext; + } + + AddServerContext getAddServerContext() { + return addServerContext; + } - LOG.debug("New server configuration : {}", cNew.toString()); + Cancellable newInstallSnapshotTimer(RaftActor raftActor) { + return raftContext.getActorSystem().scheduler().scheduleOnce( + new FiniteDuration(((raftContext.getConfigParams().getElectionTimeOutInterval().toMillis()) * 2), + TimeUnit.MILLISECONDS), raftContext.getActor(), + new FollowerCatchUpTimeout(addServerContext.getOperation().getNewServerId()), + raftContext.getActorSystem().dispatcher(), raftContext.getActor()); + } - ServerConfigurationPayload servPayload = new ServerConfigurationPayload(cNew, Collections.emptyList()); + void handleOnFollowerCatchupTimeout(RaftActor raftActor, FollowerCatchUpTimeout followerTimeout) { + String serverId = followerTimeout.getNewServerId(); - raftActor.persistData(followerInfo.getClientRequestor(), followerInfo.getContextId(), servPayload); - } + LOG.debug("{}: onFollowerCatchupTimeout for new server {}", raftContext.getId(), serverId); - private void stopFollowerTimer() { - if (followerTimeout != null && !followerTimeout.isCancelled()) { - followerTimeout.cancel(); + // cleanup + raftContext.removePeer(serverId); + + boolean isLeader = raftActor.isLeader(); + if(isLeader) { + AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior(); + leader.removeFollower(serverId); + } + + operationComplete(raftActor, getAddServerContext(), + isLeader ? ServerChangeStatus.TIMEOUT : ServerChangeStatus.NO_LEADER); } - } - private void onFollowerCatchupTimeout(RaftActor raftActor, ActorRef sender, String serverId){ - LOG.debug("onFollowerCatchupTimeout: {}", serverId); - AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior(); - // cleanup - context.removePeer(serverId); - leader.removeFollower(serverId); - LOG.warn("Timeout occured for new server {} while installing snapshot", serverId); - respondToClient(raftActor,ServerChangeStatus.TIMEOUT); - } + } - private void respondToClient(RaftActor raftActor, ServerChangeStatus result){ - // remove the entry from the queue - CatchupFollowerInfo fInfo = followerInfoQueue.remove(); + /** + * 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 { + InitialAddServerState(AddServerContext addServerContext) { + super(addServerContext); + } + + @Override + public void initiate(RaftActor raftActor) { + AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior(); + AddServer addServer = getAddServerContext().getOperation(); + + LOG.debug("{}: Initiating {}", raftContext.getId(), addServer); + + if(raftContext.getPeerInfo(addServer.getNewServerId()) != null) { + operationComplete(raftActor, getAddServerContext(), ServerChangeStatus.ALREADY_EXISTS); + return; + } - // get the sender - ActorRef toClient = fInfo.getClientRequestor(); + VotingState votingState = addServer.isVotingMember() ? VotingState.VOTING_NOT_INITIALIZED : + VotingState.NON_VOTING; + raftContext.addToPeers(addServer.getNewServerId(), addServer.getNewServerAddress(), votingState); - toClient.tell(new AddServerReply(result, raftActor.getLeaderId()), raftActor.self()); - LOG.debug("Response returned is {} for server {} ", result, fInfo.getAddServer().getNewServerId()); - if(!followerInfoQueue.isEmpty()){ - processAddServer(raftActor); + leader.addFollower(addServer.getNewServerId()); + + if(votingState == VotingState.VOTING_NOT_INITIALIZED){ + // schedule the install snapshot timeout timer + Cancellable installSnapshotTimer = newInstallSnapshotTimer(raftActor); + if(leader.initiateCaptureSnapshot(addServer.getNewServerId())) { + LOG.debug("{}: Initiating capture snapshot for new server {}", raftContext.getId(), + addServer.getNewServerId()); + + currentOperationState = new InstallingSnapshot(getAddServerContext(), installSnapshotTimer); + } else { + LOG.debug("{}: Snapshot already in progress - waiting for completion", raftContext.getId()); + + currentOperationState = new WaitingForPriorSnapshotComplete(getAddServerContext(), + installSnapshotTimer); + } + } else { + LOG.debug("{}: New follower is non-voting - directly persisting new server configuration", + raftContext.getId()); + + persistNewServerConfiguration(raftActor, getAddServerContext()); + } } - } + } + + /** + * 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 Cancellable installSnapshotTimer; + + InstallingSnapshot(AddServerContext addServerContext, Cancellable installSnapshotTimer) { + super(addServerContext); + this.installSnapshotTimer = Preconditions.checkNotNull(installSnapshotTimer); + } + + @Override + public void onFollowerCatchupTimeout(RaftActor raftActor, FollowerCatchUpTimeout followerTimeout) { + handleOnFollowerCatchupTimeout(raftActor, followerTimeout); + + LOG.warn("{}: Timeout occured for new server {} while installing snapshot", raftContext.getId(), + followerTimeout.getNewServerId()); + } + + @Override + public void onUnInitializedFollowerSnapshotReply(RaftActor raftActor, 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()) { + AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior(); + raftContext.getPeerInfo(followerId).setVotingState(VotingState.VOTING); + leader.updateMinReplicaCount(); + + persistNewServerConfiguration(raftActor, getAddServerContext()); + + installSnapshotTimer.cancel(); + } else { + LOG.debug("{}: Dropping UnInitializedFollowerSnapshotReply for server {}: {}", + raftContext.getId(), followerId, + !raftActor.isLeader() ? "not leader" : "server Id doesn't match"); + } + } + } + + /** + * 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 Cancellable snapshotTimer; + + WaitingForPriorSnapshotComplete(AddServerContext addServerContext, Cancellable snapshotTimer) { + super(addServerContext); + this.snapshotTimer = Preconditions.checkNotNull(snapshotTimer); + } + + @Override + public void onSnapshotComplete(RaftActor raftActor) { + LOG.debug("{}: onSnapshotComplete", raftContext.getId()); + + if(!raftActor.isLeader()) { + LOG.debug("{}: No longer the leader", raftContext.getId()); + return; + } + + AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior(); + if(leader.initiateCaptureSnapshot(getAddServerContext().getOperation().getNewServerId())) { + LOG.debug("{}: Initiating capture snapshot for new server {}", raftContext.getId(), + getAddServerContext().getOperation().getNewServerId()); + + currentOperationState = new InstallingSnapshot(getAddServerContext(), + newInstallSnapshotTimer(raftActor)); + + snapshotTimer.cancel(); + } + } + + @Override + public void onFollowerCatchupTimeout(RaftActor raftActor, FollowerCatchUpTimeout followerTimeout) { + handleOnFollowerCatchupTimeout(raftActor, followerTimeout); + + LOG.warn("{}: Timeout occured for new server {} while waiting for prior snapshot to complete", + raftContext.getId(), followerTimeout.getNewServerId()); + } + } - // maintain sender actorRef - private static class CatchupFollowerInfo { - private final AddServer addServer; + /** + * Stores context information for a server operation. + * + * @param the operation type + */ + private static abstract class ServerOperationContext { + private final T operation; private final ActorRef clientRequestor; private final String contextId; - CatchupFollowerInfo(AddServer addSrv, ActorRef cliReq){ - addServer = addSrv; - clientRequestor = cliReq; + ServerOperationContext(T operation, ActorRef clientRequestor){ + this.operation = operation; + this.clientRequestor = clientRequestor; contextId = UUID.randomUUID().toString(); } @@ -239,12 +475,97 @@ class RaftActorServerConfigurationSupport { return contextId; } - AddServer getAddServer(){ - return addServer; + T getOperation() { + return operation; } - ActorRef getClientRequestor(){ + ActorRef getClientRequestor() { return clientRequestor; } + + abstract Object newReply(ServerChangeStatus status, String leaderId); + + abstract InitialOperationState newInitialOperationState(RaftActorServerConfigurationSupport support); + + abstract void operationComplete(RaftActor raftActor, ServerChangeStatus serverChangeStatus); + } + + /** + * Stores context information for an AddServer operation. + */ + private static class AddServerContext extends ServerOperationContext { + AddServerContext(AddServer addServer, ActorRef clientRequestor) { + super(addServer, clientRequestor); + } + + @Override + Object newReply(ServerChangeStatus status, String leaderId) { + return new AddServerReply(status, leaderId); + } + + @Override + InitialOperationState newInitialOperationState(RaftActorServerConfigurationSupport support) { + return support.new InitialAddServerState(this); + } + + @Override + void operationComplete(RaftActor raftActor, ServerChangeStatus serverChangeStatus) { + + } + } + + private abstract class RemoveServerState extends AbstractOperationState { + private final RemoveServerContext removeServerContext; + + + protected RemoveServerState(RemoveServerContext removeServerContext) { + this.removeServerContext = Preconditions.checkNotNull(removeServerContext); + + } + + public RemoveServerContext getRemoveServerContext() { + return removeServerContext; + } + + } + + private class InitialRemoveServerState extends RemoveServerState implements InitialOperationState{ + + protected InitialRemoveServerState(RemoveServerContext removeServerContext) { + super(removeServerContext); + } + + @Override + public void initiate(RaftActor raftActor) { + raftContext.removePeer(getRemoveServerContext().getOperation().getServerId()); + persistNewServerConfiguration(raftActor, getRemoveServerContext()); + } + } + + private static class RemoveServerContext extends ServerOperationContext { + private final String peerAddress; + + RemoveServerContext(RemoveServer operation, String peerAddress, ActorRef clientRequestor) { + super(operation, clientRequestor); + this.peerAddress = peerAddress; + } + + @Override + Object newReply(ServerChangeStatus status, String leaderId) { + return new RemoveServerReply(status, leaderId); + } + + @Override + InitialOperationState newInitialOperationState(RaftActorServerConfigurationSupport support) { + return support.new InitialRemoveServerState(this); + } + + @Override + void operationComplete(RaftActor raftActor, ServerChangeStatus serverChangeStatus) { + if(peerAddress != null) { + raftActor.context().actorSelection(peerAddress).tell(new ServerRemoved(getOperation().getServerId()), raftActor.getSelf()); + } + } + } }