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/05/26 20:19:41 UTC

svn commit: r1128015 - in /activemq/activemq-apollo/trunk: apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/ apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/ apollo-broker/src/test/resources/org/apache/activemq/apo...

Author: chirino
Date: Thu May 26 18:19:40 2011
New Revision: 1128015

URL: http://svn.apache.org/viewvc?rev=1128015&view=rev
Log:
Use a String instead of a Long to identify queues, topics, and durable subscriptions.  Make the XML elements/attributes more consistent /w the REST interface.

Modified:
    activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/DestinationParser.scala
    activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/LocalRouter.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/QueueBinding.scala
    activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Router.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/security/AclAuthorizer.scala
    activemq/activemq-apollo/trunk/apollo-broker/src/test/resources/org/apache/activemq/apollo/broker/destination-config.xml
    activemq/activemq-apollo/trunk/apollo-broker/src/test/scala/org/apache/activemq/apollo/broker/DestinationConfigurationTest.scala
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/DestinationDTO.java
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/DurableSubscriptionDTO.java
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/QueueDTO.java
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/QueueDestinationDTO.java
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/QueueStatusDTO.java
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/TopicDTO.java
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/TopicDestinationDTO.java
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/TopicStatusDTO.java
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/VirtualHostDTO.java
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/VirtualHostStatusDTO.java
    activemq/activemq-apollo/trunk/apollo-dto/src/test/java/org/apache/activemq/apollo/dto/XmlCodecTest.java
    activemq/activemq-apollo/trunk/apollo-dto/src/test/resources/org/apache/activemq/apollo/dto/XmlCodecTest.xml
    activemq/activemq-apollo/trunk/apollo-openwire/src/main/scala/org/apache/activemq/apollo/openwire/OpenwireProtocolHandler.scala
    activemq/activemq-apollo/trunk/apollo-stomp/src/main/scala/org/apache/activemq/apollo/stomp/StompProtocolHandler.scala
    activemq/activemq-apollo/trunk/apollo-stomp/src/test/scala/org/apache/activemq/apollo/stomp/perf/StompRemoteClients.scala
    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/QueueStatusDTO.jade
    activemq/activemq-apollo/trunk/apollo-web/src/main/webapp/WEB-INF/org/apache/activemq/apollo/dto/TopicStatusDTO.jade
    activemq/activemq-apollo/trunk/apollo-web/src/main/webapp/WEB-INF/org/apache/activemq/apollo/dto/VirtualHostStatusDTO.jade
    activemq/activemq-apollo/trunk/apollo-website/src/documentation/management-api.md
    activemq/activemq-apollo/trunk/apollo-website/src/documentation/user-manual.md

Modified: activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/DestinationParser.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/DestinationParser.scala?rev=1128015&r1=1128014&r2=1128015&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/DestinationParser.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/DestinationParser.scala Thu May 26 18:19:40 2011
@@ -87,7 +87,7 @@ class DestinationParser extends PathPars
           case _ =>
             throw new Exception("Uknown destination type: "+dest.getClass);
         }
-        rc.append(encode_path(dest.parts.toIterable))
+        rc.append(encode_path(dest.path.toIterable))
 
       }
       rc.toString

