Improve segmented journal actor metrics
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / RaftActorServerConfigurationSupport.java
index 09020812bd5c9212c14c90161b4d49a1be95c537..fd2cd419d77344fc9284d044d089d0ca4cd21489 100644 (file)
@@ -7,34 +7,37 @@
  */
 package org.opendaylight.controller.cluster.raft;
 
+import static java.util.Objects.requireNonNull;
+
 import akka.actor.ActorRef;
 import akka.actor.ActorSelection;
 import akka.actor.Cancellable;
-import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
 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.eclipse.jdt.annotation.Nullable;
 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.base.messages.TimeoutNow;
 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.Payload;
 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.controller.cluster.raft.persisted.ServerConfigurationPayload;
+import org.opendaylight.controller.cluster.raft.persisted.ServerInfo;
+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;
@@ -47,6 +50,7 @@ import scala.concurrent.duration.FiniteDuration;
 class RaftActorServerConfigurationSupport {
     private static final Logger LOG = LoggerFactory.getLogger(RaftActorServerConfigurationSupport.class);
 
+    @SuppressWarnings("checkstyle:MemberName")
     private final OperationState IDLE = new Idle();
 
     private final RaftActor raftActor;
@@ -57,30 +61,30 @@ class RaftActorServerConfigurationSupport {
 
     private OperationState currentOperationState = IDLE;
 
-    RaftActorServerConfigurationSupport(RaftActor raftActor) {
+    RaftActorServerConfigurationSupport(final RaftActor raftActor) {
         this.raftActor = raftActor;
-        this.raftContext = raftActor.getRaftActorContext();
+        raftContext = raftActor.getRaftActorContext();
     }
 
-    boolean handleMessage(Object message, ActorRef sender) {
-        if(message instanceof AddServer) {
-            onAddServer((AddServer) message, sender);
+    boolean handleMessage(final Object message, final ActorRef sender) {
+        if (message instanceof AddServer addServer) {
+            onAddServer(addServer, sender);
             return true;
-        } else if(message instanceof RemoveServer) {
-            onRemoveServer((RemoveServer) message, sender);
+        } else if (message instanceof RemoveServer removeServer) {
+            onRemoveServer(removeServer, sender);
             return true;
-        } else if(message instanceof ChangeServersVotingStatus) {
-            onChangeServersVotingStatus((ChangeServersVotingStatus) message, sender);
+        } else if (message instanceof ChangeServersVotingStatus changeServersVotingStatus) {
+            onChangeServersVotingStatus(changeServersVotingStatus, sender);
             return true;
-        } else if (message instanceof ServerOperationTimeout) {
-            currentOperationState.onServerOperationTimeout((ServerOperationTimeout) message);
+        } else if (message instanceof ServerOperationTimeout serverOperationTimeout) {
+            currentOperationState.onServerOperationTimeout(serverOperationTimeout);
             return true;
-        } else if (message instanceof UnInitializedFollowerSnapshotReply) {
-            currentOperationState.onUnInitializedFollowerSnapshotReply((UnInitializedFollowerSnapshotReply) message);
+        } else if (message instanceof UnInitializedFollowerSnapshotReply uninitFollowerSnapshotReply) {
+            currentOperationState.onUnInitializedFollowerSnapshotReply(uninitFollowerSnapshotReply);
             return true;
-        } else if(message instanceof ApplyState) {
-            return onApplyState((ApplyState) message);
-        } else if(message instanceof SnapshotComplete) {
+        } else if (message instanceof ApplyState applyState) {
+            return onApplyState(applyState);
+        } else if (message instanceof SnapshotComplete) {
             currentOperationState.onSnapshotComplete();
             return false;
         } else {
@@ -88,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);
 
@@ -108,23 +112,23 @@ class RaftActorServerConfigurationSupport {
         // 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 localServerChangingToVoting = Boolean.TRUE.equals(message
+                .getServerVotingStatusMap().get(raftActor.getRaftActorContext().getId()));
         boolean hasNoLeader = raftActor.getLeaderId() == null;
-        if(localServerChangingToVoting && !raftContext.isVotingMember() && hasNoLeader) {
+        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) {
+    private void onRemoveServer(final RemoveServer removeServer, final ActorRef sender) {
         LOG.debug("{}: onRemoveServer: {}, state: {}", raftContext.getId(), removeServer, currentOperationState);
-        boolean isSelf = removeServer.getServerId().equals(raftActor.getId());
-        if(isSelf && !raftContext.hasFollowers()) {
+        boolean isSelf = removeServer.getServerId().equals(raftContext.getId());
+        if (isSelf && !raftContext.hasFollowers()) {
             sender.tell(new RemoveServerReply(ServerChangeStatus.NOT_SUPPORTED, raftActor.getLeaderId()),
                     raftActor.getSelf());
-        } else if(!isSelf && !raftContext.getPeerIds().contains(removeServer.getServerId())) {
+        } else if (!isSelf && !raftContext.getPeerIds().contains(removeServer.getServerId())) {
             sender.tell(new RemoveServerReply(ServerChangeStatus.DOES_NOT_EXIST, raftActor.getLeaderId()),
                     raftActor.getSelf());
         } else {
@@ -134,9 +138,9 @@ class RaftActorServerConfigurationSupport {
         }
     }
 
-    private boolean onApplyState(ApplyState applyState) {
+    private boolean onApplyState(final ApplyState applyState) {
         Payload data = applyState.getReplicatedLogEntry().getData();
-        if(data instanceof ServerConfigurationPayload) {
+        if (data instanceof ServerConfigurationPayload) {
             currentOperationState.onApplyState(applyState);
             return true;
         }
@@ -145,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>
@@ -165,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 {
@@ -198,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(),
@@ -207,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);
         }
 
@@ -223,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())));
@@ -241,8 +246,9 @@ class RaftActorServerConfigurationSupport {
             sendReply(operationContext, ServerChangeStatus.OK);
         }
 
-        protected void operationComplete(ServerOperationContext<?> operationContext, @Nullable ServerChangeStatus replyStatus) {
-            if(replyStatus != null) {
+        protected void operationComplete(final ServerOperationContext<?> operationContext,
+                final @Nullable ServerChangeStatus replyStatus) {
+            if (replyStatus != null) {
                 sendReply(operationContext, replyStatus);
             }
 
@@ -255,23 +261,24 @@ class RaftActorServerConfigurationSupport {
             currentOperationState = IDLE;
 
             ServerOperationContext<?> nextOperation = pendingOperationsQueue.poll();
-            if(nextOperation != null) {
+            if (nextOperation != null) {
                 RaftActorServerConfigurationSupport.this.onNewOperation(nextOperation);
             }
         }
 
-        protected void sendReply(ServerOperationContext<?> operationContext, ServerChangeStatus status) {
-            LOG.debug("{}: Returning {} for operation {}", raftContext.getId(), status, operationContext.getOperation());
+        protected void sendReply(final ServerOperationContext<?> operationContext, final 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) {
+        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());
@@ -288,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.
         }
     }
@@ -306,17 +313,17 @@ 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())) {
-                LOG.info("{}: {} has been successfully replicated to a majority of followers", raftActor.getId(),
+            if (operationContext.getContextId().equals(applyState.getIdentifier())) {
+                LOG.info("{}: {} has been successfully replicated to a majority of followers", raftContext.getId(),
                         applyState.getReplicatedLogEntry().getData());
 
                 timer.cancel();
@@ -325,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());
 
@@ -333,18 +340,18 @@ class RaftActorServerConfigurationSupport {
 
             // Fail any pending operations
             ServerOperationContext<?> nextOperation = pendingOperationsQueue.poll();
-            while(nextOperation != null) {
+            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);
+        public void onNewOperation(final ServerOperationContext<?> newOperationContext) {
+            if (timedOut) {
+                sendReply(newOperationContext, ServerChangeStatus.PRIOR_REQUEST_CONSENSUS_TIMEOUT);
             } else {
-                super.onNewOperation(operationContext);
+                super.onNewOperation(newOperationContext);
             }
         }
     }
@@ -355,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;
         }
 
@@ -367,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);
@@ -376,12 +383,13 @@ class RaftActorServerConfigurationSupport {
             raftContext.removePeer(serverId);
 
             boolean isLeader = raftActor.isLeader();
-            if(isLeader) {
+            if (isLeader) {
                 AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior();
                 leader.removeFollower(serverId);
             }
 
-            operationComplete(getAddServerContext(), isLeader ? ServerChangeStatus.TIMEOUT : ServerChangeStatus.NO_LEADER);
+            operationComplete(getAddServerContext(), isLeader ? ServerChangeStatus.TIMEOUT
+                    : ServerChangeStatus.NO_LEADER);
         }
 
     }
@@ -391,18 +399,18 @@ class RaftActorServerConfigurationSupport {
      * snapshot capture, if necessary.
      */
     private final class InitialAddServerState extends AddServerState implements InitialOperationState {
-        InitialAddServerState(AddServerContext addServerContext) {
+        InitialAddServerState(final AddServerContext addServerContext) {
             super(addServerContext);
         }
 
         @Override
         public void initiate() {
-            AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior();
+            final AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior();
             AddServer addServer = getAddServerContext().getOperation();
 
             LOG.debug("{}: Initiating {}", raftContext.getId(), addServer);
 
-            if(raftContext.getPeerInfo(addServer.getNewServerId()) != null) {
+            if (raftContext.getPeerInfo(addServer.getNewServerId()) != null) {
                 operationComplete(getAddServerContext(), ServerChangeStatus.ALREADY_EXISTS);
                 return;
             }
@@ -413,10 +421,10 @@ class RaftActorServerConfigurationSupport {
 
             leader.addFollower(addServer.getNewServerId());
 
-            if(votingState == VotingState.VOTING_NOT_INITIALIZED){
+            if (votingState == VotingState.VOTING_NOT_INITIALIZED) {
                 // schedule the install snapshot timeout timer
                 Cancellable installSnapshotTimer = newInstallSnapshotTimer();
-                if(leader.initiateCaptureSnapshot(addServer.getNewServerId())) {
+                if (leader.initiateCaptureSnapshot(addServer.getNewServerId())) {
                     LOG.debug("{}: Initiating capture snapshot for new server {}", raftContext.getId(),
                             addServer.getNewServerId());
 
@@ -443,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);
+            this.installSnapshotTimer = requireNonNull(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(),
@@ -457,14 +465,14 @@ class RaftActorServerConfigurationSupport {
         }
 
         @Override
-        public void onUnInitializedFollowerSnapshotReply(UnInitializedFollowerSnapshotReply reply) {
+        public void onUnInitializedFollowerSnapshotReply(final 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()) {
+            if (getAddServerContext().getOperation().getNewServerId().equals(followerId) && raftActor.isLeader()) {
                 AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior();
                 raftContext.getPeerInfo(followerId).setVotingState(VotingState.VOTING);
                 leader.updateMinReplicaCount();
@@ -487,22 +495,22 @@ 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);
+            this.snapshotTimer = requireNonNull(snapshotTimer);
         }
 
         @Override
         public void onSnapshotComplete() {
             LOG.debug("{}: onSnapshotComplete", raftContext.getId());
 
-            if(!raftActor.isLeader()) {
+            if (!raftActor.isLeader()) {
                 LOG.debug("{}: No longer the leader", raftContext.getId());
                 return;
             }
 
             AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior();
-            if(leader.initiateCaptureSnapshot(getAddServerContext().getOperation().getNewServerId())) {
+            if (leader.initiateCaptureSnapshot(getAddServerContext().getOperation().getNewServerId())) {
                 LOG.debug("{}: Initiating capture snapshot for new server {}", raftContext.getId(),
                         getAddServerContext().getOperation().getNewServerId());
 
@@ -514,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",
@@ -522,23 +530,32 @@ class RaftActorServerConfigurationSupport {
         }
     }
 
+    private static final class ServerOperationContextIdentifier
+            extends AbstractUUIDIdentifier<ServerOperationContextIdentifier> {
+        private static final long serialVersionUID = 1L;
+
+        ServerOperationContextIdentifier() {
+            super(UUID.randomUUID());
+        }
+    }
+
     /**
      * Stores context information for a server operation.
      *
      * @param <T> the operation type
      */
-    private static abstract class ServerOperationContext<T> {
+    private abstract static class ServerOperationContext<T> {
         private final T operation;
         private final ActorRef clientRequestor;
-        private final String contextId;
+        private final Identifier contextId;
 
-        ServerOperationContext(T operation, ActorRef clientRequestor){
+        ServerOperationContext(final T operation, final ActorRef clientRequestor) {
             this.operation = operation;
             this.clientRequestor = clientRequestor;
-            contextId = UUID.randomUUID().toString();
+            contextId = new ServerOperationContextIdentifier();
         }
 
-        String getContextId() {
+        Identifier getContextId() {
             return contextId;
         }
 
@@ -550,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;
         }
 
@@ -568,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);
         }
 
@@ -591,8 +608,8 @@ class RaftActorServerConfigurationSupport {
     private abstract class RemoveServerState extends OperationState {
         private final RemoveServerContext removeServerContext;
 
-        protected RemoveServerState(RemoveServerContext removeServerContext) {
-            this.removeServerContext = Preconditions.checkNotNull(removeServerContext);
+        protected RemoveServerState(final RemoveServerContext removeServerContext) {
+            this.removeServerContext = requireNonNull(removeServerContext);
 
         }
 
@@ -601,9 +618,9 @@ class RaftActorServerConfigurationSupport {
         }
     }
 
-    private final class InitialRemoveServerState extends RemoveServerState implements InitialOperationState{
+    private final class InitialRemoveServerState extends RemoveServerState implements InitialOperationState {
 
-        protected InitialRemoveServerState(RemoveServerContext removeServerContext) {
+        protected InitialRemoveServerState(final RemoveServerContext removeServerContext) {
             super(removeServerContext);
         }
 
@@ -611,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());
         }
@@ -620,30 +639,31 @@ 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) {
-            if(peerAddress != null) {
-                raftActor.context().actorSelection(peerAddress).tell(new ServerRemoved(getOperation().getServerId()), raftActor.getSelf());
+        void operationComplete(final RaftActor raftActor, final boolean succeeded) {
+            if (peerAddress != null) {
+                raftActor.context().actorSelection(peerAddress).tell(
+                        new ServerRemoved(getOperation().getServerId()), raftActor.getSelf());
             }
         }
 
         @Override
-        boolean includeSelfInNewConfiguration(RaftActor raftActor) {
+        boolean includeSelfInNewConfiguration(final RaftActor raftActor) {
             return !getOperation().getServerId().equals(raftActor.getId());
         }
 
@@ -656,50 +676,33 @@ 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().
-                    getServerVotingStatusMap().get(raftActor.getRaftActorContext().getId()));
-            if(succeeded && localServerChangedToNonVoting && raftActor.isLeader()) {
-                raftActor.initiateLeadershipTransfer(new RaftActorLeadershipTransferCohort.OnComplete() {
-                    @Override
-                    public void onSuccess(ActorRef raftActorRef, ActorRef replyTo) {
-                        LOG.debug("{}: leader transfer succeeded after change to non-voting", raftActor.persistenceId());
-                        ensureFollowerState(raftActor);
-                    }
-
-                    @Override
-                    public void onFailure(ActorRef raftActorRef, ActorRef replyTo) {
-                        LOG.debug("{}: leader transfer failed after change to non-voting", raftActor.persistenceId());
-                        ensureFollowerState(raftActor);
-                    }
-
-                    private void ensureFollowerState(RaftActor raftActor) {
-                        // Whether or not leadership transfer succeeded, we have to step down as leader and
-                        // switch to Follower so ensure that.
-                        if(raftActor.getRaftState() != RaftState.Follower) {
-                            raftActor.initializeBehavior();
-                        }
-                    }
-                });
+            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();
+            } else if (raftActor.isLeader()) {
+                raftActor.onVotingStateChangeComplete();
             }
         }
 
@@ -713,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;
         }
@@ -723,11 +726,9 @@ class RaftActorServerConfigurationSupport {
         public void initiate() {
             LOG.debug("Initiating ChangeServersVotingStatusState");
 
-            if(tryToElectLeader) {
+            if (tryToElectLeader) {
                 initiateLocalLeaderElection();
-            } else {
-                updateLocalPeerInfo();
-
+            } else if (updateLocalPeerInfo()) {
                 persistNewServerConfiguration(changeVotingStatusContext);
             }
         }
@@ -736,35 +737,55 @@ class RaftActorServerConfigurationSupport {
             LOG.debug("{}: Sending local ElectionTimeout to start leader election", raftContext.getId());
 
             ServerConfigurationPayload previousServerConfig = raftContext.getPeerServerInfo(true);
-            updateLocalPeerInfo();
+            if (!updateLocalPeerInfo()) {
+                return;
+            }
 
-            raftContext.getActor().tell(ElectionTimeout.INSTANCE, raftContext.getActor());
+            raftContext.getActor().tell(TimeoutNow.INSTANCE, raftContext.getActor());
 
             currentOperationState = new WaitingForLeaderElected(changeVotingStatusContext, previousServerConfig);
         }
 
-        private void updateLocalPeerInfo() {
-            List<ServerInfo> newServerInfoList = newServerInfoList();
+        private boolean updateLocalPeerInfo() {
+            final var 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();
+            if (raftActor.getCurrentBehavior() instanceof AbstractLeader leader) {
                 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()));
+        private ImmutableList<ServerInfo> newServerInfoList() {
+            final var serverVotingStatusMap = changeVotingStatusContext.getOperation().getServerVotingStatusMap();
+            final var peerInfos = raftContext.getPeers();
+            final var newServerInfoList = ImmutableList.<ServerInfo>builderWithExpectedSize(peerInfos.size() + 1);
+            for (var peerInfo : peerInfos) {
+                final var peerId = peerInfo.getId();
+                final var voting = serverVotingStatusMap.get(peerId);
+                newServerInfoList.add(new ServerInfo(peerId, voting != null ? voting : peerInfo.isVoting()));
             }
 
-            newServerInfoList.add(new ServerInfo(raftContext.getId(), serverVotingStatusMap.containsKey(
-                    raftContext.getId()) ? serverVotingStatusMap.get(raftContext.getId()) : raftContext.isVotingMember()));
+            final var myId = raftContext.getId();
+            final var myVoting = serverVotingStatusMap.get(myId);
+            newServerInfoList.add(new ServerInfo(myId, myVoting != null ? myVoting : raftContext.isVotingMember()));
 
-            return newServerInfoList;
+            return newServerInfoList.build();
         }
     }
 
@@ -773,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;
 
@@ -783,12 +804,16 @@ class RaftActorServerConfigurationSupport {
         }
 
         @Override
-        void onNewLeader(String newLeader) {
+        void onNewLeader(final String newLeader) {
+            if (newLeader == null) {
+                return;
+            }
+
             LOG.debug("{}: New leader {} elected", raftContext.getId(), newLeader);
 
             timer.cancel();
 
-            if(raftActor.isLeader()) {
+            if (raftActor.isLeader()) {
                 persistNewServerConfiguration(operationContext);
             } else {
                 // Edge case - some other node became leader so forward the operation.
@@ -803,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());
 
@@ -826,20 +851,20 @@ class RaftActorServerConfigurationSupport {
             // tried yet.
             Map<String, Boolean> serverVotingStatusMap = operationContext.getOperation().getServerVotingStatusMap();
             ActorSelection forwardToPeerActor = null;
-            for(Map.Entry<String, Boolean> e: serverVotingStatusMap.entrySet()) {
+            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)) {
+                if (isVoting && peerInfo != null && !peerInfo.isVoting() && !serversVisited.contains(serverId)) {
                     ActorSelection actor = raftContext.getPeerActorSelection(serverId);
-                    if(actor != null) {
+                    if (actor != null) {
                         forwardToPeerActor = actor;
                         break;
                     }
                 }
             }
 
-            if(forwardToPeerActor != null) {
+            if (forwardToPeerActor != null) {
                 LOG.debug("{}: Found server {} to forward to", raftContext.getId(), forwardToPeerActor);
 
                 forwardToPeerActor.tell(new ChangeServersVotingStatus(serverVotingStatusMap, serversVisited),
@@ -854,8 +879,8 @@ class RaftActorServerConfigurationSupport {
     static class ServerOperationTimeout {
         private final String loggingContext;
 
-        ServerOperationTimeout(String loggingContext){
-           this.loggingContext = Preconditions.checkNotNull(loggingContext, "loggingContext should not be null");
+        ServerOperationTimeout(final String loggingContext) {
+            this.loggingContext = requireNonNull(loggingContext, "loggingContext should not be null");
         }
 
         String getLoggingContext() {