git-svn-id: https://svn.apache.org/repos/asf/activemq/trunk@1128072 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Timothy A. Bish 2011-05-26 20:43:32 +00:00
parent 4f6e7e4cc3
commit 7092b19dd2
6 changed files with 452 additions and 63 deletions

View File

@ -70,7 +70,7 @@ public class BrokerView implements BrokerViewMBean {
}
public String getBrokerVersion() {
return ActiveMQConnectionMetaData.PROVIDER_VERSION;
return ActiveMQConnectionMetaData.PROVIDER_VERSION;
}
public void gc() throws Exception {
@ -103,6 +103,10 @@ public class BrokerView implements BrokerViewMBean {
return broker.getDestinationStatistics().getConsumers().getCount();
}
public long getTotalProducerCount() {
return broker.getDestinationStatistics().getProducers().getCount();
}
public long getTotalMessageCount() {
return broker.getDestinationStatistics().getMessages().getCount();
}
@ -217,6 +221,22 @@ public class BrokerView implements BrokerViewMBean {
return broker.getInactiveDurableTopicSubscribers();
}
public ObjectName[] getTopicProducers() {
return broker.getTopicProducers();
}
public ObjectName[] getQueueProducers() {
return broker.getQueueProducers();
}
public ObjectName[] getTemporaryTopicProducers() {
return broker.getTemporaryTopicProducers();
}
public ObjectName[] getTemporaryQueueProducers() {
return broker.getTemporaryQueueProducers();
}
public String addConnector(String discoveryAddress) throws Exception {
TransportConnector connector = brokerService.addConnector(discoveryAddress);
connector.start();

View File

@ -74,6 +74,9 @@ public interface BrokerViewMBean extends Service {
@MBeanInfo("Number of message consumers subscribed to destinations on the broker.")
long getTotalConsumerCount();
@MBeanInfo("Number of message producers active on destinations on the broker.")
long getTotalProducerCount();
@MBeanInfo("Number of unacknowledged messages on the broker.")
long getTotalMessageCount();
@ -154,6 +157,18 @@ public interface BrokerViewMBean extends Service {
@MBeanInfo("Temporary Queue Subscribers.")
ObjectName[] getTemporaryQueueSubscribers();
@MBeanInfo("Topic Producers.")
public ObjectName[] getTopicProducers();
@MBeanInfo("Queue Producers.")
public ObjectName[] getQueueProducers();
@MBeanInfo("Temporary Topic Producers.")
public ObjectName[] getTemporaryTopicProducers();
@MBeanInfo("Temporary Queue Producers.")
public ObjectName[] getTemporaryQueueProducers();
@MBeanInfo("Adds a Connector to the broker.")
String addConnector(@MBeanInfo("discoveryAddress") String discoveryAddress) throws Exception;

View File

@ -60,6 +60,7 @@ import org.apache.activemq.command.ActiveMQTopic;
import org.apache.activemq.command.ConsumerInfo;
import org.apache.activemq.command.Message;
import org.apache.activemq.command.MessageId;
import org.apache.activemq.command.ProducerInfo;
import org.apache.activemq.command.SubscriptionInfo;
import org.apache.activemq.store.MessageRecoveryListener;
import org.apache.activemq.store.PersistenceAdapter;
@ -88,6 +89,10 @@ public class ManagedRegionBroker extends RegionBroker {
private final Map<ObjectName, SubscriptionView> inactiveDurableTopicSubscribers = new ConcurrentHashMap<ObjectName, SubscriptionView>();
private final Map<ObjectName, SubscriptionView> temporaryQueueSubscribers = new ConcurrentHashMap<ObjectName, SubscriptionView>();
private final Map<ObjectName, SubscriptionView> temporaryTopicSubscribers = new ConcurrentHashMap<ObjectName, SubscriptionView>();
private final Map<ObjectName, ProducerView> queueProducers = new ConcurrentHashMap<ObjectName, ProducerView>();
private final Map<ObjectName, ProducerView> topicProducers = new ConcurrentHashMap<ObjectName, ProducerView>();
private final Map<ObjectName, ProducerView> temporaryQueueProducers = new ConcurrentHashMap<ObjectName, ProducerView>();
private final Map<ObjectName, ProducerView> temporaryTopicProducers = new ConcurrentHashMap<ObjectName, ProducerView>();
private final Map<SubscriptionKey, ObjectName> subscriptionKeys = new ConcurrentHashMap<SubscriptionKey, ObjectName>();
private final Map<Subscription, ObjectName> subscriptionMap = new ConcurrentHashMap<Subscription, ObjectName>();
private final Set<ObjectName> registeredMBeans = new CopyOnWriteArraySet<ObjectName>();
@ -258,6 +263,23 @@ public class ManagedRegionBroker extends RegionBroker {
super.removeConsumer(context, info);
}
@Override
public void addProducer(ConnectionContext context, ProducerInfo info)
throws Exception {
super.addProducer(context, info);
String connectionClientId = context.getClientId();
ObjectName objectName = createObjectName(info, connectionClientId);
ProducerView view = new ProducerView(info, connectionClientId, this);
registerProducer(objectName, info.getDestination(), view);
}
@Override
public void removeProducer(ConnectionContext context, ProducerInfo info) throws Exception {
ObjectName objectName = createObjectName(info, context.getClientId());
unregisterProducer(objectName);
super.removeProducer(context, info);
}
public void unregisterSubscription(Subscription sub) {
ObjectName name = subscriptionMap.remove(sub);
if (name != null) {
@ -325,6 +347,44 @@ public class ManagedRegionBroker extends RegionBroker {
}
}
protected void registerProducer(ObjectName key, ActiveMQDestination dest, ProducerView view) throws Exception {
if (dest.isQueue()) {
if (dest.isTemporary()) {
temporaryQueueProducers.put(key, view);
} else {
queueProducers.put(key, view);
}
} else {
if (dest.isTemporary()) {
temporaryTopicProducers.put(key, view);
} else {
topicProducers.put(key, view);
}
}
try {
AnnotatedMBean.registerMBean(managementContext, view, key);
registeredMBeans.add(key);
} catch (Throwable e) {
LOG.warn("Failed to register MBean: " + key);
LOG.debug("Failure reason: " + e, e);
}
}
protected void unregisterProducer(ObjectName key) throws Exception {
queueProducers.remove(key);
topicProducers.remove(key);
temporaryQueueProducers.remove(key);
temporaryTopicProducers.remove(key);
if (registeredMBeans.remove(key)) {
try {
managementContext.unregisterMBean(key);
} catch (Throwable e) {
LOG.warn("Failed to unregister MBean: " + key);
LOG.debug("Failure reason: " + e, e);
}
}
}
private void removeAndRemember(Map<ObjectName, DestinationView> map, ObjectName key, DestinationView view) {
DestinationView candidate = map.remove(key);
if (candidate != null && view == null) {
@ -570,6 +630,26 @@ public class ManagedRegionBroker extends RegionBroker {
return set.toArray(new ObjectName[set.size()]);
}
protected ObjectName[] getTopicProducers() {
Set<ObjectName> set = topicProducers.keySet();
return set.toArray(new ObjectName[set.size()]);
}
protected ObjectName[] getQueueProducers() {
Set<ObjectName> set = queueProducers.keySet();
return set.toArray(new ObjectName[set.size()]);
}
protected ObjectName[] getTemporaryTopicProducers() {
Set<ObjectName> set = temporaryTopicProducers.keySet();
return set.toArray(new ObjectName[set.size()]);
}
protected ObjectName[] getTemporaryQueueProducers() {
Set<ObjectName> set = temporaryQueueProducers.keySet();
return set.toArray(new ObjectName[set.size()]);
}
public Broker getContextBroker() {
return contextBroker;
}
@ -587,6 +667,22 @@ public class ManagedRegionBroker extends RegionBroker {
return objectName;
}
protected ObjectName createObjectName(ProducerInfo producerInfo, String connectionClientId) throws MalformedObjectNameException {
// Build the object name for the producer info
Hashtable map = brokerObjectName.getKeyPropertyList();
String destinationType = "destinationType=" + producerInfo.getDestination().getDestinationTypeAsString();
String destinationName = "destinationName=" + JMXSupport.encodeObjectNamePart(producerInfo.getDestination().getPhysicalName());
String clientId = "clientId=" + JMXSupport.encodeObjectNamePart(connectionClientId);
String producerId = "producerId=" + JMXSupport.encodeObjectNamePart(producerInfo.getProducerId().toString());
ObjectName objectName = new ObjectName(brokerObjectName.getDomain() + ":" + "BrokerName=" + map.get("BrokerName") + ","
+ "Type=Producer" + ","
+ destinationType + "," + destinationName + ","
+ clientId + "," + producerId);
return objectName;
}
public ObjectName registerSlowConsumerStrategy(AbortSlowConsumerStrategy strategy) throws MalformedObjectNameException {
ObjectName objectName = null;
try {

View File

@ -0,0 +1,114 @@
/**
* 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.broker.jmx;
import org.apache.activemq.command.ActiveMQDestination;
import org.apache.activemq.command.ProducerInfo;
public class ProducerView implements ProducerViewMBean {
protected final ProducerInfo info;
protected final String clientId;
protected final ManagedRegionBroker broker;
public ProducerView(ProducerInfo info, String clientId, ManagedRegionBroker broker) {
this.info = info;
this.clientId = clientId;
this.broker = broker;
}
@Override
public String getClientId() {
return this.clientId;
}
@Override
public String getConnectionId() {
if (info != null) {
return info.getProducerId().getConnectionId();
}
return "NOTSET";
}
@Override
public long getSessionId() {
if (info != null) {
return info.getProducerId().getSessionId();
}
return 0;
}
@Override
public String getDestinationName() {
if (info != null) {
ActiveMQDestination dest = info.getDestination();
return dest.getPhysicalName();
}
return "NOTSET";
}
@Override
public boolean isDestinationQueue() {
if (info != null) {
ActiveMQDestination dest = info.getDestination();
return dest.isQueue();
}
return false;
}
@Override
public boolean isDestinationTopic() {
if (info != null) {
ActiveMQDestination dest = info.getDestination();
return dest.isTopic();
}
return false;
}
@Override
public boolean isDestinationTemporary() {
if (info != null) {
ActiveMQDestination dest = info.getDestination();
return dest.isTemporary();
}
return false;
}
@Override
public int getProducerWindowSize() {
if (info != null) {
return info.getWindowSize();
}
return 0;
}
@Override
public boolean isDispatchAsync() {
if (info != null) {
return info.isDispatchAsync();
}
return false;
}
/**
* @return pretty print
*/
public String toString() {
return "ProducerView: " + getClientId() + ":" + getConnectionId();
}
}

View File

@ -0,0 +1,74 @@
/**
* 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.broker.jmx;
public interface ProducerViewMBean {
/**
* @return the clientId of the Connection the Producer is on
*/
@MBeanInfo("JMS Client id of the Connection the Producer is on.")
String getClientId();
/**
* @return the id of the Connection the Producer is on
*/
@MBeanInfo("ID of the Connection the Producer is on.")
String getConnectionId();
/**
* @return the id of the Session the Producer is on
*/
@MBeanInfo("ID of the Session the Producer is on.")
long getSessionId();
/**
* @return the destination name
*/
@MBeanInfo("The name of the destionation the Producer is on.")
String getDestinationName();
/**
* @return true if the destination is a Queue
*/
@MBeanInfo("Producer is on a Queue")
boolean isDestinationQueue();
/**
* @return true of the destination is a Topic
*/
@MBeanInfo("Producer is on a Topic")
boolean isDestinationTopic();
/**
* @return true if the destination is temporary
*/
@MBeanInfo("Producer is on a temporary Queue/Topic")
boolean isDestinationTemporary();
/**
* @returns the windows size configured for the producer
*/
@MBeanInfo("Configured Window Size for the Producer")
int getProducerWindowSize();
/**
* @returns if the Producer is configured for Async dispatch
*/
@MBeanInfo("Is the producer configured for Async Dispatch")
boolean isDispatchAsync();
}

View File

@ -25,6 +25,7 @@ import java.util.Map;
import javax.jms.BytesMessage;
import javax.jms.Connection;
import javax.jms.Destination;
import javax.jms.Message;
import javax.jms.MessageConsumer;
import javax.jms.MessageProducer;
@ -99,10 +100,11 @@ public class MBeanTest extends EmbeddedBrokerTestSupport {
// test all the various MBeans now we have a producer, consumer and
// messages on a queue
assertSendViaMBean();
assertQueueBrowseWorks();
assertCreateAndDestroyDurableSubscriptions();
assertConsumerCounts();
// assertSendViaMBean();
// assertQueueBrowseWorks();
// assertCreateAndDestroyDurableSubscriptions();
// assertConsumerCounts();
assertProducerCounts();
}
public void testMoveMessages() throws Exception {
@ -546,6 +548,74 @@ public class MBeanTest extends EmbeddedBrokerTestSupport {
assertEquals("topic2 Durable subscriber count", 0, topic2.getConsumerCount());
}
protected void assertProducerCounts() throws Exception {
ObjectName brokerName = assertRegisteredObjectName(domain + ":Type=Broker,BrokerName=localhost");
BrokerViewMBean broker = (BrokerViewMBean)MBeanServerInvocationHandler.newProxyInstance(mbeanServer, brokerName, BrokerViewMBean.class, true);
assertTrue("broker is not a slave", !broker.isSlave());
// create 2 topics
broker.addTopic(getDestinationString() + "1");
broker.addTopic(getDestinationString() + "2");
ObjectName topicObjName1 = assertRegisteredObjectName(domain + ":Type=Topic,BrokerName=localhost,Destination=" + getDestinationString() + "1");
ObjectName topicObjName2 = assertRegisteredObjectName(domain + ":Type=Topic,BrokerName=localhost,Destination=" + getDestinationString() + "2");
TopicViewMBean topic1 = (TopicViewMBean)MBeanServerInvocationHandler.newProxyInstance(mbeanServer, topicObjName1, TopicViewMBean.class, true);
TopicViewMBean topic2 = (TopicViewMBean)MBeanServerInvocationHandler.newProxyInstance(mbeanServer, topicObjName2, TopicViewMBean.class, true);
assertEquals("topic1 Producer count", 0, topic1.getProducerCount());
assertEquals("topic2 Producer count", 0, topic2.getProducerCount());
assertEquals("broker Topic Producer count", 0, broker.getTopicProducers().length);
// create 1 producer for each topic
Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
Destination dest1 = session.createTopic(getDestinationString() + "1");
Destination dest2 = session.createTopic(getDestinationString() + "2");
MessageProducer producer1 = session.createProducer(dest1);
MessageProducer producer2 = session.createProducer(dest2);
Thread.sleep(500);
assertEquals("topic1 Producer count", 1, topic1.getProducerCount());
assertEquals("topic2 Producer count", 1, topic2.getProducerCount());
assertEquals("broker Topic Producer count", 2, broker.getTopicProducers().length);
// create 1 more producer for topic1
MessageProducer producer3 = session.createProducer(dest1);
Thread.sleep(500);
assertEquals("topic1 Producer count", 2, topic1.getProducerCount());
assertEquals("topic2 Producer count", 1, topic2.getProducerCount());
assertEquals("broker Topic Producer count", 3, broker.getTopicProducers().length);
// destroy topic1 producer
producer1.close();
Thread.sleep(500);
assertEquals("topic1 Producer count", 1, topic1.getProducerCount());
assertEquals("topic2 Producer count", 1, topic2.getProducerCount());
assertEquals("broker Topic Producer count", 2, broker.getTopicProducers().length);
// destroy topic2 producer
producer2.close();
Thread.sleep(500);
assertEquals("topic1 Producer count", 1, topic1.getProducerCount());
assertEquals("topic2 Producer count", 0, topic2.getProducerCount());
assertEquals("broker Topic Producer count", 1, broker.getTopicProducers().length);
// destroy remaining topic1 producer
producer3.close();
Thread.sleep(500);
assertEquals("topic1 Producer count", 0, topic1.getProducerCount());
assertEquals("topic2 Producer count", 0, topic2.getProducerCount());
assertEquals("broker Topic Producer count", 0, broker.getTopicProducers().length);
}
protected ObjectName assertRegisteredObjectName(String name) throws MalformedObjectNameException, NullPointerException {
ObjectName objectName = new ObjectName(name);
if (mbeanServer.isRegistered(objectName)) {