Fix remaining CS errors in sal-akka-raft and enable enforcement
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / RaftActorServerConfigurationSupport.java
index 16c091a4c3490c56ff7e10c3038d32f86cca2f84..782ecc06a87ac11b19c6ef6b8335e6a9aa820b1a 100644 (file)
@@ -50,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;
@@ -66,13 +67,13 @@ class RaftActorServerConfigurationSupport {
     }
 
     boolean handleMessage(Object message, ActorRef sender) {
-        if(message instanceof AddServer) {
+        if (message instanceof AddServer) {
             onAddServer((AddServer) message, sender);
             return true;
-        } else if(message instanceof RemoveServer) {
+        } else if (message instanceof RemoveServer) {
             onRemoveServer((RemoveServer) message, sender);
             return true;
-        } else if(message instanceof ChangeServersVotingStatus) {
+        } else if (message instanceof ChangeServersVotingStatus) {
             onChangeServersVotingStatus((ChangeServersVotingStatus) message, sender);
             return true;
         } else if (message instanceof ServerOperationTimeout) {
@@ -81,9 +82,9 @@ class RaftActorServerConfigurationSupport {
         } else if (message instanceof UnInitializedFollowerSnapshotReply) {
             currentOperationState.onUnInitializedFollowerSnapshotReply((UnInitializedFollowerSnapshotReply) message);
             return true;
-        } else if(message instanceof ApplyState) {
+        } else if (message instanceof ApplyState) {
             return onApplyState((ApplyState) message);
-        } else if(message instanceof SnapshotComplete) {
+        } else if (message instanceof SnapshotComplete) {
             currentOperationState.onSnapshotComplete();
             return false;
         } else {
@@ -111,10 +112,10 @@ 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));
@@ -124,10 +125,10 @@ class RaftActorServerConfigurationSupport {
     private void onRemoveServer(RemoveServer removeServer, ActorRef sender) {
         LOG.debug("{}: onRemoveServer: {}, state: {}", raftContext.getId(), removeServer, currentOperationState);
         boolean isSelf = removeServer.getServerId().equals(raftContext.getId());
-        if(isSelf && !raftContext.hasFollowers()) {
+        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 {
@@ -139,7 +140,7 @@ class RaftActorServerConfigurationSupport {
 
     private boolean onApplyState(ApplyState applyState) {
         Payload data = applyState.getReplicatedLogEntry().getData();
-        if(data instanceof ServerConfigurationPayload) {
+        if (data instanceof ServerConfigurationPayload) {
             currentOperationState.onApplyState(applyState);
             return true;
         }
@@ -229,7 +230,7 @@ class RaftActorServerConfigurationSupport {
         void onNewLeader(String newLeader) {
         }
 
-        protected void persistNewServerConfiguration(ServerOperationContext<?> operationContext){
+        protected void persistNewServerConfiguration(ServerOperationContext<?> operationContext) {
             raftContext.setDynamicServerConfigurationInUse();
 
             ServerConfigurationPayload payload = raftContext.getPeerServerInfo(
@@ -244,8 +245,9 @@ class RaftActorServerConfigurationSupport {
             sendReply(operationContext, ServerChangeStatus.OK);
         }
 
-        protected void operationComplete(ServerOperationContext<?> operationContext, @Nullable ServerChangeStatus replyStatus) {
-            if(replyStatus != null) {
+        protected void operationComplete(ServerOperationContext<?> operationContext,
+                @Nullable ServerChangeStatus replyStatus) {
+            if (replyStatus != null) {
                 sendReply(operationContext, replyStatus);
             }
 
@@ -258,13 +260,14 @@ 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());
+            LOG.debug("{}: Returning {} for operation {}", raftContext.getId(), status,
+                    operationContext.getOperation());
 
             operationContext.getClientRequestor().tell(operationContext.newReply(status, raftActor.getLeaderId()),
                     raftActor.self());
@@ -318,7 +321,7 @@ class RaftActorServerConfigurationSupport {
         public void onApplyState(ApplyState applyState) {
             // Sanity check - we could get an ApplyState from a previous operation that timed out so make
             // sure it's meant for us.
-            if(operationContext.getContextId().equals(applyState.getIdentifier())) {
+            if (operationContext.getContextId().equals(applyState.getIdentifier())) {
                 LOG.info("{}: {} has been successfully replicated to a majority of followers", raftContext.getId(),
                         applyState.getReplicatedLogEntry().getData());
 
@@ -336,7 +339,7 @@ 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();
             }
@@ -344,7 +347,7 @@ class RaftActorServerConfigurationSupport {
 
         @Override
         public void onNewOperation(ServerOperationContext<?> newOperationContext) {
-            if(timedOut) {
+            if (timedOut) {
                 sendReply(newOperationContext, ServerChangeStatus.PRIOR_REQUEST_CONSENSUS_TIMEOUT);
             } else {
                 super.onNewOperation(newOperationContext);
@@ -379,12 +382,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);
         }
 
     }
@@ -400,12 +404,12 @@ class RaftActorServerConfigurationSupport {
 
         @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;
             }
@@ -416,10 +420,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());
 
@@ -467,7 +471,7 @@ 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) && raftActor.isLeader()) {
+            if (getAddServerContext().getOperation().getNewServerId().equals(followerId) && raftActor.isLeader()) {
                 AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior();
                 raftContext.getPeerInfo(followerId).setVotingState(VotingState.VOTING);
                 leader.updateMinReplicaCount();
@@ -499,13 +503,13 @@ class RaftActorServerConfigurationSupport {
         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());
 
@@ -525,7 +529,8 @@ class RaftActorServerConfigurationSupport {
         }
     }
 
-    private static final class ServerOperationContextIdentifier extends AbstractUUIDIdentifier<ServerOperationContextIdentifier> {
+    private static final class ServerOperationContextIdentifier
+            extends AbstractUUIDIdentifier<ServerOperationContextIdentifier> {
         private static final long serialVersionUID = 1L;
 
         ServerOperationContextIdentifier() {
@@ -543,7 +548,7 @@ class RaftActorServerConfigurationSupport {
         private final ActorRef clientRequestor;
         private final Identifier contextId;
 
-        ServerOperationContext(T operation, ActorRef clientRequestor){
+        ServerOperationContext(T operation, ActorRef clientRequestor) {
             this.operation = operation;
             this.clientRequestor = clientRequestor;
             contextId = new ServerOperationContextIdentifier();
@@ -612,7 +617,7 @@ class RaftActorServerConfigurationSupport {
         }
     }
 
-    private final class InitialRemoveServerState extends RemoveServerState implements InitialOperationState{
+    private final class InitialRemoveServerState extends RemoveServerState implements InitialOperationState {
 
         protected InitialRemoveServerState(RemoveServerContext removeServerContext) {
             super(removeServerContext);
@@ -648,8 +653,9 @@ class RaftActorServerConfigurationSupport {
 
         @Override
         void operationComplete(RaftActor raftActor, boolean succeeded) {
-            if(peerAddress != null) {
-                raftActor.context().actorSelection(peerAddress).tell(new ServerRemoved(getOperation().getServerId()), raftActor.getSelf());
+            if (peerAddress != null) {
+                raftActor.context().actorSelection(peerAddress).tell(
+                        new ServerRemoved(getOperation().getServerId()), raftActor.getSelf());
             }
         }
 
@@ -687,8 +693,8 @@ class RaftActorServerConfigurationSupport {
         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()));
+            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();
@@ -715,9 +721,9 @@ class RaftActorServerConfigurationSupport {
         public void initiate() {
             LOG.debug("Initiating ChangeServersVotingStatusState");
 
-            if(tryToElectLeader) {
+            if (tryToElectLeader) {
                 initiateLocalLeaderElection();
-            } else if(updateLocalPeerInfo()) {
+            } else if (updateLocalPeerInfo()) {
                 persistNewServerConfiguration(changeVotingStatusContext);
             }
         }
@@ -726,7 +732,7 @@ class RaftActorServerConfigurationSupport {
             LOG.debug("{}: Sending local ElectionTimeout to start leader election", raftContext.getId());
 
             ServerConfigurationPayload previousServerConfig = raftContext.getPeerServerInfo(true);
-            if(!updateLocalPeerInfo()) {
+            if (!updateLocalPeerInfo()) {
                 return;
             }
 
@@ -740,20 +746,20 @@ class RaftActorServerConfigurationSupport {
 
             // Check if new voting state would leave us with no voting members.
             boolean atLeastOneVoting = false;
-            for(ServerInfo info: newServerInfoList) {
-                if(info.isVoting()) {
+            for (ServerInfo info: newServerInfoList) {
+                if (info.isVoting()) {
                     atLeastOneVoting = true;
                     break;
                 }
             }
 
-            if(!atLeastOneVoting) {
+            if (!atLeastOneVoting) {
                 operationComplete(changeVotingStatusContext, ServerChangeStatus.INVALID_REQUEST);
                 return false;
             }
 
             raftContext.updatePeerIds(new ServerConfigurationPayload(newServerInfoList));
-            if(raftActor.getCurrentBehavior() instanceof AbstractLeader) {
+            if (raftActor.getCurrentBehavior() instanceof AbstractLeader) {
                 AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior();
                 leader.updateMinReplicaCount();
             }
@@ -762,15 +768,17 @@ class RaftActorServerConfigurationSupport {
         }
 
         private List<ServerInfo> newServerInfoList() {
-            Map<String, Boolean> serverVotingStatusMap = changeVotingStatusContext.getOperation().getServerVotingStatusMap();
+            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()));
+            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()));
+                    raftContext.getId()) ? serverVotingStatusMap.get(raftContext.getId())
+                            : raftContext.isVotingMember()));
 
             return newServerInfoList;
         }
@@ -792,7 +800,7 @@ class RaftActorServerConfigurationSupport {
 
         @Override
         void onNewLeader(String newLeader) {
-            if(newLeader == null) {
+            if (newLeader == null) {
                 return;
             }
 
@@ -800,7 +808,7 @@ class RaftActorServerConfigurationSupport {
 
             timer.cancel();
 
-            if(raftActor.isLeader()) {
+            if (raftActor.isLeader()) {
                 persistNewServerConfiguration(operationContext);
             } else {
                 // Edge case - some other node became leader so forward the operation.
@@ -838,20 +846,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),
@@ -866,8 +874,8 @@ class RaftActorServerConfigurationSupport {
     static class ServerOperationTimeout {
         private final String loggingContext;
 
-        ServerOperationTimeout(String loggingContext){
-           this.loggingContext = Preconditions.checkNotNull(loggingContext, "loggingContext should not be null");
+        ServerOperationTimeout(String loggingContext) {
+            this.loggingContext = Preconditions.checkNotNull(loggingContext, "loggingContext should not be null");
         }
 
         String getLoggingContext() {