Add RaftActorServerConfigurationSupport.raftActor
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / RaftActorServerConfigurationSupport.java
index 0c34158ca33c05dd751a66cb8eda8b4f252596cb..f969e3d67d46b53920e2b41b5fa3078fe5efd5f1 100644 (file)
@@ -7,27 +7,27 @@
  */
 package org.opendaylight.controller.cluster.raft;
 
-import java.util.ArrayList;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Queue;
-import java.util.concurrent.TimeUnit;
 import akka.actor.ActorRef;
 import akka.actor.ActorSelection;
 import akka.actor.Cancellable;
-import org.opendaylight.controller.cluster.raft.FollowerLogInformation.FollowerState;
-import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
+import com.google.common.base.Preconditions;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.UUID;
+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.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;
 import org.slf4j.LoggerFactory;
-import org.opendaylight.controller.cluster.raft.messages.FollowerCatchUpTimeout;
-import org.opendaylight.controller.cluster.raft.messages.UnInitializedFollowerSnapshotReply;
-import scala.concurrent.duration.FiniteDuration;
 
 /**
  * Handles server configuration related messages for a RaftActor.
@@ -36,200 +36,585 @@ 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<CatchupFollowerInfo> followerInfoQueue = new LinkedList<CatchupFollowerInfo>();
-    // timeout handle
-    private Cancellable followerTimeout = null;
 
-    RaftActorServerConfigurationSupport(RaftActorContext context) {
-        this.context = context;
+    private final OperationState IDLE = new Idle();
+
+    private final RaftActor raftActor;
+
+    private final RaftActorContext raftContext;
+
+    private final Queue<ServerOperationContext<?>> pendingOperationsQueue = new LinkedList<>();
+
+    private OperationState currentOperationState = IDLE;
+
+    RaftActorServerConfigurationSupport(RaftActor raftActor) {
+        this.raftActor = raftActor;
+        this.raftContext = raftActor.getRaftActorContext();
     }
 
-    boolean handleMessage(Object message, RaftActor raftActor, ActorRef sender) {
+    boolean handleMessage(Object message, ActorRef sender) {
         if(message instanceof AddServer) {
-            onAddServer((AddServer)message, raftActor, sender);
+            onAddServer((AddServer) message, 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 RemoveServer) {
+            onRemoveServer((RemoveServer) message, sender);
             return true;
-        } else if (message instanceof UnInitializedFollowerSnapshotReply){
-            // snapshot installation is successful
-            onUnInitializedFollowerSnapshotReply((UnInitializedFollowerSnapshotReply)message, raftActor,sender);
+        } else if (message instanceof ServerOperationTimeout) {
+            currentOperationState.onServerOperationTimeout((ServerOperationTimeout) message);
             return true;
-        } else if(message instanceof ApplyState){
-            ApplyState applyState = (ApplyState) message;
-            Payload data = applyState.getReplicatedLogEntry().getData();
-            if( data instanceof ServerConfigurationPayload){
-                 LOG.info("Server configuration : {} has been replicated to a majority of cluster servers succesfully",
-                                                                                    (ServerConfigurationPayload)data);
-                 // respond ok to follower
-                 respondToClient(raftActor, ServerChangeStatus.OK);
-                 return true;
-            }
+        } else if (message instanceof UnInitializedFollowerSnapshotReply) {
+            currentOperationState.onUnInitializedFollowerSnapshotReply((UnInitializedFollowerSnapshotReply) message);
+            return true;
+        } else if(message instanceof ApplyState) {
+            return onApplyState((ApplyState) message);
+        } else if(message instanceof SnapshotComplete) {
+            currentOperationState.onSnapshotComplete();
             return false;
         } else {
             return false;
         }
     }
 
-    private void onAddServer(AddServer addServer, RaftActor raftActor, ActorRef sender) {
-        LOG.debug("onAddServer: {}", addServer);
-        if(noLeaderOrForwardedToLeader(addServer, raftActor, sender)) {
-            return;
-        }
-
-        CatchupFollowerInfo followerInfo = new CatchupFollowerInfo(addServer,sender);
-        boolean process = !followerInfoQueue.isEmpty();
-        followerInfoQueue.add(followerInfo);
-        if(process) {
-            processAddServer(raftActor);
-        }
-    }
-
-    private void processAddServer(RaftActor raftActor){
-        LOG.debug("In processAddServer");
-        AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior();
-        AddServer addSrv = followerInfoQueue.peek().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);
-
-        // TODO
-        // if initialState == FollowerState.VOTING_NOT_INITIALIZED
-        //     Initiate snapshot via leader.initiateCaptureSnapshot(addServer.getNewServerId())
-        //     Start a timer to abort the operation after a period of time (maybe 2 times election timeout)
-        //     Set local instance state and wait for message from the AbstractLeader when install snapshot
-        //     is done and return now
-        //     When install snapshot message is received, go to step 1
-        // else
-        //     go to step 2
-        //
-        // 1) tell AbstractLeader mark the follower as VOTING and recalculate minReplicationCount and
-        //        minIsolatedLeaderPeerCount
-        // 2) persist and replicate ServerConfigurationPayload via
-        //           raftActor.persistData(sender, uuid, newServerConfigurationPayload)
-        // 3) Wait for commit complete via ApplyState message in RaftActor or time it out. In RaftActor,
-        //       on ApplyState, check if ReplicatedLogEntry payload is ServerConfigurationPayload and call
-        //       this class.
-        //
-        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 onRemoveServer(RemoveServer removeServer, ActorRef sender) {
+        LOG.debug("{}: onRemoveServer: {}, state: {}", raftContext.getId(), removeServer, currentOperationState);
+        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 {
-            LOG.debug("Directly persisting  the new server configuration : {}", addSrv.getNewServerId());
-            persistNewServerConfiguration(raftActor, followerInfoQueue.peek().getClientRequestor(),
-                                                                                 addSrv.getNewServerId());
+            onNewOperation(new RemoveServerContext(removeServer, raftContext.getPeerAddress(removeServer.getServerId()), sender));
         }
     }
 
-    private boolean noLeaderOrForwardedToLeader(Object message, RaftActor raftActor, ActorRef sender) {
-        if (raftActor.isLeader()) {
-            return false;
+    private boolean onApplyState(ApplyState applyState) {
+        Payload data = applyState.getReplicatedLogEntry().getData();
+        if(data instanceof ServerConfigurationPayload) {
+            currentOperationState.onApplyState(applyState);
+            return true;
         }
 
-        ActorSelection leader = raftActor.getLeader();
-        if (leader != null) {
-            LOG.debug("Not leader - forwarding to leader {}", leader);
-            leader.forward(message, raftActor.getContext());
+        return false;
+    }
+
+    /**
+     * The algorithm for AddServer is as follows:
+     * <ul>
+     * <li>Add the new server as a peer.</li>
+     * <li>Add the new follower to the leader.</li>
+     * <li>If new server should be voting member</li>
+     * <ul>
+     *     <li>Initialize FollowerState to VOTING_NOT_INITIALIZED.</li>
+     *     <li>Initiate install snapshot to the new follower.</li>
+     *     <li>When install snapshot complete, mark the follower as VOTING and re-calculate majority vote count.</li>
+     * </ul>
+     * <li>Persist and replicate ServerConfigurationPayload with the new server list.</li>
+     * <li>On replication consensus, respond to caller with OK.</li>
+     * </ul>
+     * If the install snapshot times out after a period of 2 * election time out
+     * <ul>
+     *     <li>Remove the new server as a peer.</li>
+     *     <li>Remove the new follower from the leader.</li>
+     *     <li>Respond to caller with TIMEOUT.</li>
+     * </ul>
+     */
+    private void onAddServer(AddServer addServer, ActorRef sender) {
+        LOG.debug("{}: onAddServer: {}, state: {}", raftContext.getId(), addServer, currentOperationState);
+
+        onNewOperation(new AddServerContext(addServer, sender));
+    }
+
+    private void onNewOperation(ServerOperationContext<?> operationContext) {
+        if (raftActor.isLeader()) {
+            currentOperationState.onNewOperation(operationContext);
         } else {
-            LOG.debug("No leader - returning NO_LEADER AddServerReply");
-            sender.tell(new AddServerReply(ServerChangeStatus.NO_LEADER, null), raftActor.self());
+            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());
+            }
         }
