BUG 2817 - Basic implementation of RemoveServer in the Raft code
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / RaftActorServerConfigurationSupport.java
index a00c0241d96b61cbb3b2a5b0330f7c992d94b0b4..207642a7213e637fc8af6539ea9e989c0a75d43e 100644 (file)
@@ -11,20 +11,21 @@ import akka.actor.ActorRef;
 import akka.actor.ActorSelection;
 import akka.actor.Cancellable;
 import com.google.common.base.Preconditions;
-import java.util.ArrayList;
-import java.util.Collections;
 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;
@@ -53,22 +54,41 @@ class RaftActorServerConfigurationSupport {
 
     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) {
-            currentOperationState.onFollowerCatchupTimeout(raftActor, (FollowerCatchUpTimeout)message);
+            currentOperationState.onFollowerCatchupTimeout(raftActor, (FollowerCatchUpTimeout) message);
             return true;
         } else if (message instanceof UnInitializedFollowerSnapshotReply) {
             currentOperationState.onUnInitializedFollowerSnapshotReply(raftActor,
-                    (UnInitializedFollowerSnapshotReply)message);
+                    (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) {
@@ -133,6 +153,8 @@ class RaftActorServerConfigurationSupport {
         void onUnInitializedFollowerSnapshotReply(RaftActor raftActor, UnInitializedFollowerSnapshotReply reply);
 
         void onApplyState(RaftActor raftActor, ApplyState applyState);
+
+        void onSnapshotComplete(RaftActor raftActor);
     }
 
     /**
@@ -171,26 +193,29 @@ class RaftActorServerConfigurationSupport {
             LOG.debug("onApplyState was called in state {}", this);
         }
 
-        protected void persistNewServerConfiguration(RaftActor raftActor, ServerOperationContext<?> operationContext){
-            List <String> newConfig = new ArrayList<String>(raftContext.getPeerAddresses().keySet());
-            newConfig.add(raftContext.getId());
-
-            LOG.debug("{}: New server configuration : {}", raftContext.getId(), newConfig);
+        @Override
+        public void onSnapshotComplete(RaftActor raftActor) {
+        }
 
-            ServerConfigurationPayload payload = new ServerConfigurationPayload(newConfig, Collections.<String>emptyList());
+        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);
         }
 
         protected void operationComplete(RaftActor raftActor, ServerOperationContext<?> operationContext,
-                ServerChangeStatus status) {
-
-            LOG.debug("{}: Returning {} for operation {}", raftContext.getId(), status, operationContext.getOperation());
+                @Nullable ServerChangeStatus replyStatus) {
+            if(replyStatus != null) {
+                sendReply(raftActor, operationContext, replyStatus);
+            }
 
-            operationContext.getClientRequestor().tell(operationContext.newReply(status, raftActor.getLeaderId()),
-                    raftActor.self());
+            operationContext.operationComplete(raftActor, replyStatus);
 
             currentOperationState = IDLE;
 
@@ -200,6 +225,14 @@ class RaftActorServerConfigurationSupport {
             }
         }
 
+        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();
@@ -236,10 +269,10 @@ class RaftActorServerConfigurationSupport {
             // Sanity check - we could get an ApplyState from a previous operation that timed out so make
             // sure it's meant for us.
             if(operationContext.getContextId().equals(applyState.getIdentifier())) {
-                LOG.info("{}: {} has been successfully replicated to a majority of followers",
+                LOG.info("{}: {} has been successfully replicated to a majority of followers", raftActor.getId(),
                         applyState.getReplicatedLogEntry().getData());
 
-                operationComplete(raftActor, operationContext, ServerChangeStatus.OK);
+                operationComplete(raftActor, operationContext, null);
             }
         }
     }
@@ -257,6 +290,33 @@ class RaftActorServerConfigurationSupport {
         AddServerContext getAddServerContext() {
             return addServerContext;
         }
+
+        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());
+        }
+
+        void handleOnFollowerCatchupTimeout(RaftActor raftActor, FollowerCatchUpTimeout followerTimeout) {
+            String serverId = followerTimeout.getNewServerId();
+
+            LOG.debug("{}: onFollowerCatchupTimeout 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(raftActor, getAddServerContext(),
+                    isLeader ? ServerChangeStatus.TIMEOUT : ServerChangeStatus.NO_LEADER);
+        }
+
     }
 
     /**
@@ -271,32 +331,35 @@ class RaftActorServerConfigurationSupport {
         @Override
         public void initiate(RaftActor raftActor) {
             AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior();
-
             AddServer addServer = getAddServerContext().getOperation();
 
             LOG.debug("{}: Initiating {}", raftContext.getId(), addServer);
 
-            raftContext.addToPeers(addServer.getNewServerId(), addServer.getNewServerAddress());
-
-            // if voting member - initialize to VOTING_NOT_INITIALIZED
-            FollowerState initialState = addServer.isVotingMember() ? FollowerState.VOTING_NOT_INITIALIZED :
-                FollowerState.NON_VOTING;
-            leader.addFollower(addServer.getNewServerId(), initialState);
+            if(raftContext.getPeerInfo(addServer.getNewServerId()) != null) {
+                operationComplete(raftActor, getAddServerContext(), ServerChangeStatus.ALREADY_EXISTS);
+                return;
+            }
 
-            if(initialState == FollowerState.VOTING_NOT_INITIALIZED){
-                LOG.debug("{}: Leader sending initiate capture snapshot to new follower {}", raftContext.getId(),
-                        addServer.getNewServerId());
+            VotingState votingState = addServer.isVotingMember() ? VotingState.VOTING_NOT_INITIALIZED :
+                    VotingState.NON_VOTING;
+            raftContext.addToPeers(addServer.getNewServerId(), addServer.getNewServerAddress(), votingState);
 
-                leader.initiateCaptureSnapshot(addServer.getNewServerId());
+            leader.addFollower(addServer.getNewServerId());
 
+            if(votingState == VotingState.VOTING_NOT_INITIALIZED){
                 // schedule the install snapshot timeout timer
-                Cancellable installSnapshotTimer = raftContext.getActorSystem().scheduler().scheduleOnce(
-                        new FiniteDuration(((raftContext.getConfigParams().getElectionTimeOutInterval().toMillis()) * 2),
-                                TimeUnit.MILLISECONDS), raftContext.getActor(),
-                                new FollowerCatchUpTimeout(addServer.getNewServerId()),
-                                raftContext.getActorSystem().dispatcher(), raftContext.getActor());
-
-                currentOperationState = new InstallingSnapshot(getAddServerContext(), installSnapshotTimer);
+                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());
@@ -320,19 +383,10 @@ class RaftActorServerConfigurationSupport {
 
         @Override
         public void onFollowerCatchupTimeout(RaftActor raftActor, FollowerCatchUpTimeout followerTimeout) {
-            String serverId = followerTimeout.getNewServerId();
-
-            LOG.debug("{}: onFollowerCatchupTimeout: {}", raftContext.getId(), serverId);
-
-            AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior();
-
-            // cleanup
-            raftContext.removePeer(serverId);
-            leader.removeFollower(serverId);
+            handleOnFollowerCatchupTimeout(raftActor, followerTimeout);
 
-            LOG.warn("{}: Timeout occured for new server {} while installing snapshot", raftContext.getId(), serverId);
-
-            operationComplete(raftActor, getAddServerContext(), ServerChangeStatus.TIMEOUT);
+            LOG.warn("{}: Timeout occured for new server {} while installing snapshot", raftContext.getId(),
+                    followerTimeout.getNewServerId());
         }
 
         @Override
@@ -343,18 +397,62 @@ class RaftActorServerConfigurationSupport {
 
             // Sanity check to guard against receiving an UnInitializedFollowerSnapshotReply from a prior
             // add server operation that timed out.
-            if(getAddServerContext().getOperation().getNewServerId().equals(followerId)) {
+            if(getAddServerContext().getOperation().getNewServerId().equals(followerId) && raftActor.isLeader()) {
                 AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior();
-                FollowerLogInformation followerLogInformation = leader.getFollower(followerId);
+                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");
+            }
+        }
+    }
 
-                followerLogInformation.setFollowerState(FollowerState.VOTING);
-                leader.updateMinReplicaCountAndMinIsolatedLeaderPeerCount();
+    /**
+     * 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;
 
-                persistNewServerConfiguration(raftActor, getAddServerContext());
+        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());
+        }
     }
 
     /**
@@ -388,6 +486,8 @@ class RaftActorServerConfigurationSupport {
         abstract Object newReply(ServerChangeStatus status, String leaderId);
 
         abstract InitialOperationState newInitialOperationState(RaftActorServerConfigurationSupport support);
+
+        abstract void operationComplete(RaftActor raftActor, ServerChangeStatus serverChangeStatus);
     }
 
     /**
@@ -407,5 +507,65 @@ class RaftActorServerConfigurationSupport {
         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<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());
+            }
+        }
+
     }
 }