Modified: activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/LocalRouter.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/LocalRouter.scala?rev=1128015&r1=1128014&r2=1128015&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/LocalRouter.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/LocalRouter.scala Thu May 26 18:19:40 2011
@@ -31,11 +31,10 @@ import java.util.concurrent.TimeUnit
 
 trait DomainDestination {
 
-  def id:Long
+  def id:String
   def virtual_host:VirtualHost
 
   def destination_dto:DestinationDTO
-  def name:String = destination_dto.name(".")
 
   def bind (destination:DestinationDTO, consumer:DeliveryConsumer)
   def unbind (consumer:DeliveryConsumer, persistent:Boolean)
@@ -112,7 +111,7 @@ class LocalRouter(val virtual_host:Virtu
   trait Domain[D <: DomainDestination] {
 
     // holds all the destinations in the domain by id
-    var destination_by_id = HashMap[Long, D]()
+    var destination_by_id = HashMap[String, D]()
     // holds all the destinations in the domain by path
     var destination_by_path = new PathMap[D]()
     // Can store consumers on wild cards paths
@@ -198,7 +197,7 @@ class LocalRouter(val virtual_host:Virtu
     }
 
     def unbind(destination:DestinationDTO, consumer:DeliveryConsumer, persistent:Boolean) = {
-      val path = destination_parser.decode_path(destination.parts)
+      val path = destination_parser.decode_path(destination.path)
       if( consumers_by_path.remove(path, new ConsumerContext(destination, consumer, null) ) ) {
         get_destination_matches(path).foreach{ dest=>
           dest.unbind(consumer, persistent)
@@ -262,7 +261,7 @@ class LocalRouter(val virtual_host:Virtu
     }
 
     def disconnect(destination:DestinationDTO, producer:BindableDeliveryProducer) = {
-      val path = destination_parser.decode_path(destination.parts)
+      val path = destination_parser.decode_path(destination.path)
       producers_by_path.remove(path, new ProducerContext(destination, producer, null))
       get_destination_matches(path).foreach { dest=>
         dest.disconnect(producer)
@@ -273,8 +272,6 @@ class LocalRouter(val virtual_host:Virtu
   val topic_domain = new TopicDomain
   class TopicDomain extends Domain[Topic] {
 
-    val topic_id_counter = new LongCounter()
-
     // Stores durable subscription queues.
     val durable_subscriptions_by_path = new PathMap[Queue]()
     val durable_subscriptions_by_id = HashMap[String, Queue]()
@@ -302,7 +299,7 @@ class LocalRouter(val virtual_host:Virtu
     def topic_config(name:Path):TopicDTO = {
       import collection.JavaConversions._
       import destination_parser._
-      virtual_host.config.topics.find( x=> decode_filter(x.name).matches(name) ).getOrElse(new TopicDTO)
+      virtual_host.config.topics.find( x=> decode_filter(x.id).matches(name) ).getOrElse(new TopicDTO)
     }
 
     def can_create_ds(config:DurableSubscriptionDTO, security:SecurityContext) = {
@@ -350,8 +347,7 @@ class LocalRouter(val virtual_host:Virtu
         return new Failure("Not authorized to create the destination")
       }
 
-      val id = topic_id_counter.incrementAndGet
-      val topic = new Topic(LocalRouter.this, destination.asInstanceOf[TopicDestinationDTO], dto, id)
+      val topic = new Topic(LocalRouter.this, destination.asInstanceOf[TopicDestinationDTO], dto, path.toString(destination_parser))
       add_destination(path, topic)
       Success(topic)
     }
@@ -403,7 +399,7 @@ class LocalRouter(val virtual_host:Virtu
       if( queue.config.unified.getOrElse(false) ) {
         // hook up the queue to be a subscriber of the topic.
 
-        val topic = topic_domain.get_or_create_destination(path, new TopicDestinationDTO(queue.binding.binding_dto.parts), null).success
+        val topic = topic_domain.get_or_create_destination(path, new TopicDestinationDTO(queue.binding.binding_dto.path), null).success
         topic.bind(null, queue)
       }
     }
@@ -415,14 +411,14 @@ class LocalRouter(val virtual_host:Virtu
       import OptionSupport._
       if( queue.config.unified.getOrElse(false) ) {
         // unhook the queue from the topic
-        val topic = topic_domain.get_or_create_destination(path, new TopicDestinationDTO(queue.binding.binding_dto.parts), null).success
+        val topic = topic_domain.get_or_create_destination(path, new TopicDestinationDTO(queue.binding.binding_dto.path), null).success
         topic.unbind(queue, false)
       }
     }
 
     def create_destination(path: Path, destination:DestinationDTO, security: SecurityContext) = {
       val dto = new QueueDestinationDTO
-      dto.parts.addAll(destination.parts)
+      dto.path.addAll(destination.path)
 
       val binding = QueueDomainQueueBinding.create(dto)
       val config = binding.config(virtual_host)
@@ -578,7 +574,7 @@ class LocalRouter(val virtual_host:Virtu
 
   def bind(destination: Array[DestinationDTO], consumer: DeliveryConsumer, security: SecurityContext) = {
     consumer.retain
-    val paths = destination.map(x=> (destination_parser.decode_path(x.parts), x) )
+    val paths = destination.map(x=> (destination_parser.decode_path(x.path), x) )
     dispatch_queue ! {
       val failures = paths.map(x=> domain(x._2).can_bind_all(x._1, x._2, consumer, security) ).flatMap( _.failure_option )
       val rc = if( !failures.isEmpty ) {
@@ -606,7 +602,7 @@ class LocalRouter(val virtual_host:Virtu
 
   def connect(destinations: Array[DestinationDTO], producer: BindableDeliveryProducer, security: SecurityContext) = {
     producer.retain
-    val paths = destinations.map(x=> (destination_parser.decode_path(x.parts), x) )
+    val paths = destinations.map(x=> (destination_parser.decode_path(x.path), x) )
     dispatch_queue ! {
 
       val failures = paths.map(x=> domain(x._2).can_connect_all(x._1, x._2, producer, security) ).flatMap( _.failure_option )
@@ -641,7 +637,7 @@ class LocalRouter(val virtual_host:Virtu
    * Returns the previously created queue if it already existed.
    */
   def _get_or_create_destination(dto: DestinationDTO, security:SecurityContext): Result[DomainDestination, String] = {
-    val path = destination_parser.decode_path(dto.parts)
+    val path = destination_parser.decode_path(dto.path)
     domain(dto).get_or_create_destination(path, dto, security)
   }
 
@@ -654,7 +650,7 @@ class LocalRouter(val virtual_host:Virtu
   /////////////////////////////////////////////////////////////////////////////
 
   var queues_by_binding = HashMap[QueueBinding, Queue]()
-  var queues_by_id = HashMap[Long, Queue]()
+  var queues_by_id = HashMap[String, Queue]()
 
   /**
    * Gets an existing queue.
@@ -666,7 +662,7 @@ class LocalRouter(val virtual_host:Virtu
   /**
    * Gets an existing queue.
    */
-  def get_queue(id:Long) = dispatch_queue ! {
+  def get_queue(id:String) = dispatch_queue ! {
     queues_by_id.get(id)
   }
 
@@ -705,9 +701,9 @@ class LocalRouter(val virtual_host:Virtu
   /**
    * Returns true if the queue no longer exists.
    */
-  def destroy_queue(id:Long, security:SecurityContext) = dispatch_queue ! { _destroy_queue(id,security) }
+  def destroy_queue(id:String, security:SecurityContext) = dispatch_queue ! { _destroy_queue(id,security) }
 
-  def _destroy_queue(id:Long, security:SecurityContext):Result[Zilch, String] = {
+  def _destroy_queue(id:String, security:SecurityContext):Result[Zilch, String] = {
     queues_by_id.get(id) match {
       case Some(queue) =>
         _destroy_queue(queue,security)
@@ -744,7 +740,7 @@ class LocalRouter(val virtual_host:Virtu
     queue.stop
     if( queue.tune_persistent ) {
       queue.dispatch_queue {
-        virtual_host.store.remove_queue(queue.id){x=> Unit}
+        virtual_host.store.remove_queue(queue.store_id){x=> Unit}
       }
     }
     Success(Zilch)

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=1128015&r1=1128014&r2=1128015&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 Thu May 26 18:19:40 2011
@@ -45,12 +45,13 @@ import Queue._
  *
  * @author <a href="http://hiramchirino.com">Hiram Chirino</a>
  */
-class Queue(val router: LocalRouter, val id:Long, val binding:QueueBinding, var config:QueueDTO) extends BaseRetained with BindableDeliveryProducer with DeliveryConsumer with BaseService with DomainDestination with Dispatched {
+class Queue(val router: LocalRouter, val store_id:Long, val binding:QueueBinding, var config:QueueDTO) extends BaseRetained with BindableDeliveryProducer with DeliveryConsumer with BaseService with DomainDestination with Dispatched {
 
   override def toString: String =  {
     "Queue(id:%d, binding:%s)".format(id, binding)
   }
 
+  def id = binding.id
 
   def virtual_host = router.virtual_host
 
@@ -60,13 +61,13 @@ class Queue(val router: LocalRouter, val
 
   val filter = binding.message_filter
 
-  override val dispatch_queue: DispatchQueue = createQueue(binding.label);
+  override val dispatch_queue: DispatchQueue = createQueue(binding.id);
   virtual_host.broker.init_dispatch_queue(dispatch_queue)
 
   def destination_dto: DestinationDTO = binding.binding_dto
 
   dispatch_queue {
-    debug("created queue for: " + binding.label)
+    debug("created queue for: " + binding.id)
   }
 
 
@@ -196,7 +197,7 @@ class Queue(val router: LocalRouter, val
 
     if( tune_persistent ) {
 
-      virtual_host.store.list_queue_entry_ranges(id, tune_swap_range_size) { ranges=>
+      virtual_host.store.list_queue_entry_ranges(store_id, tune_swap_range_size) { ranges=>
         dispatch_queue {
           if( ranges!=null && !ranges.isEmpty ) {
 
@@ -711,7 +712,7 @@ class QueueEntry(val queue:Queue, val se
 
   def toQueueEntryRecord = {
     val qer = new QueueEntryRecord
-    qer.queue_key = queue.id
+    qer.queue_key = queue.store_id
     qer.entry_seq = seq
     qer.message_key = state.message_key
     qer.size = state.size
@@ -1254,7 +1255,7 @@ class QueueEntry(val queue:Queue, val se
     override def swap_in() = {
       if( !swapping_in ) {
         swapping_in = true
-        queue.virtual_host.store.list_queue_entries(queue.id, seq, last) { records =>
+        queue.virtual_host.store.list_queue_entries(queue.store_id, seq, last) { records =>
           if( !records.isEmpty ) {
             queue.dispatch_queue {
 

Modified: activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/QueueBinding.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/QueueBinding.scala?rev=1128015&r1=1128014&r2=1128015&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/QueueBinding.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/QueueBinding.scala Thu May 26 18:19:40 2011
@@ -70,9 +70,9 @@ object QueueBinding {
 trait QueueBinding {
 
   /**
-   * A user friendly description of the binding.
+   * The name of the queue (could be the queue name or a subscription id etc)
    */
-  def label:String
+  def id:String
 
   /**
    * Wires a queue into the a virtual host based on the binding information contained
@@ -94,7 +94,7 @@ trait QueueBinding {
 
   def config(host:VirtualHost):QueueDTO
 
-  override def toString: String = label
+  override def toString: String = id
 }
 
 object QueueDomainQueueBinding extends QueueBinding.Provider {
@@ -129,7 +129,7 @@ class QueueDomainQueueBinding(val bindin
 
   import QueueDomainQueueBinding._
 
-  val destination = LocalRouter.destination_parser.decode_path(binding_dto.parts)
+  val destination = LocalRouter.destination_parser.decode_path(binding_dto.path)
   def binding_kind = POINT_TO_POINT_KIND
 
   def unbind(node: LocalRouter, queue: Queue) = {
@@ -140,7 +140,7 @@ class QueueDomainQueueBinding(val bindin
     node.queue_domain.bind(queue)
   }
 
-  def label = binding_dto.name(".")
+  val id = binding_dto.name(LocalRouter.destination_parser.path_separator)
 
   override def hashCode = binding_kind.hashCode ^ binding_data.hashCode
 
@@ -155,7 +155,7 @@ class QueueDomainQueueBinding(val bindin
     import LocalRouter.destination_parser._
 
     def matches(x:QueueDTO):Boolean = {
-      if( x.name != null && !decode_filter(x.name).matches(destination)) {
+      if( x.id != null && !decode_filter(x.id).matches(destination)) {
         return false
       }
       true
@@ -196,7 +196,7 @@ object DurableSubscriptionQueueBinding e
 class DurableSubscriptionQueueBinding(val binding_data:Buffer, val binding_dto:DurableSubscriptionDestinationDTO) extends QueueBinding {
   import DurableSubscriptionQueueBinding._
 
-  val destination = LocalRouter.destination_parser.decode_path(binding_dto.parts)
+  val destination = LocalRouter.destination_parser.decode_path(binding_dto.path)
 
   def binding_kind = DURABLE_SUB_KIND
 
@@ -209,13 +209,7 @@ class DurableSubscriptionQueueBinding(va
     router.topic_domain.bind(queue)
   }
 
-  def label = {
-    var rc = "sub: '"+binding_dto.subscription_id+"'"
-    if( binding_dto.filter!=null ) {
-      rc += " filtering '"+binding_dto.filter+"'"
-    }
-    rc
-  }
+  def id = binding_dto.subscription_id
 
   override def hashCode = binding_kind.hashCode ^ binding_data.hashCode
 
@@ -238,7 +232,7 @@ class DurableSubscriptionQueueBinding(va
       import AsciiBuffer._
 
       def matches(x:DurableSubscriptionDTO):Boolean = {
-        if( x.name != null && !decode_filter(x.name).matches(destination)) {
+        if( x.id != null && !decode_filter(x.id).matches(destination)) {
           return false
         }
         if( x.subscription_id != null && x.subscription_id!=binding_dto.subscription_id ) {
@@ -274,7 +268,7 @@ object TempQueueBinding extends QueueBin
  *
  * @author <a href="http://hiramchirino.com">Hiram Chirino</a>
  */
-class TempQueueBinding(val key:AnyRef, val label:String) extends QueueBinding {
+class TempQueueBinding(val key:AnyRef, val id:String) extends QueueBinding {
   import TempQueueBinding._
 
   def this(c:DeliveryConsumer) = this(c, c.connection.map(_.transport.getRemoteAddress).getOrElse("known") )

Modified: activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Router.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Router.scala?rev=1128015&r1=1128014&r2=1128015&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Router.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Router.scala Thu May 26 18:19:40 2011
@@ -33,7 +33,7 @@ trait Router extends Service {
 
   def virtual_host:VirtualHost
 
-  def get_queue(dto:Long):Option[Queue] @suspendable
+  def get_queue(dto:String):Option[Queue] @suspendable
 
   def bind(destinations:Array[DestinationDTO], consumer:DeliveryConsumer, security:SecurityContext) : Result[Zilch,String] @suspendable
 

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=1128015&r1=1128014&r2=1128015&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 Thu May 26 18:19:40 2011
@@ -30,7 +30,7 @@ import collection.mutable.{HashMap, List
  *
  * @author <a href="http://hiramchirino.com">Hiram Chirino</a>
  */
-class Topic(val router:LocalRouter, val destination_dto:TopicDestinationDTO, val config:TopicDTO, val id:Long) extends DomainDestination {
+class Topic(val router:LocalRouter, val destination_dto:TopicDestinationDTO, val config:TopicDTO, val id:String) extends DomainDestination {
 
   var producers = ListBuffer[BindableDeliveryProducer]()
   var consumers = ListBuffer[DeliveryConsumer]()

Modified: activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/AclAuthorizer.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/AclAuthorizer.scala?rev=1128015&r1=1128014&r2=1128015&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/AclAuthorizer.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/AclAuthorizer.scala Thu May 26 18:19:40 2011
@@ -104,7 +104,7 @@ class AclAuthorizer(val default_kinds:Li
   }
 
 
-  def name(topic: TopicDTO) = Option(topic.name).getOrElse("**")
+  def name(topic: TopicDTO) = Option(topic.id).getOrElse("**")
 
   def can_send_to(ctx: SecurityContext, host: VirtualHost, topic: TopicDTO) = log_result(ctx, "send", "topic "+name(topic)) {
     can_topic(ctx, topic)(_.sends)
@@ -141,7 +141,7 @@ class AclAuthorizer(val default_kinds:Li
     queue.acl==null || is_in(ctx, func(queue.acl))
   }
 
-  def name(queue: QueueDTO) = Option(queue.name).getOrElse("**")
+  def name(queue: QueueDTO) = Option(queue.id).getOrElse("**")
 
   def can_send_to(ctx: SecurityContext, host: VirtualHost, queue: QueueDTO) = log_result(ctx, "send", "queue "+name(queue)) {
     can_queue(ctx, queue)(_.sends)

Modified: activemq/activemq-apollo/trunk/apollo-broker/src/test/resources/org/apache/activemq/apollo/broker/destination-config.xml
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-broker/src/test/resources/org/apache/activemq/apollo/broker/destination-config.xml?rev=1128015&r1=1128014&r2=1128015&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-broker/src/test/resources/org/apache/activemq/apollo/broker/destination-config.xml (original)
+++ activemq/activemq-apollo/trunk/apollo-broker/src/test/resources/org/apache/activemq/apollo/broker/destination-config.xml Thu May 26 18:19:40 2011
@@ -20,11 +20,11 @@
   <virtual_host id="default">
     <host_name>test</host_name>
 
-    <topic name="unified.*"/>
-    <topic name="notunified.*"/>
+    <topic id="unified.*"/>
+    <topic id="notunified.*"/>
 
-    <queue name="unified.a" kind="ptp" queue_buffer="333" unified="true"/>
-    <queue name="unified.*" kind="ds" queue_buffer="444" unified="true"/>
+    <queue id="unified.a" kind="ptp" queue_buffer="333" unified="true"/>
+    <queue id="unified.*" kind="ds" queue_buffer="444" unified="true"/>
     <queue queue_buffer="111"/>
 
   </virtual_host>

Modified: activemq/activemq-apollo/trunk/apollo-broker/src/test/scala/org/apache/activemq/apollo/broker/DestinationConfigurationTest.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-broker/src/test/scala/org/apache/activemq/apollo/broker/DestinationConfigurationTest.scala?rev=1128015&r1=1128014&r2=1128015&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-broker/src/test/scala/org/apache/activemq/apollo/broker/DestinationConfigurationTest.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-broker/src/test/scala/org/apache/activemq/apollo/broker/DestinationConfigurationTest.scala Thu May 26 18:19:40 2011
@@ -62,15 +62,15 @@ class DestinationConfigurationTest exten
 
     check_tune_queue_buffer(333) {
       var p = new QueueDestinationDTO()
-      p.parts.add("unified")
-      p.parts.add("a")
+      p.path.add("unified")
+      p.path.add("a")
       p
     }
 
     check_tune_queue_buffer(111) {
       var p = new QueueDestinationDTO()
-      p.parts.add("notunified")
-      p.parts.add("other")
+      p.path.add("notunified")
+      p.path.add("other")
       p
     }
 

Modified: activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/DestinationDTO.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/DestinationDTO.java?rev=1128015&r1=1128014&r2=1128015&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/DestinationDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/DestinationDTO.java Thu May 26 18:19:40 2011
@@ -35,23 +35,23 @@ import java.util.List;
 @XmlAccessorType(XmlAccessType.FIELD)
 abstract public class DestinationDTO {
 
-    @XmlElement(name = "part")
-    public List<String> parts = new ArrayList<String>();
+    @XmlElement(name = "path")
+    public List<String> path = new ArrayList<String>();
 
     public DestinationDTO() {
     }
 
-    public DestinationDTO(List<String> parts) {
-        this.parts = parts;
+    public DestinationDTO(List<String> path) {
+        this.path = path;
     }
 
-    public DestinationDTO(String parts[]) {
-        this(Arrays.asList(parts));
+    public DestinationDTO(String path[]) {
+        this(Arrays.asList(path));
     }
 
     public String name(String separator) {
         StringBuilder sb  = new StringBuilder();
-        for( String p : parts ) {
+        for( String p : path) {
             if( sb.length() != 0 ) {
                 sb.append(separator);
             }

Modified: activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/DurableSubscriptionDTO.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/DurableSubscriptionDTO.java?rev=1128015&r1=1128014&r2=1128015&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/DurableSubscriptionDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/DurableSubscriptionDTO.java Thu May 26 18:19:40 2011
@@ -23,7 +23,7 @@ import javax.xml.bind.annotation.*;
 /**
  * @author <a href="http://hiramchirino.com">Hiram Chirino</a>
  */
-@XmlRootElement(name = "durable_subscription")
+@XmlRootElement(name = "ds")
 @XmlAccessorType(XmlAccessType.FIELD)
 public class DurableSubscriptionDTO extends QueueDTO {
 

Modified: activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/QueueDTO.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/QueueDTO.java?rev=1128015&r1=1128014&r2=1128015&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/QueueDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/QueueDTO.java Thu May 26 18:19:40 2011
@@ -25,13 +25,7 @@ import javax.xml.bind.annotation.*;
  */
 @XmlRootElement(name = "queue")
 @XmlAccessorType(XmlAccessType.FIELD)
-public class QueueDTO {
-
-    /*
-     * The destination name this queue is associated with.  You can use wild cards.
-     */
-    @XmlAttribute
-    public String name;
+public class QueueDTO extends StringIdDTO {
 
     /**
      * If set to true, then routing then there is no difference between

Modified: activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/QueueDestinationDTO.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/QueueDestinationDTO.java?rev=1128015&r1=1128014&r2=1128015&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/QueueDestinationDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/QueueDestinationDTO.java Thu May 26 18:19:40 2011
@@ -46,13 +46,13 @@ public class QueueDestinationDTO extends
 
         QueueDestinationDTO that = (QueueDestinationDTO) o;
 
-        if (parts != null ? !parts.equals(that.parts) : that.parts != null) return false;
+        if (path != null ? !path.equals(that.path) : that.path != null) return false;
 
         return true;
     }
 
     @Override
     public int hashCode() {
-        return parts != null ? parts.hashCode() : 0;
+        return path != null ? path.hashCode() : 0;
     }
 }
\ No newline at end of file

Modified: activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/QueueStatusDTO.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/QueueStatusDTO.java?rev=1128015&r1=1128014&r2=1128015&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/QueueStatusDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/QueueStatusDTO.java Thu May 26 18:19:40 2011
@@ -28,13 +28,13 @@ import java.util.List;
  */
 @XmlRootElement(name="queue_status")
 @XmlAccessorType(XmlAccessType.FIELD)
-public class QueueStatusDTO extends LongIdDTO {
+public class QueueStatusDTO extends StringIdDTO {
 
     @XmlElement
     public QueueDTO config;
 
     @XmlElement
-    public DestinationDTO destination;
+    public DestinationDTO binding;
 
     @XmlElement
     public QueueMetricsDTO metrics = new QueueMetricsDTO();

Modified: activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/TopicDTO.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/TopicDTO.java?rev=1128015&r1=1128014&r2=1128015&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/TopicDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/TopicDTO.java Thu May 26 18:19:40 2011
@@ -25,13 +25,7 @@ import javax.xml.bind.annotation.*;
  */
 @XmlRootElement(name = "topic")
 @XmlAccessorType(XmlAccessType.FIELD)
-public class TopicDTO {
-
-    /**
-     * The name of the destination.  You can use wild cards.
-     */
-	@XmlAttribute
-	public String name;
+public class TopicDTO extends StringIdDTO {
 
     @XmlElement(name="slow_consumer_policy")
     public String slow_consumer_policy;

Modified: activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/TopicDestinationDTO.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/TopicDestinationDTO.java?rev=1128015&r1=1128014&r2=1128015&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/TopicDestinationDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/TopicDestinationDTO.java Thu May 26 18:19:40 2011
@@ -17,7 +17,6 @@
 package org.apache.activemq.apollo.dto;
 
 import javax.xml.bind.annotation.*;
-import java.util.Arrays;
 import java.util.List;
 
 /**
@@ -49,14 +48,14 @@ public class TopicDestinationDTO extends
 
         TopicDestinationDTO that = (TopicDestinationDTO) o;
 
-        if (parts != null ? !parts.equals(that.parts) : that.parts != null) return false;
+        if (path != null ? !path.equals(that.path) : that.path != null) return false;
 
         return true;
     }
 
     @Override
     public int hashCode() {
-        return parts != null ? parts.hashCode() : 0;
+        return path != null ? path.hashCode() : 0;
     }
 
 }
\ No newline at end of file

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=1128015&r1=1128014&r2=1128015&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 Thu May 26 18:19:40 2011
@@ -28,13 +28,7 @@ import java.util.List;
  */
 @XmlRootElement(name="topic_status")
 @XmlAccessorType(XmlAccessType.FIELD)
-public class TopicStatusDTO extends LongIdDTO {
-
-    /**
-     * The destination name
-     */
-    @XmlAttribute
-    public String name;
+public class TopicStatusDTO extends StringIdDTO {
 
     @XmlElement
     public TopicDTO config;
@@ -55,5 +49,5 @@ public class TopicStatusDTO extends Long
      * Ids of all queues that are associated with the destination
      */
     @XmlElement(name="durable_subscription")
-    public List<LongIdLabeledDTO> durable_subscriptions = new ArrayList<LongIdLabeledDTO>();
+    public List<String> durable_subscriptions = new ArrayList<String>();
 }
\ No newline at end of file

Modified: activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/VirtualHostDTO.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/VirtualHostDTO.java?rev=1128015&r1=1128014&r2=1128015&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/VirtualHostDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/VirtualHostDTO.java Thu May 26 18:19:40 2011
@@ -61,7 +61,7 @@ public class VirtualHostDTO extends Serv
     /**
      * Holds the configuration for the queues.
      */
-    @XmlElement(name="durable_subscription")
+    @XmlElement(name="ds")
     public ArrayList<DurableSubscriptionDTO> durable_subscriptions = new ArrayList<DurableSubscriptionDTO>();
 
     /**

Modified: activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/VirtualHostStatusDTO.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/VirtualHostStatusDTO.java?rev=1128015&r1=1128014&r2=1128015&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/VirtualHostStatusDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/VirtualHostStatusDTO.java Thu May 26 18:19:40 2011
@@ -36,13 +36,13 @@ public class VirtualHostStatusDTO extend
      * Ids of all the topics running on the broker
      */
     @XmlElement(name="topics")
-    public List<LongIdLabeledDTO> topics = new ArrayList<LongIdLabeledDTO>();
+    public List<String> topics = new ArrayList<String>();
 
     /**
      * Ids of all the queues running on the broker
      */
     @XmlElement(name="queues")
-    public List<LongIdLabeledDTO> queues = new ArrayList<LongIdLabeledDTO>();
+    public List<String> queues = new ArrayList<String>();
 
     /**
      * Is the virtual host using a store.

Modified: activemq/activemq-apollo/trunk/apollo-dto/src/test/java/org/apache/activemq/apollo/dto/XmlCodecTest.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/test/java/org/apache/activemq/apollo/dto/XmlCodecTest.java?rev=1128015&r1=1128014&r2=1128015&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/test/java/org/apache/activemq/apollo/dto/XmlCodecTest.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/test/java/org/apache/activemq/apollo/dto/XmlCodecTest.java Thu May 26 18:19:40 2011
@@ -45,9 +45,9 @@ public class XmlCodecTest {
         assertEquals("localhost", host.host_names.get(0));
         assertEquals("example.com", host.host_names.get(1));
 
-        assertEquals("queue1", host.queues.get(0).name);
-        assertEquals("topic1", host.topics.get(0).name);
-        assertEquals("durable_subscription1", host.durable_subscriptions.get(0).name);
+        assertEquals("queue1", host.queues.get(0).id);
+        assertEquals("topic1", host.topics.get(0).id);
+        assertEquals("durable_subscription1", host.durable_subscriptions.get(0).id);
 
         assertNotNull(dto.acl);
         assertTrue(dto.acl.admins.contains(new PrincipalDTO("hiram")));

Modified: activemq/activemq-apollo/trunk/apollo-dto/src/test/resources/org/apache/activemq/apollo/dto/XmlCodecTest.xml
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/test/resources/org/apache/activemq/apollo/dto/XmlCodecTest.xml?rev=1128015&r1=1128014&r2=1128015&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/test/resources/org/apache/activemq/apollo/dto/XmlCodecTest.xml (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/test/resources/org/apache/activemq/apollo/dto/XmlCodecTest.xml Thu May 26 18:19:40 2011
@@ -28,9 +28,9 @@
     <host_name>localhost</host_name>
     <host_name>example.com</host_name>
 
-    <queue name="queue1"/>
-    <topic name="topic1"/>
-    <durable_subscription name="durable_subscription1"/>
+    <queue id="queue1"/>
+    <topic id="topic1"/>
+    <ds id="durable_subscription1"/>
 
   </virtual_host>
 

Modified: activemq/activemq-apollo/trunk/apollo-openwire/src/main/scala/org/apache/activemq/apollo/openwire/OpenwireProtocolHandler.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-openwire/src/main/scala/org/apache/activemq/apollo/openwire/OpenwireProtocolHandler.scala?rev=1128015&r1=1128014&r2=1128015&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-openwire/src/main/scala/org/apache/activemq/apollo/openwire/OpenwireProtocolHandler.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-openwire/src/main/scala/org/apache/activemq/apollo/openwire/OpenwireProtocolHandler.scala Thu May 26 18:19:40 2011
@@ -467,7 +467,7 @@ class OpenwireProtocolHandler extends Pr
       if( is_durable_sub ) {
         destination = destination.map { _ match {
           case x:TopicDestinationDTO=>
-            val rc = new DurableSubscriptionDestinationDTO(x.parts)
+            val rc = new DurableSubscriptionDestinationDTO(x.path)
             if( is_durable_sub ) {
               rc.subscription_id = ""
               if( parent.parent.info.getClientId != null ) {

Modified: activemq/activemq-apollo/trunk/apollo-stomp/src/main/scala/org/apache/activemq/apollo/stomp/StompProtocolHandler.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-stomp/src/main/scala/org/apache/activemq/apollo/stomp/StompProtocolHandler.scala?rev=1128015&r1=1128014&r2=1128015&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-stomp/src/main/scala/org/apache/activemq/apollo/stomp/StompProtocolHandler.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-stomp/src/main/scala/org/apache/activemq/apollo/stomp/StompProtocolHandler.scala Thu May 26 18:19:40 2011
@@ -923,7 +923,7 @@ class StompProtocolHandler extends Proto
     if( persistent ) {
       destination = destination.map { _ match {
         case x:TopicDestinationDTO=>
-          val rc = new DurableSubscriptionDestinationDTO(x.parts)
+          val rc = new DurableSubscriptionDestinationDTO(x.path)
           rc.subscription_id = decode_header(id)
           rc.filter = if (selector == null) null else selector._1
           rc

Modified: activemq/activemq-apollo/trunk/apollo-stomp/src/test/scala/org/apache/activemq/apollo/stomp/perf/StompRemoteClients.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-stomp/src/test/scala/org/apache/activemq/apollo/stomp/perf/StompRemoteClients.scala?rev=1128015&r1=1128014&r2=1128015&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-stomp/src/test/scala/org/apache/activemq/apollo/stomp/perf/StompRemoteClients.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-stomp/src/test/scala/org/apache/activemq/apollo/stomp/perf/StompRemoteClients.scala Thu May 26 18:19:40 2011
@@ -39,8 +39,8 @@ class StompRemoteConsumer extends Remote
     outboundSink.refiller = ^ {}
 
     val stompDestination = destination match {
-      case x:QueueDestinationDTO => ascii("/queue/" + x.parts);
-      case x:TopicDestinationDTO => ascii("/topic/" + x.parts);
+      case x:QueueDestinationDTO => ascii("/queue/" + x.path);
+      case x:TopicDestinationDTO => ascii("/topic/" + x.path);
     }
 
     var frame = StompFrame(CONNECT);
@@ -149,8 +149,8 @@ class StompRemoteProducer extends Remote
     outboundSink.refiller = ^ {drain}
 
     stompDestination = destination match {
-      case x:QueueDestinationDTO => ascii("/queue/" + x.parts);
-      case x:TopicDestinationDTO => ascii("/topic/" + x.parts);
+      case x:QueueDestinationDTO => ascii("/queue/" + x.path);
+      case x:TopicDestinationDTO => ascii("/topic/" + x.path);
     }
 
     outboundSink.offer(StompFrame(CONNECT));

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=1128015&r1=1128014&r2=1128015&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 Thu May 26 18:19:40 2011
@@ -27,6 +27,7 @@ import scala.collection.Iterable
 import org.apache.activemq.apollo.util.{Failure, Success, Dispatched, Result}
 import scala.Some
 import security.{SecurityContext, Authorizer}
+import org.apache.activemq.apollo.util.path.PathParser
 
 /**
  * <p>
@@ -37,7 +38,7 @@ import security.{SecurityContext, Author
  * @author <a href="http://hiramchirino.com">Hiram Chirino</a>
  */
 @Produces(Array("application/json", "application/xml","text/xml", "text/html;qs=5"))
-case class BrokerResource extends Resource {
+case class BrokerResource() extends Resource {
 
   @Path("config")
   def config_resource:ConfigurationResource = {
@@ -166,11 +167,11 @@ case class BrokerResource extends Resour
         val router:LocalRouter = host
 
         router.topic_domain.destinations.foreach { node=>
-          result.topics.add(new LongIdLabeledDTO(node.id, node.name))
+          result.topics.add(node.id)
         }
 
         router.queue_domain.destinations.foreach { node=>
-          result.queues.add(new LongIdLabeledDTO(node.id, node.binding.label))
+          result.queues.add(node.id)
         }
 
         result
@@ -215,159 +216,131 @@ case class BrokerResource extends Resour
   def link(queue:Queue) = {
     val link = new LinkDTO()
     link.kind = "queue"
-    link.id = queue.id.toString
-    link.label = queue.binding.label
+    link.id = queue.id
+    link.label = queue.id
     link
   }
 
-  @GET @Path("virtual-hosts/{id}/topics/{dest}")
-  def destination(@PathParam("id") id : String, @PathParam("dest") dest : Long):TopicStatusDTO = {
+  @GET @Path("virtual-hosts/{id}/topics/{name:.*}")
+  def topic(@PathParam("id") id : String, @PathParam("name") name : String):TopicStatusDTO = {
     with_virtual_host(id) { host =>
-
       val router:LocalRouter = host
-      val node = router.topic_domain.destination_by_id.get(dest).getOrElse(result(NOT_FOUND))
-
-      monitoring(node) {
-        val rc = new TopicStatusDTO
-        rc.id = node.id
-        rc.name = node.name
-        rc.config = node.config
-
-        node.durable_subscriptions.foreach { q=>
-          rc.durable_subscriptions.add(new LongIdLabeledDTO(q.id, q.binding.label))
-        }
-        node.consumers.foreach { consumer=>
-          consumer match {
-            case queue:Queue =>
-              rc.consumers.add(link(queue))
-            case _ =>
-              consumer.connection.foreach{c=>
-                rc.consumers.add(link(c))
-              }
-          }
-        }
-        node.producers.flatMap( _.connection ).foreach { connection=>
-          rc.producers.add(link(connection))
-        }
-
-        rc
-      }
+      val node = router.topic_domain.destination_by_id.get(name).getOrElse(result(NOT_FOUND))
+      status(node)
     }
   }
 
-  @GET @Path("virtual-hosts/{id}/all-queues/{queue}")
-  def queue(@PathParam("id") id : String, @PathParam("queue") qid : Long, @QueryParam("entries") entries:Boolean):QueueStatusDTO = {
+  @GET @Path("virtual-hosts/{id}/queues/{name:.*}")
+  def queue(@PathParam("id") id : String, @PathParam("name") name : String, @QueryParam("entries") entries:Boolean ):QueueStatusDTO = {
     with_virtual_host(id) { host =>
-      val router:LocalRouter = host
-      val queue = router.queues_by_id.get(qid)
-      status(queue, entries)
+      val router: LocalRouter = host
+      val node = router.queue_domain.destination_by_id.get(name).getOrElse(result(NOT_FOUND))
+      status(node, entries)
     }
   }
 
-  @GET @Path("virtual-hosts/{id}/queues/{queue}")
-  def destination_queue(@PathParam("id") id : String, @PathParam("queue") qid : Long, @QueryParam("entries") entries:Boolean ):QueueStatusDTO = {
+  @GET @Path("virtual-hosts/{id}/ds/{name:.*}")
+  def durable_subscription(@PathParam("id") id : String, @PathParam("name") name : String, @QueryParam("entries") entries:Boolean):QueueStatusDTO = {
     with_virtual_host(id) { host =>
       val router:LocalRouter = host
-      val queue = router.queue_domain.destination_by_id.get(qid)
-      status(queue, entries)
+      val node = router.topic_domain.durable_subscriptions_by_id.get(name).getOrElse(result(NOT_FOUND))
+      status(node, entries)
     }
   }
 
-  def get_queue_metrics(q:Queue):QueueMetricsDTO = {
-    val rc = new QueueMetricsDTO
-
-    rc.enqueue_item_counter = q.enqueue_item_counter
-    rc.enqueue_size_counter = q.enqueue_size_counter
-    rc.enqueue_ts = q.enqueue_ts
-
-    rc.dequeue_item_counter = q.dequeue_item_counter
-    rc.dequeue_size_counter = q.dequeue_size_counter
-    rc.dequeue_ts = q.dequeue_ts
-
-    rc.nack_item_counter = q.nack_item_counter
-    rc.nack_size_counter = q.nack_size_counter
-    rc.nack_ts = q.nack_ts
-
-    rc.queue_size = q.queue_size
-    rc.queue_items = q.queue_items
+  private def decode_path(name:String) = {
+    try {
+      LocalRouter.destination_parser.decode_path(name)
+    } catch {
+      case x:PathParser.PathException => result(NOT_FOUND)
+    }
+  }
 
-    rc.swap_out_item_counter = q.swap_out_item_counter
-    rc.swap_out_size_counter = q.swap_out_size_counter
-    rc.swap_in_item_counter = q.swap_in_item_counter
-    rc.swap_in_size_counter = q.swap_in_size_counter
+  def status(node: Topic): FutureResult[TopicStatusDTO] = {
+    monitoring(node) {
+      val rc = new TopicStatusDTO
+      rc.id = node.id
+      rc.config = node.config
 
-    rc.swapping_in_size = q.swapping_in_size
-    rc.swapping_out_size = q.swapping_out_size
+      node.durable_subscriptions.foreach {
+        q =>
+          rc.durable_subscriptions.add(q.id)
+      }
+      node.consumers.foreach {
+        consumer =>
+          consumer match {
+            case queue: Queue =>
+              rc.consumers.add(link(queue))
+            case _ =>
+              consumer.connection.foreach {
+                c =>
+                  rc.consumers.add(link(c))
+              }
+          }
+      }
+      node.producers.flatMap(_.connection).foreach {
+        connection =>
+          rc.producers.add(link(connection))
+      }
 
-    rc.swapped_in_items = q.swapped_in_items
-    rc.swapped_in_size = q.swapped_in_size
+      rc
+    }
+  }
 
-    rc.swapped_in_size_max = q.swapped_in_size_max
+  def status(q:Queue, entries:Boolean=false) = monitoring(q) {
+    val rc = new QueueStatusDTO
+    rc.id = q.id
+    rc.binding = q.binding.binding_dto
+    rc.config = q.config
+    rc.metrics = get_queue_metrics(q)
+
+    if( entries ) {
+      var cur = q.head_entry
+      while( cur!=null ) {
+
+        val e = new EntryStatusDTO
+        e.seq = cur.seq
+        e.count = cur.count
+        e.size = cur.size
+        e.consumer_count = cur.parked.size
+        e.is_prefetched = cur.is_prefetched
+        e.state = cur.label
 
-    rc
-  }
+        rc.entries.add(e)
 
-  def status(qo:Option[Queue], entries:Boolean=false) = qo match {
-    case None=> result(NOT_FOUND)
-    case Some(q)=> sync(q) {
-      monitoring(q) {
-        val rc = new QueueStatusDTO
-        rc.id = q.id
-        rc.destination = q.binding.binding_dto
-        rc.config = q.config
-        rc.metrics = get_queue_metrics(q)
-
-        if( entries ) {
-          var cur = q.head_entry
-          while( cur!=null ) {
-
-            val e = new EntryStatusDTO
-            e.seq = cur.seq
-            e.count = cur.count
-            e.size = cur.size
-            e.consumer_count = cur.parked.size
-            e.is_prefetched = cur.is_prefetched
-            e.state = cur.label
-
-            rc.entries.add(e)
-
-            cur = if( cur == q.tail_entry ) {
-              null
-            } else {
-              cur.nextOrTail
-            }
-          }
+        cur = if( cur == q.tail_entry ) {
+          null
         } else {
-//        rc.entries = null
+          cur.nextOrTail
         }
+      }
+    }
 
-        q.inbound_sessions.flatMap( _.producer.connection ).foreach { connection=>
-          rc.producers.add(link(connection))
-        }
-        q.all_subscriptions.valuesIterator.toSeq.foreach{ sub =>
-          val status = new QueueConsumerStatusDTO
-          sub.consumer.connection.foreach(x=> status.link = link(x))
-          status.position = sub.pos.seq
-          status.total_dispatched_count = sub.total_dispatched_count
-          status.total_dispatched_size = sub.total_dispatched_size
-          status.total_ack_count = sub.total_ack_count
-          status.total_nack_count = sub.total_nack_count
-          status.acquired_size = sub.acquired_size
-          status.acquired_count = sub.acquired_count
-          status.waiting_on = if( sub.full ) {
-            "ack"
-          } else if( sub.pos.is_tail ) {
-            "producer"
-          } else if( !sub.pos.is_loaded ) {
-            "load"
-          } else {
-            "dispatch"
-          }
-          rc.consumers.add(status)
-        }
-        rc
+    q.inbound_sessions.flatMap( _.producer.connection ).foreach { connection=>
+      rc.producers.add(link(connection))
+    }
+    q.all_subscriptions.valuesIterator.toSeq.foreach{ sub =>
+      val status = new QueueConsumerStatusDTO
+      sub.consumer.connection.foreach(x=> status.link = link(x))
+      status.position = sub.pos.seq
+      status.total_dispatched_count = sub.total_dispatched_count
+      status.total_dispatched_size = sub.total_dispatched_size
+      status.total_ack_count = sub.total_ack_count
+      status.total_nack_count = sub.total_nack_count
+      status.acquired_size = sub.acquired_size
+      status.acquired_count = sub.acquired_count
+      status.waiting_on = if( sub.full ) {
+        "ack"
+      } else if( sub.pos.is_tail ) {
+        "producer"
+      } else if( !sub.pos.is_loaded ) {
+        "load"
+      } else {
+        "dispatch"
       }
+      rc.consumers.add(status)
     }
+    rc
   }
 
 
@@ -465,4 +438,40 @@ case class BrokerResource extends Resour
     }
   }
 
+  def get_queue_metrics(q:Queue):QueueMetricsDTO = {
+    val rc = new QueueMetricsDTO
+
+    rc.enqueue_item_counter = q.enqueue_item_counter
+    rc.enqueue_size_counter = q.enqueue_size_counter
+    rc.enqueue_ts = q.enqueue_ts
+
+    rc.dequeue_item_counter = q.dequeue_item_counter
+    rc.dequeue_size_counter = q.dequeue_size_counter
+    rc.dequeue_ts = q.dequeue_ts
+
+    rc.nack_item_counter = q.nack_item_counter
+    rc.nack_size_counter = q.nack_size_counter
+    rc.nack_ts = q.nack_ts
+
+    rc.queue_size = q.queue_size
+    rc.queue_items = q.queue_items
+
+    rc.swap_out_item_counter = q.swap_out_item_counter
+    rc.swap_out_size_counter = q.swap_out_size_counter
+    rc.swap_in_item_counter = q.swap_in_item_counter
+    rc.swap_in_size_counter = q.swap_in_size_counter
+
+    rc.swapping_in_size = q.swapping_in_size
+    rc.swapping_out_size = q.swapping_out_size
+
+    rc.swapped_in_items = q.swapped_in_items
+    rc.swapped_in_size = q.swapped_in_size
+
+    rc.swapped_in_size_max = q.swapped_in_size_max
+
+    rc
+  }
+
+
+
 }

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=1128015&r1=1128014&r2=1128015&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 Thu May 26 18:19:40 2011
@@ -14,6 +14,7 @@
 -# limitations under the License.
 
 - import it._
+- import org.apache.activemq.apollo.dto._
 - val helper = new org.apache.activemq.apollo.web.resources.ViewHelper
 - import helper._
 - def percent(n:Long, d:Long) =
@@ -25,22 +26,15 @@
 .breadcumbs
   a(href={strip_resolve("..")}) Back
 
--#
-  - binding match
-    - case x:QueueBindingDTO =>
-      h1 Queue #{x.name}
-
-    - case x:SubscriptionBindingDTO =>
-      h1 Durable Subscription on #{x.name}
-      p client id: ${x.client_id}
-      p subscription id: ${x.subscription_id}
+- binding match
+  - case x:QueueDestinationDTO =>
+    h1 Queue #{id}
+  - case x:DurableSubscriptionDestinationDTO =>
+    h1 Durable Subscription on #{id}
+    - if( x.filter != null )
       p filter: ${x.filter}
-
-    - case x:TempBindingDTO =>
-      h1 Temporary Queue
-
-    - case x =>  x
-      h1 Unknown Queue Type: #{x.getClass.getName}
+  - case _ =>
+    h1 Temporary Queue
 
 h2 Current Size
 

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=1128015&r1=1128014&r2=1128015&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 Thu May 26 18:19:40 2011
@@ -20,7 +20,7 @@
 .breadcumbs
   a(href={strip_resolve("..")}) Back
 
-h1 Destination: #{name}
+h1 Destination: #{id}
 
 h3 Topic Domain
 
@@ -49,4 +49,4 @@ h4 Durable Subscribers
 ul
   - for( x <- durable_subscriptions )
     li
-      a(href={ path("../../all-queues/"+x.id) }) #{x.label}
+      a(href={ path("../../ds/"+x) }) #{x}

Modified: activemq/activemq-apollo/trunk/apollo-web/src/main/webapp/WEB-INF/org/apache/activemq/apollo/dto/VirtualHostStatusDTO.jade
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-web/src/main/webapp/WEB-INF/org/apache/activemq/apollo/dto/VirtualHostStatusDTO.jade?rev=1128015&r1=1128014&r2=1128015&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-web/src/main/webapp/WEB-INF/org/apache/activemq/apollo/dto/VirtualHostStatusDTO.jade (original)
+++ activemq/activemq-apollo/trunk/apollo-web/src/main/webapp/WEB-INF/org/apache/activemq/apollo/dto/VirtualHostStatusDTO.jade Thu May 26 18:19:40 2011
@@ -20,7 +20,7 @@
 .breadcumbs
   a(href={strip_resolve("..")}) Back
 
-h1 Virtual Host: #{id}
+h1 Virtual Host X: #{id}
 
 p state: #{state} #{ uptime(state_since) } ago
 
@@ -32,10 +32,10 @@ h2 Queues
 ul
   - for( x <- queues )
     li
-      a(href={ path("queues/"+x.id) }) #{x.label}
+      a(href={ path("queues/"+x) }) #{x}
 
 h2 Topics
 ul
   - for( x <- topics )
     li
-      a(href={ path("topics/"+x.id) }) #{x.label}
+      a(href={ path("topics/"+x) }) #{x}

Modified: activemq/activemq-apollo/trunk/apollo-website/src/documentation/management-api.md
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-website/src/documentation/management-api.md?rev=1128015&r1=1128014&r2=1128015&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-website/src/documentation/management-api.md (original)
+++ activemq/activemq-apollo/trunk/apollo-website/src/documentation/management-api.md Thu May 26 18:19:40 2011
@@ -55,7 +55,8 @@ Doing a GET against it will provide info
   
 Example:
 
-    $ curl -u "admin:password" http://localhost:61680/broker.json
+    $ curl -u "admin:password" \
+    http://localhost:61680/broker.json
 
 Results in a [Broker Status](./api/apollo-dto/org/apache/activemq/apollo/dto/BrokerStatusDTO.html) :
 
@@ -164,12 +165,12 @@ Results in a [Virtual Host Status](./api
   "state":"STARTED",
   "state_since":1305390871786,
   "topics":[
-    {"id":6,"label":"item.prices"},
-    {"id":3,"label":"inventory.level"}
+    "item.prices",
+    "inventory.level"
   ],
   "queues":[
-    {"id":10,"label":"orders.req"},
-    {"id":8,"label":"orders.res"},
+    "orders.req",
+    "orders.res",
   ],
   "store":true
 }
@@ -235,74 +236,59 @@ of the queue.
 Example:
 
     $ curl -u "admin:password" \
-    http://localhost:61680/broker/virtual-hosts/localhost/queues/1.json
+    http://localhost:61680/broker/virtual-hosts/localhost/queues/orders.req.json
 
 Results in a [Queue Status](./api/apollo-dto/org/apache/activemq/apollo/dto/QueueStatusDTO.html):
 
 {pygmentize:: js}
 {
-  "id":1,
-  "config":{
-    "name":null,
-    "unified":null,
-    "producer_buffer":null,
-    "queue_buffer":null,
-    "consumer_buffer":null,
-    "persistent":null,
-    "swap":null,
-    "swap_range_size":null,
-    "acl":null
-  },
-  "destination":{
-    "@class":"org.apache.activemq.apollo.dto.QueueDestinationDTO",
-    "name":"load-0"
-  },
-  "metrics":{
-    "enqueue_item_counter":37828413,
-    "enqueue_size_counter":41713628436,
-    "enqueue_ts":1305554574681,
-    "dequeue_item_counter":37828413,
-    "dequeue_size_counter":41713628436,
-    "dequeue_ts":1305554574681,
-    "nack_item_counter":0,
-    "nack_size_counter":0,
-    "nack_ts":1305554121093,
-    "queue_size":0,
-    "queue_items":0,
-    "swapped_in_size":0,
-    "swapped_in_items":0,
-    "swapping_in_size":0,
-    "swapping_out_size":0,
-    "swapped_in_size_max":557056,
-    "swap_out_item_counter":16,
-    "swap_out_size_counter":17634,
-    "swap_in_item_counter":16,
-    "swap_in_size_counter":17634
-  },
-  "entries":[],
-  "producers":[
-    {
-      "kind":"connection",
-      "id":"1",
-      "label":"/127.0.0.1:52690"
-    }
-  ],"consumers":[
-    {
-      "link":{
-        "kind":"connection",
-        "id":"2",
-        "label":"/127.0.0.1:52691"
-      },
-      "position":37828414,
-      "acquired_count":0,
-      "acquired_size":0,
-      "total_dispatched_count":37828413,
-      "total_dispatched_size":41713628436,
-      "total_ack_count":37828413,
-      "total_nack_count":0,
-      "waiting_on":"producer"
-    }
-  ]
+   "id":"orders.req",
+   "config":{
+      "id":null,
+      "unified":null,
+      "producer_buffer":null,
+      "queue_buffer":null,
+      "consumer_buffer":null,
+      "persistent":null,
+      "swap":null,
+      "swap_range_size":null,
+      "acl":null
+   },
+   "binding":{
+      "@class":"org.apache.activemq.apollo.dto.QueueDestinationDTO",
+      "path":[
+         "orders",
+         "req"
+      ]
+   },
+   "metrics":{
+      "enqueue_item_counter":72292,
+      "enqueue_size_counter":79943846,
+      "enqueue_ts":1306433477839,
+      "dequeue_item_counter":72144,
+      "dequeue_size_counter":79780158,
+      "dequeue_ts":1306433477839,
+      "nack_item_counter":0,
+      "nack_size_counter":0,
+      "nack_ts":1306433476813,
+      "queue_size":163688,
+      "queue_items":148,
+      "swapped_in_size":0,
+      "swapped_in_items":0,
+      "swapping_in_size":0,
+      "swapping_out_size":0,
+      "swapped_in_size_max":32768,
+      "swap_out_item_counter":148,
+      "swap_out_size_counter":163688,
+      "swap_in_item_counter":0,
+      "swap_in_size_counter":0
+   },
+   "entries":[
+   ],
+   "producers":[
+   ],
+   "consumers":[
+   ]
 }
 {pygmentize}
 
@@ -319,16 +305,15 @@ of the topic.
 Example:
 
     $ curl -u "admin:password" \
-    http://localhost:61680/broker/virtual-hosts/localhost/topics/1.json
+    http://localhost:61680/broker/virtual-hosts/localhost/topics/item.prices.json
 
 Results in a [Topic Status](./api/apollo-dto/org/apache/activemq/apollo/dto/TopicStatusDTO.html):
 
 {pygmentize:: js}
 {
-  "id":1,
-  "name":"load-0",
+  "id":"item.prices",
   "config":{
-    "name":null,
+    "id":null,
     "slow_consumer_policy":null,
     "acl":null
   },
@@ -347,6 +332,7 @@ Results in a [Topic Status](./api/apollo
     }
   ],
   "durable_subscriptions":[
+     "mysubname"
   ]
 }
 {pygmentize}

Modified: activemq/activemq-apollo/trunk/apollo-website/src/documentation/user-manual.md
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-website/src/documentation/user-manual.md?rev=1128015&r1=1128014&r2=1128015&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-website/src/documentation/user-manual.md (original)
+++ activemq/activemq-apollo/trunk/apollo-website/src/documentation/user-manual.md Thu May 26 18:19:40 2011
@@ -198,7 +198,7 @@ When a new queue is first created in the
 determined by the first `queue` element which matches the queue being
 created. The attributes matched against are:
 
-* `name` : The name of the queue, you can use wild cards to match
+* `id` : The name of the queue, you can use wild cards to match
   multiple.
 
 A `queue` element may be configured with the following attributes:
@@ -234,7 +234,7 @@ When a new topic is first created in the
 determined by the first `topic` element which matches the topic being
 created. The attributes matched against are:
 
-* `name` : The name of the topic, you can use wild cards to match
+* `id` : The name of the topic, you can use wild cards to match
   against multiple
 
 A `topic` element may be configured with the following attributes:
@@ -254,11 +254,9 @@ configuration will be determined by the 
 which matches the durable subscription being created. The attributes matched
 against are:
 
-* `name` : The name of the topic, you can use wild cards to match
+* `id` : The name of the topic, you can use wild cards to match
   multiple.
 
-* `client_id` This specify which client id this configuration should match.
-
 * `subscription_id` : This specify which subscription id this configuration 
   should match.