You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@qpid.apache.org by kw...@apache.org on 2014/09/01 15:41:34 UTC

svn commit: r1621775 - in /qpid/trunk/qpid/java/bdbstore: jmx/src/main/java/org/apache/qpid/server/store/berkeleydb/jmx/ jmx/src/test/java/org/apache/qpid/server/store/berkeleydb/jmx/ src/main/java/org/apache/qpid/server/virtualhostnode/berkeleydb/ src...

Author: kwall
Date: Mon Sep  1 13:41:33 2014
New Revision: 1621775

URL: http://svn.apache.org/r1621775
Log:
QPID-6063: [Java Broker] Disambiguate the concept of a node being unreachable from a node awaiting election

Added:
    qpid/trunk/qpid/java/bdbstore/src/main/java/org/apache/qpid/server/virtualhostnode/berkeleydb/NodeRole.java
Modified:
    qpid/trunk/qpid/java/bdbstore/jmx/src/main/java/org/apache/qpid/server/store/berkeleydb/jmx/BDBHAMessageStoreManagerMBean.java
    qpid/trunk/qpid/java/bdbstore/jmx/src/test/java/org/apache/qpid/server/store/berkeleydb/jmx/BDBHAMessageStoreManagerMBeanTest.java
    qpid/trunk/qpid/java/bdbstore/src/main/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHARemoteReplicationNode.java
    qpid/trunk/qpid/java/bdbstore/src/main/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHARemoteReplicationNodeImpl.java
    qpid/trunk/qpid/java/bdbstore/src/main/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHAVirtualHostNode.java
    qpid/trunk/qpid/java/bdbstore/src/main/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHAVirtualHostNodeImpl.java
    qpid/trunk/qpid/java/bdbstore/src/main/java/resources/js/qpid/management/virtualhostnode/bdb_ha/show.js
    qpid/trunk/qpid/java/bdbstore/src/test/java/org/apache/qpid/server/store/berkeleydb/BDBHAVirtualHostNodeTest.java
    qpid/trunk/qpid/java/bdbstore/src/test/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHARemoteReplicationNodeTest.java
    qpid/trunk/qpid/java/bdbstore/src/test/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHAVirtualHostNodeOperationalLoggingTest.java
    qpid/trunk/qpid/java/bdbstore/src/test/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHAVirtualHostNodeTestHelper.java
    qpid/trunk/qpid/java/bdbstore/systests/src/test/java/org/apache/qpid/server/store/berkeleydb/replication/BDBHAVirtualHostNodeRestTest.java