+    }
+
+    /**
+     * 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();
+    }
 
-        return true;
+    /**
+     * Interface for the initial state for a server operation.
+     */
+    private interface InitialOperationState {
+        void initiate();
     }
 
-    private void onUnInitializedFollowerSnapshotReply(UnInitializedFollowerSnapshotReply reply,
-                                                 RaftActor raftActor, ActorRef sender){
+    /**
+     * 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) {
+            // 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);
+        }
+
+        @Override
+        public void onServerOperationTimeout(ServerOperationTimeout timeout) {
+            LOG.debug("onServerOperationTimeout should not be called in state {}", this);
+        }
+
+        @Override
+        public void onUnInitializedFollowerSnapshotReply(UnInitializedFollowerSnapshotReply reply) {
+            LOG.debug("onUnInitializedFollowerSnapshotReply was called in state {}", this);
+        }
+
+        @Override
+        public void onApplyState(ApplyState applyState) {
+            LOG.debug("onApplyState was called in state {}", this);
+        }
+
+        @Override
+        public void onSnapshotComplete() {
+        }
+
+        protected void persistNewServerConfiguration(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, newTimer(new ServerOperationTimeout(operationContext.getServerId())));
 
-        String followerId = reply.getFollowerId();
-        AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior();
-        FollowerLogInformation followerLogInformation = leader.getFollower(followerId);
-        stopFollowerTimer();
-        followerLogInformation.setFollowerState(FollowerState.VOTING);
-        leader.updateMinReplicaCountAndMinIsolatedLeaderPeerCount();
-        persistNewServerConfiguration(raftActor, sender, followerId);
+            sendReply(operationContext, ServerChangeStatus.OK);
+        }
+
+        protected void operationComplete(ServerOperationContext<?> operationContext, @Nullable ServerChangeStatus replyStatus) {
+            if(replyStatus != null) {
+                sendReply(operationContext, replyStatus);
+            }
+
+            operationContext.operationComplete(raftActor, replyStatus);
+
+            currentOperationState = IDLE;
+
+            ServerOperationContext<?> nextOperation = pendingOperationsQueue.poll();
+            if(nextOperation != null) {
+                RaftActorServerConfigurationSupport.this.onNewOperation(nextOperation);
+            }
+        }
+
+        protected void sendReply(ServerOperationContext<?> operationContext, 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) {
+            return raftContext.getActorSystem().scheduler().scheduleOnce(
+                    raftContext.getConfigParams().getElectionTimeOutInterval().$times(2), raftContext.getActor(), message,
+                            raftContext.getActorSystem().dispatcher(), raftContext.getActor());
+        }
+
+        @Override
+        public String toString() {
+            return getClass().getSimpleName();
+        }
+    }
+
+    /**
+     * The state when no server operation is in progress. It immediately initiates new server operations.
+     */
+    private class Idle extends AbstractOperationState {
+        @Override
+        public void onNewOperation(ServerOperationContext<?> operationContext) {
+            operationContext.newInitialOperationState(RaftActorServerConfigurationSupport.this).initiate();
+        }
+
+        @Override
+        public void onApplyState(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;
+        private final Cancellable timer;
+        private boolean timedOut = false;
+
+        Persisting(ServerOperationContext<?> operationContext, Cancellable timer) {
+            this.operationContext = operationContext;
+            this.timer = timer;
+        }
+
+        @Override
+        public void onApplyState(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());
+
+                timer.cancel();
+                operationComplete(operationContext, null);
+            }
+        }
+
+        @Override
+        public void onServerOperationTimeout(ServerOperationTimeout timeout) {
+            LOG.warn("{}: Timeout occured while replicating the new server configuration for {}", raftContext.getId(),
+                    timeout.getServerId());
+
+            timedOut = true;
+
+            // Fail any pending operations
+            ServerOperationContext<?> nextOperation = pendingOperationsQueue.poll();
+            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);
+            } else {
+                super.onNewOperation(operationContext);
+            }
+        }
     }
 
