You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@activemq.apache.org by ch...@apache.org on 2011/06/14 20:50:59 UTC

svn commit: r1135742 - in /activemq/activemq-apollo/trunk: apollo-bdb/src/main/webapp/WEB-INF/org/apache/activemq/apollo/broker/store/bdb/dto/ apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/ apollo-broker/src/main/scala/org/apache/activ...

Author: chirino
Date: Tue Jun 14 18:50:58 2011
New Revision: 1135742

URL: http://svn.apache.org/viewvc?rev=1135742&view=rev
Log:
Fixes https://issues.apache.org/jira/browse/APLO-47 : Add connection_counter, consumer_counter, and producer_counter metrics to the management interface

Also expose more details in the HTML views that was present in the JSON version of the data.

Modified:
    activemq/activemq-apollo/trunk/apollo-bdb/src/main/webapp/WEB-INF/org/apache/activemq/apollo/broker/store/bdb/dto/BDBStoreStatusDTO.jade
    activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Broker.scala
    activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Connector.scala
    activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Queue.scala
    activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Topic.scala
    activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/store/DelayingStoreSupport.scala
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerStatusDTO.java
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/ConnectorStatusDTO.java
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/QueueMetricsDTO.java
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/StoreStatusDTO.java
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/TopicStatusDTO.java
    activemq/activemq-apollo/trunk/apollo-hawtdb/src/main/webapp/WEB-INF/org/apache/activemq/apollo/broker/store/hawtdb/dto/HawtDBStoreStatusDTO.jade
    activemq/activemq-apollo/trunk/apollo-jdbm2/src/main/webapp/WEB-INF/org/apache/activemq/apollo/broker/store/jdbm2/dto/JDBM2StoreStatusDTO.jade
    activemq/activemq-apollo/trunk/apollo-openwire/src/test/scala/org/apache/activemq/apollo/openwire/OpenwireTestSupport.scala
    activemq/activemq-apollo/trunk/apollo-stomp/src/test/scala/org/apache/activemq/apollo/stomp/StompTest.scala
    activemq/activemq-apollo/trunk/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/TransportServer.java
    activemq/activemq-apollo/trunk/apollo-web/src/main/scala/org/apache/activemq/apollo/web/resources/BrokerResource.scala
    activemq/activemq-apollo/trunk/apollo-web/src/main/webapp/WEB-INF/org/apache/activemq/apollo/dto/BrokerStatusDTO.jade
    activemq/activemq-apollo/trunk/apollo-web/src/main/webapp/WEB-INF/org/apache/activemq/apollo/dto/ConnectorStatusDTO.jade
    activemq/activemq-apollo/trunk/apollo-web/src/main/webapp/WEB-INF/org/apache/activemq/apollo/dto/QueueStatusDTO.jade
    activemq/activemq-apollo/trunk/apollo-web/src/main/webapp/WEB-INF/org/apache/activemq/apollo/dto/TopicStatusDTO.jade

Modified: activemq/activemq-apollo/trunk/apollo-bdb/src/main/webapp/WEB-INF/org/apache/activemq/apollo/broker/store/bdb/dto/BDBStoreStatusDTO.jade
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-bdb/src/main/webapp/WEB-INF/org/apache/activemq/apollo/broker/store/bdb/dto/BDBStoreStatusDTO.jade?rev=1135742&r1=1135741&r2=1135742&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-bdb/src/main/webapp/WEB-INF/org/apache/activemq/apollo/broker/store/bdb/dto/BDBStoreStatusDTO.jade (original)
+++ activemq/activemq-apollo/trunk/apollo-bdb/src/main/webapp/WEB-INF/org/apache/activemq/apollo/broker/store/bdb/dto/BDBStoreStatusDTO.jade Tue Jun 14 18:50:58 2011
@@ -22,6 +22,7 @@
 .breadcumbs
   a(href={strip_resolve(".")}) Back
 
+h1 Store: #{id}
 p state: #{state} for #{ uptime(state_since) }
 
 p pending stores: #{pending_stores}