Modified: qpid/trunk/qpid/java/bdbstore/jmx/src/main/java/org/apache/qpid/server/store/berkeleydb/jmx/BDBHAMessageStoreManagerMBean.java
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/java/bdbstore/jmx/src/main/java/org/apache/qpid/server/store/berkeleydb/jmx/BDBHAMessageStoreManagerMBean.java?rev=1621775&r1=1621774&r2=1621775&view=diff
==============================================================================
--- qpid/trunk/qpid/java/bdbstore/jmx/src/main/java/org/apache/qpid/server/store/berkeleydb/jmx/BDBHAMessageStoreManagerMBean.java (original)
+++ qpid/trunk/qpid/java/bdbstore/jmx/src/main/java/org/apache/qpid/server/store/berkeleydb/jmx/BDBHAMessageStoreManagerMBean.java Mon Sep  1 13:41:33 2014
@@ -149,7 +149,7 @@ public class BDBHAMessageStoreManagerMBe
     {
         try
         {
-            return _virtualHostNode.getRole();
+            return _virtualHostNode.getRole().name();
         }
         catch (RuntimeException e)
         {

Modified: qpid/trunk/qpid/java/bdbstore/jmx/src/test/java/org/apache/qpid/server/store/berkeleydb/jmx/BDBHAMessageStoreManagerMBeanTest.java
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/java/bdbstore/jmx/src/test/java/org/apache/qpid/server/store/berkeleydb/jmx/BDBHAMessageStoreManagerMBeanTest.java?rev=1621775&r1=1621774&r2=1621775&view=diff
==============================================================================
--- qpid/trunk/qpid/java/bdbstore/jmx/src/test/java/org/apache/qpid/server/store/berkeleydb/jmx/BDBHAMessageStoreManagerMBeanTest.java (original)
+++ qpid/trunk/qpid/java/bdbstore/jmx/src/test/java/org/apache/qpid/server/store/berkeleydb/jmx/BDBHAMessageStoreManagerMBeanTest.java Mon Sep  1 13:41:33 2014
@@ -44,6 +44,7 @@ import org.apache.qpid.server.model.Remo
 import org.apache.qpid.server.virtualhost.berkeleydb.BDBHAVirtualHost;
 import org.apache.qpid.server.virtualhostnode.berkeleydb.BDBHARemoteReplicationNode;
 import org.apache.qpid.server.virtualhostnode.berkeleydb.BDBHAVirtualHostNode;
+import org.apache.qpid.server.virtualhostnode.berkeleydb.NodeRole;
 
 public class BDBHAMessageStoreManagerMBeanTest extends TestCase
 {
@@ -53,7 +54,7 @@ public class BDBHAMessageStoreManagerMBe
     private static final String TEST_NODE_HOST_PORT = "host:1234";
     private static final String TEST_HELPER_HOST_PORT = "host:5678";
     private static final String TEST_DURABILITY = "sync,sync,all";
-    private static final String TEST_NODE_STATE = "MASTER";
+    private static final NodeRole TEST_NODE_ROLE = NodeRole.MASTER;
     private static final boolean TEST_DESIGNATED_PRIMARY_FLAG = false;
 
     private BDBHAVirtualHostNode<?> _virtualHostNode;
@@ -133,9 +134,9 @@ public class BDBHAMessageStoreManagerMBe
 
     public void testNodeState() throws Exception
     {
-        when(_virtualHostNode.getRole()).thenReturn(TEST_NODE_STATE);
+        when(_virtualHostNode.getRole()).thenReturn(TEST_NODE_ROLE);
 
-        assertEquals(TEST_NODE_STATE, _mBean.getAttribute(ManagedBDBHAMessageStore.ATTR_NODE_STATE));
+        assertEquals(TEST_NODE_ROLE.name(), _mBean.getAttribute(ManagedBDBHAMessageStore.ATTR_NODE_STATE));
     }
 
     public void testDesignatedPrimaryFlag() throws Exception

Modified: qpid/trunk/qpid/java/bdbstore/src/main/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHARemoteReplicationNode.java
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/java/bdbstore/src/main/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHARemoteReplicationNode.java?rev=1621775&r1=1621774&r2=1621775&view=diff
==============================================================================
--- qpid/trunk/qpid/java/bdbstore/src/main/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHARemoteReplicationNode.java (original)
+++ qpid/trunk/qpid/java/bdbstore/src/main/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHARemoteReplicationNode.java Mon Sep  1 13:41:33 2014
@@ -42,7 +42,7 @@ public interface BDBHARemoteReplicationN
     String getAddress();
 
     @ManagedAttribute(persist = false)
-    String getRole();
+    NodeRole getRole();
 
     @DerivedAttribute
     long getJoinTime();

Modified: qpid/trunk/qpid/java/bdbstore/src/main/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHARemoteReplicationNodeImpl.java
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/java/bdbstore/src/main/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHARemoteReplicationNodeImpl.java?rev=1621775&r1=1621774&r2=1621775&view=diff
==============================================================================
--- qpid/trunk/qpid/java/bdbstore/src/main/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHARemoteReplicationNodeImpl.java (original)
+++ qpid/trunk/qpid/java/bdbstore/src/main/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHARemoteReplicationNodeImpl.java Mon Sep  1 13:41:33 2014
@@ -21,16 +21,12 @@
 
 package org.apache.qpid.server.virtualhostnode.berkeleydb;
 
-import static com.sleepycat.je.rep.ReplicatedEnvironment.State.MASTER;
-import static com.sleepycat.je.rep.ReplicatedEnvironment.State.REPLICA;
-
 import java.security.AccessControlException;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicReference;
 
 import com.sleepycat.je.rep.MasterStateException;
-import com.sleepycat.je.rep.ReplicatedEnvironment;
 
 import org.apache.log4j.Logger;
 import org.apache.qpid.server.configuration.IllegalConfigurationException;
@@ -60,16 +56,16 @@ public class BDBHARemoteReplicationNodeI
 
     private volatile long _joinTime;
     private volatile long _lastTransactionId;
-    private volatile String _lastReplicatedEnvironmentState = ReplicatedEnvironment.State.UNKNOWN.name();
 
     @ManagedAttributeField(afterSet="afterSetRole")
-    private volatile String _role = ReplicatedEnvironment.State.UNKNOWN.name();
+    private volatile NodeRole _role;
 
     private final AtomicReference<State> _state;
     private final boolean _isMonitor;
     private boolean _detached;
     private BDBHAVirtualHostNodeLogSubject _virtualHostNodeLogSubject;
     private GroupLogSubject _groupLogSubject;
+    private volatile NodeRole _lastKnownRole;
 
     public BDBHARemoteReplicationNodeImpl(BDBHAVirtualHostNode<?> virtualHostNode, Map<String, Object> attributes, ReplicatedEnvironmentFacade replicatedEnvironmentFacade)
     {
@@ -77,7 +73,11 @@ public class BDBHARemoteReplicationNodeI
         _broker = virtualHostNode.getParent(Broker.class);
         _address = (String)attributes.get(ADDRESS);
         _replicatedEnvironmentFacade = replicatedEnvironmentFacade;
-        _state = new AtomicReference<State>(State.ACTIVE);
+        _state = new AtomicReference<>(State.ACTIVE);
+
+        _role = NodeRole.UNREACHABLE;
+        _lastKnownRole = NodeRole.UNREACHABLE;
+
         _isMonitor = (Boolean)attributes.get(MONITOR);
     }
 
@@ -100,9 +100,9 @@ public class BDBHARemoteReplicationNodeI
     }
 
     @Override
-    public String getRole()
+    public NodeRole getRole()
     {
-        return _lastReplicatedEnvironmentState;
+        return _lastKnownRole;
     }
 
     @Override
@@ -201,15 +201,16 @@ public class BDBHARemoteReplicationNodeI
         super.validateChange(proxyForValidation, changedAttributes);
         if (changedAttributes.contains(ROLE))
         {
-            String currentRole = getRole();
-            if (!REPLICA.name().equals(currentRole))
+            NodeRole currentRole = getRole();
+            if (NodeRole.REPLICA != currentRole)
             {
                 throw new IllegalArgumentException("Cannot transfer mastership when not in replica role."
                                                  + " Current role " + currentRole);
             }
-            if (!MASTER.name().equals(((BDBHARemoteReplicationNode<?>)proxyForValidation).getRole()))
+            NodeRole newRole = (NodeRole) ((BDBHARemoteReplicationNode<?>) proxyForValidation).getAttribute(ROLE);
+            if (NodeRole.MASTER != newRole)
             {
-                throw new IllegalArgumentException("Changing role to other value then " + MASTER.name() + " is unsupported");
+                throw new IllegalArgumentException("Changing role to other value then " + NodeRole.MASTER + " is unsupported");
             }
         }
 
@@ -224,9 +225,9 @@ public class BDBHARemoteReplicationNodeI
         }
     }
 
-    void setRole(String role)
+    void setRole(NodeRole role)
     {
-        _lastReplicatedEnvironmentState = role;
+        _lastKnownRole = role;
         _role = role;
         updateModelStateFromRole(role);
     }
@@ -241,7 +242,7 @@ public class BDBHARemoteReplicationNodeI
         _lastTransactionId = lastTransactionId;
     }
 
-    private void updateModelStateFromRole(final String role)
+    private void updateModelStateFromRole(NodeRole role)
     {
         State currentState = _state.get();
         if (currentState == State.DELETED)
@@ -249,7 +250,7 @@ public class BDBHARemoteReplicationNodeI
             return;
         }
 
-        boolean isActive = MASTER.name().equals(role) || REPLICA.name().equals(role);
+        boolean isActive = NodeRole.MASTER == role || NodeRole.REPLICA == role;
         _state.compareAndSet(currentState, isActive ? State.ACTIVE : State.UNAVAILABLE);
     }
 
@@ -260,7 +261,7 @@ public class BDBHARemoteReplicationNodeI
 
     public void setDetached(boolean detached)
     {
-        this._detached = detached;
+        _detached = detached;
     }
 
     @Override

Modified: qpid/trunk/qpid/java/bdbstore/src/main/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHAVirtualHostNode.java
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/java/bdbstore/src/main/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHAVirtualHostNode.java?rev=1621775&r1=1621774&r2=1621775&view=diff
==============================================================================
--- qpid/trunk/qpid/java/bdbstore/src/main/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHAVirtualHostNode.java (original)
+++ qpid/trunk/qpid/java/bdbstore/src/main/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHAVirtualHostNode.java Mon Sep  1 13:41:33 2014
@@ -60,8 +60,8 @@ public interface BDBHAVirtualHostNode<X 
     @ManagedAttribute(defaultValue = "0")
     int getQuorumOverride();
 
-    @ManagedAttribute(persist = false, defaultValue = "UNKNOWN")
-    String getRole();
+    @ManagedAttribute(persist = false, defaultValue = "WAITING")
+    NodeRole getRole();
 
     @DerivedAttribute
     Long getLastKnownReplicationTransactionId();