-    private void persistNewServerConfiguration(RaftActor raftActor, ActorRef sender, String followerId){
-        /* get old server configuration list */
-        Map<String, String> tempMap =  context.getPeerAddresses();
-        List<String> cOld = new ArrayList<String>();
-        for (Map.Entry<String, String> entry : tempMap.entrySet()) {
-            if(!entry.getKey().equals(followerId)){
-                cOld.add(entry.getKey());
+    /**
+     * 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;
+        }
+
+        Cancellable newInstallSnapshotTimer() {
+            return newTimer(new ServerOperationTimeout(addServerContext.getOperation().getNewServerId()));
+        }
+
+        void handleInstallSnapshotTimeout(ServerOperationTimeout timeout) {
+            String serverId = timeout.getServerId();
+
+            LOG.debug("{}: handleInstallSnapshotTimeout for new server {}", raftContext.getId(), serverId);
+
+            // cleanup
+            raftContext.removePeer(serverId);
+
+            boolean isLeader = raftActor.isLeader();
+            if(isLeader) {
+                AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior();
+                leader.removeFollower(serverId);
             }
+
+            operationComplete(getAddServerContext(), isLeader ? ServerChangeStatus.TIMEOUT : ServerChangeStatus.NO_LEADER);
+        }
+
+    }
+
+    /**
+     * 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);
         }
-        LOG.debug("Cold server configuration : {}",  cOld.toString());
-        /* get new server configuration list */
-        List <String> cNew = new ArrayList<String>(cOld);
-        cNew.add(followerId);
-        LOG.debug("Cnew server configuration : {}",  cNew.toString());
-        // construct the peer list
-        ServerConfigurationPayload servPayload = new ServerConfigurationPayload(cOld, cNew);
-        /* TODO - persist new configuration - CHECK WHETHER USING getId below is correct */
-        raftActor.persistData(sender, context.getId(), servPayload);
-   }
-
-   private void stopFollowerTimer() {
-        if (followerTimeout != null && !followerTimeout.isCancelled()) {
-            followerTimeout.cancel();
-        }
-   }
-
-   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("onFollowerCatchupTimeout - Timeout occured for server - {} while installing snapshot", serverId);
-        respondToClient(raftActor,ServerChangeStatus.TIMEOUT);
-   }
-
-   private void respondToClient(RaftActor raftActor, ServerChangeStatus result){
-
-        int size = followerInfoQueue.size();
-
-        // remove the entry from the queue
-        CatchupFollowerInfo fInfo = followerInfoQueue.remove();
-        // get the sender
-        ActorRef toClient = fInfo.getClientRequestor();
-
-        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);
-        }
-   }
-
-    // mantain sender actorRef
-    private class CatchupFollowerInfo {
-        private final AddServer addServer;
+
+        @Override
+        public void initiate() {
+            AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior();
+            AddServer addServer = getAddServerContext().getOperation();
+
+            LOG.debug("{}: Initiating {}", raftContext.getId(), addServer);
+
+            if(raftContext.getPeerInfo(addServer.getNewServerId()) != null) {
+                operationComplete(getAddServerContext(), ServerChangeStatus.ALREADY_EXISTS);
+                return;
+            }
+
+            VotingState votingState = addServer.isVotingMember() ? VotingState.VOTING_NOT_INITIALIZED :
+                    VotingState.NON_VOTING;
+            raftContext.addToPeers(addServer.getNewServerId(), addServer.getNewServerAddress(), votingState);
+
+            leader.addFollower(addServer.getNewServerId());
+
+            if(votingState == VotingState.VOTING_NOT_INITIALIZED){
+                // schedule the install snapshot timeout timer
+                Cancellable installSnapshotTimer = newInstallSnapshotTimer();
+                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(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 onServerOperationTimeout(ServerOperationTimeout timeout) {
+            handleInstallSnapshotTimeout(timeout);
+
+            LOG.warn("{}: Timeout occured for new server {} while installing snapshot", raftContext.getId(),
+                    timeout.getServerId());
+        }
+
+        @Override
+        public void onUnInitializedFollowerSnapshotReply(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(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() {
+            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());
+
+                snapshotTimer.cancel();
+            }
+        }
+
+        @Override
+        public void onServerOperationTimeout(ServerOperationTimeout timeout) {
+            handleInstallSnapshotTimeout(timeout);
+
+            LOG.warn("{}: Timeout occured for new server {} while waiting for prior snapshot to complete",
+                    raftContext.getId(), timeout.getServerId());
+        }
+    }
+
+    /**
+     * Stores context information for a server operation.
+     *
+     * @param <T> the operation type
+     */
+    private static abstract class ServerOperationContext<T> {
+        private final T operation;
         private final ActorRef clientRequestor;
+        private final String contextId;
+
+        ServerOperationContext(T operation, ActorRef clientRequestor){
+            this.operation = operation;
+            this.clientRequestor = clientRequestor;
+            contextId = UUID.randomUUID().toString();
+        }
 
-        CatchupFollowerInfo(AddServer addSrv, ActorRef cliReq){
-            addServer = addSrv;
-            clientRequestor = cliReq;
+        String getContextId() {
+            return contextId;
         }
-        public AddServer getAddServer(){
-            return addServer;
+
+        T getOperation() {
+            return operation;
         }
-        public 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);
+
+        abstract String getServerId();
+    }
+
+    /**
+     * Stores context information for an AddServer operation.
+     */
+    private static class AddServerContext extends ServerOperationContext<AddServer> {
+        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) {
+
+        }
+
+        @Override
+        String getServerId() {
+            return getOperation().getNewServerId();
+        }
+    }
+
+    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() {
+            raftContext.removePeer(getRemoveServerContext().getOperation().getServerId());
+            persistNewServerConfiguration(getRemoveServerContext());
+        }
+    }
+
+    private static class RemoveServerContext extends ServerOperationContext<RemoveServer> {
+        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());
+            }
+        }
+
+        @Override
+        String getServerId() {
+            return getOperation().getServerId();
+        }
+    }
+
+    static class ServerOperationTimeout {
+        private final String serverId;
+
+        ServerOperationTimeout(String serverId){
+           this.serverId = Preconditions.checkNotNull(serverId, "serverId should not be null");
+        }
+
+        String getServerId() {
+            return serverId;
+        }
     }
 }