Adds username to some MBeans when enabled in BrokerService to allow for tracking of the authenticated user of connections and other resources.

git-svn-id: https://svn.apache.org/repos/asf/activemq/trunk@1292599 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Timothy A. Bish 2012-02-23 00:40:13 +00:00
parent 757107d7aa
commit f5dcc9dfbf
13 changed files with 221 additions and 66 deletions

View File

@ -129,6 +129,7 @@ public class BrokerService implements Service {
private boolean persistent = true;
private boolean populateJMSXUserID;
private boolean useAuthenticatedPrincipalForJMSXUserID;
private boolean populateUserNameInMBeans;
private boolean useShutdownHook = true;
private boolean useLoggingForShutdownErrors;
@ -2578,6 +2579,24 @@ public class BrokerService implements Service {
this.useAuthenticatedPrincipalForJMSXUserID = useAuthenticatedPrincipalForJMSXUserID;
}
/**
* Should MBeans that support showing the Authenticated User Name information have this
* value filled in or not.
*
* @return true if user names should be exposed in MBeans
*/
public boolean isPopulateUserNameInMBeans() {
return this.populateUserNameInMBeans;
}
/**
* Sets whether Authenticated User Name information is shown in MBeans that support this field.
* @param true if MBeans should expose user name information.
*/
public void setPopulateUserNameInMBeans(boolean value) {
this.populateUserNameInMBeans = value;
}
public boolean isNetworkConnectorStartAsync() {
return networkConnectorStartAsync;
}

View File

@ -21,6 +21,7 @@ import org.apache.activemq.broker.Connection;
public class ConnectionView implements ConnectionViewMBean {
private final Connection connection;
private String userName;
public ConnectionView(Connection connection) {
this.connection = connection;
@ -81,4 +82,12 @@ public class ConnectionView implements ConnectionViewMBean {
return connection.getConnectionId();
}
@Override
public String getUserName() {
return userName;
}
public void setUserName(String userName) {
this.userName = userName;
}
}

View File

@ -42,16 +42,16 @@ public interface ConnectionViewMBean extends Service {
*/
@MBeanInfo("Connection is active (both connected and receiving messages).")
boolean isActive();
/**
* Resets the statistics
*/
@MBeanInfo("Resets the statistics")
void resetStatistics();
/**
* Returns the source address for this connection
*
*
* @return the source address for this connection
*/
@MBeanInfo("Source address for this connection")
@ -64,4 +64,12 @@ public interface ConnectionViewMBean extends Service {
@MBeanInfo("The number of messages pending dispatch")
public int getDispatchQueueSize();
/**
* Returns the User Name used to authorize creation of this Connection.
* This value can be null if display of user name information is disabled.
*
* @return the name of the user that created this Connection
*/
@MBeanInfo("User Name used to authorize creation of this connection")
String getUserName();
}

View File

@ -26,7 +26,7 @@ import org.apache.activemq.broker.region.Subscription;
import org.apache.activemq.command.RemoveSubscriptionInfo;
/**
*
*
*/
public class DurableSubscriptionView extends SubscriptionView implements DurableSubscriptionViewMBean {
@ -36,12 +36,12 @@ public class DurableSubscriptionView extends SubscriptionView implements Durable
/**
* Constructor
*
*
* @param clientId
* @param sub
*/
public DurableSubscriptionView(ManagedRegionBroker broker, String clientId, Subscription sub) {
super(clientId, sub);
public DurableSubscriptionView(ManagedRegionBroker broker, String clientId, String userName, Subscription sub) {
super(clientId, userName, sub);
this.broker = broker;
this.durableSub=(DurableTopicSubscription) sub;
if (sub != null) {
@ -58,7 +58,7 @@ public class DurableSubscriptionView extends SubscriptionView implements Durable
/**
* Browse messages for this durable subscriber
*
*
* @return messages
* @throws OpenDataException
*/
@ -68,7 +68,7 @@ public class DurableSubscriptionView extends SubscriptionView implements Durable
/**
* Browse messages for this durable subscriber
*
*
* @return messages
* @throws OpenDataException
*/
@ -94,7 +94,7 @@ public class DurableSubscriptionView extends SubscriptionView implements Durable
return "ActiveDurableSubscriptionView: " + getClientId() + ":" + getSubscriptionName();
}
public int cursorSize() {
if (durableSub != null && durableSub.getPending() != null) {
return durableSub.getPending().size();
@ -102,7 +102,7 @@ public class DurableSubscriptionView extends SubscriptionView implements Durable
return 0;
}
public boolean doesCursorHaveMessagesBuffered() {
if (durableSub != null && durableSub.getPending() != null) {
return durableSub.getPending().hasMessagesBufferedToDeliver();
@ -110,7 +110,7 @@ public class DurableSubscriptionView extends SubscriptionView implements Durable
return false;
}
public boolean doesCursorHaveSpace() {
if (durableSub != null && durableSub.getPending() != null) {
return durableSub.getPending().hasSpace();
@ -128,7 +128,7 @@ public class DurableSubscriptionView extends SubscriptionView implements Durable
return 0;
}
public int getCursorPercentUsage() {
if (durableSub != null && durableSub.getPending() != null && durableSub.getPending().getSystemUsage()!=null) {
return durableSub.getPending().getSystemUsage().getMemoryUsage().getPercentUsage();
@ -147,6 +147,6 @@ public class DurableSubscriptionView extends SubscriptionView implements Durable
public boolean isActive() {
return durableSub.isActive();
}
}

View File

@ -27,21 +27,22 @@ import org.apache.activemq.command.RemoveSubscriptionInfo;
import org.apache.activemq.command.SubscriptionInfo;
/**
*
*
*
*
*/
public class InactiveDurableSubscriptionView extends DurableSubscriptionView implements DurableSubscriptionViewMBean {
protected SubscriptionInfo subscriptionInfo;
/**
* Constructor
*
*
* @param broker
* @param clientId
* @param userName
* @param subInfo
*/
public InactiveDurableSubscriptionView(ManagedRegionBroker broker, String clientId, SubscriptionInfo subInfo, Subscription subscription) {
super(broker,clientId, subscription);
super(broker, clientId, null, subscription);
this.broker = broker;
this.subscriptionInfo = subInfo;
}
@ -104,7 +105,7 @@ public class InactiveDurableSubscriptionView extends DurableSubscriptionView imp
/**
* Browse messages for this durable subscriber
*
*
* @return messages
* @throws OpenDataException
*/
@ -114,7 +115,7 @@ public class InactiveDurableSubscriptionView extends DurableSubscriptionView imp
/**
* Browse messages for this durable subscriber
*
*
* @return messages
* @throws OpenDataException
*/

View File

@ -23,11 +23,12 @@ import java.util.Hashtable;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.Map.Entry;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CopyOnWriteArraySet;
import java.util.concurrent.ThreadPoolExecutor;
import javax.management.InstanceNotFoundException;
import javax.management.MalformedObjectNameException;
import javax.management.ObjectName;
@ -38,6 +39,7 @@ import javax.management.openmbean.OpenDataException;
import javax.management.openmbean.TabularData;
import javax.management.openmbean.TabularDataSupport;
import javax.management.openmbean.TabularType;
import org.apache.activemq.broker.Broker;
import org.apache.activemq.broker.BrokerService;
import org.apache.activemq.broker.ConnectionContext;
@ -199,13 +201,14 @@ public class ManagedRegionBroker extends RegionBroker {
info.setSelector(sub.getSelector());
addInactiveSubscription(key, info, sub);
} else {
String userName = brokerService.isPopulateUserNameInMBeans() ? context.getUserName() : null;
if (sub.getConsumerInfo().isDurable()) {
view = new DurableSubscriptionView(this, context.getClientId(), sub);
view = new DurableSubscriptionView(this, context.getClientId(), userName, sub);
} else {
if (sub instanceof TopicSubscription) {
view = new TopicSubscriptionView(context.getClientId(), (TopicSubscription) sub);
view = new TopicSubscriptionView(context.getClientId(), userName, (TopicSubscription) sub);
} else {
view = new SubscriptionView(context.getClientId(), sub);
view = new SubscriptionView(context.getClientId(), userName, sub);
}
}
registerSubscription(objectName, sub.getConsumerInfo(), key, view);
@ -219,7 +222,7 @@ public class ManagedRegionBroker extends RegionBroker {
}
public static String getSubscriptionObjectName(ConsumerInfo info, String connectionClientId, ObjectName brokerJmxObjectName) {
Hashtable map = brokerJmxObjectName.getKeyPropertyList();
Hashtable<String, String> map = brokerJmxObjectName.getKeyPropertyList();
String brokerDomain = brokerJmxObjectName.getDomain();
String objectNameStr = brokerDomain + ":" + "BrokerName=" + map.get("BrokerName") + ",Type=Subscription,";
String destinationType = "destinationType=" + info.getDestination().getDestinationTypeAsString();
@ -272,7 +275,8 @@ public class ManagedRegionBroker extends RegionBroker {
super.addProducer(context, info);
String connectionClientId = context.getClientId();
ObjectName objectName = createObjectName(info, connectionClientId);
ProducerView view = new ProducerView(info, connectionClientId, this);
String userName = brokerService.isPopulateUserNameInMBeans() ? context.getUserName() : null;
ProducerView view = new ProducerView(info, connectionClientId, userName, this);
registerProducer(objectName, info.getDestination(), view);
}
@ -491,10 +495,9 @@ public class ManagedRegionBroker extends RegionBroker {
protected void buildExistingSubscriptions() throws Exception {
Map<SubscriptionKey, SubscriptionInfo> subscriptions = new HashMap<SubscriptionKey, SubscriptionInfo>();
Set destinations = destinationFactory.getDestinations();
Set<ActiveMQDestination> destinations = destinationFactory.getDestinations();
if (destinations != null) {
for (Iterator iter = destinations.iterator(); iter.hasNext();) {
ActiveMQDestination dest = (ActiveMQDestination)iter.next();
for (ActiveMQDestination dest : destinations) {
if (dest.isTopic()) {
SubscriptionInfo[] infos = destinationFactory.getAllDurableSubscriptions((ActiveMQTopic)dest);
if (infos != null) {
@ -510,11 +513,9 @@ public class ManagedRegionBroker extends RegionBroker {
}
}
}
for (Iterator i = subscriptions.entrySet().iterator(); i.hasNext();) {
Map.Entry entry = (Entry)i.next();
SubscriptionKey key = (SubscriptionKey)entry.getKey();
SubscriptionInfo info = (SubscriptionInfo)entry.getValue();
addInactiveSubscription(key, info, null);
for (Map.Entry<SubscriptionKey, SubscriptionInfo> entry : subscriptions.entrySet()) {
addInactiveSubscription(entry.getKey(), entry.getValue(), null);
}
}
@ -693,7 +694,7 @@ public class ManagedRegionBroker extends RegionBroker {
protected ObjectName createObjectName(ActiveMQDestination destName) throws MalformedObjectNameException {
// Build the object name for the destination
Hashtable map = brokerObjectName.getKeyPropertyList();
Hashtable<String, String> map = brokerObjectName.getKeyPropertyList();
ObjectName objectName = new ObjectName(brokerObjectName.getDomain() + ":" + "BrokerName=" + map.get("BrokerName") + "," + "Type="
+ JMXSupport.encodeObjectNamePart(destName.getDestinationTypeAsString()) + "," + "Destination="
+ JMXSupport.encodeObjectNamePart(destName.getPhysicalName()));
@ -702,7 +703,7 @@ public class ManagedRegionBroker extends RegionBroker {
protected ObjectName createObjectName(ProducerInfo producerInfo, String connectionClientId) throws MalformedObjectNameException {
// Build the object name for the producer info
Hashtable map = brokerObjectName.getKeyPropertyList();
Hashtable<String, String> map = brokerObjectName.getKeyPropertyList();
String destinationType = "destinationType=";
String destinationName = "destinationName=";
@ -743,7 +744,7 @@ public class ManagedRegionBroker extends RegionBroker {
}
protected ObjectName createObjectName(XATransaction transaction) throws MalformedObjectNameException {
Hashtable map = brokerObjectName.getKeyPropertyList();
Hashtable<String, String> map = brokerObjectName.getKeyPropertyList();
ObjectName objectName = new ObjectName(brokerObjectName.getDomain() + ":" + "BrokerName=" + map.get("BrokerName")
+ "," + "Type=RecoveredXaTransaction"
+ "," + "Xid="
@ -782,7 +783,7 @@ public class ManagedRegionBroker extends RegionBroker {
}
private ObjectName createObjectName(AbortSlowConsumerStrategy strategy) throws MalformedObjectNameException{
Hashtable map = brokerObjectName.getKeyPropertyList();
Hashtable<String, String> map = brokerObjectName.getKeyPropertyList();
ObjectName objectName = new ObjectName(brokerObjectName.getDomain() + ":" + "BrokerName=" + map.get("BrokerName") + ","
+ "Type=SlowConsumerStrategy," + "InstanceName=" + JMXSupport.encodeObjectNamePart(strategy.getName()));
return objectName;

View File

@ -16,6 +16,11 @@
*/
package org.apache.activemq.broker.jmx;
import java.io.IOException;
import java.util.Hashtable;
import javax.management.ObjectName;
import org.apache.activemq.broker.Broker;
import org.apache.activemq.broker.TransportConnection;
import org.apache.activemq.broker.TransportConnector;
@ -27,14 +32,9 @@ import org.apache.activemq.util.IOExceptionSupport;
import org.apache.activemq.util.JMXSupport;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.Hashtable;
import javax.management.ObjectName;
/**
* A managed transport connection
*
*
*/
public class ManagedTransportConnection extends TransportConnection {
private static final Logger LOG = LoggerFactory.getLogger(ManagedTransportConnection.class);
@ -46,6 +46,8 @@ public class ManagedTransportConnection extends TransportConnection {
private ObjectName byClientIdName;
private ObjectName byAddressName;
private final boolean populateUserName;
public ManagedTransportConnection(TransportConnector connector, Transport transport, Broker broker,
TaskRunnerFactory factory, ManagementContext context, ObjectName connectorName)
throws IOException {
@ -53,6 +55,7 @@ public class ManagedTransportConnection extends TransportConnection {
this.managementContext = context;
this.connectorName = connectorName;
this.mbean = new ConnectionView(this);
this.populateUserName = broker.getBrokerService().isPopulateUserNameInMBeans();
if (managementContext.isAllowRemoteAddressInMBeanNames()) {
byAddressName = createByAddressObjectName("address", transport.getRemoteAddress());
registerMBean(byAddressName);
@ -76,6 +79,9 @@ public class ManagedTransportConnection extends TransportConnection {
public Response processAddConnection(ConnectionInfo info) throws Exception {
Response answer = super.processAddConnection(info);
String clientId = info.getClientId();
if (populateUserName) {
((ConnectionView) mbean).setUserName(info.getUserName());
}
if (clientId != null) {
if (byClientIdName == null) {
byClientIdName = createByClientIdObjectName(clientId);
@ -110,7 +116,7 @@ public class ManagedTransportConnection extends TransportConnection {
}
protected ObjectName createByAddressObjectName(String type, String value) throws IOException {
Hashtable map = connectorName.getKeyPropertyList();
Hashtable<String, String> map = connectorName.getKeyPropertyList();
try {
return new ObjectName(connectorName.getDomain() + ":" + "BrokerName="
+ JMXSupport.encodeObjectNamePart((String)map.get("BrokerName")) + ","
@ -124,7 +130,7 @@ public class ManagedTransportConnection extends TransportConnection {
}
protected ObjectName createByClientIdObjectName(String value) throws IOException {
Hashtable map = connectorName.getKeyPropertyList();
Hashtable<String, String> map = connectorName.getKeyPropertyList();
try {
return new ObjectName(connectorName.getDomain() + ":" + "BrokerName="
+ JMXSupport.encodeObjectNamePart((String)map.get("BrokerName")) + ","

View File

@ -23,13 +23,15 @@ public class ProducerView implements ProducerViewMBean {
protected final ProducerInfo info;
protected final String clientId;
protected final String userName;
protected final ManagedRegionBroker broker;
protected ActiveMQDestination lastUsedDestination;
public ProducerView(ProducerInfo info, String clientId, ManagedRegionBroker broker) {
public ProducerView(ProducerInfo info, String clientId, String userName, ManagedRegionBroker broker) {
this.info = info;
this.clientId = clientId;
this.userName = userName;
this.broker = broker;
}
@ -141,4 +143,9 @@ public class ProducerView implements ProducerViewMBean {
void setLastUsedDestinationName(ActiveMQDestination destinationName) {
this.lastUsedDestination = destinationName;
}
@Override
public String getUserName() {
return userName;
}
}

View File

@ -77,4 +77,13 @@ public interface ProducerViewMBean {
*/
@MBeanInfo("Is the producer configured for Async Dispatch")
boolean isDispatchAsync();
/**
* Returns the User Name used to authorize creation of this Producer.
* This value can be null if display of user name information is disabled.
*
* @return the name of the user that created this Producer
*/
@MBeanInfo("User Name used to authorize creation of this Producer")
String getUserName();
}

View File

@ -20,27 +20,29 @@ import javax.jms.InvalidSelectorException;
import org.apache.activemq.broker.region.Subscription;
import org.apache.activemq.command.ActiveMQDestination;
import org.apache.activemq.command.ConsumerInfo;
import org.apache.activemq.command.ActiveMQQueue;
import org.apache.activemq.command.ActiveMQTopic;
import org.apache.activemq.command.ConsumerInfo;
import org.apache.activemq.filter.DestinationFilter;
/**
*
*
*/
public class SubscriptionView implements SubscriptionViewMBean {
protected final Subscription subscription;
protected final String clientId;
protected final String userName;
/**
* Constructor
*
*
* @param subs
*/
public SubscriptionView(String clientId, Subscription subs) {
public SubscriptionView(String clientId, String userName, Subscription subs) {
this.clientId = clientId;
this.subscription = subs;
this.userName = userName;
}
/**
@ -236,7 +238,7 @@ public class SubscriptionView implements SubscriptionViewMBean {
public int getDispatchedQueueSize() {
return subscription != null ? subscription.getDispatchedQueueSize() : 0;
}
public int getMessageCountAwaitingAcknowledge() {
return getDispatchedQueueSize();
}
@ -309,4 +311,9 @@ public class SubscriptionView implements SubscriptionViewMBean {
public boolean isSlowConsumer() {
return subscription.isSlowConsumer();
}
@Override
public String getUserName() {
return userName;
}
}

View File

@ -19,7 +19,7 @@ package org.apache.activemq.broker.jmx;
import javax.jms.InvalidSelectorException;
/**
*
*
*/
public interface SubscriptionViewMBean {
@ -100,9 +100,9 @@ public interface SubscriptionViewMBean {
*/
@MBeanInfo("Number of messages dispatched awaiting acknowledgement.")
int getDispatchedQueueSize();
/**
* The same as the number of messages dispatched -
* The same as the number of messages dispatched -
* making it explicit
* @return
*/
@ -205,4 +205,12 @@ public interface SubscriptionViewMBean {
@MBeanInfo("Returns true if the subscription is slow")
boolean isSlowConsumer();
/**
* Returns the User Name used to authorize creation of this Subscription.
* This value can be null if display of user name information is disabled.
*
* @return the name of the user that created this Subscription
*/
@MBeanInfo("User Name used to authorize creation of this Subscription")
String getUserName();
}

View File

@ -20,13 +20,13 @@ import org.apache.activemq.broker.region.DurableTopicSubscription;
import org.apache.activemq.broker.region.TopicSubscription;
/**
*
*
*
*
*/
public class TopicSubscriptionView extends SubscriptionView implements TopicSubscriptionViewMBean {
public TopicSubscriptionView(String clientId, TopicSubscription subs) {
super(clientId, subs);
public TopicSubscriptionView(String clientId, String userName, TopicSubscription subs) {
super(clientId, userName, subs);
}
protected TopicSubscription getTopicSubscription() {
@ -50,7 +50,7 @@ public class TopicSubscriptionView extends SubscriptionView implements TopicSubs
}
/**
*
*
*/
public void setMaximumPendingQueueSize(int max) {
TopicSubscription topicSubscription = getTopicSubscription();
@ -68,6 +68,6 @@ public class TopicSubscriptionView extends SubscriptionView implements TopicSubs
}
}
}

View File

@ -22,6 +22,7 @@ import java.net.URI;
import java.net.URL;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
import javax.jms.BytesMessage;
import javax.jms.Connection;
@ -31,6 +32,7 @@ import javax.jms.Message;
import javax.jms.MessageConsumer;
import javax.jms.MessageProducer;
import javax.jms.Session;
import javax.jms.Topic;
import javax.management.MBeanServer;
import javax.management.MBeanServerInvocationHandler;
import javax.management.MalformedObjectNameException;
@ -60,8 +62,6 @@ import org.slf4j.LoggerFactory;
* A test case of the various MBeans in ActiveMQ. If you want to look at the
* various MBeans after the test has been run then run this test case as a
* command line application.
*
*
*/
public class MBeanTest extends EmbeddedBrokerTestSupport {
private static final Logger LOG = LoggerFactory.getLogger(MBeanTest.class);
@ -211,7 +211,6 @@ public class MBeanTest extends EmbeddedBrokerTestSupport {
connection = connectionFactory.createConnection();
useConnection(connection);
ObjectName queueViewMBeanName = assertRegisteredObjectName(domain + ":Type=Queue,Destination=" + getDestinationString() + ",BrokerName=localhost");
QueueViewMBean queue = (QueueViewMBean)MBeanServerInvocationHandler.newProxyInstance(mbeanServer, queueViewMBeanName, QueueViewMBean.class, true);
@ -231,7 +230,6 @@ public class MBeanTest extends EmbeddedBrokerTestSupport {
consumer.close();
session.close();
// now lets get the dead letter queue
Thread.sleep(1000);
@ -260,7 +258,6 @@ public class MBeanTest extends EmbeddedBrokerTestSupport {
assertTrue("dlq has some memory usage", dlqMemUsage > 0);
assertEquals("dest has no memory usage", 0, queue.getMemoryPercentUsage());
echo("About to retry " + messageCount + " messages");
for (String messageID : messageIDs) {
@ -897,6 +894,89 @@ public class MBeanTest extends EmbeddedBrokerTestSupport {
assertTrue("dest has some memory usage", queue.getMemoryPercentUsage() > 0);
}
public void testUserNamePopulated() throws Exception {
doTestUserNameInMBeans(true);
}
public void testUserNameNotPopulated() throws Exception {
doTestUserNameInMBeans(false);
}
@SuppressWarnings("unused")
private void doTestUserNameInMBeans(boolean expect) throws Exception {
broker.setPopulateUserNameInMBeans(expect);
connection = connectionFactory.createConnection("admin", "admin");
connection.setClientID("MBeanTest");
Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
Destination queue = session.createQueue(getDestinationString() + ".Queue");
Topic topic = session.createTopic(getDestinationString() + ".Topic");
MessageProducer producer = session.createProducer(queue);
MessageConsumer queueConsumer = session.createConsumer(queue);
MessageConsumer topicConsumer = session.createConsumer(topic);
MessageConsumer durable = session.createDurableSubscriber(topic, "Durable");
ObjectName brokerName = assertRegisteredObjectName(domain + ":Type=Broker,BrokerName=localhost");
BrokerViewMBean broker = (BrokerViewMBean)MBeanServerInvocationHandler.newProxyInstance(mbeanServer, brokerName, BrokerViewMBean.class, true);
Thread.sleep(100);
assertTrue(broker.getQueueProducers().length == 1);
assertTrue(broker.getTopicSubscribers().length == 2);
assertTrue(broker.getQueueSubscribers().length == 1);
ObjectName producerName = broker.getQueueProducers()[0];
ProducerViewMBean producerView = (ProducerViewMBean)MBeanServerInvocationHandler.newProxyInstance(mbeanServer, producerName, ProducerViewMBean.class, true);
assertNotNull(producerView);
if (expect) {
assertEquals("admin", producerView.getUserName());
} else {
assertNull(producerView.getUserName());
}
for (ObjectName name : broker.getTopicSubscribers()) {
SubscriptionViewMBean subscriberView = (SubscriptionViewMBean)MBeanServerInvocationHandler.newProxyInstance(mbeanServer, name, SubscriptionViewMBean.class, true);
if (expect) {
assertEquals("admin", subscriberView.getUserName());
} else {
assertNull(subscriberView.getUserName());
}
}
for (ObjectName name : broker.getQueueSubscribers()) {
SubscriptionViewMBean subscriberView = (SubscriptionViewMBean)MBeanServerInvocationHandler.newProxyInstance(mbeanServer, name, SubscriptionViewMBean.class, true);
if (expect) {
assertEquals("admin", subscriberView.getUserName());
} else {
assertNull(subscriberView.getUserName());
}
}
Set<ObjectName> names = mbeanServer.queryNames(null, null);
boolean found = false;
for (ObjectName name : names) {
if (name.toString().startsWith(domain + ":BrokerName=localhost,Type=Connection,ConnectorName=tcp") &&
name.toString().endsWith("Connection=MBeanTest")) {
ConnectionViewMBean connectionView =
(ConnectionViewMBean)MBeanServerInvocationHandler.newProxyInstance(mbeanServer, name, ConnectionViewMBean.class, true);
assertNotNull(connectionView);
if (expect) {
assertEquals("admin", connectionView.getUserName());
} else {
assertNull(connectionView.getUserName());
}
found = true;
break;
}
}
assertTrue("Should find the connection's ManagedTransportConnection", found);
}
public void testBrowseBytesMessages() throws Exception {
connection = connectionFactory.createConnection();
useConnectionWithByteMessage(connection);