Modified: qpid/trunk/qpid/java/bdbstore/src/main/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHAVirtualHostNodeImpl.java
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/java/bdbstore/src/main/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHAVirtualHostNodeImpl.java?rev=1621775&r1=1621774&r2=1621775&view=diff
==============================================================================
--- qpid/trunk/qpid/java/bdbstore/src/main/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHAVirtualHostNodeImpl.java (original)
+++ qpid/trunk/qpid/java/bdbstore/src/main/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHAVirtualHostNodeImpl.java Mon Sep  1 13:41:33 2014
@@ -95,7 +95,7 @@ public class BDBHAVirtualHostNodeImpl ex
 
     private final AtomicReference<ReplicatedEnvironmentFacade> _environmentFacade = new AtomicReference<>();
 
-    private final AtomicReference<ReplicatedEnvironment.State> _lastReplicatedEnvironmentState = new AtomicReference<>(ReplicatedEnvironment.State.UNKNOWN);
+    private final AtomicReference<NodeRole> _lastReplicatedEnvironmentState = new AtomicReference<>(NodeRole.WAITING);
     private BDBHAVirtualHostNodeLogSubject _virtualHostNodeLogSubject;
     private GroupLogSubject _groupLogSubject;
     private String _virtualHostNodePrincipalName;
@@ -123,7 +123,7 @@ public class BDBHAVirtualHostNodeImpl ex
     private int _quorumOverride;
 
     @ManagedAttributeField(afterSet="postSetRole")
-    private String _role;
+    private NodeRole _role;
 
     @ManagedAttributeField
     private String _helperNodeName;
@@ -145,15 +145,14 @@ public class BDBHAVirtualHostNodeImpl ex
 
         if (changedAttributes.contains(ROLE))
         {
-            String currentRole = getRole();
-            if (!ReplicatedEnvironment.State.REPLICA.name().equals(currentRole))
+            NodeRole currentRole = getRole();
+            if (NodeRole.REPLICA != currentRole)
             {
-                throw new IllegalStateException("Cannot transfer mastership when not a replica, current role is " + currentRole);
+                throw new IllegalStateException("Cannot transfer mastership when not a " + NodeRole.REPLICA + ", current role is " + currentRole);
             }
-
-            if (!ReplicatedEnvironment.State.MASTER.name().equals(proposed.getRole()))
+            if (NodeRole.MASTER != proposed.getAttribute(ROLE))
             {
-                throw new IllegalArgumentException("Changing role to other value then " + ReplicatedEnvironment.State.MASTER.name() + " is unsupported");
+                throw new IllegalArgumentException("Changing role to other value then " + NodeRole.MASTER + " is unsupported");
             }
         }
 
@@ -206,9 +205,9 @@ public class BDBHAVirtualHostNodeImpl ex
     }
 
     @Override
-    public String getRole()
+    public NodeRole getRole()
     {
-        return _lastReplicatedEnvironmentState.get().name();
+        return _lastReplicatedEnvironmentState.get();
     }
 
     @Override
@@ -350,9 +349,8 @@ public class BDBHAVirtualHostNodeImpl ex
 
             // closing the environment does not cause a state change.  Adjust the role
             // so that our observers will see DETACHED rather than our previous role in the group.
-            ReplicatedEnvironment.State detached = ReplicatedEnvironment.State.DETACHED;
-            _lastReplicatedEnvironmentState.set(detached);
-            attributeSet(ROLE, _role, detached);
+            _lastReplicatedEnvironmentState.set(NodeRole.DETACHED);
+            attributeSet(ROLE, _role, NodeRole.DETACHED);
         }
     }
 
@@ -576,7 +574,7 @@ public class BDBHAVirtualHostNodeImpl ex
             {
                 LOGGER.info("Received BDB event indicating transition to state " + state);
             }
-            String previousRole = getRole();
+            NodeRole previousRole = getRole();
             try
             {
                 switch (state)
@@ -598,10 +596,11 @@ public class BDBHAVirtualHostNodeImpl ex
             }
             finally
             {
-                _lastReplicatedEnvironmentState.set(state);
+                NodeRole newRole = NodeRole.fromJeState(state);
+                _lastReplicatedEnvironmentState.set(newRole);
                 attributeSet(ROLE, _role, state.name());
                 getEventLogger().message(getGroupLogSubject(),
-                        HighAvailabilityMessages.ROLE_CHANGED(getName(), getAddress(), previousRole, state.name()));
+                        HighAvailabilityMessages.ROLE_CHANGED(getName(), getAddress(), previousRole.name(), newRole.name()));
             }
         }
     }
