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/31 20:58:25 UTC

svn commit: r1129849 - in /activemq/activemq-apollo/trunk: apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/ apollo-distro/src/main/descriptors/ apollo-dto/src/main/java/org/apache/activemq/apollo/dto/ apollo-stomp/src/main/scala/org/apac...

Author: chirino
Date: Tue May 31 18:58:24 2011
New Revision: 1129849

URL: http://svn.apache.org/viewvc?rev=1129849&view=rev
Log:
Fixes https://issues.apache.org/jira/browse/APLO-33 : Support deleting queues from the management interface. Also show the status/uptime of the destinations.

Modified:
    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/Topic.scala
    activemq/activemq-apollo/trunk/apollo-distro/src/main/descriptors/common-bin.xml
    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/TopicStatusDTO.java
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/VirtualHostStatusDTO.java
    activemq/activemq-apollo/trunk/apollo-stomp/src/main/scala/org/apache/activemq/apollo/stomp/StompProtocolHandler.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

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=1129849&r1=1129848&r2=1129849&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 Tue May 31 18:58:24 2011
@@ -894,17 +894,24 @@ class LocalRouter(val virtual_host:Virtu
         return Some("Not authorized to destroy")
       }
     }
+    _destroy_queue(queue)
+    None
+  }
+
 
