Remove use of {String,UUID}Identifier
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / RaftActorServerConfigurationSupport.java
index 960ef15a6f7888792454699c987a93b023e37115..bbc692e885c66a92fdfa8c01a0423cba56a4cc26 100644 (file)
@@ -12,22 +12,34 @@ import akka.actor.ActorSelection;
 import akka.actor.Cancellable;
 import com.google.common.base.Preconditions;
 import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
 import java.util.Queue;
 import java.util.UUID;
 import javax.annotation.Nullable;
+import org.opendaylight.controller.cluster.raft.ServerConfigurationPayload.ServerInfo;
 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
+import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
 import org.opendaylight.controller.cluster.raft.base.messages.SnapshotComplete;
 import org.opendaylight.controller.cluster.raft.behaviors.AbstractLeader;
 import org.opendaylight.controller.cluster.raft.messages.AddServer;
 import org.opendaylight.controller.cluster.raft.messages.AddServerReply;
+import org.opendaylight.controller.cluster.raft.messages.ChangeServersVotingStatus;
 import org.opendaylight.controller.cluster.raft.messages.RemoveServer;
 import org.opendaylight.controller.cluster.raft.messages.RemoveServerReply;
+import org.opendaylight.controller.cluster.raft.messages.ServerChangeReply;
 import org.opendaylight.controller.cluster.raft.messages.ServerChangeStatus;
 import org.opendaylight.controller.cluster.raft.messages.ServerRemoved;
 import org.opendaylight.controller.cluster.raft.messages.UnInitializedFollowerSnapshotReply;
 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
+import org.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.
@@ -59,6 +71,9 @@ class RaftActorServerConfigurationSupport {
         } else if(message instanceof RemoveServer) {
             onRemoveServer((RemoveServer) message, sender);
             return true;
+        } else if(message instanceof ChangeServersVotingStatus) {
+            onChangeServersVotingStatus((ChangeServersVotingStatus) message, sender);
+            return true;
         } else if (message instanceof ServerOperationTimeout) {
             currentOperationState.onServerOperationTimeout((ServerOperationTimeout) message);
             return true;
@@ -75,9 +90,39 @@ class RaftActorServerConfigurationSupport {
         }
     }
 