@@ -868,10 +867,10 @@ public class BDBHAVirtualHostNodeImpl ex
             BDBHARemoteReplicationNodeImpl remoteNode = getChildByName(BDBHARemoteReplicationNodeImpl.class, node.getName());
             if (remoteNode != null)
             {
-                String currentRole = remoteNode.getRole();
+                NodeRole currentRole = remoteNode.getRole();
                 if (nodeState == null)
                 {
-                    remoteNode.setRole(ReplicatedEnvironment.State.UNKNOWN.name());
+                    remoteNode.setRole(NodeRole.UNREACHABLE);
                     remoteNode.setLastTransactionId(-1);
                     if (!remoteNode.isDetached())
                     {
@@ -883,7 +882,8 @@ public class BDBHAVirtualHostNodeImpl ex
                 {
                     remoteNode.setJoinTime(nodeState.getJoinTime());
                     remoteNode.setLastTransactionId(nodeState.getCurrentTxnEndVLSN());
-                    remoteNode.setRole(nodeState.getNodeState().name());
+                    ReplicatedEnvironment.State state = nodeState.getNodeState();
+                    remoteNode.setRole(NodeRole.fromJeState(state));
                     if (remoteNode.isDetached())
                     {
                         getEventLogger().message(getGroupLogSubject(), HighAvailabilityMessages.JOINED(remoteNode.getName(), remoteNode.getAddress()));
@@ -908,10 +908,10 @@ public class BDBHAVirtualHostNodeImpl ex
                     }
                 }
 
-                String newRole = remoteNode.getRole();
-                if (!newRole.equals(currentRole))
+                NodeRole newRole = remoteNode.getRole();
+                if (newRole != currentRole)
                 {
-                    getEventLogger().message(getGroupLogSubject(), HighAvailabilityMessages.ROLE_CHANGED(remoteNode.getName(), remoteNode.getAddress(), currentRole, newRole));
+                    getEventLogger().message(getGroupLogSubject(), HighAvailabilityMessages.ROLE_CHANGED(remoteNode.getName(), remoteNode.getAddress(), currentRole.name(), newRole.name()));
                 }
             }
         }

Added: qpid/trunk/qpid/java/bdbstore/src/main/java/org/apache/qpid/server/virtualhostnode/berkeleydb/NodeRole.java
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/java/bdbstore/src/main/java/org/apache/qpid/server/virtualhostnode/berkeleydb/NodeRole.java?rev=1621775&view=auto
==============================================================================
--- qpid/trunk/qpid/java/bdbstore/src/main/java/org/apache/qpid/server/virtualhostnode/berkeleydb/NodeRole.java (added)
+++ qpid/trunk/qpid/java/bdbstore/src/main/java/org/apache/qpid/server/virtualhostnode/berkeleydb/NodeRole.java Mon Sep  1 13:41:33 2014
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.qpid.server.virtualhostnode.berkeleydb;
+
+import com.sleepycat.je.rep.ReplicatedEnvironment.State;
+
+public enum NodeRole
+{
+    /** Node is master. */
+    MASTER,
+    /** Node is replica. */
+    REPLICA,
+    /** Node is awaiting an election result, or may be awaiting more nodes to join in order that an election may be held.  */
+    WAITING,
+    /**
+     * (Remote) node is unreachable.  Its virtual host node may be stopped, its Broker down,  or a network problem may
+     * be preventing it from being contacted.
+     */
+    UNREACHABLE,
+    /** (Local) node is not connected to the group */
+    DETACHED;
+
+    public static NodeRole fromJeState(final State state)
+    {
+        switch(state)
+        {
+            case DETACHED:
+                return DETACHED;
+            case UNKNOWN:
+                return WAITING;
+            case MASTER:
+                return MASTER;
+            case REPLICA:
+                return REPLICA;
+            default:
+                throw new IllegalArgumentException("Unrecognised JE node state " + state);
+        }
+    }
+}

Modified: qpid/trunk/qpid/java/bdbstore/src/main/java/resources/js/qpid/management/virtualhostnode/bdb_ha/show.js
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/java/bdbstore/src/main/java/resources/js/qpid/management/virtualhostnode/bdb_ha/show.js?rev=1621775&r1=1621774&r2=1621775&view=diff
==============================================================================
--- qpid/trunk/qpid/java/bdbstore/src/main/java/resources/js/qpid/management/virtualhostnode/bdb_ha/show.js (original)
+++ qpid/trunk/qpid/java/bdbstore/src/main/java/resources/js/qpid/management/virtualhostnode/bdb_ha/show.js Mon Sep  1 13:41:33 2014
@@ -96,8 +96,8 @@ define(["dojo/_base/xhr",
           findNode("groupMembers", containerNode),
           [
            { name: 'Name', field: 'name', width: '10%' },
-           { name: 'Role', field: 'role', width: '10%' },
-           { name: 'Address', field: 'address', width: '35%' },
+           { name: 'Role', field: 'role', width: '15%' },
+           { name: 'Address', field: 'address', width: '30%' },
            { name: 'Join Time', field: 'joinTime', width: '25%', formatter: function(value){ return value ? UserPreferences.formatDateTime(value) : "";} },
            { name: 'Replication Transaction ID', field: 'lastKnownReplicationTransactionId', width: '20%', formatter: function(value){ return value > 0 ? value : "N/A";} }
           ],

Modified: qpid/trunk/qpid/java/bdbstore/src/test/java/org/apache/qpid/server/store/berkeleydb/BDBHAVirtualHostNodeTest.java
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/java/bdbstore/src/test/java/org/apache/qpid/server/store/berkeleydb/BDBHAVirtualHostNodeTest.java?rev=1621775&r1=1621774&r2=1621775&view=diff
==============================================================================
--- qpid/trunk/qpid/java/bdbstore/src/test/java/org/apache/qpid/server/store/berkeleydb/BDBHAVirtualHostNodeTest.java (original)
+++ qpid/trunk/qpid/java/bdbstore/src/test/java/org/apache/qpid/server/store/berkeleydb/BDBHAVirtualHostNodeTest.java Mon Sep  1 13:41:33 2014
@@ -50,6 +50,7 @@ import org.apache.qpid.server.virtualhos
 import org.apache.qpid.server.virtualhostnode.berkeleydb.BDBHARemoteReplicationNodeImpl;
 import org.apache.qpid.server.virtualhostnode.berkeleydb.BDBHAVirtualHostNode;
 import org.apache.qpid.server.virtualhostnode.berkeleydb.BDBHAVirtualHostNodeTestHelper;
+import org.apache.qpid.server.virtualhostnode.berkeleydb.NodeRole;
 import org.apache.qpid.test.utils.QpidTestCase;
 
 public class BDBHAVirtualHostNodeTest extends QpidTestCase
@@ -91,7 +92,7 @@ public class BDBHAVirtualHostNodeTest ex
         BDBHAVirtualHostNode<?> node = _helper.createHaVHN(attributes);
 
         node.start();
-        _helper.assertNodeRole(node, "MASTER", "REPLICA");
+        _helper.assertNodeRole(node, NodeRole.MASTER, NodeRole.REPLICA);
 
         assertEquals("Unexpected node state", State.ACTIVE, node.getState());
 
@@ -155,7 +156,7 @@ public class BDBHAVirtualHostNodeTest ex
         assertEquals("Unexpected electable group override value after mutation", 1, environment.getRepMutableConfig().getElectableGroupSizeOverride());
 
         assertNotNull("Join time should be set", node.getJoinTime());
-        assertNotNull("Last known replication transaction idshould be set", node.getLastKnownReplicationTransactionId());
+        assertNotNull("Last known replication transaction id should be set", node.getLastKnownReplicationTransactionId());
     }
 
     public void testTransferMasterToSelf() throws Exception
@@ -177,11 +178,11 @@ public class BDBHAVirtualHostNodeTest ex
         Map<String, Object> node3Attributes = _helper.createNodeAttributes("node3", groupName, "localhost:" + node3PortNumber, helperAddress, nodeName);
         _helper.createAndStartHaVHN(node3Attributes);
 
-        BDBHAVirtualHostNode<?> replica = _helper.awaitAndFindNodeInRole("REPLICA");
+        BDBHAVirtualHostNode<?> replica = _helper.awaitAndFindNodeInRole(NodeRole.REPLICA);
 
-        replica.setAttribute(BDBHAVirtualHostNode.ROLE, "REPLICA", "MASTER");
+        replica.setAttribute(BDBHAVirtualHostNode.ROLE, replica.getRole(), NodeRole.MASTER);
 
-        _helper.assertNodeRole(replica, "MASTER");
+        _helper.assertNodeRole(replica, NodeRole.MASTER);
     }
 
     public void testTransferMasterToRemoteReplica() throws Exception
@@ -222,12 +223,12 @@ public class BDBHAVirtualHostNodeTest ex
         assertTrue("Replication nodes have not been seen during 5s", remoteNodeLatch.await(5, TimeUnit.SECONDS));
 
         BDBHARemoteReplicationNodeImpl replicaRemoteNode = (BDBHARemoteReplicationNodeImpl)lastSeenReplica.get();
-        _helper.awaitForAttributeChange(replicaRemoteNode, BDBHARemoteReplicationNodeImpl.ROLE, "REPLICA");
+        _helper.awaitForAttributeChange(replicaRemoteNode, BDBHARemoteReplicationNodeImpl.ROLE, NodeRole.REPLICA);
 
-        replicaRemoteNode.setAttributes(Collections.<String,Object>singletonMap(BDBHARemoteReplicationNode.ROLE, "MASTER"));
+        replicaRemoteNode.setAttributes(Collections.<String,Object>singletonMap(BDBHARemoteReplicationNode.ROLE, NodeRole.MASTER));
 
         BDBHAVirtualHostNode<?> replica = replicaRemoteNode.getName().equals(node2.getName())? node2 : node3;
-        _helper.assertNodeRole(replica, "MASTER");
+        _helper.assertNodeRole(replica, NodeRole.MASTER);
     }
 
     public void testMutatingRoleWhenNotReplica_IsDisallowed() throws Exception
