You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@activemq.apache.org by cl...@apache.org on 2016/12/07 18:18:19 UTC

[16/55] [abbrv] activemq-artemis git commit: remove JMS JMX Objects and add new Address JMX objects

remove JMS JMX Objects and add new Address JMX objects

more fixes to handle null or emty address on cluster connection

removal of all JMS control objects

more fixes after JMX JMS removal

remove unwanted tests

added missing managament methods


Project: http://git-wip-us.apache.org/repos/asf/activemq-artemis/repo
Commit: http://git-wip-us.apache.org/repos/asf/activemq-artemis/commit/2b46f6a4
Tree: http://git-wip-us.apache.org/repos/asf/activemq-artemis/tree/2b46f6a4
Diff: http://git-wip-us.apache.org/repos/asf/activemq-artemis/diff/2b46f6a4

Branch: refs/heads/ARTEMIS-780
Commit: 2b46f6a4e7e44887971f21937aed4e430359be96
Parents: 13f4928
Author: Andy Taylor <an...@gmail.com>
Authored: Sun Nov 6 10:43:16 2016 +0000
Committer: Clebert Suconic <cl...@apache.org>
Committed: Wed Dec 7 13:16:32 2016 -0500

----------------------------------------------------------------------
 .../cli/commands/tools/XmlDataImporter.java     |   21 +-
 .../core/management/ActiveMQServerControl.java  |   20 +
 .../api/core/management/AddressControl.java     |   30 +
 .../api/core/management/ObjectNameBuilder.java  |   34 +-
 .../api/core/management/QueueControl.java       |    5 +-
 .../api/core/management/ResourceNames.java      |   29 +-
 .../management/ConnectionFactoryControl.java    |  399 -----
 .../api/jms/management/DestinationControl.java  |   79 -
 .../api/jms/management/JMSConnectionInfo.java   |   87 -
 .../api/jms/management/JMSConsumerInfo.java     |  120 --
 .../api/jms/management/JMSQueueControl.java     |  443 -----
 .../api/jms/management/JMSServerControl.java    |  381 ----
 .../api/jms/management/JMSSessionInfo.java      |   54 -
 .../api/jms/management/SubscriptionInfo.java    |  130 --
 .../api/jms/management/TopicControl.java        |  153 --
 .../impl/JMSConnectionFactoryControlImpl.java   |  471 -----
 .../management/impl/JMSQueueControlImpl.java    |  532 ------
 .../management/impl/JMSServerControlImpl.java   |  876 ---------
 .../management/impl/JMSTopicControlImpl.java    |  370 ----
 .../openmbean/JMSCompositeDataConstants.java    |   57 -
 .../impl/openmbean/JMSOpenTypeSupport.java      |  357 ----
 .../jms/server/impl/JMSServerManagerImpl.java   |   32 +-
 .../server/management/JMSManagementService.java |   48 -
 .../impl/JMSManagementServiceImpl.java          |  155 --
 .../config/ClusterConnectionConfiguration.java  |    7 -
 .../deployers/impl/FileConfigurationParser.java |    2 +-
 .../impl/ActiveMQServerControlImpl.java         |   72 +-
 .../management/impl/AddressControlImpl.java     |  132 +-
 .../core/management/impl/QueueControlImpl.java  |   12 +-
 .../artemis/core/postoffice/PostOffice.java     |    2 +-
 .../core/postoffice/impl/PostOfficeImpl.java    |    6 +-
 .../artemis/core/server/ActiveMQServer.java     |    2 +-
 .../core/server/cluster/ClusterManager.java     |    2 +-
 .../cluster/impl/ClusterConnectionBridge.java   |    4 +-
 .../cluster/impl/ClusterConnectionImpl.java     |    2 +-
 .../core/server/impl/ActiveMQServerImpl.java    |    8 +-
 .../artemis/core/server/impl/AddressInfo.java   |    7 +
 .../server/impl/PostOfficeJournalLoader.java    |    4 +-
 .../core/server/impl/ScaleDownHandler.java      |    4 +-
 .../server/management/ManagementService.java    |    5 +-
 .../management/impl/ManagementServiceImpl.java  |   49 +-
 .../resources/schema/artemis-configuration.xsd  |    2 +-
 .../group/impl/ClusteredResetMockTest.java      |    5 +-
 .../artemis/jms/example/JMXExample.java         |    9 +-
 .../jms/example/MessageCounterExample.java      |    4 +-
 .../artemiswrapper/OpenwireArtemisBaseTest.java |   12 -
 .../org/apache/activemq/JMSConsumerTest.java    |   42 +-
 .../failover/FailoverBackupLeakTest.java        |    5 +-
 .../tests/extras/jms/bridge/BridgeTestBase.java |   13 +-
 .../client/AutoCreateJmsDestinationTest.java    |    8 +-
 .../client/JMSMessageCounterTest.java           |    7 +-
 .../client/MultipleProducersTest.java           |    7 +-
 .../cluster/bridge/BridgeReconnectTest.java     |    2 +-
 .../ClusteredMessageCounterTest.java            |    6 +-
 .../crossprotocol/AMQPToOpenwireTest.java       |    1 -
 .../jms/client/ExpiryMessageTest.java           |    5 +-
 .../ConnectionFactoryControlTest.java           |  170 --
 .../server/management/JMSMessagingProxy.java    |   95 -
 .../server/management/JMSQueueControlTest.java  | 1688 ------------------
 .../management/JMSQueueControlUsingJMSTest.java |  409 -----
 .../management/JMSServerControl2Test.java       | 1150 ------------
 .../management/JMSServerControlRestartTest.java |  157 --
 .../server/management/JMSServerControlTest.java | 1113 ------------
 .../JMSServerControlUsingJMSTest.java           |  436 -----
 .../management/TopicControlClusterTest.java     |   13 +-
 .../jms/server/management/TopicControlTest.java |  669 -------
 .../management/TopicControlUsingJMSTest.java    |  436 -----
 .../AcceptorControlUsingCoreTest.java           |    2 +-
 .../management/ActiveMQServerControlTest.java   |   12 +-
 .../ActiveMQServerControlUsingCoreTest.java     |   22 +-
 .../management/AddressControlUsingCoreTest.java |    2 +-
 .../management/BridgeControlUsingCoreTest.java  |    2 +-
 .../BroadcastGroupControlUsingCoreTest.java     |    2 +-
 .../management/DivertControlTest.java           |    8 +-
 .../management/DivertControlUsingCoreTest.java  |    4 +-
 .../management/ManagementControlHelper.java     |   33 +-
 .../management/ManagementServiceImplTest.java   |   11 +-
 .../management/ManagementTestBase.java          |   11 -
 .../ManagementWithPagingServerTest.java         |   17 +-
 .../management/ManagementWithStompTest.java     |    4 +-
 .../management/QueueControlTest.java            |    4 +-
 .../management/QueueControlUsingCoreTest.java   |   16 +-
 .../management/SecurityManagementTestBase.java  |    2 +-
 .../integration/openwire/OpenWireTestBase.java  |    6 -
 .../artemis/tests/util/JMSTestBase.java         |    6 -
 .../jms/tests/ActiveMQServerTestCase.java       |   23 +-
 .../tests/tools/container/LocalTestServer.java  |   28 +-
 .../jms/tests/tools/container/Server.java       |    2 +-
 .../activemq/artemis/common/AbstractAdmin.java  |   14 +-
 .../activemq/artemis/jms/ActiveMQCoreAdmin.java |   20 +-
 .../jms/bridge/impl/JMSBridgeImplTest.java      |    7 +-
 91 files changed, 487 insertions(+), 11438 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2b46f6a4/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/tools/XmlDataImporter.java