Modified: activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Broker.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Broker.scala?rev=1135742&r1=1135741&r2=1135742&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Broker.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Broker.scala Tue Jun 14 18:50:58 2011
@@ -528,7 +528,7 @@ class Broker() extends BaseService {
   }
 
   def get_socket_address = {
-    first_accepting_connector.get.transport_server.getSocketAddress
+    first_accepting_connector.get.socket_address
   }
 
   def first_accepting_connector = connectors.values.find(_.isInstanceOf[AcceptingConnector]).map(_.asInstanceOf[AcceptingConnector])

Modified: activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Connector.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Connector.scala?rev=1135742&r1=1135741&r2=1135742&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Connector.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Connector.scala Tue Jun 14 18:50:58 2011
@@ -23,6 +23,7 @@ import protocol.{ProtocolFactory, Protoc
 import org.apache.activemq.apollo.transport._
 import org.apache.activemq.apollo.util._
 import org.apache.activemq.apollo.util.OptionSupport._
+import java.net.SocketAddress
 
 
 /**
@@ -43,6 +44,7 @@ trait Connector extends BaseService {
   def accepted:LongCounter
   def connected:LongCounter
   def update(config: ConnectorDTO, on_complete:Runnable):Unit
+  def socket_address:SocketAddress
 
 }
 
@@ -69,6 +71,8 @@ class AcceptingConnector(val broker:Brok
 
   override def toString = "connector: "+config.id
 
+  def socket_address = Option(transport_server).map(_.getSocketAddress).getOrElse(null)
+
   object BrokerAcceptListener extends TransportAcceptListener {
     def onAcceptError(e: Exception): Unit = {
       warn(e, "Error occured while accepting client connection.")

Modified: activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Queue.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Queue.scala?rev=1135742&r1=1135741&r2=1135742&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Queue.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Queue.scala Tue Jun 14 18:50:58 2011
@@ -181,6 +181,9 @@ class Queue(val router: LocalRouter, val
   var swap_in_item_counter = 0L
   var swap_in_size_counter = 0L
 
+  var producer_counter = 0L
+  var consumer_counter = 0L
+
   var individual_swapped_items = 0
 
   val swap_source = createSource(EventAggregators.INTEGER_ADD, dispatch_queue)
@@ -649,6 +652,7 @@ class Queue(val router: LocalRouter, val
     } else {
       dispatch_queue {
         producers += producer
+        producer_counter += 1
         check_idle
       }
       producer.bind(this::Nil)
@@ -1492,6 +1496,7 @@ class Subscription(val queue:Queue, val 
     queue.head_entry ::= this
 
     queue.all_subscriptions += consumer -> this
+    queue.consumer_counter += 1
     queue.addCapacity( queue.tune_consumer_buffer )
 
     if( exclusive ) {

Modified: activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Topic.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Topic.scala?rev=1135742&r1=1135741&r2=1135742&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Topic.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Topic.scala Tue Jun 14 18:50:58 2011
@@ -40,6 +40,8 @@ class Topic(val router:LocalRouter, val 
   var idled_at = 0L
   val created_at = System.currentTimeMillis()
   var auto_delete_after = 0
+  var producer_counter = 0L
+  var consumer_counter = 0L
 
   var config:TopicDTO = _
 
@@ -116,6 +118,7 @@ class Topic(val router:LocalRouter, val 
         }
 
         consumers += target
+        consumer_counter += 1
         val list = target :: Nil
         producers.foreach({ r=>
           r.bind(list)
@@ -193,6 +196,7 @@ class Topic(val router:LocalRouter, val 
 
   def connect (destination:DestinationDTO, producer:BindableDeliveryProducer) = {
     producers += producer
+    producer_counter += 1
     producer.bind(consumers.toList ::: durable_subscriptions.toList)
     check_idle
   }

Modified: activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/store/DelayingStoreSupport.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/store/DelayingStoreSupport.scala?rev=1135742&r1=1135741&r2=1135742&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/store/DelayingStoreSupport.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/store/DelayingStoreSupport.scala Tue Jun 14 18:50:58 2011
@@ -235,6 +235,7 @@ trait DelayingStoreSupport extends Store
   var canceled_enqueue:Long = 0
 
   protected def fill_store_status(rc: StoreStatusDTO) {
+    rc.id = this.toString
     rc.state = service_state.toString
     rc.state_since = service_state.since
 

Modified: activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerStatusDTO.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerStatusDTO.java?rev=1135742&r1=1135741&r2=1135742&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerStatusDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerStatusDTO.java Tue Jun 14 18:50:58 2011
@@ -66,4 +66,9 @@ public class BrokerStatusDTO extends Ser
     @XmlElement(name="connection")
     public List<LongIdLabeledDTO> connections = new ArrayList<LongIdLabeledDTO>();
 
+    /**
+     * The total number of connections accepted by the broker.
+     */
+	@XmlAttribute(name="connection_counter")
+	public long connection_counter;
 }