@@ -239,11 +240,11 @@ public class BDBHAVirtualHostNodeTest ex
 
         Map<String, Object> node1Attributes = _helper.createNodeAttributes(nodeName, groupName, helperAddress, helperAddress, nodeName, nodePortNumber);
         BDBHAVirtualHostNode<?> node = _helper.createAndStartHaVHN(node1Attributes);
-        _helper.assertNodeRole(node, "MASTER");
+        _helper.assertNodeRole(node, NodeRole.MASTER);
 
         try
         {
-            node.setAttributes(Collections.<String,Object>singletonMap(BDBHAVirtualHostNode.ROLE, "REPLICA"));
+            node.setAttributes(Collections.<String,Object>singletonMap(BDBHAVirtualHostNode.ROLE, NodeRole.REPLICA));
             fail("Role mutation should fail");
         }
         catch(IllegalStateException e)
@@ -272,10 +273,10 @@ public class BDBHAVirtualHostNodeTest ex
         Map<String, Object> node3Attributes = _helper.createNodeAttributes("node3", groupName, "localhost:" + node3PortNumber, helperAddress, nodeName);
         _helper.createAndStartHaVHN(node3Attributes);
 
-        BDBHAVirtualHostNode<?> master = _helper.awaitAndFindNodeInRole("MASTER");
+        BDBHAVirtualHostNode<?> master = _helper.awaitAndFindNodeInRole(NodeRole.MASTER);
         _helper.awaitRemoteNodes(master, 2);
 
-        BDBHAVirtualHostNode<?> replica = _helper.awaitAndFindNodeInRole("REPLICA");
+        BDBHAVirtualHostNode<?> replica = _helper.awaitAndFindNodeInRole(NodeRole.REPLICA);
 
         assertNotNull("Remote node " + replica.getName() + " is not found", _helper.findRemoteNode(master, replica.getName()));
         replica.delete();
@@ -297,7 +298,7 @@ public class BDBHAVirtualHostNodeTest ex
         BDBHAVirtualHostNode<?> node = _helper.createHaVHN(nodeAttributes);
 
         node.start();
-        _helper.assertNodeRole(node, "MASTER", "REPLICA");
+        _helper.assertNodeRole(node, NodeRole.MASTER, NodeRole.REPLICA);
         assertEquals("Unexpected node state", State.ACTIVE, node.getState());
 
         _helper.awaitForVirtualhost(node,30000);
@@ -324,7 +325,7 @@ public class BDBHAVirtualHostNodeTest ex
         try
         {
             virtualHost.setAttributes(Collections.<String, Object>singletonMap(BDBHAVirtualHost.LOCAL_TRANSACTION_SYNCHRONIZATION_POLICY, "INVALID"));
-            fail("Invalid syncronization policy is set");
+            fail("Invalid synchronization policy is set");
         }
         catch(IllegalArgumentException e)
         {
@@ -334,7 +335,7 @@ public class BDBHAVirtualHostNodeTest ex
         try
         {
             virtualHost.setAttributes(Collections.<String, Object>singletonMap(BDBHAVirtualHost.REMOTE_TRANSACTION_SYNCHRONIZATION_POLICY, "INVALID"));
-            fail("Invalid syncronization policy is set");
+            fail("Invalid synchronization policy is set");
         }
         catch(IllegalArgumentException e)
         {

Modified: qpid/trunk/qpid/java/bdbstore/src/test/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHARemoteReplicationNodeTest.java
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/java/bdbstore/src/test/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHARemoteReplicationNodeTest.java?rev=1621775&r1=1621774&r2=1621775&view=diff
==============================================================================
--- qpid/trunk/qpid/java/bdbstore/src/test/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHARemoteReplicationNodeTest.java (original)
+++ qpid/trunk/qpid/java/bdbstore/src/test/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHARemoteReplicationNodeTest.java Mon Sep  1 13:41:33 2014
@@ -82,7 +82,7 @@ public class BDBHARemoteReplicationNodeT
         String remoteReplicationName = getName();
         BDBHARemoteReplicationNode remoteReplicationNode = createRemoteReplicationNode(remoteReplicationName);
 
-        remoteReplicationNode.setAttribute(BDBHARemoteReplicationNode.ROLE, "UNKNOWN", "MASTER");
+        remoteReplicationNode.setAttribute(BDBHARemoteReplicationNode.ROLE, remoteReplicationNode.getRole(), NodeRole.MASTER);
 
         verify(_facade).transferMasterAsynchronously(remoteReplicationName);
     }

Modified: qpid/trunk/qpid/java/bdbstore/src/test/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHAVirtualHostNodeOperationalLoggingTest.java
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/java/bdbstore/src/test/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHAVirtualHostNodeOperationalLoggingTest.java?rev=1621775&r1=1621774&r2=1621775&view=diff
==============================================================================
--- qpid/trunk/qpid/java/bdbstore/src/test/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHAVirtualHostNodeOperationalLoggingTest.java (original)
+++ qpid/trunk/qpid/java/bdbstore/src/test/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHAVirtualHostNodeOperationalLoggingTest.java Mon Sep  1 13:41:33 2014
@@ -76,7 +76,7 @@ public class BDBHAVirtualHostNodeOperati
         Map<String, Object> node1Attributes = _helper.createNodeAttributes(nodeName, groupName, helperAddress, helperAddress, nodeName, node1PortNumber);
         BDBHAVirtualHostNodeImpl node1 = (BDBHAVirtualHostNodeImpl)_helper.createHaVHN(node1Attributes);
 
-        _helper.assertNodeRole(node1, "MASTER");
+        _helper.assertNodeRole(node1, NodeRole.MASTER);
 
         assertEquals("Unexpected VHN log subject", "[grp(/group)/vhn(/node1)] ", node1.getVirtualHostNodeLogSubject().getLogString());
         assertEquals("Unexpected group log subject", "[grp(/group)] ", node1.getGroupLogSubject().getLogString());
@@ -85,7 +85,7 @@ public class BDBHAVirtualHostNodeOperati
         verify(_eventLogger).message(argThat(new LogSubjectMatcher(node1.getVirtualHostNodeLogSubject())),
                 argThat(new LogMessageMatcher(expectedMessage, HighAvailabilityMessages.CREATED_LOG_HIERARCHY)));
 
-        expectedMessage = HighAvailabilityMessages.ROLE_CHANGED(node1.getName(), node1.getAddress(), "UNKNOWN", "MASTER").toString();
+        expectedMessage = HighAvailabilityMessages.ROLE_CHANGED(node1.getName(), node1.getAddress(), NodeRole.WAITING.name(), NodeRole.MASTER.name()).toString();
         verify(_eventLogger).message(argThat(new LogSubjectMatcher(node1.getGroupLogSubject())),
                 argThat(new LogMessageMatcher(expectedMessage, HighAvailabilityMessages.ROLE_CHANGED_LOG_HIERARCHY)));
     }
