Prevent non-voting member from becoming entity owner
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / RaftActorServerConfigurationSupport.java
index 782ecc06a87ac11b19c6ef6b8335e6a9aa820b1a..3a3511668c4bb82c7c216b56b5a89aa356521375 100644 (file)
@@ -61,12 +61,12 @@ class RaftActorServerConfigurationSupport {
 
     private OperationState currentOperationState = IDLE;
 
-    RaftActorServerConfigurationSupport(RaftActor raftActor) {
+    RaftActorServerConfigurationSupport(final RaftActor raftActor) {
         this.raftActor = raftActor;
         this.raftContext = raftActor.getRaftActorContext();
     }
 
-    boolean handleMessage(Object message, ActorRef sender) {
+    boolean handleMessage(final Object message, final ActorRef sender) {
         if (message instanceof AddServer) {
             onAddServer((AddServer) message, sender);
             return true;
@@ -92,11 +92,11 @@ class RaftActorServerConfigurationSupport {
         }
     }
 
-    void onNewLeader(String leaderId) {
+    void onNewLeader(final String leaderId) {
         currentOperationState.onNewLeader(leaderId);
     }
 
-    private void onChangeServersVotingStatus(ChangeServersVotingStatus message, ActorRef sender) {
+    private void onChangeServersVotingStatus(final ChangeServersVotingStatus message, final ActorRef sender) {
         LOG.debug("{}: onChangeServersVotingStatus: {}, state: {}", raftContext.getId(), message,
                 currentOperationState);
 
@@ -122,7 +122,7 @@ class RaftActorServerConfigurationSupport {
         }
     }
 
-    private void onRemoveServer(RemoveServer removeServer, ActorRef sender) {
+    private void onRemoveServer(final RemoveServer removeServer, final ActorRef sender) {
         LOG.debug("{}: onRemoveServer: {}, state: {}", raftContext.getId(), removeServer, currentOperationState);
         boolean isSelf = removeServer.getServerId().equals(raftContext.getId());
         if (isSelf && !raftContext.hasFollowers()) {
@@ -138,7 +138,7 @@ class RaftActorServerConfigurationSupport {
         }
     }
 
-    private boolean onApplyState(ApplyState applyState) {
+    private boolean onApplyState(final ApplyState applyState) {
         Payload data = applyState.getReplicatedLogEntry().getData();
         if (data instanceof ServerConfigurationPayload) {
             currentOperationState.onApplyState(applyState);
@@ -149,7 +149,7 @@ class RaftActorServerConfigurationSupport {
     }
 
     /**
-     * The algorithm for AddServer is as follows:
+     * Add a server. 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>
@@ -169,13 +169,13 @@ class RaftActorServerConfigurationSupport {
      *     <li>Respond to caller with TIMEOUT.</li>
      * </ul>
      */
-    private void onAddServer(AddServer addServer, ActorRef sender) {
+    private void onAddServer(final AddServer addServer, final ActorRef sender) {
         LOG.debug("{}: onAddServer: {}, state: {}", raftContext.getId(), addServer, currentOperationState);
 
         onNewOperation(new AddServerContext(addServer, sender));
     }
 
-    private void onNewOperation(ServerOperationContext<?> operationContext) {
+    private void onNewOperation(final ServerOperationContext<?> operationContext) {
         if (raftActor.isLeader()) {
             currentOperationState.onNewOperation(operationContext);
         } else {
@@ -202,7 +202,7 @@ class RaftActorServerConfigurationSupport {
      * Abstract base class for a server operation FSM state. Handles common behavior for all states.
      */
     private abstract class OperationState {
-        void onNewOperation(ServerOperationContext<?> operationContext) {
+        void onNewOperation(final ServerOperationContext<?> operationContext) {
             // We're currently processing another operation so queue it to be processed later.
 
             LOG.debug("{}: Server operation already in progress - queueing {}", raftContext.getId(),
@@ -211,15 +211,15 @@ class RaftActorServerConfigurationSupport {
             pendingOperationsQueue.add(operationContext);
         }
 
-        void onServerOperationTimeout(ServerOperationTimeout timeout) {
+        void onServerOperationTimeout(final ServerOperationTimeout timeout) {
             LOG.debug("onServerOperationTimeout should not be called in state {}", this);
         }
 
-        void onUnInitializedFollowerSnapshotReply(UnInitializedFollowerSnapshotReply reply) {
+        void onUnInitializedFollowerSnapshotReply(final UnInitializedFollowerSnapshotReply reply) {
             LOG.debug("onUnInitializedFollowerSnapshotReply was called in state {}", this);
         }
 
-        void onApplyState(ApplyState applyState) {
+        void onApplyState(final ApplyState applyState) {
             LOG.debug("onApplyState was called in state {}", this);
         }
 
@@ -227,17 +227,18 @@ class RaftActorServerConfigurationSupport {
 
         }
 
-        void onNewLeader(String newLeader) {
+        void onNewLeader(final String newLeader) {
         }
 
-        protected void persistNewServerConfiguration(ServerOperationContext<?> operationContext) {
+        protected void persistNewServerConfiguration(final ServerOperationContext<?> operationContext) {
             raftContext.setDynamicServerConfigurationInUse();
 
             ServerConfigurationPayload payload = raftContext.getPeerServerInfo(
                     operationContext.includeSelfInNewConfiguration(raftActor));
             LOG.debug("{}: New server configuration : {}", raftContext.getId(), payload.getServerConfig());
 
-            raftActor.persistData(operationContext.getClientRequestor(), operationContext.getContextId(), payload);
+            raftActor.persistData(operationContext.getClientRequestor(), operationContext.getContextId(),
+                    payload, false);
 
             currentOperationState = new Persisting(operationContext, newTimer(new ServerOperationTimeout(
                     operationContext.getLoggingContext())));
@@ -245,8 +246,8 @@ class RaftActorServerConfigurationSupport {
             sendReply(operationContext, ServerChangeStatus.OK);
         }
 
-        protected void operationComplete(ServerOperationContext<?> operationContext,
-                @Nullable ServerChangeStatus replyStatus) {
+        protected void operationComplete(final ServerOperationContext<?> operationContext,
+                @Nullable final ServerChangeStatus replyStatus) {
             if (replyStatus != null) {
                 sendReply(operationContext, replyStatus);
             }
@@ -265,7 +266,7 @@ class RaftActorServerConfigurationSupport {
             }
         }
 
-        protected void sendReply(ServerOperationContext<?> operationContext, ServerChangeStatus status) {
+        protected void sendReply(final ServerOperationContext<?> operationContext, final ServerChangeStatus status) {
             LOG.debug("{}: Returning {} for operation {}", raftContext.getId(), status,
                     operationContext.getOperation());
 
@@ -273,11 +274,11 @@ class RaftActorServerConfigurationSupport {
                     raftActor.self());
         }
 
-        Cancellable newTimer(Object message) {
+        Cancellable newTimer(final Object message) {
             return newTimer(raftContext.getConfigParams().getElectionTimeOutInterval().$times(2), message);
         }
 
-        Cancellable newTimer(FiniteDuration timeout, Object message) {
+        Cancellable newTimer(final FiniteDuration timeout, final Object message) {
             return raftContext.getActorSystem().scheduler().scheduleOnce(
                     timeout, raftContext.getActor(), message,
                             raftContext.getActorSystem().dispatcher(), raftContext.getActor());
@@ -294,12 +295,12 @@ class RaftActorServerConfigurationSupport {
      */
     private final class Idle extends OperationState {
         @Override
-        public void onNewOperation(ServerOperationContext<?> operationContext) {
+        public void onNewOperation(final ServerOperationContext<?> operationContext) {
             operationContext.newInitialOperationState(RaftActorServerConfigurationSupport.this).initiate();
         }
 
         @Override
-        public void onApplyState(ApplyState applyState) {
+        public void onApplyState(final ApplyState applyState) {
             // Noop - we override b/c ApplyState is called normally for followers in the idle state.
         }
     }
@@ -312,13 +313,13 @@ class RaftActorServerConfigurationSupport {
         private final Cancellable timer;
         private boolean timedOut = false;
 
-        Persisting(ServerOperationContext<?> operationContext, Cancellable timer) {
+        Persisting(final ServerOperationContext<?> operationContext, final Cancellable timer) {
             this.operationContext = operationContext;
             this.timer = timer;
         }
 
         @Override
-        public void onApplyState(ApplyState applyState) {
+        public void onApplyState(final ApplyState applyState) {
             // Sanity check - we could get an ApplyState from a previous operation that timed out so make
             // sure it's meant for us.
             if (operationContext.getContextId().equals(applyState.getIdentifier())) {
@@ -331,7 +332,7 @@ class RaftActorServerConfigurationSupport {
         }
 
         @Override
-        public void onServerOperationTimeout(ServerOperationTimeout timeout) {
+        public void onServerOperationTimeout(final ServerOperationTimeout timeout) {
             LOG.warn("{}: Timeout occured while replicating the new server configuration for {}", raftContext.getId(),
                     timeout.getLoggingContext());
 
@@ -346,7 +347,7 @@ class RaftActorServerConfigurationSupport {
         }
 
         @Override
-        public void onNewOperation(ServerOperationContext<?> newOperationContext) {
+        public void onNewOperation(final ServerOperationContext<?> newOperationContext) {
             if (timedOut) {
                 sendReply(newOperationContext, ServerChangeStatus.PRIOR_REQUEST_CONSENSUS_TIMEOUT);
             } else {
@@ -361,7 +362,7 @@ class RaftActorServerConfigurationSupport {
     private abstract class AddServerState extends OperationState {
         private final AddServerContext addServerContext;
 
-        AddServerState(AddServerContext addServerContext) {
+        AddServerState(final AddServerContext addServerContext) {
             this.addServerContext = addServerContext;
         }
 
@@ -373,7 +374,7 @@ class RaftActorServerConfigurationSupport {
             return newTimer(new ServerOperationTimeout(addServerContext.getOperation().getNewServerId()));
         }
 
-        void handleInstallSnapshotTimeout(ServerOperationTimeout timeout) {
+        void handleInstallSnapshotTimeout(final ServerOperationTimeout timeout) {
             String serverId = timeout.getLoggingContext();
 
             LOG.debug("{}: handleInstallSnapshotTimeout for new server {}", raftContext.getId(), serverId);
@@ -398,7 +399,7 @@ class RaftActorServerConfigurationSupport {
      * snapshot capture, if necessary.
      */
     private final class InitialAddServerState extends AddServerState implements InitialOperationState {
-        InitialAddServerState(AddServerContext addServerContext) {
+        InitialAddServerState(final AddServerContext addServerContext) {
             super(addServerContext);
         }
 
@@ -450,13 +451,13 @@ class RaftActorServerConfigurationSupport {
     private final class InstallingSnapshot extends AddServerState {
         private final Cancellable installSnapshotTimer;
 
-        InstallingSnapshot(AddServerContext addServerContext, Cancellable installSnapshotTimer) {
+        InstallingSnapshot(final AddServerContext addServerContext, final Cancellable installSnapshotTimer) {
             super(addServerContext);
             this.installSnapshotTimer = Preconditions.checkNotNull(installSnapshotTimer);
         }
 
         @Override
-        public void onServerOperationTimeout(ServerOperationTimeout timeout) {
+        public void onServerOperationTimeout(final ServerOperationTimeout timeout) {
             handleInstallSnapshotTimeout(timeout);
 
             LOG.warn("{}: Timeout occured for new server {} while installing snapshot", raftContext.getId(),
@@ -464,7 +465,7 @@ class RaftActorServerConfigurationSupport {
         }
 
         @Override
-        public void onUnInitializedFollowerSnapshotReply(UnInitializedFollowerSnapshotReply reply) {
+        public void onUnInitializedFollowerSnapshotReply(final UnInitializedFollowerSnapshotReply reply) {
             LOG.debug("{}: onUnInitializedFollowerSnapshotReply: {}", raftContext.getId(), reply);
 
             String followerId = reply.getFollowerId();
@@ -494,7 +495,7 @@ class RaftActorServerConfigurationSupport {
     private final class WaitingForPriorSnapshotComplete extends AddServerState {
         private final Cancellable snapshotTimer;
 
-        WaitingForPriorSnapshotComplete(AddServerContext addServerContext, Cancellable snapshotTimer) {
+        WaitingForPriorSnapshotComplete(final AddServerContext addServerContext, final Cancellable snapshotTimer) {
             super(addServerContext);
             this.snapshotTimer = Preconditions.checkNotNull(snapshotTimer);
         }
@@ -521,7 +522,7 @@ class RaftActorServerConfigurationSupport {
         }
 
         @Override
-        public void onServerOperationTimeout(ServerOperationTimeout timeout) {
+        public void onServerOperationTimeout(final ServerOperationTimeout timeout) {
             handleInstallSnapshotTimeout(timeout);
 
             LOG.warn("{}: Timeout occured for new server {} while waiting for prior snapshot to complete",
@@ -548,7 +549,7 @@ class RaftActorServerConfigurationSupport {
         private final ActorRef clientRequestor;
         private final Identifier contextId;
 
-        ServerOperationContext(T operation, ActorRef clientRequestor) {
+        ServerOperationContext(final T operation, final ActorRef clientRequestor) {
             this.operation = operation;
             this.clientRequestor = clientRequestor;
             contextId = new ServerOperationContextIdentifier();
@@ -566,10 +567,10 @@ class RaftActorServerConfigurationSupport {
             return clientRequestor;
         }
 
-        void operationComplete(RaftActor raftActor, boolean succeeded) {
+        void operationComplete(final RaftActor raftActor, final boolean succeeded) {
         }
 
-        boolean includeSelfInNewConfiguration(RaftActor raftActor) {
+        boolean includeSelfInNewConfiguration(final RaftActor raftActor) {
             return true;
         }
 
@@ -584,17 +585,17 @@ class RaftActorServerConfigurationSupport {
      * Stores context information for an AddServer operation.
      */
     private static class AddServerContext extends ServerOperationContext<AddServer> {
-        AddServerContext(AddServer addServer, ActorRef clientRequestor) {
+        AddServerContext(final AddServer addServer, final ActorRef clientRequestor) {
             super(addServer, clientRequestor);
         }
 
         @Override
-        Object newReply(ServerChangeStatus status, String leaderId) {
+        Object newReply(final ServerChangeStatus status, final String leaderId) {
             return new AddServerReply(status, leaderId);
         }
 
         @Override
-        InitialOperationState newInitialOperationState(RaftActorServerConfigurationSupport support) {
+        InitialOperationState newInitialOperationState(final RaftActorServerConfigurationSupport support) {
             return support.new InitialAddServerState(this);
         }
 
@@ -607,7 +608,7 @@ class RaftActorServerConfigurationSupport {
     private abstract class RemoveServerState extends OperationState {
         private final RemoveServerContext removeServerContext;
 
-        protected RemoveServerState(RemoveServerContext removeServerContext) {
+        protected RemoveServerState(final RemoveServerContext removeServerContext) {
             this.removeServerContext = Preconditions.checkNotNull(removeServerContext);
 
         }
@@ -619,7 +620,7 @@ class RaftActorServerConfigurationSupport {
 
     private final class InitialRemoveServerState extends RemoveServerState implements InitialOperationState {
 
-        protected InitialRemoveServerState(RemoveServerContext removeServerContext) {
+        protected InitialRemoveServerState(final RemoveServerContext removeServerContext) {
             super(removeServerContext);
         }
 
@@ -627,7 +628,9 @@ class RaftActorServerConfigurationSupport {
         public void initiate() {
             String serverId = getRemoveServerContext().getOperation().getServerId();
             raftContext.removePeer(serverId);
-            ((AbstractLeader)raftActor.getCurrentBehavior()).removeFollower(serverId);
+            AbstractLeader leader = (AbstractLeader)raftActor.getCurrentBehavior();
+            leader.removeFollower(serverId);
+            leader.updateMinReplicaCount();
 
             persistNewServerConfiguration(getRemoveServerContext());
         }
@@ -636,23 +639,23 @@ class RaftActorServerConfigurationSupport {
     private static class RemoveServerContext extends ServerOperationContext<RemoveServer> {
         private final String peerAddress;
 
-        RemoveServerContext(RemoveServer operation, String peerAddress, ActorRef clientRequestor) {
+        RemoveServerContext(final RemoveServer operation, final String peerAddress, final ActorRef clientRequestor) {
             super(operation, clientRequestor);
             this.peerAddress = peerAddress;
         }
 
         @Override
-        Object newReply(ServerChangeStatus status, String leaderId) {
+        Object newReply(final ServerChangeStatus status, final String leaderId) {
             return new RemoveServerReply(status, leaderId);
         }
 
         @Override
-        InitialOperationState newInitialOperationState(RaftActorServerConfigurationSupport support) {
+        InitialOperationState newInitialOperationState(final RaftActorServerConfigurationSupport support) {
             return support.new InitialRemoveServerState(this);
         }
 
         @Override
-        void operationComplete(RaftActor raftActor, boolean succeeded) {
+        void operationComplete(final RaftActor raftActor, final boolean succeeded) {
             if (peerAddress != null) {
                 raftActor.context().actorSelection(peerAddress).tell(
                         new ServerRemoved(getOperation().getServerId()), raftActor.getSelf());
@@ -660,7 +663,7 @@ class RaftActorServerConfigurationSupport {
         }
 
         @Override
-        boolean includeSelfInNewConfiguration(RaftActor raftActor) {
+        boolean includeSelfInNewConfiguration(final RaftActor raftActor) {
             return !getOperation().getServerId().equals(raftActor.getId());
         }
 
@@ -673,24 +676,24 @@ class RaftActorServerConfigurationSupport {
     private static class ChangeServersVotingStatusContext extends ServerOperationContext<ChangeServersVotingStatus> {
         private final boolean tryToElectLeader;
 
-        ChangeServersVotingStatusContext(ChangeServersVotingStatus convertMessage, ActorRef clientRequestor,
-                boolean tryToElectLeader) {
+        ChangeServersVotingStatusContext(final ChangeServersVotingStatus convertMessage, final ActorRef clientRequestor,
+                final boolean tryToElectLeader) {
             super(convertMessage, clientRequestor);
             this.tryToElectLeader = tryToElectLeader;
         }
 
         @Override
-        InitialOperationState newInitialOperationState(RaftActorServerConfigurationSupport support) {
+        InitialOperationState newInitialOperationState(final RaftActorServerConfigurationSupport support) {
             return support.new ChangeServersVotingStatusState(this, tryToElectLeader);
         }
 
         @Override
-        Object newReply(ServerChangeStatus status, String leaderId) {
+        Object newReply(final ServerChangeStatus status, final String leaderId) {
             return new ServerChangeReply(status, leaderId);
         }
 
         @Override
-        void operationComplete(final RaftActor raftActor, boolean succeeded) {
+        void operationComplete(final RaftActor raftActor, final boolean succeeded) {
             // If this leader changed to non-voting we need to step down as leader so we'll try to transfer
             // leadership.
             boolean localServerChangedToNonVoting = Boolean.FALSE.equals(getOperation()
@@ -698,6 +701,8 @@ class RaftActorServerConfigurationSupport {
             if (succeeded && localServerChangedToNonVoting) {
                 LOG.debug("Leader changed to non-voting - trying leadership transfer");
                 raftActor.becomeNonVoting();
+            } else if (raftActor.isLeader()) {
+                raftActor.onVotingStateChangeComplete();
             }
         }
 
@@ -711,8 +716,8 @@ class RaftActorServerConfigurationSupport {
         private final ChangeServersVotingStatusContext changeVotingStatusContext;
         private final boolean tryToElectLeader;
 
-        ChangeServersVotingStatusState(ChangeServersVotingStatusContext changeVotingStatusContext,
-                boolean tryToElectLeader) {
+        ChangeServersVotingStatusState(final ChangeServersVotingStatusContext changeVotingStatusContext,
+                final boolean tryToElectLeader) {
             this.changeVotingStatusContext = changeVotingStatusContext;
             this.tryToElectLeader = tryToElectLeader;
         }
@@ -789,8 +794,8 @@ class RaftActorServerConfigurationSupport {
         private final ChangeServersVotingStatusContext operationContext;
         private final Cancellable timer;
 
-        WaitingForLeaderElected(ChangeServersVotingStatusContext operationContext,
-                ServerConfigurationPayload previousServerConfig) {
+        WaitingForLeaderElected(final ChangeServersVotingStatusContext operationContext,
+                final ServerConfigurationPayload previousServerConfig) {
             this.operationContext = operationContext;
             this.previousServerConfig = previousServerConfig;
 
@@ -799,7 +804,7 @@ class RaftActorServerConfigurationSupport {
         }
 
         @Override
-        void onNewLeader(String newLeader) {
+        void onNewLeader(final String newLeader) {
             if (newLeader == null) {
                 return;
             }
@@ -823,7 +828,7 @@ class RaftActorServerConfigurationSupport {
         }
 
         @Override
-        void onServerOperationTimeout(ServerOperationTimeout timeout) {
+        void onServerOperationTimeout(final ServerOperationTimeout timeout) {
             LOG.warn("{}: Leader election timed out - cannot apply operation {}",
                     raftContext.getId(), timeout.getLoggingContext());
 
@@ -874,7 +879,7 @@ class RaftActorServerConfigurationSupport {
     static class ServerOperationTimeout {
         private final String loggingContext;
 
-        ServerOperationTimeout(String loggingContext) {
+        ServerOperationTimeout(final String loggingContext) {
             this.loggingContext = Preconditions.checkNotNull(loggingContext, "loggingContext should not be null");
         }