Modified: activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/ConnectorStatusDTO.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/ConnectorStatusDTO.java?rev=1135742&r1=1135741&r2=1135742&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/ConnectorStatusDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/ConnectorStatusDTO.java Tue Jun 14 18:50:58 2011
@@ -33,10 +33,19 @@ import java.util.List;
 public class ConnectorStatusDTO extends ServiceStatusDTO {
 
     /**
+     * The local address the connector has bound
+     */
+    @XmlAttribute(name="local_address")
+    public String local_address;
+
+    @XmlAttribute(name="protocol")
+    public String protocol;
+
+    /**
      * The number of connections that this connector has accepted.
      */
-    @XmlAttribute
-    public long accepted;
+    @XmlAttribute(name="connection_counter")
+    public long connection_counter;
 
     /**
      * The number of connections that this connector has currently connected.

Modified: activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/QueueMetricsDTO.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/QueueMetricsDTO.java?rev=1135742&r1=1135741&r2=1135742&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/QueueMetricsDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/QueueMetricsDTO.java Tue Jun 14 18:50:58 2011
@@ -179,4 +179,18 @@ public class QueueMetricsDTO {
     @XmlAttribute(name="swap_in_size_counter")
     public long swap_in_size_counter;
 
+    /**
+     * The total number of producers that have sent to
+     * the queue.
+     */
+    @XmlAttribute(name="producer_counter")
+    public long producer_counter;
+
+    /**
+     * The total number of consumers that have ever subscribed to
+     * the queue.
+     */
+    @XmlAttribute(name="consumer_counter")
+    public long consumer_counter;
+
 }

Modified: activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/StoreStatusDTO.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/StoreStatusDTO.java?rev=1135742&r1=1135741&r2=1135742&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/StoreStatusDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/StoreStatusDTO.java Tue Jun 14 18:50:58 2011
@@ -28,7 +28,7 @@ import javax.xml.bind.annotation.XmlType
  */
 @XmlType(name = "store_status_type")
 @JsonTypeInfo(use=JsonTypeInfo.Id.CLASS, include=JsonTypeInfo.As.PROPERTY, property="@class")