@@ -99,7 +99,7 @@ public class BDBHAVirtualHostNodeOperati
 
         Map<String, Object> node1Attributes = _helper.createNodeAttributes(nodeName, groupName, helperAddress, helperAddress, nodeName, node1PortNumber);
         BDBHAVirtualHostNodeImpl node1 = (BDBHAVirtualHostNodeImpl)_helper.createHaVHN(node1Attributes);
-        _helper.assertNodeRole(node1, "MASTER");
+        _helper.assertNodeRole(node1, NodeRole.MASTER);
 
         reset(_eventLogger);
 
@@ -120,7 +120,7 @@ public class BDBHAVirtualHostNodeOperati
 
         Map<String, Object> node1Attributes = _helper.createNodeAttributes(nodeName, groupName, helperAddress, helperAddress, nodeName, node1PortNumber);
         BDBHAVirtualHostNodeImpl node1 = (BDBHAVirtualHostNodeImpl)_helper.createHaVHN(node1Attributes);
-        _helper.assertNodeRole(node1, "MASTER");
+        _helper.assertNodeRole(node1, NodeRole.MASTER);
 
         reset(_eventLogger);
 
@@ -143,7 +143,7 @@ public class BDBHAVirtualHostNodeOperati
 
         Map<String, Object> node1Attributes = _helper.createNodeAttributes(nodeName, groupName, helperAddress, helperAddress, nodeName, node1PortNumber);
         BDBHAVirtualHostNodeImpl node1 = (BDBHAVirtualHostNodeImpl)_helper.createHaVHN(node1Attributes);
-        _helper.assertNodeRole(node1, "MASTER");
+        _helper.assertNodeRole(node1, NodeRole.MASTER);
 
         reset(_eventLogger);
 
@@ -166,7 +166,7 @@ public class BDBHAVirtualHostNodeOperati
 
         Map<String, Object> node1Attributes = _helper.createNodeAttributes(nodeName, groupName, helperAddress, helperAddress, nodeName, node1PortNumber);
         BDBHAVirtualHostNodeImpl node1 = (BDBHAVirtualHostNodeImpl)_helper.createHaVHN(node1Attributes);
-        _helper.assertNodeRole(node1, "MASTER");
+        _helper.assertNodeRole(node1, NodeRole.MASTER);
 
         reset(_eventLogger);
 
@@ -190,7 +190,7 @@ public class BDBHAVirtualHostNodeOperati
 
         Map<String, Object> node1Attributes = _helper.createNodeAttributes(nodeName, groupName, helperAddress, helperAddress, nodeName, node1PortNumber, node2PortNumber);
         BDBHAVirtualHostNodeImpl node1 = (BDBHAVirtualHostNodeImpl)_helper.createHaVHN(node1Attributes);
-        _helper.assertNodeRole(node1, "MASTER");
+        _helper.assertNodeRole(node1, NodeRole.MASTER);
 
         reset(_eventLogger);
 
@@ -219,7 +219,7 @@ public class BDBHAVirtualHostNodeOperati
         Map<String, Object> node1Attributes = _helper.createNodeAttributes(nodeName, groupName, helperAddress, helperAddress, nodeName, node1PortNumber, node2PortNumber);
         node1Attributes.put(BDBHAVirtualHostNode.DESIGNATED_PRIMARY, true);
         BDBHAVirtualHostNodeImpl node1 = (BDBHAVirtualHostNodeImpl)_helper.createHaVHN(node1Attributes);
-        _helper.assertNodeRole(node1, "MASTER");
+        _helper.assertNodeRole(node1, NodeRole.MASTER);
 
         resetEventLogger();
 
@@ -251,7 +251,7 @@ public class BDBHAVirtualHostNodeOperati
         Map<String, Object> node1Attributes = _helper.createNodeAttributes(nodeName, groupName, helperAddress, helperAddress, nodeName, node1PortNumber, node2PortNumber);
         node1Attributes.put(BDBHAVirtualHostNode.DESIGNATED_PRIMARY, true);
         BDBHAVirtualHostNodeImpl node1 = (BDBHAVirtualHostNodeImpl)_helper.createHaVHN(node1Attributes);
-        _helper.assertNodeRole(node1, "MASTER");
+        _helper.assertNodeRole(node1, NodeRole.MASTER);
 
         Map<String, Object> node2Attributes = _helper.createNodeAttributes("node2", groupName, "localhost:" + node2PortNumber, helperAddress, nodeName);
         BDBHAVirtualHostNodeImpl node2 = (BDBHAVirtualHostNodeImpl)_helper.createHaVHN(node2Attributes);
@@ -284,7 +284,7 @@ public class BDBHAVirtualHostNodeOperati
         Map<String, Object> node1Attributes = _helper.createNodeAttributes(nodeName, groupName, helperAddress, helperAddress, nodeName, node1PortNumber, node2PortNumber);
         node1Attributes.put(BDBHAVirtualHostNode.DESIGNATED_PRIMARY, true);
         BDBHAVirtualHostNodeImpl node1 = (BDBHAVirtualHostNodeImpl)_helper.createHaVHN(node1Attributes);
-        _helper.assertNodeRole(node1, "MASTER");
+        _helper.assertNodeRole(node1, NodeRole.MASTER);
 
         resetEventLogger();
 
@@ -302,7 +302,7 @@ public class BDBHAVirtualHostNodeOperati
 
         node2Attributes.put(BDBHAVirtualHostNode.PERMITTED_NODES, node1Attributes.get(BDBHAVirtualHostNode.PERMITTED_NODES));
         node2 = (BDBHAVirtualHostNodeImpl)_helper.recoverHaVHN(node2.getId(), node2Attributes);
-        _helper.assertNodeRole(node2, "REPLICA", "MASTER");
+        _helper.assertNodeRole(node2, NodeRole.REPLICA, NodeRole.MASTER);
         waitForNodeDetachedField(remoteNode, false);
 
         final String expectedMessage = HighAvailabilityMessages.JOINED(node2.getName(), node2.getAddress()).toString();