----------------------------------------------------------------------
diff --git a/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/tools/XmlDataImporter.java b/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/tools/XmlDataImporter.java
index 587e402..225b14c 100644
--- a/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/tools/XmlDataImporter.java
+++ b/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/tools/XmlDataImporter.java
@@ -328,7 +328,7 @@ public final class XmlDataImporter extends ActionAbstract {
             // address may need the message
             try (ClientRequestor requestor = new ClientRequestor(managementSession, "activemq.management")) {
                ClientMessage managementMessage = managementSession.createMessage(false);
-               ManagementHelper.putAttribute(managementMessage, "core.queue." + queue, "ID");
+               ManagementHelper.putAttribute(managementMessage, ResourceNames.QUEUE + queue, "ID");
                managementSession.start();
                if (logger.isDebugEnabled()) {
                   logger.debug("Requesting ID for: " + queue);
@@ -825,20 +825,7 @@ public final class XmlDataImporter extends ActionAbstract {
          reader.next();
       }
 
-      try (ClientRequestor requestor = new ClientRequestor(managementSession, "activemq.management")) {
-         ClientMessage managementMessage = managementSession.createMessage(false);
-         ManagementHelper.putOperationInvocation(managementMessage, ResourceNames.JMS_SERVER, "createConnectionFactory", name, Boolean.parseBoolean(ha), discoveryGroupName.length() > 0, Integer.parseInt(type), connectors, entries, clientId, Long.parseLong(clientFailureCheckPeriod), Long.parseLong(connectionTtl), Long.parseLong(callTimeout), Long.parseLong(callFailoverTimeout), Integer.parseInt(minLargeMessageSize), Boolean.parseBoolean(compressLargeMessages), Integer.parseInt(consumerWindowSize), Integer.parseInt(consumerMaxRate), Integer.parseInt(confirmationWindowSize), Integer.parseInt(producerWindowSize), Integer.parseInt(producerMaxRate), Boolean.parseBoolean(blockOnAcknowledge), Boolean.parseBoolean(blockOnDurableSend), Boolean.parseBoolean(blockOnNonDurableSend), Boolean.parseBoolean(autoGroup), Boolean.parseBoolean(preacknowledge), loadBalancingPolicyClassName, Integer.parseInt(transactionBatchSize), Integer.parseInt(dupsOkBatchSize), Boolean.parseBoolean(useGlobalPools), In
 teger.parseInt(scheduledThreadMaxPoolSize), Integer.parseInt(threadMaxPoolSize), Long.parseLong(retryInterval), Double.parseDouble(retryIntervalMultiplier), Long.parseLong(maxRetryInterval), Integer.parseInt(reconnectAttempts), Boolean.parseBoolean(failoverOnInitialConnection), groupId);
-         //Boolean.parseBoolean(cacheLargeMessagesClient));
-         managementSession.start();
-         ClientMessage reply = requestor.request(managementMessage);
-         if (ManagementHelper.hasOperationSucceeded(reply)) {
-            if (logger.isDebugEnabled()) {
-               logger.debug("Created connection factory " + name);
-            }
-         } else {
-            ActiveMQServerLogger.LOGGER.error("Problem creating " + name);
-         }
-      }
+      ActiveMQServerLogger.LOGGER.error("Ignoring Connection Factory " + name);
    }
 
    private void createJmsDestination() throws Exception {
@@ -886,9 +873,9 @@ public final class XmlDataImporter extends ActionAbstract {
       try (ClientRequestor requestor = new ClientRequestor(managementSession, "activemq.management")) {
          ClientMessage managementMessage = managementSession.createMessage(false);
          if ("Queue".equals(type)) {
-            ManagementHelper.putOperationInvocation(managementMessage, ResourceNames.JMS_SERVER, "createQueue", name, entries, selector);
+            ManagementHelper.putOperationInvocation(managementMessage, ResourceNames.BROKER, "createQueue", name, entries, selector);
          } else if ("Topic".equals(type)) {
-            ManagementHelper.putOperationInvocation(managementMessage, ResourceNames.JMS_SERVER, "createTopic", name, entries);
+            ManagementHelper.putOperationInvocation(managementMessage, ResourceNames.BROKER, "createAddress", name, entries);
          }
          managementSession.start();
          ClientMessage reply = requestor.request(managementMessage);

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2b46f6a4/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/ActiveMQServerControl.java
----------------------------------------------------------------------
diff --git a/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/ActiveMQServerControl.java b/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/ActiveMQServerControl.java
index 87a4a79..7772459 100644
--- a/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/ActiveMQServerControl.java
+++ b/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/ActiveMQServerControl.java
@@ -297,6 +297,14 @@ public interface ActiveMQServerControl {
    String getManagementAddress();
 
    /**
+    * Returns the node ID of this server.
+    * <br>
+    * Clients can send management messages to this address to manage this server.
+    */
+   @Attribute(desc = "Node ID of this server")
+   String getNodeID();
+
+   /**
     * Returns the management notification address of this server.
     * <br>
     * Clients can bind queues to this address to receive management notifications emitted by this server.
@@ -424,6 +432,15 @@ public interface ActiveMQServerControl {
 
    // Operations ----------------------------------------------------
 
+   @Operation(desc = "create an address", impact = MBeanOperationInfo.ACTION)
+   void createAddress(@Parameter(name = "name", desc = "The name of the address") String name,
+                      @Parameter(name = "routingType", desc = "the routing type of the address either 0 for multicast or 1 for anycast") int routingType,
+                      @Parameter(name = "defaultDeleteOnNoConsumers", desc = "Whether or not a queue with this address is deleted when it has no consumers") boolean defaultDeleteOnNoConsumers,
+                      @Parameter(name = "defaultMaxConsumers", desc = "The maximim number of consumer a queue with this address can have") int defaultMaxConsumers) throws Exception;
+
+   @Operation(desc = "delete an address", impact = MBeanOperationInfo.ACTION)
+   void deleteAddress(@Parameter(name = "name", desc = "The name of the address") String name) throws Exception;
+
    /**
     * Create a durable queue.
     * <br>
@@ -881,5 +898,8 @@ public interface ActiveMQServerControl {
 
    @Operation(desc = "force the server to stop and to scale down to another server", impact = MBeanOperationInfo.UNKNOWN)
    void scaleDown(@Parameter(name = "name", desc = "The connector to use to scale down, if not provided the first appropriate connector will be used") String connector) throws Exception;
+
+   @Operation(desc = "List the Network Topology", impact = MBeanOperationInfo.INFO)
+   String listNetworkTopology() throws Exception;
 }
 

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2b46f6a4/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/AddressControl.java
----------------------------------------------------------------------
diff --git a/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/AddressControl.java b/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/AddressControl.java
index fbecf25..5e7d600 100644
--- a/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/AddressControl.java
+++ b/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/AddressControl.java
@@ -16,6 +16,9 @@
  */
 package org.apache.activemq.artemis.api.core.management;
 
+import javax.management.MBeanOperationInfo;
+import java.util.Map;
+
 /**
  * An AddressControl is used to manage an address.
  */
@@ -27,6 +30,12 @@ public interface AddressControl {
    @Attribute(desc = "managed address")
    String getAddress();
 
+   /*
+   * The routing type of this address, either multicast (topic subscriptions) or anycast (queue semantics).
+   * */
+   @Attribute(desc = "The routing type of this address")
+   String getRoutingType();
+
    /**
     * Returns the roles (name and permissions) associated with this address.
     */
@@ -85,4 +94,25 @@ public interface AddressControl {
     */
    @Attribute(desc = "names of all bindings (both queues and diverts) bound to this address")
    String[] getBindingNames() throws Exception;
+
+   @Attribute(desc = "number of messages added to all the queues for this address")
+   long getMessageCount();
+
+
+   /**
+    * @param headers  the message headers and properties to set. Can only
+    *                 container Strings maped to primitive types.
+    * @param body     the text to send
+    * @param durable
+    * @param user
+    * @param password @return
+    * @throws Exception
+    */
+   @Operation(desc = "Sends a TextMessage to a password-protected address.", impact = MBeanOperationInfo.ACTION)
+   String sendMessage(@Parameter(name = "headers", desc = "The headers to add to the message") Map<String, String> headers,
+                      @Parameter(name = "headers", desc = "A type for the message") final int type,
+                      @Parameter(name = "body", desc = "The body (byte[]) of the message encoded as a string using Base64") String body,
+                      @Parameter(name = "durable", desc = "Whether the message is durable") boolean durable,
+                      @Parameter(name = "user", desc = "The user to authenticate with") String user,
+                      @Parameter(name = "password", desc = "The users password to authenticate with") String password) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2b46f6a4/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/ObjectNameBuilder.java
----------------------------------------------------------------------
diff --git a/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/ObjectNameBuilder.java b/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/ObjectNameBuilder.java
index ef7b483..019996a 100644
--- a/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/ObjectNameBuilder.java
+++ b/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/ObjectNameBuilder.java
@@ -33,10 +33,6 @@ public final class ObjectNameBuilder {
     */
    public static final ObjectNameBuilder DEFAULT = new ObjectNameBuilder(ActiveMQDefaultConfiguration.getDefaultJmxDomain(), "localhost", true);
 
-   static final String JMS_MODULE = "JMS";
-
-   static final String CORE_MODULE = "Core";
-
    // Attributes ----------------------------------------------------
 
    private final String domain;
@@ -85,7 +81,7 @@ public final class ObjectNameBuilder {
     * Returns the ObjectName used by the single {@link ActiveMQServerControl}.
     */
    public ObjectName getActiveMQServerObjectName() throws Exception {
-      return ObjectName.getInstance(domain + ":" + getBrokerProperties() + "module=Core," + getObjectType() + "=Server");
+      return ObjectName.getInstance(domain + ":" + getBrokerProperties() + getObjectType() + "=Broker");
    }
 
    /**
@@ -94,7 +90,7 @@ public final class ObjectNameBuilder {
     * @see AddressControl
     */
    public ObjectName getAddressObjectName(final SimpleString address) throws Exception {
-      return createObjectName(ObjectNameBuilder.CORE_MODULE, "Address", address.toString());
+      return createObjectName("Address", address.toString());
    }
 
    /**
@@ -103,7 +99,7 @@ public final class ObjectNameBuilder {
     * @see QueueControl
     */
    public ObjectName getQueueObjectName(final SimpleString address, final SimpleString name) throws Exception {
-      return ObjectName.getInstance(String.format("%s:" + getBrokerProperties() + "module=%s," + getObjectType() + "=%s,address=%s,name=%s", domain, ObjectNameBuilder.CORE_MODULE, "Queue", ObjectName.quote(address.toString()), ObjectName.quote(name.toString())));
+      return ObjectName.getInstance(String.format("%s:" + getBrokerProperties() + "parentType=%s,parentName=%s," + getObjectType() + "=%s,name=%s", domain, "Address", ObjectName.quote(address.toString()), "Queue", ObjectName.quote(name.toString())));
    }
 
    /**
@@ -111,8 +107,8 @@ public final class ObjectNameBuilder {
     *
     * @see DivertControl
     */
-   public ObjectName getDivertObjectName(final String name) throws Exception {
-      return createObjectName(ObjectNameBuilder.CORE_MODULE, "Divert", name);
+   public ObjectName getDivertObjectName(final String name, String address) throws Exception {
+      return ObjectName.getInstance(String.format("%s:" + getBrokerProperties() + "parentType=%s,parentName=%s," + getObjectType() + "=%s,name=%s", domain, "Address", ObjectName.quote(address.toString()), "Divert", ObjectName.quote(name.toString())));
    }
 
    /**
@@ -121,7 +117,7 @@ public final class ObjectNameBuilder {
     * @see AcceptorControl
     */
    public ObjectName getAcceptorObjectName(final String name) throws Exception {
-      return createObjectName(ObjectNameBuilder.CORE_MODULE, "Acceptor", name);
+      return createObjectName("Acceptor", name);
    }
 
    /**
@@ -130,7 +126,7 @@ public final class ObjectNameBuilder {
     * @see BroadcastGroupControl
     */
    public ObjectName getBroadcastGroupObjectName(final String name) throws Exception {
-      return createObjectName(ObjectNameBuilder.CORE_MODULE, "BroadcastGroup", name);
+      return createObjectName("BroadcastGroup", name);
    }
 
    /**
@@ -139,7 +135,7 @@ public final class ObjectNameBuilder {
     * @see BridgeControl
     */
    public ObjectName getBridgeObjectName(final String name) throws Exception {
-      return createObjectName(ObjectNameBuilder.CORE_MODULE, "Bridge", name);
+      return createObjectName("Bridge", name);
    }
 
    /**
@@ -148,14 +144,14 @@ public final class ObjectNameBuilder {
     * @see ClusterConnectionControl
     */
    public ObjectName getClusterConnectionObjectName(final String name) throws Exception {
-      return createObjectName(ObjectNameBuilder.CORE_MODULE, "ClusterConnection", name);
+      return createObjectName("ClusterConnection", name);
    }
 
    /**
     * Returns the ObjectName used by DiscoveryGroupControl.
     */
    public ObjectName getDiscoveryGroupObjectName(final String name) throws Exception {
-      return createObjectName(ObjectNameBuilder.CORE_MODULE, "DiscoveryGroup", name);
+      return createObjectName("DiscoveryGroup", name);
    }
 
    /**
@@ -169,25 +165,25 @@ public final class ObjectNameBuilder {
     * Returns the ObjectName used by JMSQueueControl.
     */
    public ObjectName getJMSQueueObjectName(final String name) throws Exception {
-      return createObjectName(ObjectNameBuilder.JMS_MODULE, "Queue", name);
+      return createObjectName("Queue", name);
    }
 
    /**
     * Returns the ObjectName used by TopicControl.
     */
    public ObjectName getJMSTopicObjectName(final String name) throws Exception {
-      return createObjectName(ObjectNameBuilder.JMS_MODULE, "Topic", name);
+      return createObjectName("Topic", name);
    }
 
    /**
     * Returns the ObjectName used by ConnectionFactoryControl.
     */
    public ObjectName getConnectionFactoryObjectName(final String name) throws Exception {
-      return createObjectName(ObjectNameBuilder.JMS_MODULE, "ConnectionFactory", name);
+      return createObjectName("ConnectionFactory", name);
    }
 
-   private ObjectName createObjectName(final String module, final String type, final String name) throws Exception {
-      String format = String.format("%s:" + getBrokerProperties() + "module=%s," + getObjectType() + "=%s,name=%s", domain, module, type, ObjectName.quote(name));
+   private ObjectName createObjectName(final String type, final String name) throws Exception {
+      String format = String.format("%s:" + getBrokerProperties() + getObjectType() + "=%s,name=%s", domain, type, ObjectName.quote(name));
       return ObjectName.getInstance(format);
    }
 

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2b46f6a4/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/QueueControl.java
----------------------------------------------------------------------
diff --git a/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/QueueControl.java b/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/QueueControl.java
index 3336aae..bbf365c 100644
--- a/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/QueueControl.java
+++ b/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/QueueControl.java
@@ -338,7 +338,6 @@ public interface QueueControl {
     * @param headers  the message headers and properties to set. Can only
     *                 container Strings maped to primitive types.
     * @param body     the text to send
-    * @param userID
     * @param durable
     * @param user
     * @param password @return
@@ -348,7 +347,6 @@ public interface QueueControl {
    String sendMessage(@Parameter(name = "headers", desc = "The headers to add to the message") Map<String, String> headers,
                       @Parameter(name = "headers", desc = "A type for the message") final int type,
                       @Parameter(name = "body", desc = "The body (byte[]) of the message encoded as a string using Base64") String body,
-                      @Parameter(name = "body", desc = "The user ID to set on the message") String userID,
                       @Parameter(name = "durable", desc = "Whether the message is durable") boolean durable,
                       @Parameter(name = "user", desc = "The user to authenticate with") String user,
                       @Parameter(name = "password", desc = "The users password to authenticate with") String password) throws Exception;
@@ -431,6 +429,9 @@ public interface QueueControl {
    @Attribute(desc = "whether the queue is paused")
    boolean isPaused() throws Exception;
 
+   @Operation(desc = "Browse Messages", impact = MBeanOperationInfo.ACTION)
+   CompositeData[] browse() throws Exception;
+
    /**
     * Resets the MessagesAdded property
     */

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2b46f6a4/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/ResourceNames.java
----------------------------------------------------------------------
diff --git a/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/ResourceNames.java b/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/ResourceNames.java
index a8c7632..574cada 100644
--- a/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/ResourceNames.java
+++ b/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/ResourceNames.java
@@ -20,35 +20,26 @@ package org.apache.activemq.artemis.api.core.management;
  * Helper class used to build resource names used by management messages.
  * <br>
  * Resource's name is build by appending its <em>name</em> to its corresponding type.
- * For example, the resource name of the "foo" queue is {@code CORE_QUEUE + "foo"}.
+ * For example, the resource name of the "foo" queue is {@code QUEUE + "foo"}.
  */
 public final class ResourceNames {
+   public static final String BROKER = "broker";
 
-   public static final String CORE_SERVER = "core.server";
+   public static final String QUEUE = "queue.";
 
-   public static final String CORE_QUEUE = "core.queue.";
+   public static final String ADDRESS = "address.";
 
-   public static final String CORE_ADDRESS = "core.address.";
+   public static final String BRIDGE = "bridge.";
 
-   public static final String CORE_BRIDGE = "core.bridge.";
+   public static final String ACCEPTOR = "acceptor.";
 
-   public static final String CORE_ACCEPTOR = "core.acceptor.";
+   public static final String DIVERT = "divert.";
 
-   public static final String CORE_DIVERT = "core.divert.";
+   public static final String CORE_CLUSTER_CONNECTION = "clusterconnection.";
 
-   public static final String CORE_CLUSTER_CONNECTION = "core.clusterconnection.";
+   public static final String BROADCAST_GROUP = "broadcastgroup.";
 
-   public static final String CORE_BROADCAST_GROUP = "core.broadcastgroup.";
-
-   public static final String CORE_DISCOVERY_GROUP = "core.discovery.";
-
-   public static final String JMS_SERVER = "jms.server";
-
-//   public static final String JMS_QUEUE = "jms.queue.";
-
-//   public static final String JMS_TOPIC = "jms.topic.";
-
-   public static final String JMS_CONNECTION_FACTORY = "jms.connectionfactory.";
+   public static final String DISCOVERY_GROUP = "discovery.";
 
    private ResourceNames() {
    }

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2b46f6a4/artemis-jms-client/src/main/java/org/apache/activemq/artemis/api/jms/management/ConnectionFactoryControl.java
----------------------------------------------------------------------
diff --git a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/api/jms/management/ConnectionFactoryControl.java b/artemis-jms-client/src/main/java/org/apache/activemq/artemis/api/jms/management/ConnectionFactoryControl.java
deleted file mode 100644
index f3ddd62..0000000
--- a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/api/jms/management/ConnectionFactoryControl.java
+++ /dev/null
@@ -1,399 +0,0 @@
-/*
- * 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.activemq.artemis.api.jms.management;
-
-import org.apache.activemq.artemis.api.core.DiscoveryGroupConfiguration;
-import org.apache.activemq.artemis.api.core.TransportConfiguration;
-import org.apache.activemq.artemis.api.core.management.Operation;
-import org.apache.activemq.artemis.api.core.management.Parameter;
-
-/**
- * A ConnectionFactoryControl is used to manage a JMS ConnectionFactory. <br>
- * ActiveMQ Artemis JMS ConnectionFactory uses an underlying ClientSessionFactory to connect to ActiveMQ
- * servers. Please refer to the ClientSessionFactory for a detailed description.
- *
- * @see org.apache.activemq.artemis.api.core.client.ServerLocator
- * @see org.apache.activemq.artemis.api.core.client.ClientSessionFactory
- */
-public interface ConnectionFactoryControl {
-
-   /**
-    * Returns the configuration name of this connection factory.
-    */
-   String getName();
-
-   /**
-    * Returns the Registry bindings associated  to this connection factory.
-    */
-   String[] getRegistryBindings();
-
-   /**
-    * does ths cf support HA
-    *
-    * @return true if it supports HA
-    */
-   boolean isHA();
-
-   /**
-    * return the type of factory
-    *
-    * @return 0 = jms cf, 1 = queue cf, 2 = topic cf, 3 = xa cf, 4 = xa queue cf, 5 = xa topic cf
-    */
-   int getFactoryType();
-
-   /**
-    * Returns the Client ID of this connection factory (or {@code null} if it is not set.
-    */
-   String getClientID();
-
-   /**
-    * Sets the Client ID for this connection factory.
-    */
-   void setClientID(String clientID);
-
-   /**
-    * @return whether large messages are compressed
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#isCompressLargeMessage()
-    */
-   boolean isCompressLargeMessages();
-
-   void setCompressLargeMessages(boolean compress);
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#getClientFailureCheckPeriod()
-    */
-   long getClientFailureCheckPeriod();
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#setClientFailureCheckPeriod
-    */
-   void setClientFailureCheckPeriod(long clientFailureCheckPeriod);
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#getCallTimeout()
-    */
-   long getCallTimeout();
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#setCallTimeout(long)
-    */
-   void setCallTimeout(long callTimeout);
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#getCallFailoverTimeout()
-    */
-   long getCallFailoverTimeout();
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#setCallFailoverTimeout(long)
-    */
-
-   void setCallFailoverTimeout(long callTimeout);
-
-   /**
-    * Returns the batch size (in bytes) between acknowledgements when using DUPS_OK_ACKNOWLEDGE
-    * mode.
-    *
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#getAckBatchSize()
-    * @see javax.jms.Session#DUPS_OK_ACKNOWLEDGE
-    */
-   int getDupsOKBatchSize();
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#setAckBatchSize(int)
-    */
-   void setDupsOKBatchSize(int dupsOKBatchSize);
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#getConsumerMaxRate()
-    */
-   int getConsumerMaxRate();
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#setConsumerMaxRate(int)
-    */
-   void setConsumerMaxRate(int consumerMaxRate);
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#getConsumerWindowSize()
-    */
-   int getConsumerWindowSize();
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#setConfirmationWindowSize(int)
-    */
-   void setConsumerWindowSize(int consumerWindowSize);
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#getProducerMaxRate()
-    */
-   int getProducerMaxRate();
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#setProducerMaxRate(int)
-    */
-   void setProducerMaxRate(int producerMaxRate);
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#getConfirmationWindowSize()
-    */
-   int getConfirmationWindowSize();
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#setConfirmationWindowSize(int)
-    */
-   void setConfirmationWindowSize(int confirmationWindowSize);
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#isBlockOnAcknowledge()
-    */
-   boolean isBlockOnAcknowledge();
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#setBlockOnAcknowledge(boolean)
-    */
-   void setBlockOnAcknowledge(boolean blockOnAcknowledge);
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#isBlockOnDurableSend()
-    */
-   boolean isBlockOnDurableSend();
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#setBlockOnDurableSend(boolean)
-    */
-   void setBlockOnDurableSend(boolean blockOnDurableSend);
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#isBlockOnNonDurableSend()
-    */
-   boolean isBlockOnNonDurableSend();
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#setBlockOnNonDurableSend(boolean)
-    */
-   void setBlockOnNonDurableSend(boolean blockOnNonDurableSend);
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#isPreAcknowledge()
-    */
-   boolean isPreAcknowledge();
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#setPreAcknowledge(boolean)
-    */
-   void setPreAcknowledge(boolean preAcknowledge);
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#getConnectionTTL()
-    */
-   long getConnectionTTL();
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#setConnectionTTL(long)
-    */
-   void setConnectionTTL(long connectionTTL);
-
-   /**
-    * Returns the batch size (in bytes) between acknowledgements when using a transacted session.
-    *
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#getAckBatchSize()
-    */
-   int getTransactionBatchSize();
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#setAckBatchSize(int)
-    */
-   void setTransactionBatchSize(int transactionBatchSize);
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#getMinLargeMessageSize()
-    */
-   int getMinLargeMessageSize();
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#setMinLargeMessageSize(int)
-    */
-   void setMinLargeMessageSize(int minLargeMessageSize);
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#isAutoGroup()
-    */
-   boolean isAutoGroup();
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#setAutoGroup(boolean)
-    */
-   void setAutoGroup(boolean autoGroup);
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#getRetryInterval()
-    */
-   long getRetryInterval();
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#setRetryInterval(long)
-    */
-   void setRetryInterval(long retryInterval);
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#getRetryIntervalMultiplier()
-    */
-   double getRetryIntervalMultiplier();
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#setRetryIntervalMultiplier(double)
-    */
-   void setRetryIntervalMultiplier(double retryIntervalMultiplier);
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#getReconnectAttempts()
-    */
-   int getReconnectAttempts();
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#setReconnectAttempts(int)
-    */
-   void setReconnectAttempts(int reconnectAttempts);
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#isFailoverOnInitialConnection()
-    */
-   boolean isFailoverOnInitialConnection();
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#setFailoverOnInitialConnection(boolean)
-    */
-   void setFailoverOnInitialConnection(boolean failoverOnInitialConnection);
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#getProducerWindowSize()
-    */
-   int getProducerWindowSize();
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#setProducerWindowSize(int)
-    */
-   void setProducerWindowSize(int producerWindowSize);
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#isCacheLargeMessagesClient()
-    */
-   boolean isCacheLargeMessagesClient();
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#setCacheLargeMessagesClient(boolean)
-    */
-   void setCacheLargeMessagesClient(boolean cacheLargeMessagesClient);
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#getMaxRetryInterval()
-    */
-   long getMaxRetryInterval();
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#setMaxRetryInterval(long)
-    */
-   void setMaxRetryInterval(long retryInterval);
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#getScheduledThreadPoolMaxSize()
-    */
-   int getScheduledThreadPoolMaxSize();
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#setScheduledThreadPoolMaxSize(int)
-    */
-   void setScheduledThreadPoolMaxSize(int scheduledThreadPoolMaxSize);
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#getThreadPoolMaxSize()
-    */
-   int getThreadPoolMaxSize();
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#setThreadPoolMaxSize(int)
-    */
-   void setThreadPoolMaxSize(int threadPoolMaxSize);
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#getGroupID()
-    */
-   String getGroupID();
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#setGroupID(String)
-    */
-   void setGroupID(String groupID);
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#getInitialMessagePacketSize()
-    */
-   int getInitialMessagePacketSize();
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#isUseGlobalPools()
-    */
-   boolean isUseGlobalPools();
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#setUseGlobalPools(boolean)
-    */
-   void setUseGlobalPools(boolean useGlobalPools);
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#getConnectionLoadBalancingPolicyClassName()
-    */
-   String getConnectionLoadBalancingPolicyClassName();
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#setConnectionLoadBalancingPolicyClassName(String)
-    */
-   void setConnectionLoadBalancingPolicyClassName(String connectionLoadBalancingPolicyClassName);
-
-   /**
-    * @see org.apache.activemq.artemis.api.core.client.ServerLocator#getStaticTransportConfigurations()
-    */
-   TransportConfiguration[] getStaticConnectors();
-
-   /**
-    * get the discovery group configuration
-    */
-   DiscoveryGroupConfiguration getDiscoveryGroupConfiguration();
-
-   /**
-    * get the protocol manager factory name
-    */
-   String getProtocolManagerFactoryStr();
-
-   /**
-    * set the protocol manager factory name
-    */
-   void setProtocolManagerFactoryStr(String protocolManagerFactoryStr);
-
-   /**
-    * Add the Registry binding to this destination
-    */
-   @Operation(desc = "Adds the factory to another Registry binding")
-   void addBinding(@Parameter(name = "binding", desc = "the name of the binding for the Registry") String binding) throws Exception;
-
-   /**
-    * Remove a Registry binding
-    */
-   @Operation(desc = "Remove an existing Registry binding")
-   void removeBinding(@Parameter(name = "binding", desc = "the name of the binding for Registry") String binding) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2b46f6a4/artemis-jms-client/src/main/java/org/apache/activemq/artemis/api/jms/management/DestinationControl.java
----------------------------------------------------------------------
diff --git a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/api/jms/management/DestinationControl.java b/artemis-jms-client/src/main/java/org/apache/activemq/artemis/api/jms/management/DestinationControl.java
deleted file mode 100644
index afd07ca..0000000
--- a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/api/jms/management/DestinationControl.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * 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.activemq.artemis.api.jms.management;
-
-import javax.management.MBeanOperationInfo;
-
-import org.apache.activemq.artemis.api.core.management.Attribute;
-import org.apache.activemq.artemis.api.core.management.Operation;
-import org.apache.activemq.artemis.api.core.management.Parameter;
-
-/**
- * A DestinationControl is used to manage a JMS Destination.
- */
-public interface DestinationControl {
-   // Attributes ----------------------------------------------------
-
-   /**
-    * Returns the name of this destination.
-    */
-   @Attribute(desc = "the name of this destination")
-   String getName();
-
-   /**
-    * Returns the address corresponding to this destination.
-    */
-   @Attribute(desc = "the address corresponding to this destination")
-   String getAddress();
-
-   /**
-    * Returns whether this destination is temporary.
-    */
-   @Attribute(desc = "whether this destination is temporary")
-   boolean isTemporary();
-
-   /**
-    * Returns the number of messages currently in this destination.
-    */
-   @Attribute(desc = "the number of messages currently in this destination")
-   long getMessageCount() throws Exception;
-
-   /**
-    * Returns the number of messages that this queue is currently delivering to its consumers.
-    */
-   @Attribute(desc = "the number of messages that this queue is currently delivering to its consumers")
-   int getDeliveringCount();
-
-   /**
-    * Returns the number of messages added to this queue since it was created.
-    */
-   @Attribute(desc = "the number of messages added to this queue since it was created")
-   long getMessagesAdded();
-
-   // Operations ----------------------------------------------------
-
-   /**
-    * Removed all the messages which matches the specified JMS filter from this destination.
-    * <br>
-    * Using {@code null} or an empty filter will remove <em>all</em> messages from this destination.
-    *
-    * @return the number of removed messages
-    */
-   @Operation(desc = "Remove messages matching the given filter from the destination", impact = MBeanOperationInfo.ACTION)
-   int removeMessages(@Parameter(name = "filter", desc = "A JMS message filter (can be empty)") String filter) throws Exception;
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2b46f6a4/artemis-jms-client/src/main/java/org/apache/activemq/artemis/api/jms/management/JMSConnectionInfo.java
----------------------------------------------------------------------
diff --git a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/api/jms/management/JMSConnectionInfo.java b/artemis-jms-client/src/main/java/org/apache/activemq/artemis/api/jms/management/JMSConnectionInfo.java
deleted file mode 100644
index 64da481..0000000
--- a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/api/jms/management/JMSConnectionInfo.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/*
- * 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.activemq.artemis.api.jms.management;
-
-import javax.json.JsonArray;
-import javax.json.JsonObject;
-
-import org.apache.activemq.artemis.api.core.JsonUtil;
-
-public class JMSConnectionInfo {
-
-   private final String connectionID;
-
-   private final String clientAddress;
-
-   private final long creationTime;
-
-   private final String clientID;
-
-   private final String username;
-
-   // Static --------------------------------------------------------
-
-   public static JMSConnectionInfo[] from(final String jsonString) throws Exception {
-      JsonArray array = JsonUtil.readJsonArray(jsonString);
-      JMSConnectionInfo[] infos = new JMSConnectionInfo[array.size()];
-      for (int i = 0; i < array.size(); i++) {
-         JsonObject obj = array.getJsonObject(i);
-         String cid = obj.containsKey("clientID") ? obj.getString("clientID") : null;
-         String uname = obj.containsKey("principal") ? obj.getString("principal") : null;
-
-         JMSConnectionInfo info = new JMSConnectionInfo(obj.getString("connectionID"), obj.getString("clientAddress"), obj.getJsonNumber("creationTime").longValue(), cid, uname);
-         infos[i] = info;
-      }
-      return infos;
-   }
-
-   // Constructors --------------------------------------------------
-
-   private JMSConnectionInfo(final String connectionID,
-                             final String clientAddress,
-                             final long creationTime,
-                             final String clientID,
-                             final String username) {
-      this.connectionID = connectionID;
-      this.clientAddress = clientAddress;
-      this.creationTime = creationTime;
-      this.clientID = clientID;
-      this.username = username;
-   }
-
-   // Public --------------------------------------------------------
-
-   public String getConnectionID() {
-      return connectionID;
-   }
-
-   public String getClientAddress() {
-      return clientAddress;
-   }
-
-   public long getCreationTime() {
-      return creationTime;
-   }
-
-   public String getClientID() {
-      return clientID;
-   }
-
-   public String getUsername() {
-      return username;
-   }
-}

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2b46f6a4/artemis-jms-client/src/main/java/org/apache/activemq/artemis/api/jms/management/JMSConsumerInfo.java
----------------------------------------------------------------------
diff --git a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/api/jms/management/JMSConsumerInfo.java b/artemis-jms-client/src/main/java/org/apache/activemq/artemis/api/jms/management/JMSConsumerInfo.java
deleted file mode 100644
index 5b7f5e3..0000000
--- a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/api/jms/management/JMSConsumerInfo.java
+++ /dev/null
@@ -1,120 +0,0 @@
-/*
- * 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.activemq.artemis.api.jms.management;
-
-import javax.json.JsonArray;
-import javax.json.JsonObject;
-
-import org.apache.activemq.artemis.api.core.JsonUtil;
-
-/**
- * Helper class to create Java Objects from the
- * JSON serialization returned by {@link JMSServerControl#listConsumersAsJSON(String)} and related methods.
- */
-public class JMSConsumerInfo {
-
-   private final String consumerID;
-
-   private final String connectionID;
-
-   private final String destinationName;
-
-   private final String destinationType;
-
-   private final boolean browseOnly;
-
-   private final long creationTime;
-
-   private final boolean durable;
-
-   private final String filter;
-
-   // Static --------------------------------------------------------
-
-   /**
-    * Returns an array of SubscriptionInfo corresponding to the JSON serialization returned
-    * by {@link TopicControl#listAllSubscriptionsAsJSON()} and related methods.
-    */
-   public static JMSConsumerInfo[] from(final String jsonString) throws Exception {
-      JsonArray array = JsonUtil.readJsonArray(jsonString);
-      JMSConsumerInfo[] infos = new JMSConsumerInfo[array.size()];
-      for (int i = 0; i < array.size(); i++) {
-         JsonObject sub = array.getJsonObject(i);
-         JMSConsumerInfo info = new JMSConsumerInfo(sub.getJsonNumber("consumerID").toString(), sub.getString("connectionID"), sub.getString("destinationName"), sub.getString("destinationType"), sub.getBoolean("browseOnly"), sub.getJsonNumber("creationTime").longValue(), sub.getBoolean("durable"), sub.getString("filter", null));
-         infos[i] = info;
-      }
-
-      return infos;
-   }
-
-   // Constructors --------------------------------------------------
-
-   private JMSConsumerInfo(final String consumerID,
-                           final String connectionID,
-                           final String destinationName,
-                           final String destinationType,
-                           final boolean browseOnly,
-                           final long creationTime,
-                           final boolean durable,
-                           final String filter) {
-      this.consumerID = consumerID;
-      this.connectionID = connectionID;
-      this.destinationName = destinationName;
-      this.destinationType = destinationType;
-      this.browseOnly = browseOnly;
-      this.creationTime = creationTime;
-      this.durable = durable;
-      this.filter = filter;
-   }
-
-   // Public --------------------------------------------------------
-
-   public String getConsumerID() {
-      return consumerID;
-   }
-
-   public String getConnectionID() {
-      return connectionID;
-   }
-
-   public String getDestinationName() {
-      return destinationName;
-   }
-
-   public String getDestinationType() {
-      return destinationType;
-   }
-
-   public boolean isBrowseOnly() {
-      return browseOnly;
-   }
-
-   public long getCreationTime() {
-      return creationTime;
-   }
-
-   /**
-    * @return the durable
-    */
-   public boolean isDurable() {
-      return durable;
-   }
-
-   public String getFilter() {
-      return filter;
-   }
-}

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2b46f6a4/artemis-jms-client/src/main/java/org/apache/activemq/artemis/api/jms/management/JMSQueueControl.java
----------------------------------------------------------------------
diff --git a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/api/jms/management/JMSQueueControl.java b/artemis-jms-client/src/main/java/org/apache/activemq/artemis/api/jms/management/JMSQueueControl.java
deleted file mode 100644
index 3a4101a..0000000
--- a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/api/jms/management/JMSQueueControl.java
+++ /dev/null
@@ -1,443 +0,0 @@
-/*
- * 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.activemq.artemis.api.jms.management;
-
-import javax.management.MBeanOperationInfo;
-import javax.management.openmbean.CompositeData;
-import java.util.Map;
-
-import org.apache.activemq.artemis.api.core.management.Attribute;
-import org.apache.activemq.artemis.api.core.management.Operation;
-import org.apache.activemq.artemis.api.core.management.Parameter;
-
-/**
- * A JMSQueueControl is used to manage a JMS queue.
- */
-public interface JMSQueueControl extends DestinationControl {
-   // Attributes ----------------------------------------------------
-
-   /**
-    * Returns the expiry address associated with this queue.
-    */
-   @Attribute(desc = "expiry address associated with this queue")
-   String getExpiryAddress();
-
-   /**
-    * Returns the dead-letter address associated with this queue.
-    */
-   @Attribute(desc = "dead-letter address associated with this queue")
-   String getDeadLetterAddress();
-
-   /**
-    * Returns the number of scheduled messages in this queue.
-    */
-   @Attribute(desc = "number of scheduled messages in this queue")
-   long getScheduledCount();
-
-   /**
-    * Returns the number of consumers consuming messages from this queue.
-    */
-   @Attribute(desc = "number of consumers consuming messages from this queue")
-   int getConsumerCount();
-
-   /**
-    * Returns the number of messages expired from this queue since it was created.
-    */
-   @Attribute(desc = "the number of messages expired from this queue since it was created")
-   long getMessagesExpired();
-
-   /**
-    * Returns the number of messages removed from this queue since it was created due to exceeding the max delivery attempts.
-    */
-   @Attribute(desc = "number of messages removed from this queue since it was created due to exceeding the max delivery attempts")
-   long getMessagesKilled();
-
-   /**
-    * returns the selector for the queue
-    */
-   @Attribute(desc = "selector for the queue")
-   String getSelector();
-
-   /**
-    * Returns the first message on the queue as JSON
-    */
-   @Attribute(desc = "first message on the queue as JSON")
-   String getFirstMessageAsJSON() throws Exception;
-
-   /**
-    * Returns the timestamp of the first message in milliseconds.
-    */
-   @Attribute(desc = "timestamp of the first message in milliseconds")
-   Long getFirstMessageTimestamp() throws Exception;
-
-   /**
-    * Returns the age of the first message in milliseconds.
-    */
-   @Attribute(desc = "age of the first message in milliseconds")
-   Long getFirstMessageAge() throws Exception;
-
-   // Operations ----------------------------------------------------
-
-   /**
-    * Returns the Registry bindings associated with this queue.
-    */
-   @Attribute(desc = "Returns the list of Registry bindings associated with this queue")
-   String[] getRegistryBindings();
-
-   /**
-    * Add the JNDI binding to this destination
-    */
-   @Operation(desc = "Adds the queue to another Registry binding")
-   void addBinding(@Parameter(name = "binding", desc = "the name of the binding for the registry") String binding) throws Exception;
-
-   /**
-    * Lists all the JMS messages in this queue matching the specified filter.
-    * <br>
-    * 1 Map represents 1 message, keys are the message's properties and headers, values are the corresponding values.
-    * <br>
-    * Using {@code null} or an empty filter will list <em>all</em> messages from this queue.
-    */
-   @Operation(desc = "List all messages in the queue which matches the filter", impact = MBeanOperationInfo.INFO)
-   Map<String, Object>[] listMessages(@Parameter(name = "filter", desc = "A JMS Message filter") String filter) throws Exception;
-
-   /**
-    * Lists all the JMS messages in this queue matching the specified filter using JSON serialization.
-    * <br>
-    * Using {@code null} or an empty filter will list <em>all</em> messages from this queue.
-    */
-   @Operation(desc = "List all messages in the queue which matches the filter and return them using JSON", impact = MBeanOperationInfo.INFO)
-   String listMessagesAsJSON(@Parameter(name = "filter", desc = "A JMS Message filter (can be empty)") String filter) throws Exception;
-
-   /**
-    * Counts the number of messages in this queue matching the specified filter.
-    * <br>
-    * Using {@code null} or an empty filter will count <em>all</em> messages from this queue.
-    */
-   @Operation(desc = "Returns the number of the messages in the queue matching the given filter", impact = MBeanOperationInfo.INFO)
-   long countMessages(@Parameter(name = "filter", desc = "A JMS message filter (can be empty)") String filter) throws Exception;
-
-   /**
-    * Removes the message corresponding to the specified message ID.
-    *
-    * @return {@code true} if the message was removed, {@code false} else
-    */
-   @Operation(desc = "Remove the message corresponding to the given messageID", impact = MBeanOperationInfo.ACTION)
-   boolean removeMessage(@Parameter(name = "messageID", desc = "A message ID") String messageID) throws Exception;
-
-   /**
-    * Removes all the message corresponding to the specified filter.
-    * <br>
-    * Using {@code null} or an empty filter will remove <em>all</em> messages from this queue.
-    *
-    * @return the number of removed messages
-    */
-   @Override
-   @Operation(desc = "Remove the messages corresponding to the given filter (and returns the number of removed messages)", impact = MBeanOperationInfo.ACTION)
-   int removeMessages(@Parameter(name = "filter", desc = "A message filter (can be empty)") String filter) throws Exception;
-
-   /**
-    * Expires all the message corresponding to the specified filter.
-    * <br>
-    * Using {@code null} or an empty filter will expire <em>all</em> messages from this queue.
-    *
-    * @return the number of expired messages
-    */
-   @Operation(desc = "Expire the messages corresponding to the given filter (and returns the number of expired messages)", impact = MBeanOperationInfo.ACTION)
-   int expireMessages(@Parameter(name = "filter", desc = "A message filter (can be empty)") String filter) throws Exception;
-
-   /**
-    * Expires the message corresponding to the specified message ID.
-    *
-    * @return {@code true} if the message was expired, {@code false} else
-    */
-   @Operation(desc = "Expire the message corresponding to the given messageID", impact = MBeanOperationInfo.ACTION)
-   boolean expireMessage(@Parameter(name = "messageID", desc = "A message ID") String messageID) throws Exception;
-
-   /**
-    * Sends the message corresponding to the specified message ID to this queue's dead letter address.
-    *
-    * @return {@code true} if the message was sent to the dead letter address, {@code false} else
-    */
-   @Operation(desc = "Send the message corresponding to the given messageID to this queue's Dead Letter Address", impact = MBeanOperationInfo.ACTION)
-   boolean sendMessageToDeadLetterAddress(@Parameter(name = "messageID", desc = "A message ID") String messageID) throws Exception;
-
-   /**
-    * Sends all the message corresponding to the specified filter to this queue's dead letter address.
-    * <br>
-    * Using {@code null} or an empty filter will send <em>all</em> messages from this queue.
-    *
-    * @return the number of sent messages
-    */
-   @Operation(desc = "Send the messages corresponding to the given filter to this queue's Dead Letter Address", impact = MBeanOperationInfo.ACTION)
-   int sendMessagesToDeadLetterAddress(@Parameter(name = "filter", desc = "A message filter (can be empty)") String filterStr) throws Exception;
-
-   /**
-    * Sends a TextMesage to the destination.
-    *
-    * @param body the text to send
-    * @return the message id of the message sent.
-    * @throws Exception
-    */
-   @Operation(desc = "Sends a TextMessage to a password-protected destination.", impact = MBeanOperationInfo.ACTION)
-   String sendTextMessage(@Parameter(name = "body") String body) throws Exception;
-
-   /**
-    * Sends a TextMessage to the destination.
-    *
-    * @param properties the message properties to set as a comma sep name=value list. Can only
-    *                   contain Strings maped to primitive types or JMS properties. eg: body=hi,JMSReplyTo=Queue2
-    * @return the message id of the message sent.
-    * @throws Exception
-    */
-   @Operation(desc = "Sends a TextMessage to a password-protected destination.", impact = MBeanOperationInfo.ACTION)
-   String sendTextMessageWithProperties(String properties) throws Exception;
-
-   /**
-    * Sends a TextMesage to the destination.
-    *
-    * @param headers the message headers and properties to set. Can only
-    *                container Strings maped to primitive types.
-    * @param body    the text to send
-    * @return the message id of the message sent.
-    * @throws Exception
-    */
-   @Operation(desc = "Sends a TextMessage to a password-protected destination.", impact = MBeanOperationInfo.ACTION)
-   String sendTextMessage(@Parameter(name = "headers") Map<String, String> headers,
-                          @Parameter(name = "body") String body) throws Exception;
-
-   /**
-    * Sends a TextMesage to the destination.
-    *
-    * @param body     the text to send
-    * @param user
-    * @param password
-    * @return
-    * @throws Exception
-    */
-   @Operation(desc = "Sends a TextMessage to a password-protected destination.", impact = MBeanOperationInfo.ACTION)
-   String sendTextMessage(@Parameter(name = "body") String body,
-                          @Parameter(name = "user") String user,
-                          @Parameter(name = "password") String password) throws Exception;
-
-   /**
-    * @param headers  the message headers and properties to set. Can only
-    *                 container Strings maped to primitive types.
-    * @param body     the text to send
-    * @param user
-    * @param password
-    * @return
-    * @throws Exception
-    */
-   @Operation(desc = "Sends a TextMessage to a password-protected destination.", impact = MBeanOperationInfo.ACTION)
-   String sendTextMessage(@Parameter(name = "headers") Map<String, String> headers,
-                          @Parameter(name = "body") String body,
-                          @Parameter(name = "user") String user,
-                          @Parameter(name = "password") String password) throws Exception;
-
-   /**
-    * Changes the message's priority corresponding to the specified message ID to the specified priority.
-    *
-    * @param newPriority between 0 and 9 inclusive.
-    * @return {@code true} if the message priority was changed
-    */
-   @Operation(desc = "Change the priority of the message corresponding to the given messageID", impact = MBeanOperationInfo.ACTION)
-   boolean changeMessagePriority(@Parameter(name = "messageID", desc = "A message ID") String messageID,
-                                 @Parameter(name = "newPriority", desc = "the new priority (between 0 and 9)") int newPriority) throws Exception;
-
-   /**
-    * Changes the priority for all the message corresponding to the specified filter to the specified priority.
-    * <br>
-    * Using {@code null} or an empty filter will change <em>all</em> messages from this queue.
-    *
-    * @return the number of changed messages
-    */
-   @Operation(desc = "Change the priority of the messages corresponding to the given filter", impact = MBeanOperationInfo.ACTION)
-   int changeMessagesPriority(@Parameter(name = "filter", desc = "A message filter") String filter,
-                              @Parameter(name = "newPriority", desc = "the new priority (between 0 and 9)") int newPriority) throws Exception;
-
-   /**
-    * Moves the message corresponding to the specified message ID to the specified other queue.
-    *
-    * @return {@code true} if the message was moved, {@code false} else
-    */
-   @Operation(desc = "Move the message corresponding to the given messageID to another queue, ignoring duplicates (rejectDuplicates=false on this case)", impact = MBeanOperationInfo.ACTION)
-   boolean moveMessage(@Parameter(name = "messageID", desc = "A message ID") String messageID,
-                       @Parameter(name = "otherQueueName", desc = "The name of the queue to move the message to") String otherQueueName) throws Exception;
-
-   /**
-    * Moves the message corresponding to the specified message ID to the specified other queue.
-    *
-    * @return {@code true} if the message was moved, {@code false} else
-    */
-   @Operation(desc = "Move the message corresponding to the given messageID to another queue", impact = MBeanOperationInfo.ACTION)
-   boolean moveMessage(@Parameter(name = "messageID", desc = "A message ID") String messageID,
-                       @Parameter(name = "otherQueueName", desc = "The name of the queue to move the message to") String otherQueueName,
-                       @Parameter(name = "rejectDuplicates", desc = "Reject messages identified as duplicate by the duplicate message") boolean rejectDuplicates) throws Exception;
-
-   /**
-    * Moves all the message corresponding to the specified filter  to the specified other queue.
-    * RejectDuplicates=false on this case
-    * <br>
-    * Using {@code null} or an empty filter will move <em>all</em> messages from this queue.
-    *
-    * @return the number of moved messages
-    */
-   @Operation(desc = "Move the messages corresponding to the given filter (and returns the number of moved messages). rejectDuplicates=false on this case", impact = MBeanOperationInfo.ACTION)
-   int moveMessages(@Parameter(name = "filter", desc = "A message filter (can be empty)") String filter,
-                    @Parameter(name = "otherQueueName", desc = "The name of the queue to move the messages to") String otherQueueName) throws Exception;
-
-   /**
-    * Moves all the message corresponding to the specified filter  to the specified other queue.
-    * <br>
-    * Using {@code null} or an empty filter will move <em>all</em> messages from this queue.
-    *
-    * @return the number of moved messages
-    */
-   @Operation(desc = "Move the messages corresponding to the given filter (and returns the number of moved messages)", impact = MBeanOperationInfo.ACTION)
-   int moveMessages(@Parameter(name = "filter", desc = "A message filter (can be empty)") String filter,
-                    @Parameter(name = "otherQueueName", desc = "The name of the queue to move the messages to") String otherQueueName,
-                    @Parameter(name = "rejectDuplicates", desc = "Reject messages identified as duplicate by the duplicate message") boolean rejectDuplicates) throws Exception;
-
-   /**
-    * Retries the message corresponding to the given messageID to the original queue.
-    * This is appropriate on dead messages on Dead letter queues only.
-    *
-    * @param messageID
-    * @return {@code true} if the message was retried, {@code false}�else
-    * @throws Exception
-    */
-   @Operation(desc = "Retry the message corresponding to the given messageID to the original queue", impact = MBeanOperationInfo.ACTION)
-   boolean retryMessage(@Parameter(name = "messageID", desc = "A message ID") String messageID) throws Exception;
-
-   /**
-    * Retries all messages on a DLQ to their respective original queues.
-    * This is appropriate on dead messages on Dead letter queues only.
-    *
-    * @return the number of retried messages.
-    * @throws Exception
-    */
-   @Operation(desc = "Retry all messages on a DLQ to their respective original queues", impact = MBeanOperationInfo.ACTION)
-   int retryMessages() throws Exception;
-
-   /**
-    * Lists the message counter for this queue.
-    */
-   @Operation(desc = "List the message counters", impact = MBeanOperationInfo.INFO)
-   String listMessageCounter() throws Exception;
-
-   /**
-    * Resets the message counter for this queue.
-    */
-   @Operation(desc = "Reset the message counters", impact = MBeanOperationInfo.INFO)
-   void resetMessageCounter() throws Exception;
-
-   /**
-    * Lists the message counter for this queue as a HTML table.
-    */
-   @Operation(desc = "List the message counters as HTML", impact = MBeanOperationInfo.INFO)
-   String listMessageCounterAsHTML() throws Exception;
-
-   /**
-    * Lists the message counter history for this queue.
-    */
-   @Operation(desc = "List the message counters history", impact = MBeanOperationInfo.INFO)
-   String listMessageCounterHistory() throws Exception;
-
-   /**
-    * Lists the message counter history for this queue as a HTML table.
-    */
-   @Operation(desc = "List the message counters history as HTML", impact = MBeanOperationInfo.INFO)
-   String listMessageCounterHistoryAsHTML() throws Exception;
-
-   /**
-    * Pauses the queue. Messages are no longer delivered to its consumers.
-    */
-   @Operation(desc = "Pause the queue.", impact = MBeanOperationInfo.ACTION)
-   void pause() throws Exception;
-
-   /**
-    * Pauses the queue. Messages are no longer delivered to its consumers.
-    */
-   @Operation(desc = "Pauses the Queue", impact = MBeanOperationInfo.ACTION)
-   void pause(@Parameter(name = "persist", desc = "if true, the pause state will be persisted.") boolean persist) throws Exception;
-
-   /**
-    * Returns whether the queue is paused.
-    */
-   @Attribute(desc = "Returns true if the queue is paused.")
-   boolean isPaused() throws Exception;
-
-   /**
-    * Resumes the queue. Messages are again delivered to its consumers.
-    */
-   @Operation(desc = "Resume the queue.", impact = MBeanOperationInfo.ACTION)
-   void resume() throws Exception;
-
-   /**
-    * Resumes the queue. Messages are again delivered to its consumers.
-    */
-   @Operation(desc = "Browse the queue.", impact = MBeanOperationInfo.ACTION)
-   CompositeData[] browse() throws Exception;
-
-   /**
-    * Resumes the queue. Messages are again delivered to its consumers.
-    */
-   @Operation(desc = "Browse the queue.", impact = MBeanOperationInfo.ACTION)
-   CompositeData[] browse(String filter) throws Exception;
-
-   @Operation(desc = "List all the existent consumers on the Queue")
-   String listConsumersAsJSON() throws Exception;
-
-   /**
-    * it will flush one cycle on internal executors, so you would be sure that any pending tasks are done before you call
-    * any other measure.
-    * It is useful if you need the exact number of counts on a message
-    */
-   void flushExecutor();
-
-   /**
-    * Lists all the messages scheduled for delivery for this queue.
-    * <br>
-    * 1 Map represents 1 message, keys are the message's properties and headers, values are the corresponding values.
-    */
-   @Operation(desc = "List the messages scheduled for delivery", impact = MBeanOperationInfo.INFO)
-   Map<String, Object>[] listScheduledMessages() throws Exception;
-
-   /**
-    * Lists all the messages scheduled for delivery for this queue using JSON serialization.
-    */
-   @Operation(desc = "List the messages scheduled for delivery and returns them using JSON", impact = MBeanOperationInfo.INFO)
-   String listScheduledMessagesAsJSON() throws Exception;
-
-   /**
-    * Lists all the messages being deliver per consumer.
-    * <br>
-    * The Map's key is a toString representation for the consumer. Each consumer will then return a {@code Map<String,Object>[]} same way is returned by {@link #listScheduledMessages()}
-    */
-   @Operation(desc = "List all messages being delivered per consumer")
-   Map<String, Map<String, Object>[]> listDeliveringMessages() throws Exception;
-
-   /**
-    * Executes a conversion of {@link #listDeliveringMessages()} to JSON
-    *
-    * @return
-    * @throws Exception
-    */
-   @Operation(desc = "list all messages being delivered per consumer using JSON form")
-   String listDeliveringMessagesAsJSON() throws Exception;
-
-}