+    void onNewLeader(String leaderId) {
+        currentOperationState.onNewLeader(leaderId);
+    }
+
+    private void onChangeServersVotingStatus(ChangeServersVotingStatus message, ActorRef sender) {
+        LOG.debug("{}: onChangeServersVotingStatus: {}, state: {}", raftContext.getId(), message,
+                currentOperationState);
+
+        // The following check is a special case. Normally we fail an operation if there's no leader.
+        // Consider a scenario where one has 2 geographically-separated 3-node clusters, one a primary and
+        // the other a backup such that if the primary cluster is lost, the backup can take over. In this
+        // scenario, we have a logical 6-node cluster where the primary sub-cluster is configured as voting
+        // and the backup sub-cluster as non-voting such that the primary cluster can make progress without
+        // consensus from the backup cluster while still replicating to the backup. On fail-over to the backup,
+        // a request would be sent to a member of the backup cluster to flip the voting states, ie make the
+        // backup sub-cluster voting and the lost primary non-voting. However since the primary majority
+        // cluster is lost, there would be no leader to apply, persist and replicate the server config change.
+        // Therefore, if the local server is currently non-voting and is to be changed to voting and there is
+        // no current leader, we will try to elect a leader using the new server config in order to replicate
+        // the change and progress.
+        boolean localServerChangingToVoting = Boolean.TRUE.equals(message.
+                getServerVotingStatusMap().get(raftActor.getRaftActorContext().getId()));
+        boolean hasNoLeader = raftActor.getLeaderId() == null;
+        if(localServerChangingToVoting && !raftContext.isVotingMember() && hasNoLeader) {
+            currentOperationState.onNewOperation(new ChangeServersVotingStatusContext(message, sender, true));
+        } else {
+            onNewOperation(new ChangeServersVotingStatusContext(message, sender, false));
+        }
+    }
+
     private void onRemoveServer(RemoveServer removeServer, ActorRef sender) {
         LOG.debug("{}: onRemoveServer: {}, state: {}", raftContext.getId(), removeServer, currentOperationState);
-        boolean isSelf = removeServer.getServerId().equals(raftActor.getId());
+        boolean isSelf = removeServer.getServerId().equals(raftContext.getId());
         if(isSelf && !raftContext.hasFollowers()) {
             sender.tell(new RemoveServerReply(ServerChangeStatus.NOT_SUPPORTED, raftActor.getLeaderId()),
                     raftActor.getSelf());
@@ -135,7 +180,7 @@ class RaftActorServerConfigurationSupport {
             ActorSelection leader = raftActor.getLeader();
             if (leader != null) {
                 LOG.debug("{}: Not leader - forwarding to leader {}", raftContext.getId(), leader);
-                leader.forward(operationContext.getOperation(), raftActor.getContext());
+                leader.tell(operationContext.getOperation(), operationContext.getClientRequestor());
             } else {
                 LOG.debug("{}: No leader - returning NO_LEADER reply", raftContext.getId());
                 operationContext.getClientRequestor().tell(operationContext.newReply(
@@ -180,16 +225,20 @@ class RaftActorServerConfigurationSupport {
 
         }
 
+        void onNewLeader(String newLeader) {
+        }
+
         protected void persistNewServerConfiguration(ServerOperationContext<?> operationContext){
             raftContext.setDynamicServerConfigurationInUse();
 
-            boolean includeSelf = !operationContext.getServerId().equals(raftActor.getId());
-            ServerConfigurationPayload payload = raftContext.getPeerServerInfo(includeSelf);
+            ServerConfigurationPayload payload = raftContext.getPeerServerInfo(
+                    operationContext.includeSelfInNewConfiguration(raftActor));
             LOG.debug("{}: New server configuration : {}", raftContext.getId(), payload.getServerConfig());
 
             raftActor.persistData(operationContext.getClientRequestor(), operationContext.getContextId(), payload);
 
-            currentOperationState = new Persisting(operationContext, newTimer(new ServerOperationTimeout(operationContext.getServerId())));
+            currentOperationState = new Persisting(operationContext, newTimer(new ServerOperationTimeout(
+                    operationContext.getLoggingContext())));
 
             sendReply(operationContext, ServerChangeStatus.OK);
         }
@@ -199,8 +248,12 @@ class RaftActorServerConfigurationSupport {
                 sendReply(operationContext, replyStatus);
             }
 
-            operationContext.operationComplete(raftActor, replyStatus);
+            operationContext.operationComplete(raftActor, replyStatus == null || replyStatus == ServerChangeStatus.OK);
+
+            changeToIdleState();
+        }
 
+        protected void changeToIdleState() {
             currentOperationState = IDLE;
 
             ServerOperationContext<?> nextOperation = pendingOperationsQueue.poll();
@@ -217,8 +270,12 @@ class RaftActorServerConfigurationSupport {
         }
 
         Cancellable newTimer(Object message) {
+            return newTimer(raftContext.getConfigParams().getElectionTimeOutInterval().$times(2), message);
+        }
+
+        Cancellable newTimer(FiniteDuration timeout, Object message) {
             return raftContext.getActorSystem().scheduler().scheduleOnce(
-                    raftContext.getConfigParams().getElectionTimeOutInterval().$times(2), raftContext.getActor(), message,
+                    timeout, raftContext.getActor(), message,
                             raftContext.getActorSystem().dispatcher(), raftContext.getActor());
         }
 
@@ -261,7 +318,7 @@ class RaftActorServerConfigurationSupport {
             // Sanity check - we could get an ApplyState from a previous operation that timed out so make
             // sure it's meant for us.
             if(operationContext.getContextId().equals(applyState.getIdentifier())) {
-                LOG.info("{}: {} has been successfully replicated to a majority of followers", raftActor.getId(),
+                LOG.info("{}: {} has been successfully replicated to a majority of followers", raftContext.getId(),
                         applyState.getReplicatedLogEntry().getData());
 
                 timer.cancel();
@@ -272,7 +329,7 @@ class RaftActorServerConfigurationSupport {
         @Override
         public void onServerOperationTimeout(ServerOperationTimeout timeout) {
             LOG.warn("{}: Timeout occured while replicating the new server configuration for {}", raftContext.getId(),
-                    timeout.getServerId());
+                    timeout.getLoggingContext());
 
             timedOut = true;
 
@@ -313,7 +370,7 @@ class RaftActorServerConfigurationSupport {
         }
 
         void handleInstallSnapshotTimeout(ServerOperationTimeout timeout) {
-            String serverId = timeout.getServerId();
+            String serverId = timeout.getLoggingContext();
 
             LOG.debug("{}: handleInstallSnapshotTimeout for new server {}", raftContext.getId(), serverId);
 
@@ -398,7 +455,7 @@ class RaftActorServerConfigurationSupport {
             handleInstallSnapshotTimeout(timeout);
 
             LOG.warn("{}: Timeout occured for new server {} while installing snapshot", raftContext.getId(),
-                    timeout.getServerId());
+                    timeout.getLoggingContext());
         }
 
         @Override
@@ -463,7 +520,15 @@ class RaftActorServerConfigurationSupport {
             handleInstallSnapshotTimeout(timeout);
 
             LOG.warn("{}: Timeout occured for new server {} while waiting for prior snapshot to complete",
-                    raftContext.getId(), timeout.getServerId());
+                    raftContext.getId(), timeout.getLoggingContext());
+        }
+    }
+
+    private static final class ServerOperationContextIdentifier extends AbstractUUIDIdentifier<ServerOperationContextIdentifier> {
+        private static final long serialVersionUID = 1L;
+
+        ServerOperationContextIdentifier() {
+            super(UUID.randomUUID());
         }
     }
 
@@ -475,15 +540,15 @@ class RaftActorServerConfigurationSupport {
     private static abstract class ServerOperationContext<T> {
         private final T operation;
         private final ActorRef clientRequestor;
-        private final String contextId;
+        private final Identifier contextId;
 
         ServerOperationContext(T operation, ActorRef clientRequestor){
             this.operation = operation;
             this.clientRequestor = clientRequestor;
-            contextId = UUID.randomUUID().toString();
+            contextId = new ServerOperationContextIdentifier();
         }
 
-        String getContextId() {
+        Identifier getContextId() {
             return contextId;
         }
 
@@ -495,13 +560,18 @@ class RaftActorServerConfigurationSupport {
             return clientRequestor;
         }
 
+        void operationComplete(RaftActor raftActor, boolean succeeded) {
+        }
+
+        boolean includeSelfInNewConfiguration(RaftActor raftActor) {
+            return true;
+        }
+
         abstract Object newReply(ServerChangeStatus status, String leaderId);
 
         abstract InitialOperationState newInitialOperationState(RaftActorServerConfigurationSupport support);
 
-        abstract void operationComplete(RaftActor raftActor, ServerChangeStatus serverChangeStatus);
-
-        abstract String getServerId();
+        abstract String getLoggingContext();
     }
 
     /**
@@ -523,12 +593,7 @@ class RaftActorServerConfigurationSupport {
         }
 
         @Override
-        void operationComplete(RaftActor raftActor, ServerChangeStatus serverChangeStatus) {
-
-        }
-
-        @Override
-        String getServerId() {
+        String getLoggingContext() {
             return getOperation().getNewServerId();
         }
     }
@@ -581,27 +646,227 @@ class RaftActorServerConfigurationSupport {
         }
 
         @Override
-        void operationComplete(RaftActor raftActor, ServerChangeStatus serverChangeStatus) {
+        void operationComplete(RaftActor raftActor, boolean succeeded) {
             if(peerAddress != null) {
                 raftActor.context().actorSelection(peerAddress).tell(new ServerRemoved(getOperation().getServerId()), raftActor.getSelf());
             }
         }
 
         @Override
-        String getServerId() {
+        boolean includeSelfInNewConfiguration(RaftActor raftActor) {
+            return !getOperation().getServerId().equals(raftActor.getId());
+        }
+
+        @Override
+        String getLoggingContext() {
             return getOperation().getServerId();
         }
     }
 
+    private static class ChangeServersVotingStatusContext extends ServerOperationContext<ChangeServersVotingStatus> {
+        private final boolean tryToElectLeader;
+
+        ChangeServersVotingStatusContext(ChangeServersVotingStatus convertMessage, ActorRef clientRequestor,
+                boolean tryToElectLeader) {
+            super(convertMessage, clientRequestor);
+            this.tryToElectLeader = tryToElectLeader;
+        }
+
+        @Override
+        InitialOperationState newInitialOperationState(RaftActorServerConfigurationSupport support) {
+            return support.new ChangeServersVotingStatusState(this, tryToElectLeader);
+        }
+
+        @Override
+        Object newReply(ServerChangeStatus status, String leaderId) {
+            return new ServerChangeReply(status, leaderId);
+        }
+
+        @Override
+        void operationComplete(final RaftActor raftActor, boolean succeeded) {
+            // If this leader changed to non-voting we need to step down as leader so we'll try to transfer
+            // leadership.
+            boolean localServerChangedToNonVoting = Boolean.FALSE.equals(getOperation().
+                    getServerVotingStatusMap().get(raftActor.getRaftActorContext().getId()));
+            if (succeeded && localServerChangedToNonVoting) {
+                LOG.debug("Leader changed to non-voting - trying leadership transfer");
+                raftActor.becomeNonVoting();
+            }
+        }
+
+        @Override
+        String getLoggingContext() {
+            return getOperation().toString();
+        }
+    }
+
+    private class ChangeServersVotingStatusState extends OperationState implements InitialOperationState {
+        private final ChangeServersVotingStatusContext changeVotingStatusContext;
+        private final boolean tryToElectLeader;
+
+        ChangeServersVotingStatusState(ChangeServersVotingStatusContext changeVotingStatusContext,
+                boolean tryToElectLeader) {
+            this.changeVotingStatusContext = changeVotingStatusContext;
+            this.tryToElectLeader = tryToElectLeader;
+        }
+
+        @Override
+        public void initiate() {
+            LOG.debug("Initiating ChangeServersVotingStatusState");
+
+            if(tryToElectLeader) {
+                initiateLocalLeaderElection();
+            } else 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(ElectionTimeout.INSTANCE, raftContext.getActor());
+
+            currentOperationState = new WaitingForLeaderElected(changeVotingStatusContext, previousServerConfig);
+        }
+
+        private boolean updateLocalPeerInfo() {
+            List<ServerInfo> 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));
+            if(raftActor.getCurrentBehavior() instanceof AbstractLeader) {
+                AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior();
+                leader.updateMinReplicaCount();
+            }
+
+            return true;
+        }
+
+        private List<ServerInfo> newServerInfoList() {
+            Map<String, Boolean> serverVotingStatusMap = changeVotingStatusContext.getOperation().getServerVotingStatusMap();
+            List<ServerInfo> newServerInfoList = new ArrayList<>();
+            for(String peerId: raftContext.getPeerIds()) {
+                newServerInfoList.add(new ServerInfo(peerId, serverVotingStatusMap.containsKey(peerId) ?
+                        serverVotingStatusMap.get(peerId) : raftContext.getPeerInfo(peerId).isVoting()));
+            }
+
+            newServerInfoList.add(new ServerInfo(raftContext.getId(), serverVotingStatusMap.containsKey(
+                    raftContext.getId()) ? serverVotingStatusMap.get(raftContext.getId()) : raftContext.isVotingMember()));
+
+            return newServerInfoList;
+        }
+    }
+
+    private class WaitingForLeaderElected extends OperationState {
+        private final ServerConfigurationPayload previousServerConfig;
+        private final ChangeServersVotingStatusContext operationContext;
+        private final Cancellable timer;
+
+        WaitingForLeaderElected(ChangeServersVotingStatusContext operationContext,
+                ServerConfigurationPayload previousServerConfig) {
+            this.operationContext = operationContext;
+            this.previousServerConfig = previousServerConfig;
+
+            timer = newTimer(raftContext.getConfigParams().getElectionTimeOutInterval(),
+                    new ServerOperationTimeout(operationContext.getLoggingContext()));
+        }
+
+        @Override
+        void onNewLeader(String newLeader) {
+            LOG.debug("{}: New leader {} elected", raftContext.getId(), newLeader);
+
+            timer.cancel();
+
+            if(raftActor.isLeader()) {
+                persistNewServerConfiguration(operationContext);
+            } else {
+                // Edge case - some other node became leader so forward the operation.
+                LOG.debug("{}: Forwarding {} to new leader", raftContext.getId(), operationContext.getOperation());
+
+                // Revert the local server config change.
+                raftContext.updatePeerIds(previousServerConfig);
+
+                changeToIdleState();
+                RaftActorServerConfigurationSupport.this.onNewOperation(operationContext);
+            }
+        }
+
+        @Override
+        void onServerOperationTimeout(ServerOperationTimeout timeout) {
+            LOG.warn("{}: Leader election timed out - cannot apply operation {}",
+                    raftContext.getId(), timeout.getLoggingContext());
+
+            // Revert the local server config change.
+            raftContext.updatePeerIds(previousServerConfig);
+            raftActor.initializeBehavior();
+
+            tryToForwardOperationToAnotherServer();
+        }
+
+        private void tryToForwardOperationToAnotherServer() {
+            Collection<String> 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<String, Boolean> serverVotingStatusMap = operationContext.getOperation().getServerVotingStatusMap();
+            ActorSelection forwardToPeerActor = null;
+            for(Map.Entry<String, Boolean> e: serverVotingStatusMap.entrySet()) {
+                Boolean isVoting = e.getValue();
+                String serverId = e.getKey();
+                PeerInfo peerInfo = raftContext.getPeerInfo(serverId);
+                if(isVoting && peerInfo != null && !peerInfo.isVoting() && !serversVisited.contains(serverId)) {
+                    ActorSelection actor = raftContext.getPeerActorSelection(serverId);
+                    if(actor != null) {
+                        forwardToPeerActor = actor;
+                        break;
+                    }
+                }
+            }
+
+            if(forwardToPeerActor != null) {
+                LOG.debug("{}: Found server {} to forward to", raftContext.getId(), forwardToPeerActor);
+
+                forwardToPeerActor.tell(new ChangeServersVotingStatus(serverVotingStatusMap, serversVisited),
+                        operationContext.getClientRequestor());
+                changeToIdleState();
+            } else {
+                operationComplete(operationContext, ServerChangeStatus.NO_LEADER);
+            }
+        }
+    }
+
     static class ServerOperationTimeout {
-        private final String serverId;
+        private final String loggingContext;
 
-        ServerOperationTimeout(String serverId){
-           this.serverId = Preconditions.checkNotNull(serverId, "serverId should not be null");
+        ServerOperationTimeout(String loggingContext){
+           this.loggingContext = Preconditions.checkNotNull(loggingContext, "loggingContext should not be null");
         }
 
-        String getServerId() {
-            return serverId;
+        String getLoggingContext() {
+            return loggingContext;
         }
     }
 }