Modified: qpid/trunk/qpid/java/bdbstore/src/test/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHAVirtualHostNodeTestHelper.java
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/java/bdbstore/src/test/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHAVirtualHostNodeTestHelper.java?rev=1621775&r1=1621774&r2=1621775&view=diff
==============================================================================
--- qpid/trunk/qpid/java/bdbstore/src/test/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHAVirtualHostNodeTestHelper.java (original)
+++ qpid/trunk/qpid/java/bdbstore/src/test/java/org/apache/qpid/server/virtualhostnode/berkeleydb/BDBHAVirtualHostNodeTestHelper.java Mon Sep  1 13:41:33 2014
@@ -164,31 +164,31 @@ public class BDBHAVirtualHostNodeTestHel
         assertEquals("Unexpected attribute " + name + " on " + object, expectedValue, object.getAttribute(name) );
     }
 
-    public BDBHAVirtualHostNode<?> awaitAndFindNodeInRole(String role) throws InterruptedException
+    public BDBHAVirtualHostNode<?> awaitAndFindNodeInRole(NodeRole desiredRole) throws InterruptedException
     {
         BDBHAVirtualHostNode<?> replica = null;
         int findReplicaCount = 0;
         while(replica == null)
         {
-            replica = findNodeInRole(role);
+            replica = findNodeInRole(desiredRole);
             if (replica == null)
             {
                 Thread.sleep(100);
             }
             if (findReplicaCount > 50)
             {
-                fail("Could not find a node in replica role");
+                fail("Could not find a node in role " + desiredRole);
             }
             findReplicaCount++;
         }
         return replica;
     }
 