-    queue.binding.unbind(this, queue)
-    queues_by_binding.remove(queue.binding)
-    queues_by_id.remove(queue.id)
-    queue.stop
-    if( queue.tune_persistent ) {
-      queue.dispatch_queue {
-        virtual_host.store.remove_queue(queue.store_id){x=> Unit}
+  def _destroy_queue(queue: Queue) {
+    queue.stop(^{
+      queue.binding.unbind(this, queue)
+      queues_by_binding.remove(queue.binding)
+      queues_by_id.remove(queue.id)
+      if (queue.tune_persistent) {
+        queue.dispatch_queue {
+          virtual_host.store.remove_queue(queue.store_id) {
+            x => Unit
+          }
+        }
       }
-    }
-    None
+    })
   }
 
-}
+}
\ No newline at end of file

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=1129849&r1=1129848&r2=1129849&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 May 31 18:58:24 2011
@@ -53,6 +53,7 @@ class Queue(val router: LocalRouter, val
 
   def virtual_host = router.virtual_host
 
+  var producers = ListBuffer[BindableDeliveryProducer]()
   var inbound_sessions = Set[DeliverySession]()
   var all_subscriptions = Map[DeliveryConsumer, Subscription]()
   var exclusive_subscriptions = ListBuffer[Subscription]()
@@ -218,7 +219,14 @@ class Queue(val router: LocalRouter, val
   }
 
   protected def _stop(on_completed: Runnable) = {
-    // TODO: perhaps we should remove all the entries
+    // Disconnect the producers..
+    producers.foreach { producer =>
+      disconnect(producer)
+    }
+    // Close all the subscriptions..
+    all_subscriptions.values.toArray.foreach { sub:Subscription =>
+      sub.close()
+    }
     on_completed.run
   }
 
@@ -454,6 +462,8 @@ class Queue(val router: LocalRouter, val
   def connect(p: DeliveryProducer) = new DeliverySession {
     retain
 
+    override def toString = Queue.this.toString
+
     override def consumer = Queue.this
 
     override def producer = p
@@ -560,12 +570,24 @@ class Queue(val router: LocalRouter, val
       val topic = router.topic_domain.get_or_create_destination(binding.destination, binding.binding_dto, null).success
       topic.connect(destination, producer)
     } else {
+      dispatch_queue {
+        producers += producer
+      }
       producer.bind(this::Nil)
     }
   }
 
   def disconnect (producer:BindableDeliveryProducer) = {
-    producer.unbind(this::Nil)
+    import OptionSupport._
+    if( config.unified.getOrElse(false) ) {
+      val topic = router.topic_domain.get_or_create_destination(binding.destination, binding.binding_dto, null).success
+      topic.disconnect(producer)
+    } else {
+      dispatch_queue {
+        producers -= producer
+      }
+      producer.unbind(this::Nil)
+    }
   }
 
   override def connection:Option[BrokerConnection] = None

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=1129849&r1=1129848&r2=1129849&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 May 31 18:58:24 2011
@@ -36,6 +36,7 @@ class Topic(val router:LocalRouter, val 
   var consumers = ListBuffer[DeliveryConsumer]()
   var durable_subscriptions = ListBuffer[Queue]()
   var consumer_queues = HashMap[DeliveryConsumer, Queue]()
+  val created_at = System.currentTimeMillis()
 
   import OptionSupport._
 

Modified: activemq/activemq-apollo/trunk/apollo-distro/src/main/descriptors/common-bin.xml
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-distro/src/main/descriptors/common-bin.xml?rev=1129849&r1=1129848&r2=1129849&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-distro/src/main/descriptors/common-bin.xml (original)
+++ activemq/activemq-apollo/trunk/apollo-distro/src/main/descriptors/common-bin.xml Tue May 31 18:58:24 2011
@@ -50,6 +50,8 @@
         <include>org.codehaus.jackson:jackson-jaxrs</include>
         <include>org.fusesource.scalate:scalate-core</include>
         <include>org.fusesource.scalate:scalate-util</include>
+        <include>net.sf.josql:josql</include>
+        <include>net.sf.josql:gentlyweb-utils</include>
         
         <!-- the logging apis used -->
         <include>org.slf4j:slf4j-api</include>

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=1129849&r1=1129848&r2=1129849&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 Tue May 31 18:58:24 2011
@@ -28,7 +28,7 @@ import java.util.List;
  */
 @XmlRootElement(name="queue_status")
 @XmlAccessorType(XmlAccessType.FIELD)
-public class QueueStatusDTO extends StringIdDTO {
+public class QueueStatusDTO extends ServiceStatusDTO {
 
     @XmlElement
     public QueueDTO config;

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=1129849&r1=1129848&r2=1129849&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 May 31 18:58:24 2011
@@ -28,7 +28,7 @@ import java.util.List;
  */
 @XmlRootElement(name="topic_status")
 @XmlAccessorType(XmlAccessType.FIELD)
-public class TopicStatusDTO extends StringIdDTO {
+public class TopicStatusDTO extends ServiceStatusDTO {
 
     @XmlElement
     public TopicDTO config;

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=1129849&r1=1129848&r2=1129849&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 Tue May 31 18:58:24 2011
@@ -33,18 +33,24 @@ import java.util.List;
 public class VirtualHostStatusDTO extends ServiceStatusDTO {
 
     /**
-     * Ids of all the topics running on the broker
+     * Ids of all the topics that exist on the broker
      */
-    @XmlElement(name="topics")
+    @XmlElement(name="topic")
     public List<String> topics = new ArrayList<String>();
 
     /**
-     * Ids of all the queues running on the broker
+     * Ids of all the queues that exist on the broker
      */
-    @XmlElement(name="queues")
+    @XmlElement(name="queue")
     public List<String> queues = new ArrayList<String>();
 
     /**
+     * Ids of all the durable subscriptions that exist on the broker
+     */
+    @XmlElement(name="dsub")
+    public List<String> dsubs = new ArrayList<String>();
+
+    /**
      * Is the virtual host using a store.
      */
     @XmlElement(name="store")

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=1129849&r1=1129848&r2=1129849&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 Tue May 31 18:58:24 2011
@@ -38,6 +38,7 @@ import org.apache.activemq.apollo.transp
 import java.security.cert.X509Certificate
 import collection.mutable.{ListBuffer, HashMap}
 import java.io.IOException
+import javax.management.remote.rmi._RMIConnection_Stub
 
 
 case class RichBuffer(self:Buffer) extends Proxy {
@@ -274,6 +275,8 @@ class StompProtocolHandler extends Proto
 
       retain
 
+      override def toString = "connection to "+StompProtocolHandler.this.connection.transport.getRemoteAddress
+
       def producer = p
       def consumer = StompConsumer.this
       var closed = false

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=1129849&r1=1129848&r2=1129849&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 May 31 18:58:24 2011
@@ -32,6 +32,7 @@ import javax.ws.rs._
 import core.Response
 import Response.Status._
 import org.josql.expressions.SelectItemExpression
+import org.apache.activemq.apollo.util.BaseService._
 
 /**
  * <p>
@@ -169,12 +170,14 @@ case class BrokerResource() extends Reso
 
         val router:LocalRouter = host
 
+        router.queue_domain.destinations.foreach { node=>
+          result.queues.add(node.id)
+        }
         router.topic_domain.destinations.foreach { node=>
           result.topics.add(node.id)
         }
-
-        router.queue_domain.destinations.foreach { node=>
-          result.queues.add(node.id)
+        router.topic_domain.durable_subscriptions_by_id.keys.foreach { id=>
+          result.dsubs.add(id)
         }
 
         result
@@ -294,10 +297,31 @@ case class BrokerResource() extends Reso
     with_virtual_host(id) { host =>
       val router: LocalRouter = host
       val node = router.queue_domain.destination_by_id.get(name).getOrElse(result(NOT_FOUND))
-      status(node, entries)
+      sync(node) {
+        status(node, entries)
+      }
     }
   }
 
+  @DELETE @Path("virtual-hosts/{id}/queues/{name:.*}")
+  @Produces(Array("application/json", "application/xml","text/xml"))
+  def queue_delete(@PathParam("id") id : String, @PathParam("name") name : String):Unit = {
+    with_virtual_host(id) { host =>
+      val router: LocalRouter = host
+      val node = router.queue_domain.destination_by_id.get(name).getOrElse(result(NOT_FOUND))
+      admining(node) {
+        router._destroy_queue(node)
+      }
+    }
+  }
+
+  @POST @Path("virtual-hosts/{id}/queues/{name:.*}/action/delete")
+  @Produces(Array("text/html;qs=5"))
+  def post_queue_delete_and_redirect(@PathParam("id") id : String, @PathParam("name") name : String):Unit = {
+    queue_delete(id, name)
+    result(strip_resolve("../../.."))
+  }
+
   @GET @Path("virtual-hosts/{id}/dsubs")
   @Produces(Array("application/json"))
   def durable_subscriptions(@PathParam("id") id : String, @QueryParam("f") f:java.util.List[String],
@@ -319,10 +343,33 @@ case class BrokerResource() extends Reso
     with_virtual_host(id) { host =>
       val router:LocalRouter = host
       val node = router.topic_domain.durable_subscriptions_by_id.get(name).getOrElse(result(NOT_FOUND))
-      status(node, entries)
+      sync(node) {
+        status(node, entries)
+      }
     }
   }
 
+
+  @DELETE @Path("virtual-hosts/{id}/dsubs/{name:.*}")
+  @Produces(Array("application/json", "application/xml","text/xml"))
+  def dsub_delete(@PathParam("id") id : String, @PathParam("name") name : String):Unit = {
+    with_virtual_host(id) { host =>
+      val router: LocalRouter = host
+      val node = router.topic_domain.durable_subscriptions_by_id.get(name).getOrElse(result(NOT_FOUND))
+      admining(node) {
+        router._destroy_queue(node)
+      }
+    }
+  }
+
+  @POST @Path("virtual-hosts/{id}/dsubs/{name:.*}/action/delete")
+  @Produces(Array("text/html;qs=5"))
+  def post_dsub_delete_and_redirect(@PathParam("id") id : String, @PathParam("name") name : String):Unit = {
+    dsub_delete(id, name)
+    result(strip_resolve("../../.."))
+  }
+
+
   private def decode_path(name:String) = {
     try {
       LocalRouter.destination_parser.decode_path(name)
@@ -335,6 +382,8 @@ case class BrokerResource() extends Reso
     monitoring(node) {
       val rc = new TopicStatusDTO
       rc.id = node.id
+      rc.state = "STARTED"
+      rc.state_since = node.created_at
       rc.config = node.config
 
       node.durable_subscriptions.foreach {
@@ -365,6 +414,8 @@ case class BrokerResource() extends Reso
   def status(q:Queue, entries:Boolean=false) = monitoring(q) {
     val rc = new QueueStatusDTO
     rc.id = q.id
+    rc.state = q.service_state.toString
+    rc.state_since = q.service_state.since
     rc.binding = q.binding.binding_dto
     rc.config = q.config
     rc.metrics = get_queue_metrics(q)
@@ -514,9 +565,9 @@ case class BrokerResource() extends Reso
     }
   }
 
-  @POST @Path("connections/{id}/action/shutdown")
+  @DELETE @Path("connections/{id}")
   @Produces(Array("application/json", "application/xml","text/xml"))
-  def post_connection_shutdown(@PathParam("id") id : Long):Unit = {
+  def connection_delete(@PathParam("id") id : Long):Unit = {
     with_connection(id){ connection=>
       admining(connection.connector.broker) {
         connection.stop
@@ -525,10 +576,10 @@ case class BrokerResource() extends Reso
   }
 
 
-  @POST @Path("connections/{id}/action/shutdown")
+  @POST @Path("connections/{id}/action/delete")
   @Produces(Array("text/html;qs=5"))
-  def post_connection_shutdown_and_redirect(@PathParam("id") id : Long):Unit = {
-    post_connection_shutdown(id)
+  def post_connection_delete_and_redirect(@PathParam("id") id : Long):Unit = {
+    connection_delete(id)
     result(strip_resolve("../../.."))
   }
 

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=1129849&r1=1129848&r2=1129849&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 May 31 18:58:24 2011
@@ -36,6 +36,11 @@
   - case _ =>
     h1 Temporary Queue
 
+p state: #{state} #{ uptime(state_since) } ago
+- if( state == "STARTED" )
+  form(method="post" action={path("action/delete")})
+    input(type="submit" value="delete")
+
 h2 Current Size
 
 p queue size: #{metrics.queue_items} messages #{memory(metrics.queue_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=1129849&r1=1129848&r2=1129849&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 May 31 18:58:24 2011
@@ -20,8 +20,9 @@
 .breadcumbs
   a(href={strip_resolve("..")}) Back
 
-h1 Destination: #{id}
+h1 Topic: #{id}
 
+p state: #{state} #{ uptime(state_since) } ago
 h3 Topic Domain
 
 h4 Publishers

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=1129849&r1=1129848&r2=1129849&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 Tue May 31 18:58:24 2011
@@ -39,3 +39,9 @@ ul
   - for( x <- topics )
     li
       a(href={ path("topics/"+x) }) #{x}
+
+h2 Durable Subscriptions
+ul
+  - for( x <- dsubs )
+    li
+      a(href={ path("dsubs/"+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=1129849&r1=1129848&r2=1129849&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 Tue May 31 18:58:24 2011
@@ -245,14 +245,18 @@ Results in a [Connection Status](./api/a
 }
 {pygmentize}
 
-To shutdown a connection send a POST to:
+To shutdown a connection, send a DELETE  to
 
-    /broker/connections/:id/action/shutdown
+    /broker/connections/:id
+    
+or send a POST to:
+
+    /broker/connections/:id/action/delete
 
 Example:
 
-    curl -X POST -u "admin:password" \
-    http://localhost:61680/broker/connections/5/action/shutdown.json
+    curl -X DELETE -u "admin:password" \
+    http://localhost:61680/broker/connections/5.json
 
 
 ### Virtual Host Management
@@ -406,6 +410,19 @@ Results in a [Queue Status](./api/apollo
 }
 {pygmentize}
 
+To delete a queue, send a DELETE  to
+
+    /broker/virtual-hosts/:name/queues/:qid
+    
+or send a POST to:
+
+    /broker/virtual-hosts/:name/queues/:qid/action/delete
+
+Example:
+
+    curl -X DELETE -u "admin:password" \
+    http://localhost:61680/broker/virtual-hosts/localhost/queues/orders.req.json
+
 
 #### Topic Management
 
@@ -476,6 +493,19 @@ Example:
 
 Results in a [Queue Status](./api/apollo-dto/org/apache/activemq/apollo/dto/QueueStatusDTO.html):
 
+To delete a durable subscription, send a DELETE  to
+
+    /broker/virtual-hosts/:name/dsubs/:sub
+    
+or send a POST to:
+
+    /broker/virtual-hosts/:name/dsubs/:sub/action/delete
+
+Example:
+
+    curl -X DELETE -u "admin:password" \
+    http://localhost:61680/broker/virtual-hosts/localhost/dsubs/mysub.json
+
 
 ### Getting the Broker's Configuration