-public abstract class StoreStatusDTO {
+public abstract class StoreStatusDTO extends StringIdDTO {
 
     /**
      * The state of the service.

Modified: activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/TopicStatusDTO.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/TopicStatusDTO.java?rev=1135742&r1=1135741&r2=1135742&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/TopicStatusDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/TopicStatusDTO.java Tue Jun 14 18:50:58 2011
@@ -50,4 +50,18 @@ public class TopicStatusDTO extends Serv
      */
     @XmlElement(name="dsub")
     public List<String> dsubs = new ArrayList<String>();
+
+    /**
+     * The total number of producers that have sent to
+     * the topic.
+     */
+    @XmlAttribute(name="producer_counter")
+    public long producer_counter;
+
+    /**
+     * The total number of consumers that have ever subscribed to
+     * the topic.
+     */
+    @XmlAttribute(name="consumer_counter")
+    public long consumer_counter;
 }
\ No newline at end of file

Modified: activemq/activemq-apollo/trunk/apollo-hawtdb/src/main/webapp/WEB-INF/org/apache/activemq/apollo/broker/store/hawtdb/dto/HawtDBStoreStatusDTO.jade
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-hawtdb/src/main/webapp/WEB-INF/org/apache/activemq/apollo/broker/store/hawtdb/dto/HawtDBStoreStatusDTO.jade?rev=1135742&r1=1135741&r2=1135742&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-hawtdb/src/main/webapp/WEB-INF/org/apache/activemq/apollo/broker/store/hawtdb/dto/HawtDBStoreStatusDTO.jade (original)
+++ activemq/activemq-apollo/trunk/apollo-hawtdb/src/main/webapp/WEB-INF/org/apache/activemq/apollo/broker/store/hawtdb/dto/HawtDBStoreStatusDTO.jade Tue Jun 14 18:50:58 2011
@@ -22,6 +22,7 @@
 .breadcumbs
   a(href={strip_resolve(".")}) Back
 
+h1 Store: #{id}
 p state: #{state} for #{ uptime(state_since) }
 
 h2 Cancel Stats

Modified: activemq/activemq-apollo/trunk/apollo-jdbm2/src/main/webapp/WEB-INF/org/apache/activemq/apollo/broker/store/jdbm2/dto/JDBM2StoreStatusDTO.jade
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-jdbm2/src/main/webapp/WEB-INF/org/apache/activemq/apollo/broker/store/jdbm2/dto/JDBM2StoreStatusDTO.jade?rev=1135742&r1=1135741&r2=1135742&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-jdbm2/src/main/webapp/WEB-INF/org/apache/activemq/apollo/broker/store/jdbm2/dto/JDBM2StoreStatusDTO.jade (original)
+++ activemq/activemq-apollo/trunk/apollo-jdbm2/src/main/webapp/WEB-INF/org/apache/activemq/apollo/broker/store/jdbm2/dto/JDBM2StoreStatusDTO.jade Tue Jun 14 18:50:58 2011
@@ -22,8 +22,10 @@
 .breadcumbs
   a(href={strip_resolve(".")}) Back
 
+h1 Store: #{id}
 p state: #{state} for #{ uptime(state_since) }
 
+
 h2 Cancel Stats
 p canceled message stores: #{canceled_message_counter}
 p canceled message enqueues: #{canceled_enqueue_counter}

Modified: activemq/activemq-apollo/trunk/apollo-openwire/src/test/scala/org/apache/activemq/apollo/openwire/OpenwireTestSupport.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-openwire/src/test/scala/org/apache/activemq/apollo/openwire/OpenwireTestSupport.scala?rev=1135742&r1=1135741&r2=1135742&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-openwire/src/test/scala/org/apache/activemq/apollo/openwire/OpenwireTestSupport.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-openwire/src/test/scala/org/apache/activemq/apollo/openwire/OpenwireTestSupport.scala Tue Jun 14 18:50:58 2011
@@ -26,6 +26,7 @@ import FileSupport._
 import javax.jms.Connection
 import org.apache.activemq.ActiveMQConnectionFactory
 import org.apache.activemq.command.{ActiveMQTopic, ActiveMQQueue}
+import java.net.InetSocketAddress
 
 class OpenwireTestSupport extends FunSuiteSupport with ShouldMatchers with BeforeAndAfterEach with Logging {
   var broker: Broker = null
@@ -37,7 +38,7 @@ class OpenwireTestSupport extends FunSui
     info("Loading broker configuration from the classpath with URI: " + broker_config_uri)
     broker = BrokerFactory.createBroker(broker_config_uri)
     ServiceControl.start(broker, "Starting broker")
-    port = broker.get_socket_address.getPort
+    port = broker.get_socket_address.asInstanceOf[InetSocketAddress].getPort
   }
 
   var default_connection:Connection = _

Modified: activemq/activemq-apollo/trunk/apollo-stomp/src/test/scala/org/apache/activemq/apollo/stomp/StompTest.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-stomp/src/test/scala/org/apache/activemq/apollo/stomp/StompTest.scala?rev=1135742&r1=1135741&r2=1135742&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-stomp/src/test/scala/org/apache/activemq/apollo/stomp/StompTest.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-stomp/src/test/scala/org/apache/activemq/apollo/stomp/StompTest.scala Tue Jun 14 18:50:58 2011
@@ -23,6 +23,7 @@ import org.apache.activemq.apollo.broker
 import org.apache.activemq.apollo.util.{FileSupport, Logging, FunSuiteSupport, ServiceControl}
 import FileSupport._
 import org.apache.activemq.apollo.dto.KeyStorageDTO
+import java.net.InetSocketAddress
 
 class StompTestSupport extends FunSuiteSupport with ShouldMatchers with BeforeAndAfterEach with Logging {
   var broker: Broker = null
@@ -35,7 +36,7 @@ class StompTestSupport extends FunSuiteS
       info("Loading broker configuration from the classpath with URI: " + broker_config_uri)
       broker = BrokerFactory.createBroker(broker_config_uri)
       ServiceControl.start(broker, "Starting broker")
-      port = broker.get_socket_address.getPort
+      port = broker.get_socket_address.asInstanceOf[InetSocketAddress].getPort
     }
     catch {
       case e:Throwable => e.printStackTrace

Modified: activemq/activemq-apollo/trunk/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/TransportServer.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/TransportServer.java?rev=1135742&r1=1135741&r2=1135742&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/TransportServer.java (original)
+++ activemq/activemq-apollo/trunk/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/TransportServer.java Tue Jun 14 18:50:58 2011
@@ -17,6 +17,7 @@
 package org.apache.activemq.apollo.transport;
 
 import java.net.InetSocketAddress;
+import java.net.SocketAddress;
 import java.net.URI;
 
 import org.apache.activemq.apollo.util.Service;
@@ -47,7 +48,7 @@ public interface TransportServer extends
      *         on or null if this does not or is not currently accepting
      *         connections on a socket.
      */
-    InetSocketAddress getSocketAddress();
+    SocketAddress getSocketAddress();
 
     /**
      * Returns the dispatch queue used by the transport

Modified: activemq/activemq-apollo/trunk/apollo-web/src/main/scala/org/apache/activemq/apollo/web/resources/BrokerResource.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-web/src/main/scala/org/apache/activemq/apollo/web/resources/BrokerResource.scala?rev=1135742&r1=1135741&r2=1135742&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-web/src/main/scala/org/apache/activemq/apollo/web/resources/BrokerResource.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-web/src/main/scala/org/apache/activemq/apollo/web/resources/BrokerResource.scala Tue Jun 14 18:50:58 2011
@@ -36,6 +36,7 @@ import org.apache.activemq.apollo.util.B
 import management.ManagementFactory
 import javax.management.ObjectName
 import javax.management.openmbean.CompositeData
+import javax.management.remote.rmi._RMIConnection_Stub
 
 /**
  * <p>
@@ -69,7 +70,7 @@ case class BrokerResource() extends Reso
         result.state = broker.service_state.toString
         result.state_since = broker.service_state.since
         result.version = Broker.version
-
+        result.connection_counter = broker.connection_id_counter.get()
         broker.virtual_hosts.values.foreach{ host=>
           // TODO: may need to sync /w virtual host's dispatch queue
           result.virtual_hosts.add( host.id )
@@ -195,6 +196,8 @@ case class BrokerResource() extends Reso
       rc.swapped_in_size += q.swapped_in_size
 
       rc.swapped_in_size_max += q.swapped_in_size_max
+      rc.producer_counter += q.producer_counter
+      rc.consumer_counter += q.consumer_counter
 
       if( q.isInstanceOf[AggregateQueueMetricsDTO] ) {
         rc.queues += q.asInstanceOf[AggregateQueueMetricsDTO].queues
@@ -460,6 +463,8 @@ case class BrokerResource() extends Reso
       rc.state = "STARTED"
       rc.state_since = node.created_at
       rc.config = node.config
+      rc.producer_counter = node.producer_counter
+      rc.consumer_counter = node.consumer_counter
 
       node.durable_subscriptions.foreach {
         q =>
@@ -574,8 +579,10 @@ case class BrokerResource() extends Reso
     result.id = connector.id.toString
     result.state = connector.service_state.toString
     result.state_since = connector.service_state.since
-    result.accepted = connector.accepted.get
+    result.connection_counter = connector.accepted.get
     result.connected = connector.connected.get
+    result.protocol = Option(connector.config.protocol).getOrElse("any")
+    result.local_address = Option(connector.socket_address).map(_.toString).getOrElse("any")
     result
   }
 
@@ -705,6 +712,9 @@ case class BrokerResource() extends Reso
 
     rc.swapped_in_size_max = q.swapped_in_size_max
 
+    rc.producer_counter = q.producer_counter
+    rc.consumer_counter = q.consumer_counter
+
     rc
   }
 

Modified: activemq/activemq-apollo/trunk/apollo-web/src/main/webapp/WEB-INF/org/apache/activemq/apollo/dto/BrokerStatusDTO.jade
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-web/src/main/webapp/WEB-INF/org/apache/activemq/apollo/dto/BrokerStatusDTO.jade?rev=1135742&r1=1135741&r2=1135742&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-web/src/main/webapp/WEB-INF/org/apache/activemq/apollo/dto/BrokerStatusDTO.jade (original)
+++ activemq/activemq-apollo/trunk/apollo-web/src/main/webapp/WEB-INF/org/apache/activemq/apollo/dto/BrokerStatusDTO.jade Tue Jun 14 18:50:58 2011
@@ -18,15 +18,13 @@
 - import helper._
 
 h1 Apache ActiveMQ Apollo #{version}
-
 p state: #{state} #{ uptime(state_since) } ago
 
+a(href={path("config/files")}) Edit Configuration
 - if( state == "STARTED" )
   form(method="post" action={path("action/shutdown")})
     input(type="submit" value="shutdown")
 
-a(href={path("config/files")}) Edit Configuration
-
 h2 Virtual Hosts
 ul
   - for( x <- virtual_hosts )
@@ -40,7 +38,39 @@ ul
       a(href={ path("connectors/"+x) }) #{x}
 
 h2 Connections
+p total connections ever : #{connection_counter}
 ul
   - for( x <- connections )
     li
       a(href={ path("connections/"+x.id) }) #{x.label}
+
+h2 Operating System
+p Name : #{jvm_metrics.os_name}
+p Architecture : #{jvm_metrics.os_arch}
+p Memory total : #{memory(jvm_metrics.os_memory_total)}
+p Memory free : #{memory(jvm_metrics.os_memory_free)}
+p Swap total : #{memory(jvm_metrics.os_swap_total)}
+p Swap free : #{memory(jvm_metrics.os_swap_free)}
+p File descriptors open: #{jvm_metrics.os_fd_open}
+p File descriptors max: #{jvm_metrics.os_fd_max}
+
+h2 Java Virtual Machine
+p Name: #{jvm_metrics.jvm_name}
+p Runtime : #{jvm_metrics.runtime_name}
+p Start time : #{ new java.util.Date(jvm_metrics.start_time) }
+p Up time : #{ uptime(jvm_metrics.start_time) }
+
+h2 Memory
+p Heap memory used : #{memory(jvm_metrics.heap_memory.used)}
+p Heap memory allocated : #{memory(jvm_metrics.heap_memory.alloc)}
+p Heap memory max : #{memory(jvm_metrics.heap_memory.max)}
+p Non-heap memory allocated : #{memory(jvm_metrics.non_heap_memory.alloc)}
+
+h2 Classes
+p Classes loaded : #{jvm_metrics.classes_loaded}
+p Classes unloaded : #{jvm_metrics.classes_unloaded}
+
+h2 Threads
+p Threads current : #{jvm_metrics.threads_current}
+p Threads peak : #{jvm_metrics.threads_peak}
+

Modified: activemq/activemq-apollo/trunk/apollo-web/src/main/webapp/WEB-INF/org/apache/activemq/apollo/dto/ConnectorStatusDTO.jade
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-web/src/main/webapp/WEB-INF/org/apache/activemq/apollo/dto/ConnectorStatusDTO.jade?rev=1135742&r1=1135741&r2=1135742&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-web/src/main/webapp/WEB-INF/org/apache/activemq/apollo/dto/ConnectorStatusDTO.jade (original)
+++ activemq/activemq-apollo/trunk/apollo-web/src/main/webapp/WEB-INF/org/apache/activemq/apollo/dto/ConnectorStatusDTO.jade Tue Jun 14 18:50:58 2011
@@ -29,8 +29,10 @@ p state: #{state} #{ uptime(state_since)
   form(method="post" action={path("action/start")})
     input(type="submit" value="start")
 
-p total accepted: #{accepted}
 
+p local address: #{local_address}
+p protocol: #{protocol}
 p currently connected: #{connected}
+p total ever connected: #{connection_counter}
 
 

Modified: activemq/activemq-apollo/trunk/apollo-web/src/main/webapp/WEB-INF/org/apache/activemq/apollo/dto/QueueStatusDTO.jade
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-web/src/main/webapp/WEB-INF/org/apache/activemq/apollo/dto/QueueStatusDTO.jade?rev=1135742&r1=1135741&r2=1135742&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-web/src/main/webapp/WEB-INF/org/apache/activemq/apollo/dto/QueueStatusDTO.jade (original)
+++ activemq/activemq-apollo/trunk/apollo-web/src/main/webapp/WEB-INF/org/apache/activemq/apollo/dto/QueueStatusDTO.jade Tue Jun 14 18:50:58 2011
@@ -69,6 +69,7 @@ p total swap outs : #{metrics.swap_out_i
 p total swap ins : #{metrics.swap_in_item_counter} messages (#{memory(metrics.swap_in_size_counter)})
 
 h3 Producers
+p total producers ever : #{metrics.producer_counter}
 ul
   - for( x <- producers )
     - x.kind match
@@ -79,6 +80,7 @@ ul
 
 
 h3 Consumers
+p total consumers ever : #{metrics.consumer_counter}
 ul
   - for( consumer <- consumers )
     - import consumer._

Modified: activemq/activemq-apollo/trunk/apollo-web/src/main/webapp/WEB-INF/org/apache/activemq/apollo/dto/TopicStatusDTO.jade
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-web/src/main/webapp/WEB-INF/org/apache/activemq/apollo/dto/TopicStatusDTO.jade?rev=1135742&r1=1135741&r2=1135742&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-web/src/main/webapp/WEB-INF/org/apache/activemq/apollo/dto/TopicStatusDTO.jade (original)
+++ activemq/activemq-apollo/trunk/apollo-web/src/main/webapp/WEB-INF/org/apache/activemq/apollo/dto/TopicStatusDTO.jade Tue Jun 14 18:50:58 2011
@@ -25,7 +25,8 @@ h1 Topic: #{id}
 p state: #{state} #{ uptime(state_since) } ago
 h3 Topic Domain
 
-h4 Publishers
+h4 Producers
+p total producers ever : #{producer_counter}
 ul
   - for( x <- producers )
     - x.kind match
@@ -34,7 +35,8 @@ ul
           a(href={ path("../../../../connections/"+x.id) }) #{x.label}
       - case _ =>
 
-h4 Subscribers
+h4 Consumers
+p total consumers ever : #{consumer_counter}
 ul
   - for( x <- consumers )
     - x.kind match