-    public BDBHAVirtualHostNode<?> findNodeInRole(String role)
+    public BDBHAVirtualHostNode<?> findNodeInRole(NodeRole role)
     {
         for (BDBHAVirtualHostNode<?> node : _nodes)
         {
-            if (role.equals(node.getRole()))
+            if (role == node.getRole())
             {
                 return node;
             }
@@ -218,15 +218,15 @@ public class BDBHAVirtualHostNodeTestHel
         return node;
     }
 
-    public void assertNodeRole(BDBHAVirtualHostNode<?> node, String... roleName) throws InterruptedException
+    public void assertNodeRole(BDBHAVirtualHostNode<?> node, NodeRole... roleName) throws InterruptedException
     {
         int iterationCounter = 0;
         boolean inRole =false;
         do
         {
-            for (String role : roleName)
+            for (NodeRole role : roleName)
             {
-                if (role.equals(node.getRole()))
+                if (role == node.getRole())
                 {
                     inRole = true;
                     break;
@@ -252,7 +252,7 @@ public class BDBHAVirtualHostNodeTestHel
     public BDBHAVirtualHostNode<?> startNodeAndWait(BDBHAVirtualHostNode<?> node) throws InterruptedException
     {
         node.start();
-        assertNodeRole(node, "MASTER", "REPLICA");
+        assertNodeRole(node, NodeRole.MASTER, NodeRole.REPLICA);
         assertEquals("Unexpected node state", State.ACTIVE, node.getState());
         return node;
     }
@@ -273,7 +273,7 @@ public class BDBHAVirtualHostNodeTestHel
     {
         Map<String, Object> node1Attributes = new HashMap<String, Object>();
         node1Attributes.put(BDBHAVirtualHostNode.ID, UUID.randomUUID());
-        node1Attributes.put(BDBHAVirtualHostNode.TYPE, "BDB_HA");
+        node1Attributes.put(BDBHAVirtualHostNode.TYPE, BDBHAVirtualHostNodeImpl.VIRTUAL_HOST_NODE_TYPE);
         node1Attributes.put(BDBHAVirtualHostNode.NAME, nodeName);
         node1Attributes.put(BDBHAVirtualHostNode.GROUP_NAME, groupName);
         node1Attributes.put(BDBHAVirtualHostNode.ADDRESS, address);

Modified: qpid/trunk/qpid/java/bdbstore/systests/src/test/java/org/apache/qpid/server/store/berkeleydb/replication/BDBHAVirtualHostNodeRestTest.java
URL: http://svn.apache.org/viewvc/qpid/trunk/qpid/java/bdbstore/systests/src/test/java/org/apache/qpid/server/store/berkeleydb/replication/BDBHAVirtualHostNodeRestTest.java?rev=1621775&r1=1621774&r2=1621775&view=diff
==============================================================================
--- qpid/trunk/qpid/java/bdbstore/systests/src/test/java/org/apache/qpid/server/store/berkeleydb/replication/BDBHAVirtualHostNodeRestTest.java (original)
+++ qpid/trunk/qpid/java/bdbstore/systests/src/test/java/org/apache/qpid/server/store/berkeleydb/replication/BDBHAVirtualHostNodeRestTest.java Mon Sep  1 13:41:33 2014
@@ -43,6 +43,7 @@ import org.apache.qpid.server.virtualhos
 import org.apache.qpid.server.virtualhostnode.AbstractVirtualHostNode;
 import org.apache.qpid.server.virtualhostnode.berkeleydb.BDBHARemoteReplicationNode;
 import org.apache.qpid.server.virtualhostnode.berkeleydb.BDBHAVirtualHostNode;
+import org.apache.qpid.server.virtualhostnode.berkeleydb.BDBHAVirtualHostNodeImpl;
 import org.apache.qpid.systest.rest.Asserts;
 import org.apache.qpid.systest.rest.QpidRestTestCase;
 import org.apache.qpid.test.utils.TestBrokerConfiguration;
@@ -124,7 +125,7 @@ public class BDBHAVirtualHostNodeRestTes
         assertEquals("Node 1 observed from node 2 is in the wrong state",
                 "UNAVAILABLE", remoteNode1.get(BDBHARemoteReplicationNode.STATE));
         assertEquals("Node 1 observed from node 2 has the wrong role",
-                     "UNKNOWN", remoteNode1.get(BDBHARemoteReplicationNode.ROLE));
+                     "UNREACHABLE", remoteNode1.get(BDBHARemoteReplicationNode.ROLE));
 
     }
 
@@ -170,8 +171,7 @@ public class BDBHAVirtualHostNodeRestTes
         List<Map<String,Object>> data = getRestTestHelper().getJsonAsList("replicationnode/" + NODE1);
         assertEquals("Unexpected number of remote nodes on " + NODE1, 2, data.size());
 
-        int responseCode = getRestTestHelper().submitRequest(_baseNodeRestUrl + NODE2, "DELETE");
-        assertEquals("Unexpected response code on deletion of virtual host node " + NODE2, 200, responseCode);
+        getRestTestHelper().submitRequest(_baseNodeRestUrl + NODE2, "DELETE", HttpServletResponse.SC_OK);
 
         int counter = 0;
         while (data.size() != 1 && counter<50)
@@ -195,23 +195,23 @@ public class BDBHAVirtualHostNodeRestTes
         assertNode(NODE1, _node1HaPort, _node1HaPort, NODE1);
         assertRemoteNodes(NODE1, NODE2, NODE3);
 
-        // change priority to make Node2 a master
-        int responseCode = getRestTestHelper().submitRequest(_baseNodeRestUrl + NODE2, "PUT", Collections.<String,Object>singletonMap(BDBHAVirtualHostNode.PRIORITY, 100));
-        assertEquals("Unexpected response code on priority update of virtual host node " + NODE2, 200, responseCode);
+        // change priority to ensure that Node2 becomes a master
+        getRestTestHelper().submitRequest(_baseNodeRestUrl + NODE2,
+                                          "PUT",
+                                          Collections.<String, Object>singletonMap(BDBHAVirtualHostNode.PRIORITY, 100),
+                                          HttpServletResponse.SC_OK);
 
         List<Map<String,Object>> data = getRestTestHelper().getJsonAsList("replicationnode/" + NODE2);
         assertEquals("Unexpected number of remote nodes on " + NODE2, 2, data.size());
 
         // delete master
-        responseCode = getRestTestHelper().submitRequest(_baseNodeRestUrl + NODE1, "DELETE");
-        assertEquals("Unexpected response code on deletion of virtual host node " + NODE1, 200, responseCode);
+        getRestTestHelper().submitRequest(_baseNodeRestUrl + NODE1, "DELETE", HttpServletResponse.SC_OK);
 
         // wait for new master
         waitForAttributeChanged(_baseNodeRestUrl + NODE2 + "?depth=0", BDBHAVirtualHostNode.ROLE, "MASTER");
 
         // delete remote node
-        responseCode = getRestTestHelper().submitRequest("replicationnode/" + NODE2 + "/" + NODE1, "DELETE");
-        assertEquals("Unexpected response code on deletion of remote node " + NODE1, 200, responseCode);
+        getRestTestHelper().submitRequest("replicationnode/" + NODE2 + "/" + NODE1, "DELETE", HttpServletResponse.SC_OK);
 
         int counter = 0;
         while (data.size() != 1 && counter<50)
@@ -233,7 +233,7 @@ public class BDBHAVirtualHostNodeRestTes
 
         // add permitted node
         Map<String, Object> node3Data = createNodeAttributeMap(NODE3, _node3HaPort, _node1HaPort);
-        getRestTestHelper().submitRequest(_baseNodeRestUrl + NODE3, "PUT", node3Data, 201);
+        getRestTestHelper().submitRequest(_baseNodeRestUrl + NODE3, "PUT", node3Data, HttpServletResponse.SC_CREATED);
         assertNode(NODE3, _node3HaPort, _node1HaPort, NODE1);
         assertRemoteNodes(NODE1, NODE3);
 
@@ -241,7 +241,7 @@ public class BDBHAVirtualHostNodeRestTes
 
         // try to add not permitted node
         Map<String, Object> nodeData = createNodeAttributeMap(NODE2, intruderPort, _node1HaPort);
-        getRestTestHelper().submitRequest(_baseNodeRestUrl + NODE2, "PUT", nodeData, 409);
+        getRestTestHelper().submitRequest(_baseNodeRestUrl + NODE2, "PUT", nodeData, HttpServletResponse.SC_CONFLICT);
 
         assertRemoteNodes(NODE1, NODE3);
     }
@@ -257,7 +257,7 @@ public class BDBHAVirtualHostNodeRestTes
 
         // add permitted node
         Map<String, Object> node3Data = createNodeAttributeMap(NODE3, _node3HaPort, _node1HaPort);
-        getRestTestHelper().submitRequest(_baseNodeRestUrl + NODE3, "PUT", node3Data, 201);
+        getRestTestHelper().submitRequest(_baseNodeRestUrl + NODE3, "PUT", node3Data, HttpServletResponse.SC_CREATED);
         assertNode(NODE3, _node3HaPort, _node1HaPort, NODE1);
         assertRemoteNodes(NODE1, NODE3);
 
@@ -313,8 +313,7 @@ public class BDBHAVirtualHostNodeRestTes
     {
         Map<String, Object> nodeData = createNodeAttributeMap(nodeName, nodePort, helperPort);
 
-        int responseCode = getRestTestHelper().submitRequest(_baseNodeRestUrl + nodeName, "PUT", nodeData);
-        assertEquals("Unexpected response code for virtual host node " + nodeName + " creation request", 201, responseCode);
+        getRestTestHelper().submitRequest(_baseNodeRestUrl + nodeName, "PUT", nodeData, HttpServletResponse.SC_CREATED);
         String hostExpectedState = nodePort == helperPort ? State.ACTIVE.name(): State.UNAVAILABLE.name();
         waitForAttributeChanged("virtualhost/" + nodeName + "/" + _hostName, BDBHAVirtualHost.STATE, hostExpectedState);
     }
@@ -323,11 +322,12 @@ public class BDBHAVirtualHostNodeRestTes
     {
         Map<String, Object> nodeData = new HashMap<String, Object>();
         nodeData.put(BDBHAVirtualHostNode.NAME, nodeName);
-        nodeData.put(BDBHAVirtualHostNode.TYPE, "BDB_HA");
+        nodeData.put(BDBHAVirtualHostNode.TYPE, BDBHAVirtualHostNodeImpl.VIRTUAL_HOST_NODE_TYPE);
         nodeData.put(BDBHAVirtualHostNode.GROUP_NAME, _hostName);
         nodeData.put(BDBHAVirtualHostNode.ADDRESS, "localhost:" + nodePort);
         nodeData.put(BDBHAVirtualHostNode.HELPER_ADDRESS, "localhost:" + helperPort);
         nodeData.put(BDBHAVirtualHostNode.HELPER_NODE_NAME, NODE1);
+
         Map<String,String> context = new HashMap<>();
         nodeData.put(BDBHAVirtualHostNode.CONTEXT, context);
         if (nodePort == helperPort)
@@ -347,7 +347,7 @@ public class BDBHAVirtualHostNodeRestTes
 
         Map<String, Object> nodeData = getRestTestHelper().getJsonAsSingletonList(_baseNodeRestUrl + nodeName + "?depth=0");
         assertEquals("Unexpected name", nodeName, nodeData.get(BDBHAVirtualHostNode.NAME));
-        assertEquals("Unexpected type", "BDB_HA", nodeData.get(BDBHAVirtualHostNode.TYPE));
+        assertEquals("Unexpected type", BDBHAVirtualHostNodeImpl.VIRTUAL_HOST_NODE_TYPE, nodeData.get(BDBHAVirtualHostNode.TYPE));
         assertEquals("Unexpected address", "localhost:" + nodePort, nodeData.get(BDBHAVirtualHostNode.ADDRESS));
         assertEquals("Unexpected helper address", "localhost:" + nodeHelperPort, nodeData.get(BDBHAVirtualHostNode.HELPER_ADDRESS));
         assertEquals("Unexpected group name", _hostName, nodeData.get(BDBHAVirtualHostNode.GROUP_NAME));



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@qpid.apache.org
For additional commands, e-mail: commits-help@qpid.apache.org