You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@activemq.apache.org by cl...@apache.org on 2016/06/06 20:01:26 UTC

[3/3] activemq-artemis git commit: ARTEMIS-322 auto-create/delete JMS topic

ARTEMIS-322 auto-create/delete JMS topic

Implements a new feature for the broker whereby it may automatically create and
delete JMS topics which are not explicitly defined through the management API
or file-based configuration. A JMS topic is created in response to a sent
message or connected subscriber. The topic may subsequently be deleted when it
no longer has any subscribers. Auto-creation and auto-deletion can both be
turned on/off via address-setting.


Project: http://git-wip-us.apache.org/repos/asf/activemq-artemis/repo
Commit: http://git-wip-us.apache.org/repos/asf/activemq-artemis/commit/e53649a6
Tree: http://git-wip-us.apache.org/repos/asf/activemq-artemis/tree/e53649a6
Diff: http://git-wip-us.apache.org/repos/asf/activemq-artemis/diff/e53649a6

Branch: refs/heads/master
Commit: e53649a6b9052b4b3a8387274b59c1e5585c9500
Parents: 25316e4
Author: jbertram <jb...@apache.org>
Authored: Wed Jun 1 12:50:23 2016 -0500
Committer: Clebert Suconic <cl...@apache.org>
Committed: Mon Jun 6 16:00:57 2016 -0400

----------------------------------------------------------------------
 .../artemis/api/core/client/ClientSession.java  |   6 +
 .../core/management/ActiveMQServerControl.java  |   4 +-
 .../core/management/AddressSettingsInfo.java    |  20 +-
 .../core/client/impl/AddressQueryImpl.java      |  11 +-
 .../core/impl/ActiveMQSessionContext.java       |   8 +-
 .../core/protocol/core/impl/ChannelImpl.java    |   2 +
 .../core/protocol/core/impl/PacketDecoder.java  |   6 +
 .../core/protocol/core/impl/PacketImpl.java     |   2 +
 .../SessionBindingQueryResponseMessage.java     |   4 +-
 .../SessionBindingQueryResponseMessage_V2.java  |   6 +-
 .../SessionBindingQueryResponseMessage_V3.java  |  95 +++++++
 .../main/resources/activemq-version.properties  |   2 +-
 .../jms/client/ActiveMQMessageProducer.java     |   2 +-
 .../artemis/jms/client/ActiveMQSession.java     |   6 +-
 .../artemis/jms/server/JMSServerManager.java    |  11 +
 .../jms/server/impl/JMSServerManagerImpl.java   | 100 ++++++-
 .../client/HornetQClientSessionContext.java     |   2 +-
 .../protocol/openwire/OpenWireConnection.java   |   5 +-
 .../core/protocol/openwire/amq/AMQConsumer.java |   9 +-
 .../core/protocol/openwire/amq/AMQSession.java  |   2 +-
 .../protocol/openwire/util/OpenWireUtil.java    |  21 +-
 .../core/protocol/stomp/StompConnection.java    |  29 +--
 .../deployers/impl/FileConfigurationParser.java |  10 +
 .../impl/ActiveMQServerControlImpl.java         |   8 +-
 .../core/ServerSessionPacketHandler.java        |   6 +-
 .../artemis/core/server/ActiveMQServer.java     |  32 ++-
 .../artemis/core/server/BindingQueryResult.java |  11 +-
 .../core/server/PostQueueCreationCallback.java  |  27 ++
 .../core/server/PostQueueDeletionCallback.java  |  27 ++
 .../artemis/core/server/QueueDeleter.java       |   2 +-
 .../core/server/impl/ActiveMQServerImpl.java    |  59 ++++-
 .../impl/AutoCreatedQueueManagerImpl.java       |  31 +--
 .../server/impl/PostOfficeJournalLoader.java    |   2 +-
 .../core/server/impl/ServerSessionImpl.java     |   6 +-
 .../core/settings/impl/AddressSettings.java     |  64 ++++-
 .../resources/schema/artemis-configuration.xsd  |  17 ++
 .../core/config/impl/FileConfigurationTest.java |   4 +
 .../core/settings/AddressSettingsTest.java      |   2 +
 .../resources/ConfigurationTest-full-config.xml |   4 +
 docs/user-manual/en/queue-attributes.md         |  17 +-
 pom.xml                                         |   4 +-
 .../client/AutoCreateJmsDestinationTest.java    | 258 +++++++++++++++++++
 .../client/AutoCreateJmsQueueTest.java          | 204 ---------------
 .../client/AutoDeleteJmsDestinationTest.java    | 199 ++++++++++++++
 .../client/AutoDeleteJmsQueueTest.java          | 127 ---------
 .../integration/client/PagingOrderTest.java     |   4 +-
 .../jms/jms2client/NonExistentQueueTest.java    |   2 +
 .../management/ActiveMQServerControlTest.java   |  14 +-
 .../ActiveMQServerControlUsingCoreTest.java     |   6 +-
 .../management/ManagementWithStompTest.java     |   8 +-
 .../openwire/amq/JmsTopicRequestReplyTest.java  |  18 +-
 .../tests/integration/stomp/StompTest.java      |  37 +++
 .../artemis/jms/tests/MessageProducerTest.java  |   2 +
 .../activemq/artemis/jms/tests/SessionTest.java |   2 +
 54 files changed, 1126 insertions(+), 441 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/e53649a6/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/client/ClientSession.java
----------------------------------------------------------------------
diff --git a/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/client/ClientSession.java b/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/client/ClientSession.java
index 40db6cf..dd1c45c 100644
--- a/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/client/ClientSession.java
+++ b/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/client/ClientSession.java
@@ -64,6 +64,12 @@ public interface ClientSession extends XAResource, AutoCloseable {
        * queue, <code>false</code> else.
        */
       boolean isAutoCreateJmsQueues();
+
+      /**
+       * Returns <code>true</code> if auto-creation for this address is enabled and if the address queried is for a JMS
+       * topic, <code>false</code> else.
+       */
+      boolean isAutoCreateJmsTopics();
    }
 
    /**

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/e53649a6/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/ActiveMQServerControl.java
----------------------------------------------------------------------
diff --git a/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/ActiveMQServerControl.java b/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/ActiveMQServerControl.java
index 7784992..677bfb6 100644
--- a/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/ActiveMQServerControl.java
+++ b/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/ActiveMQServerControl.java
@@ -656,7 +656,9 @@ public interface ActiveMQServerControl {
                            @Parameter(desc = "how often (in seconds) to check for slow consumers", name = "slowConsumerCheckPeriod") long slowConsumerCheckPeriod,
                            @Parameter(desc = "the policy to use when a slow consumer is detected", name = "slowConsumerPolicy") String slowConsumerPolicy,
                            @Parameter(desc = "allow queues to be created automatically", name = "autoCreateJmsQueues") boolean autoCreateJmsQueues,
-                           @Parameter(desc = "allow auto-created queues to be deleted automatically", name = "autoDeleteJmsQueues") boolean autoDeleteJmsQueues) throws Exception;
+                           @Parameter(desc = "allow auto-created queues to be deleted automatically", name = "autoDeleteJmsQueues") boolean autoDeleteJmsQueues,
+                           @Parameter(desc = "allow topics to be created automatically", name = "autoCreateJmsTopics")  boolean autoCreateJmsTopics,
+                           @Parameter(desc = "allow auto-created topics to be deleted automatically", name = "autoDeleteJmsTopics") boolean autoDeleteJmsTopics) throws Exception;
 
    void removeAddressSettings(String addressMatch) throws Exception;
 

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/e53649a6/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/AddressSettingsInfo.java
----------------------------------------------------------------------
diff --git a/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/AddressSettingsInfo.java b/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/AddressSettingsInfo.java
index 63b80cb..d4f5232 100644
--- a/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/AddressSettingsInfo.java
+++ b/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/management/AddressSettingsInfo.java
@@ -57,11 +57,15 @@ public final class AddressSettingsInfo {
 
    private final boolean autoDeleteJmsQueues;
 
+   private final boolean autoCreateJmsTopics;
+
+   private final boolean autoDeleteJmsTopics;
+
    // Static --------------------------------------------------------
 
    public static AddressSettingsInfo from(final String jsonString) throws Exception {
       JSONObject object = new JSONObject(jsonString);
-      return new AddressSettingsInfo(object.getString("addressFullMessagePolicy"), object.getLong("maxSizeBytes"), object.getInt("pageSizeBytes"), object.getInt("pageCacheMaxSize"), object.getInt("maxDeliveryAttempts"), object.getLong("redeliveryDelay"), object.getDouble("redeliveryMultiplier"), object.getLong("maxRedeliveryDelay"), object.getString("DLA"), object.getString("expiryAddress"), object.getBoolean("lastValueQueue"), object.getLong("redistributionDelay"), object.getBoolean("sendToDLAOnNoRoute"), object.getLong("slowConsumerThreshold"), object.getLong("slowConsumerCheckPeriod"), object.getString("slowConsumerPolicy"), object.getBoolean("autoCreateJmsQueues"), object.getBoolean("autoDeleteJmsQueues"));
+      return new AddressSettingsInfo(object.getString("addressFullMessagePolicy"), object.getLong("maxSizeBytes"), object.getInt("pageSizeBytes"), object.getInt("pageCacheMaxSize"), object.getInt("maxDeliveryAttempts"), object.getLong("redeliveryDelay"), object.getDouble("redeliveryMultiplier"), object.getLong("maxRedeliveryDelay"), object.getString("DLA"), object.getString("expiryAddress"), object.getBoolean("lastValueQueue"), object.getLong("redistributionDelay"), object.getBoolean("sendToDLAOnNoRoute"), object.getLong("slowConsumerThreshold"), object.getLong("slowConsumerCheckPeriod"), object.getString("slowConsumerPolicy"), object.getBoolean("autoCreateJmsQueues"), object.getBoolean("autoDeleteJmsQueues"), object.getBoolean("autoCreateJmsTopics"), object.getBoolean("autoDeleteJmsTopics"));
    }
 
    // Constructors --------------------------------------------------
@@ -83,7 +87,9 @@ public final class AddressSettingsInfo {
                               long slowConsumerCheckPeriod,
                               String slowConsumerPolicy,
                               boolean autoCreateJmsQueues,
-                              boolean autoDeleteJmsQueues) {
+                              boolean autoDeleteJmsQueues,
+                              boolean autoCreateJmsTopics,
+                              boolean autoDeleteJmsTopics) {
       this.addressFullMessagePolicy = addressFullMessagePolicy;
       this.maxSizeBytes = maxSizeBytes;
       this.pageSizeBytes = pageSizeBytes;
@@ -102,6 +108,8 @@ public final class AddressSettingsInfo {
       this.slowConsumerPolicy = slowConsumerPolicy;
       this.autoCreateJmsQueues = autoCreateJmsQueues;
       this.autoDeleteJmsQueues = autoDeleteJmsQueues;
+      this.autoCreateJmsTopics = autoCreateJmsTopics;
+      this.autoDeleteJmsTopics = autoDeleteJmsTopics;
    }
 
    // Public --------------------------------------------------------
@@ -181,5 +189,13 @@ public final class AddressSettingsInfo {
    public boolean isAutoDeleteJmsQueues() {
       return autoDeleteJmsQueues;
    }
+
+   public boolean isAutoCreateJmsTopics() {
+      return autoCreateJmsTopics;
+   }
+
+   public boolean isAutoDeleteJmsTopics() {
+      return autoDeleteJmsTopics;
+   }
 }
 

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/e53649a6/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/client/impl/AddressQueryImpl.java
----------------------------------------------------------------------
diff --git a/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/client/impl/AddressQueryImpl.java b/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/client/impl/AddressQueryImpl.java
index 3a1f62b..9d38dbb 100644
--- a/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/client/impl/AddressQueryImpl.java
+++ b/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/client/impl/AddressQueryImpl.java
@@ -30,12 +30,16 @@ public class AddressQueryImpl implements ClientSession.AddressQuery {
 
    private final boolean autoCreateJmsQueues;
 
+   private final boolean autoCreateJmsTopics;
+
    public AddressQueryImpl(final boolean exists,
                            final List<SimpleString> queueNames,
-                           final boolean autoCreateJmsQueues) {
+                           final boolean autoCreateJmsQueues,
+                           final boolean autoCreateJmsTopics) {
       this.exists = exists;
       this.queueNames = new ArrayList<>(queueNames);
       this.autoCreateJmsQueues = autoCreateJmsQueues;
+      this.autoCreateJmsTopics = autoCreateJmsTopics;
    }
 
    @Override
@@ -52,4 +56,9 @@ public class AddressQueryImpl implements ClientSession.AddressQuery {
    public boolean isAutoCreateJmsQueues() {
       return autoCreateJmsQueues;
    }
+
+   @Override
+   public boolean isAutoCreateJmsTopics() {
+      return autoCreateJmsTopics;
+   }
 }

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/e53649a6/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/ActiveMQSessionContext.java
----------------------------------------------------------------------
diff --git a/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/ActiveMQSessionContext.java b/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/ActiveMQSessionContext.java
index 794df52..9f0edce 100644
--- a/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/ActiveMQSessionContext.java
+++ b/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/ActiveMQSessionContext.java
@@ -49,18 +49,18 @@ import org.apache.activemq.artemis.core.protocol.core.ChannelHandler;
 import org.apache.activemq.artemis.core.protocol.core.CommandConfirmationHandler;
 import org.apache.activemq.artemis.core.protocol.core.CoreRemotingConnection;
 import org.apache.activemq.artemis.core.protocol.core.Packet;
+import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.ActiveMQExceptionMessage;
 import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.CreateQueueMessage;
 import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.CreateSessionMessage;
 import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.CreateSharedQueueMessage;
 import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.DisconnectConsumerMessage;
-import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.ActiveMQExceptionMessage;
 import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.ReattachSessionMessage;
 import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.ReattachSessionResponseMessage;
 import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.RollbackMessage;
 import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SessionAcknowledgeMessage;
 import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SessionAddMetaDataMessageV2;
 import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SessionBindingQueryMessage;
-import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SessionBindingQueryResponseMessage_V2;
+import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SessionBindingQueryResponseMessage_V3;
 import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SessionCloseMessage;
 import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SessionConsumerCloseMessage;
 import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SessionConsumerFlowCreditMessage;
@@ -283,9 +283,9 @@ public class ActiveMQSessionContext extends SessionContext {
 
    @Override
    public ClientSession.AddressQuery addressQuery(final SimpleString address) throws ActiveMQException {
-      SessionBindingQueryResponseMessage_V2 response = (SessionBindingQueryResponseMessage_V2) sessionChannel.sendBlocking(new SessionBindingQueryMessage(address), PacketImpl.SESS_BINDINGQUERY_RESP_V2);
+      SessionBindingQueryResponseMessage_V3 response = (SessionBindingQueryResponseMessage_V3) sessionChannel.sendBlocking(new SessionBindingQueryMessage(address), PacketImpl.SESS_BINDINGQUERY_RESP_V3);
 
-      return new AddressQueryImpl(response.isExists(), response.getQueueNames(), response.isAutoCreateJmsQueues());
+      return new AddressQueryImpl(response.isExists(), response.getQueueNames(), response.isAutoCreateJmsQueues(), response.isAutoCreateJmsTopics());
    }
 
    @Override

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/e53649a6/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/ChannelImpl.java
----------------------------------------------------------------------
diff --git a/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/ChannelImpl.java b/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/ChannelImpl.java
index b4ac75d..89fa4f1 100644
--- a/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/ChannelImpl.java
+++ b/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/ChannelImpl.java
@@ -164,6 +164,8 @@ public final class ChannelImpl implements Channel {
             return version >= 126;
          case PacketImpl.SESS_BINDINGQUERY_RESP_V2:
             return version >= 126;
+         case PacketImpl.SESS_BINDINGQUERY_RESP_V3:
+            return version >= 127;
          default:
             return true;
       }

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/e53649a6/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/PacketDecoder.java
----------------------------------------------------------------------
diff --git a/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/PacketDecoder.java b/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/PacketDecoder.java
index 804a26a..e04f3d0 100644
--- a/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/PacketDecoder.java
+++ b/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/PacketDecoder.java
@@ -40,6 +40,7 @@ import static org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl.SES
 import static org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl.SESS_BINDINGQUERY;
 import static org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl.SESS_BINDINGQUERY_RESP;
 import static org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl.SESS_BINDINGQUERY_RESP_V2;
+import static org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl.SESS_BINDINGQUERY_RESP_V3;
 import static org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl.SESS_CLOSE;
 import static org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl.SESS_COMMIT;
 import static org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl.SESS_CONSUMER_CLOSE;
@@ -110,6 +111,7 @@ import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SessionAdd
 import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SessionBindingQueryMessage;
 import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SessionBindingQueryResponseMessage;
 import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SessionBindingQueryResponseMessage_V2;
+import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SessionBindingQueryResponseMessage_V3;
 import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SessionCloseMessage;
 import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SessionCommitMessage;
 import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SessionConsumerCloseMessage;
@@ -257,6 +259,10 @@ public abstract class PacketDecoder implements Serializable {
             packet = new SessionBindingQueryResponseMessage_V2();
             break;
          }
+         case SESS_BINDINGQUERY_RESP_V3: {
+            packet = new SessionBindingQueryResponseMessage_V3();
+            break;
+         }
          case SESS_XA_START: {
             packet = new SessionXAStartMessage();
             break;

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/e53649a6/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/PacketImpl.java
----------------------------------------------------------------------
diff --git a/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/PacketImpl.java b/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/PacketImpl.java
index 9aa8d3c..ac1edf7 100644
--- a/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/PacketImpl.java
+++ b/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/PacketImpl.java
@@ -245,6 +245,8 @@ public class PacketImpl implements Packet {
 
    public static final byte REPLICATION_RESPONSE_V2 = -9;
 
+   public static final byte SESS_BINDINGQUERY_RESP_V3 = -10;
+
    // Static --------------------------------------------------------
 
    public PacketImpl(final byte type) {

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/e53649a6/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/wireformat/SessionBindingQueryResponseMessage.java
----------------------------------------------------------------------
diff --git a/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/wireformat/SessionBindingQueryResponseMessage.java b/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/wireformat/SessionBindingQueryResponseMessage.java
index 6197faf..410dac1 100644
--- a/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/wireformat/SessionBindingQueryResponseMessage.java
+++ b/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/wireformat/SessionBindingQueryResponseMessage.java
@@ -44,8 +44,8 @@ public class SessionBindingQueryResponseMessage extends PacketImpl {
       super(SESS_BINDINGQUERY_RESP);
    }
 
-   public SessionBindingQueryResponseMessage(byte v2) {
-      super(v2);
+   public SessionBindingQueryResponseMessage(byte v) {
+      super(v);
    }
 
    @Override

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/e53649a6/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/wireformat/SessionBindingQueryResponseMessage_V2.java
----------------------------------------------------------------------
diff --git a/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/wireformat/SessionBindingQueryResponseMessage_V2.java b/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/wireformat/SessionBindingQueryResponseMessage_V2.java
index 55da268..0a48743 100644
--- a/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/wireformat/SessionBindingQueryResponseMessage_V2.java
+++ b/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/wireformat/SessionBindingQueryResponseMessage_V2.java
@@ -23,7 +23,7 @@ import org.apache.activemq.artemis.api.core.SimpleString;
 
 public class SessionBindingQueryResponseMessage_V2 extends SessionBindingQueryResponseMessage {
 
-   private boolean autoCreateJmsQueues;
+   protected boolean autoCreateJmsQueues;
 
    public SessionBindingQueryResponseMessage_V2(final boolean exists,
                                                 final List<SimpleString> queueNames,
@@ -41,6 +41,10 @@ public class SessionBindingQueryResponseMessage_V2 extends SessionBindingQueryRe
       super(SESS_BINDINGQUERY_RESP_V2);
    }
 
+   public SessionBindingQueryResponseMessage_V2(byte v) {
+      super(v);
+   }
+
    public boolean isAutoCreateJmsQueues() {
       return autoCreateJmsQueues;
    }

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/e53649a6/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/wireformat/SessionBindingQueryResponseMessage_V3.java
----------------------------------------------------------------------
diff --git a/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/wireformat/SessionBindingQueryResponseMessage_V3.java b/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/wireformat/SessionBindingQueryResponseMessage_V3.java
new file mode 100644
index 0000000..3ed489f
--- /dev/null
+++ b/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/wireformat/SessionBindingQueryResponseMessage_V3.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.activemq.artemis.core.protocol.core.impl.wireformat;
+
+import java.util.List;
+
+import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
+import org.apache.activemq.artemis.api.core.SimpleString;
+
+public class SessionBindingQueryResponseMessage_V3 extends SessionBindingQueryResponseMessage_V2 {
+
+   private boolean autoCreateJmsTopics;
+
+   public SessionBindingQueryResponseMessage_V3(final boolean exists,
+                                                final List<SimpleString> queueNames,
+                                                final boolean autoCreateJmsQueues,
+                                                final boolean autoCreateJmsTopics) {
+      super(SESS_BINDINGQUERY_RESP_V3);
+
+      this.exists = exists;
+
+      this.queueNames = queueNames;
+
+      this.autoCreateJmsQueues = autoCreateJmsQueues;
+
+      this.autoCreateJmsTopics = autoCreateJmsTopics;
+   }
+
+   public SessionBindingQueryResponseMessage_V3() {
+      super(SESS_BINDINGQUERY_RESP_V3);
+   }
+
+   public boolean isAutoCreateJmsTopics() {
+      return autoCreateJmsTopics;
+   }
+
+   @Override
+   public void encodeRest(final ActiveMQBuffer buffer) {
+      super.encodeRest(buffer);
+      buffer.writeBoolean(autoCreateJmsTopics);
+   }
+
+   @Override
+   public void decodeRest(final ActiveMQBuffer buffer) {
+      super.decodeRest(buffer);
+      autoCreateJmsTopics = buffer.readBoolean();
+   }
+
+   @Override
+   public int hashCode() {
+      final int prime = 31;
+      int result = super.hashCode();
+      result = prime * result + (autoCreateJmsTopics ? 1231 : 1237);
+      return result;
+   }
+
+   @Override
+   public String toString() {
+      StringBuffer buff = new StringBuffer(getParentString());
+      buff.append(", exists=" + exists);
+      buff.append(", queueNames=" + queueNames);
+      buff.append(", autoCreateJmsQueues=" + autoCreateJmsQueues);
+      buff.append(", autoCreateJmsTopics=" + autoCreateJmsTopics);
+      buff.append("]");
+      return buff.toString();
+   }
+
+   @Override
+   public boolean equals(Object obj) {
+      if (this == obj)
+         return true;
+      if (!super.equals(obj))
+         return false;
+      if (!(obj instanceof SessionBindingQueryResponseMessage_V3))
+         return false;
+      SessionBindingQueryResponseMessage_V3 other = (SessionBindingQueryResponseMessage_V3) obj;
+      if (autoCreateJmsTopics != other.autoCreateJmsTopics)
+         return false;
+      return true;
+   }
+}

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/e53649a6/artemis-core-client/src/main/resources/activemq-version.properties
----------------------------------------------------------------------
diff --git a/artemis-core-client/src/main/resources/activemq-version.properties b/artemis-core-client/src/main/resources/activemq-version.properties
index 4144bf0..07685e1 100644
--- a/artemis-core-client/src/main/resources/activemq-version.properties
+++ b/artemis-core-client/src/main/resources/activemq-version.properties
@@ -20,4 +20,4 @@ activemq.version.minorVersion=${activemq.version.minorVersion}
 activemq.version.microVersion=${activemq.version.microVersion}
 activemq.version.incrementingVersion=${activemq.version.incrementingVersion}
 activemq.version.versionTag=${activemq.version.versionTag}
-activemq.version.compatibleVersionList=121,122,123,124,125,126
+activemq.version.compatibleVersionList=121,122,123,124,125,126,127

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/e53649a6/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQMessageProducer.java
----------------------------------------------------------------------
diff --git a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQMessageProducer.java b/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQMessageProducer.java
index 0f33c04..112b535 100644
--- a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQMessageProducer.java
+++ b/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQMessageProducer.java
@@ -405,7 +405,7 @@ public class ActiveMQMessageProducer implements MessageProducer, QueueSender, To
 
                // if it's autoCreateJMSQueue we will let the PostOffice.route to execute the creation at the server's side
                // as that's a more efficient path for such operation
-               if (!query.isExists() && !query.isAutoCreateJmsQueues()) {
+               if (!query.isExists() && ((address.toString().startsWith(ActiveMQDestination.JMS_QUEUE_ADDRESS_PREFIX) && !query.isAutoCreateJmsQueues()) || (address.toString().startsWith(ActiveMQDestination.JMS_TOPIC_ADDRESS_PREFIX) && !query.isAutoCreateJmsTopics()))) {
                   throw new InvalidDestinationException("Destination " + address + " does not exist");
                }
                else {

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/e53649a6/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQSession.java
----------------------------------------------------------------------
diff --git a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQSession.java b/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQSession.java
index da2dd80..ee81cf1 100644
--- a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQSession.java
+++ b/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQSession.java
@@ -299,7 +299,7 @@ public class ActiveMQSession implements QueueSession, TopicSession {
          if (jbd != null) {
             ClientSession.AddressQuery response = session.addressQuery(jbd.getSimpleAddress());
 
-            if (!response.isExists() && !response.isAutoCreateJmsQueues()) {
+            if (!response.isExists() && ((jbd.getAddress().startsWith(ActiveMQDestination.JMS_QUEUE_ADDRESS_PREFIX) && !response.isAutoCreateJmsQueues()) || (jbd.getAddress().startsWith(ActiveMQDestination.JMS_TOPIC_ADDRESS_PREFIX) && !response.isAutoCreateJmsTopics()))) {
                throw new InvalidDestinationException("Destination " + jbd.getName() + " does not exist");
             }
 
@@ -659,7 +659,7 @@ public class ActiveMQSession implements QueueSession, TopicSession {
          else {
             AddressQuery response = session.addressQuery(dest.getSimpleAddress());
 
-            if (!response.isExists()) {
+            if (!response.isExists() && !response.isAutoCreateJmsTopics()) {
                throw new InvalidDestinationException("Topic " + dest.getName() + " does not exist");
             }
 
@@ -1106,7 +1106,7 @@ public class ActiveMQSession implements QueueSession, TopicSession {
 
       AddressQuery query = session.addressQuery(topic.getSimpleAddress());
 
-      if (!query.isExists()) {
+      if (!query.isExists() && !query.isAutoCreateJmsTopics()) {
          return null;
       }
       else {

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/e53649a6/artemis-jms-server/src/main/java/org/apache/activemq/artemis/jms/server/JMSServerManager.java
----------------------------------------------------------------------
diff --git a/artemis-jms-server/src/main/java/org/apache/activemq/artemis/jms/server/JMSServerManager.java b/artemis-jms-server/src/main/java/org/apache/activemq/artemis/jms/server/JMSServerManager.java
index 879e7cb..69314b6 100644
--- a/artemis-jms-server/src/main/java/org/apache/activemq/artemis/jms/server/JMSServerManager.java
+++ b/artemis-jms-server/src/main/java/org/apache/activemq/artemis/jms/server/JMSServerManager.java
@@ -77,6 +77,17 @@ public interface JMSServerManager extends ActiveMQComponent {
    boolean createTopic(boolean storeConfig, String topicName, String... bindings) throws Exception;
 
    /**
+    *
+    * @param storeConfig
+    * @param topicName
+    * @param autoCreated
+    * @param bindings
+    * @return
+    * @throws Exception
+    */
+   boolean createTopic(boolean storeConfig, String topicName, boolean autoCreated, String... bindings) throws Exception;
+
+   /**
     * Remove the topic from the Binding Registry or BindingRegistry.
     * Calling this method does <em>not</em> destroy the destination.
     *

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/e53649a6/artemis-jms-server/src/main/java/org/apache/activemq/artemis/jms/server/impl/JMSServerManagerImpl.java
----------------------------------------------------------------------
diff --git a/artemis-jms-server/src/main/java/org/apache/activemq/artemis/jms/server/impl/JMSServerManagerImpl.java b/artemis-jms-server/src/main/java/org/apache/activemq/artemis/jms/server/impl/JMSServerManagerImpl.java
index 85084c1..336c9d2 100644
--- a/artemis-jms-server/src/main/java/org/apache/activemq/artemis/jms/server/impl/JMSServerManagerImpl.java
+++ b/artemis-jms-server/src/main/java/org/apache/activemq/artemis/jms/server/impl/JMSServerManagerImpl.java
@@ -21,6 +21,7 @@ import javax.transaction.xa.Xid;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
@@ -48,6 +49,8 @@ import org.apache.activemq.artemis.core.remoting.impl.netty.TransportConstants;
 import org.apache.activemq.artemis.core.security.Role;
 import org.apache.activemq.artemis.core.server.ActivateCallback;
 import org.apache.activemq.artemis.core.server.ActiveMQServer;
+import org.apache.activemq.artemis.core.server.PostQueueCreationCallback;
+import org.apache.activemq.artemis.core.server.PostQueueDeletionCallback;
 import org.apache.activemq.artemis.core.server.Queue;
 import org.apache.activemq.artemis.core.server.QueueCreator;
 import org.apache.activemq.artemis.core.server.QueueDeleter;
@@ -371,11 +374,15 @@ public class JMSServerManagerImpl implements JMSServerManager, ActivateCallback
          return;
       }
 
-      server.setJMSQueueCreator(new JMSQueueCreator());
+      server.setJMSQueueCreator(new JMSDestinationCreator());
 
       server.setJMSQueueDeleter(new JMSQueueDeleter());
 
       server.registerActivateCallback(this);
+
+      server.registerPostQueueCreationCallback(new JMSPostQueueCreationCallback());
+
+      server.registerPostQueueDeletionCallback(new JMSPostQueueDeletionCallback());
       /**
        * See this method's javadoc.
        * <p>
@@ -523,10 +530,19 @@ public class JMSServerManagerImpl implements JMSServerManager, ActivateCallback
       return true;
    }
 
+
    @Override
    public synchronized boolean createTopic(final boolean storeConfig,
                                            final String topicName,
                                            final String... bindings) throws Exception {
+      return createTopic(storeConfig, topicName, false, bindings);
+   }
+
+   @Override
+   public synchronized boolean createTopic(final boolean storeConfig,
+                                           final String topicName,
+                                           final boolean autoCreated,
+                                           final String... bindings) throws Exception {
       if (active && topics.get(topicName) != null) {
          return false;
       }
@@ -541,7 +557,7 @@ public class JMSServerManagerImpl implements JMSServerManager, ActivateCallback
          public void runException() throws Exception {
             checkBindings(bindings);
 
-            if (internalCreateTopic(topicName)) {
+            if (internalCreateTopic(topicName, autoCreated)) {
                ActiveMQDestination destination = topics.get(topicName);
 
                if (destination == null) {
@@ -1082,6 +1098,8 @@ public class JMSServerManagerImpl implements JMSServerManager, ActivateCallback
       }
    }
 
+
+
    /**
     * Performs the internal creation without activating any storage.
     * The storage load will call this method
@@ -1091,6 +1109,10 @@ public class JMSServerManagerImpl implements JMSServerManager, ActivateCallback
     * @throws Exception
     */
    private synchronized boolean internalCreateTopic(final String topicName) throws Exception {
+      return internalCreateTopic(topicName, false);
+   }
+
+   private synchronized boolean internalCreateTopic(final String topicName, final boolean autoCreated) throws Exception {
 
       if (topics.get(topicName) != null) {
          return false;
@@ -1101,7 +1123,7 @@ public class JMSServerManagerImpl implements JMSServerManager, ActivateCallback
          // checks when routing messages to a topic that
          // does not exist - otherwise we would not be able to distinguish from a non existent topic and one with no
          // subscriptions - core has no notion of a topic
-         server.deployQueue(SimpleString.toSimpleString(activeMQTopic.getAddress()), SimpleString.toSimpleString(activeMQTopic.getAddress()), SimpleString.toSimpleString(JMSServerManagerImpl.REJECT_FILTER), true, false);
+         server.deployQueue(SimpleString.toSimpleString(activeMQTopic.getAddress()), SimpleString.toSimpleString(activeMQTopic.getAddress()), SimpleString.toSimpleString(JMSServerManagerImpl.REJECT_FILTER), true, false, autoCreated);
 
          topics.put(topicName, activeMQTopic);
 
@@ -1619,13 +1641,19 @@ public class JMSServerManagerImpl implements JMSServerManager, ActivateCallback
       }
    }
 
-   class JMSQueueCreator implements QueueCreator {
+   /**
+    * This class is responsible for auto-creating the JMS (and underlying core) resources when a client sends a message
+    * to a non-existent JMS queue or topic
+    */
+   class JMSDestinationCreator implements QueueCreator {
       @Override
       public boolean create(SimpleString address) throws Exception {
          AddressSettings settings = server.getAddressSettingsRepository().getMatch(address.toString());
          if (address.toString().startsWith(ActiveMQDestination.JMS_QUEUE_ADDRESS_PREFIX) && settings.isAutoCreateJmsQueues()) {
-            JMSServerManagerImpl.this.internalCreateJMSQueue(false, address.toString().substring(ActiveMQDestination.JMS_QUEUE_ADDRESS_PREFIX.length()), null, true, true);
-            return true;
+            return internalCreateJMSQueue(false, address.toString().substring(ActiveMQDestination.JMS_QUEUE_ADDRESS_PREFIX.length()), null, true, true);
+         }
+         else if (address.toString().startsWith(ActiveMQDestination.JMS_TOPIC_ADDRESS_PREFIX) && settings.isAutoCreateJmsTopics()) {
+            return createTopic(false, address.toString().substring(ActiveMQDestination.JMS_TOPIC_ADDRESS_PREFIX.length()), true);
          }
          else {
             return false;
@@ -1635,8 +1663,64 @@ public class JMSServerManagerImpl implements JMSServerManager, ActivateCallback
 
    class JMSQueueDeleter implements QueueDeleter {
       @Override
-      public boolean delete(SimpleString address) throws Exception {
-         return JMSServerManagerImpl.this.destroyQueue(address.toString().substring(ActiveMQDestination.JMS_QUEUE_ADDRESS_PREFIX.length()), false);
+      public boolean delete(SimpleString queueName) throws Exception {
+         Queue queue = server.locateQueue(queueName);
+         SimpleString address = queue.getAddress();
+         AddressSettings settings = server.getAddressSettingsRepository().getMatch(address.toString());
+         long consumerCount = queue.getConsumerCount();
+         long messageCount = queue.getMessageCount();
+
+         if (address.toString().startsWith(ActiveMQDestination.JMS_QUEUE_ADDRESS_PREFIX) && settings.isAutoDeleteJmsQueues() && queue.getMessageCount() == 0) {
+            if (ActiveMQJMSServerLogger.LOGGER.isDebugEnabled()) {
+               ActiveMQJMSServerLogger.LOGGER.debug("deleting auto-created queue \"" + queueName + ".\" consumerCount = " + consumerCount + "; messageCount = " + messageCount + "; isAutoDeleteJmsQueues = " + settings.isAutoDeleteJmsQueues());
+            }
+
+            return destroyQueue(queueName.toString().substring(ActiveMQDestination.JMS_QUEUE_ADDRESS_PREFIX.length()), false);
+         }
+         else {
+            return false;
+         }
+      }
+   }
+
+   /**
+    * When a core queue is created with a jms.topic prefix this class will create the associated JMS resources
+    * retroactively.  This would happen if, for example, a client created a subscription a non-existent JMS topic and
+    * autoCreateJmsTopics = true.
+    */
+   class JMSPostQueueCreationCallback implements PostQueueCreationCallback {
+      @Override
+      public void callback(SimpleString queueName) throws Exception {
+         Queue queue = server.locateQueue(queueName);
+         String address = queue.getAddress().toString();
+
+         AddressSettings settings = server.getAddressSettingsRepository().getMatch(address.toString());
+         /* When a topic is created a dummy subscription is created which never receives any messages; when the queue
+          * for that dummy subscription is created we don't want to call createTopic again. Therefore we make sure the
+          * queue name doesn't start with the topic prefix.
+          */
+         if (address.toString().startsWith(ActiveMQDestination.JMS_TOPIC_ADDRESS_PREFIX) && settings.isAutoCreateJmsTopics() && !queueName.toString().startsWith(ActiveMQDestination.JMS_TOPIC_ADDRESS_PREFIX)) {
+            createTopic(false, address.toString().substring(ActiveMQDestination.JMS_TOPIC_ADDRESS_PREFIX.length()), true);
+         }
+      }
+   }
+
+   /**
+    * When a core queue representing a JMS topic subscription is deleted this class will check to see if that was the
+    * last subscription on the topic and if so and autoDeleteJmsTopics = true then it will delete the JMS resources
+    * for that topic.
+    */
+   class JMSPostQueueDeletionCallback implements PostQueueDeletionCallback {
+      @Override
+      public void callback(SimpleString address, SimpleString queueName) throws Exception {
+         Queue queue = server.locateQueue(address);
+         Collection<Binding> bindings = server.getPostOffice().getBindingsForAddress(address).getBindings();
+
+         AddressSettings settings = server.getAddressSettingsRepository().getMatch(address.toString());
+
+         if (address.toString().startsWith(ActiveMQDestination.JMS_TOPIC_ADDRESS_PREFIX) && settings.isAutoDeleteJmsTopics() && bindings.size() == 1 && queue != null && queue.isAutoCreated()) {
+            destroyTopic(address.toString().substring(ActiveMQDestination.JMS_TOPIC_ADDRESS_PREFIX.length()));
+         }
       }
    }
 }

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/e53649a6/artemis-protocols/artemis-hqclient-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/hornetq/client/HornetQClientSessionContext.java
----------------------------------------------------------------------
diff --git a/artemis-protocols/artemis-hqclient-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/hornetq/client/HornetQClientSessionContext.java b/artemis-protocols/artemis-hqclient-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/hornetq/client/HornetQClientSessionContext.java
index 15d3ba1..d5e8df4 100644
--- a/artemis-protocols/artemis-hqclient-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/hornetq/client/HornetQClientSessionContext.java
+++ b/artemis-protocols/artemis-hqclient-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/hornetq/client/HornetQClientSessionContext.java
@@ -74,7 +74,7 @@ public class HornetQClientSessionContext extends ActiveMQSessionContext {
    public ClientSession.AddressQuery addressQuery(final SimpleString address) throws ActiveMQException {
       SessionBindingQueryResponseMessage response = (SessionBindingQueryResponseMessage) getSessionChannel().sendBlocking(new SessionBindingQueryMessage(address), PacketImpl.SESS_BINDINGQUERY_RESP);
 
-      return new AddressQueryImpl(response.isExists(), response.getQueueNames(), false);
+      return new AddressQueryImpl(response.isExists(), response.getQueueNames(), false, false);
    }
 
    @Override

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/e53649a6/artemis-protocols/artemis-openwire-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/openwire/OpenWireConnection.java
----------------------------------------------------------------------
diff --git a/artemis-protocols/artemis-openwire-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/openwire/OpenWireConnection.java b/artemis-protocols/artemis-openwire-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/openwire/OpenWireConnection.java
index b1f892a..fad17af 100644
--- a/artemis-protocols/artemis-openwire-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/openwire/OpenWireConnection.java
+++ b/artemis-protocols/artemis-openwire-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/openwire/OpenWireConnection.java
@@ -801,11 +801,10 @@ public class OpenWireConnection extends AbstractRemotingConnection implements Se
 
    public void removeDestination(ActiveMQDestination dest) throws Exception {
       if (dest.isQueue()) {
-         SimpleString qName = new SimpleString("jms.queue." + dest.getPhysicalName());
-         server.destroyQueue(qName);
+         server.destroyQueue(OpenWireUtil.toCoreAddress(dest));
       }
       else {
-         Bindings bindings = server.getPostOffice().getBindingsForAddress(SimpleString.toSimpleString("jms.topic." + dest.getPhysicalName()));
+         Bindings bindings = server.getPostOffice().getBindingsForAddress(OpenWireUtil.toCoreAddress(dest));
          Iterator<Binding> iterator = bindings.getBindings().iterator();
 
          while (iterator.hasNext()) {

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/e53649a6/artemis-protocols/artemis-openwire-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/openwire/amq/AMQConsumer.java
----------------------------------------------------------------------
diff --git a/artemis-protocols/artemis-openwire-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/openwire/amq/AMQConsumer.java b/artemis-protocols/artemis-openwire-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/openwire/amq/AMQConsumer.java
index e2deb80..70dfde3 100644
--- a/artemis-protocols/artemis-openwire-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/openwire/amq/AMQConsumer.java
+++ b/artemis-protocols/artemis-openwire-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/openwire/amq/AMQConsumer.java
@@ -81,7 +81,12 @@ public class AMQConsumer {
       SimpleString address;
 
       if (openwireDestination.isTopic()) {
-         address = new SimpleString("jms.topic." + physicalName);
+         if (openwireDestination.isTemporary()) {
+            address = new SimpleString("jms.temptopic." + physicalName);
+         }
+         else {
+            address = new SimpleString("jms.topic." + physicalName);
+         }
 
          SimpleString queueName = createTopicSubscription(info.isDurable(), info.getClientId(), physicalName, info.getSubscriptionName(), selector, address);
 
@@ -90,7 +95,7 @@ public class AMQConsumer {
       }
       else {
          SimpleString queueName = OpenWireUtil.toCoreAddress(openwireDestination);
-         session.getCoreServer().getJMSQueueCreator().create(queueName);
+         session.getCoreServer().getJMSDestinationCreator().create(queueName);
          serverConsumer = session.getCoreSession().createConsumer(nativeId, queueName, selector, info.isBrowser(), false, -1);
          serverConsumer.setlowConsumerDetection(slowConsumerDetectionListener);
          AddressSettings addrSettings = session.getCoreServer().getAddressSettingsRepository().getMatch(queueName.toString());

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/e53649a6/artemis-protocols/artemis-openwire-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/openwire/amq/AMQSession.java
----------------------------------------------------------------------
diff --git a/artemis-protocols/artemis-openwire-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/openwire/amq/AMQSession.java b/artemis-protocols/artemis-openwire-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/openwire/amq/AMQSession.java
index 218e0df..17d3e18 100644
--- a/artemis-protocols/artemis-openwire-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/openwire/amq/AMQSession.java
+++ b/artemis-protocols/artemis-openwire-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/openwire/amq/AMQSession.java
@@ -149,7 +149,7 @@ public class AMQSession implements SessionCallback {
       for (ActiveMQDestination openWireDest : dests) {
          if (openWireDest.isQueue()) {
             SimpleString queueName = OpenWireUtil.toCoreAddress(openWireDest);
-            getCoreServer().getJMSQueueCreator().create(queueName);
+            getCoreServer().getJMSDestinationCreator().create(queueName);
          }
          AMQConsumer consumer = new AMQConsumer(this, openWireDest, info, scheduledPool);
 

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/e53649a6/artemis-protocols/artemis-openwire-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/openwire/util/OpenWireUtil.java
----------------------------------------------------------------------
diff --git a/artemis-protocols/artemis-openwire-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/openwire/util/OpenWireUtil.java b/artemis-protocols/artemis-openwire-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/openwire/util/OpenWireUtil.java
index 1c64676..3e7410c 100644
--- a/artemis-protocols/artemis-openwire-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/openwire/util/OpenWireUtil.java
+++ b/artemis-protocols/artemis-openwire-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/openwire/util/OpenWireUtil.java
@@ -28,6 +28,11 @@ import org.apache.activemq.command.TransactionId;
 import org.apache.activemq.command.XATransactionId;
 import org.apache.activemq.util.ByteSequence;
 
+import static org.apache.activemq.artemis.jms.client.ActiveMQDestination.JMS_QUEUE_ADDRESS_PREFIX;
+import static org.apache.activemq.artemis.jms.client.ActiveMQDestination.JMS_TEMP_QUEUE_ADDRESS_PREFIX;
+import static org.apache.activemq.artemis.jms.client.ActiveMQDestination.JMS_TEMP_TOPIC_ADDRESS_PREFIX;
+import static org.apache.activemq.artemis.jms.client.ActiveMQDestination.JMS_TOPIC_ADDRESS_PREFIX;
+
 public class OpenWireUtil {
 
    public static ActiveMQBuffer toActiveMQBuffer(ByteSequence bytes) {
@@ -39,10 +44,20 @@ public class OpenWireUtil {
 
    public static SimpleString toCoreAddress(ActiveMQDestination dest) {
       if (dest.isQueue()) {
-         return new SimpleString("jms.queue." + dest.getPhysicalName());
+         if (dest.isTemporary()) {
+            return new SimpleString(JMS_TEMP_QUEUE_ADDRESS_PREFIX + dest.getPhysicalName());
+         }
+         else {
+            return new SimpleString(JMS_QUEUE_ADDRESS_PREFIX + dest.getPhysicalName());
+         }
       }
       else {
-         return new SimpleString("jms.topic." + dest.getPhysicalName());
+         if (dest.isTemporary()) {
+            return new SimpleString(JMS_TEMP_TOPIC_ADDRESS_PREFIX + dest.getPhysicalName());
+         }
+         else {
+            return new SimpleString(JMS_TOPIC_ADDRESS_PREFIX + dest.getPhysicalName());
+         }
       }
    }
 
@@ -54,7 +69,7 @@ public class OpenWireUtil {
     */
    public static ActiveMQDestination toAMQAddress(ServerMessage message, ActiveMQDestination actualDestination) {
       String address = message.getAddress().toString();
-      String strippedAddress = address.replace("jms.queue.", "").replace("jms.topic.", "");
+      String strippedAddress = address.replace(JMS_QUEUE_ADDRESS_PREFIX, "").replace(JMS_TEMP_QUEUE_ADDRESS_PREFIX, "").replace(JMS_TOPIC_ADDRESS_PREFIX, "").replace(JMS_TEMP_TOPIC_ADDRESS_PREFIX, "");
       if (actualDestination.isQueue()) {
          return new ActiveMQQueue(strippedAddress);
       }

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/e53649a6/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompConnection.java
----------------------------------------------------------------------
diff --git a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompConnection.java b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompConnection.java
index 568e5b1..0812867 100644
--- a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompConnection.java
+++ b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompConnection.java
@@ -29,13 +29,13 @@ import org.apache.activemq.artemis.api.core.ActiveMQBuffers;
 import org.apache.activemq.artemis.api.core.ActiveMQException;
 import org.apache.activemq.artemis.api.core.SimpleString;
 import org.apache.activemq.artemis.api.core.client.ActiveMQClient;
-import org.apache.activemq.artemis.api.core.management.ResourceNames;
 import org.apache.activemq.artemis.core.protocol.stomp.v10.StompFrameHandlerV10;
 import org.apache.activemq.artemis.core.protocol.stomp.v12.StompFrameHandlerV12;
 import org.apache.activemq.artemis.core.remoting.CloseListener;
 import org.apache.activemq.artemis.core.remoting.FailureListener;
 import org.apache.activemq.artemis.core.remoting.impl.netty.TransportConstants;
 import org.apache.activemq.artemis.core.server.ActiveMQServerLogger;
+import org.apache.activemq.artemis.core.server.QueueCreator;
 import org.apache.activemq.artemis.core.server.ServerMessage;
 import org.apache.activemq.artemis.core.server.impl.ServerMessageImpl;
 import org.apache.activemq.artemis.spi.core.protocol.RemotingConnection;
@@ -227,30 +227,23 @@ public final class StompConnection implements RemotingConnection {
    }
 
    public void checkDestination(String destination) throws ActiveMQStompException {
-      if (autoCreateQueueIfPossible(destination)) {
-         return;
-      }
+      autoCreateDestinationIfPossible(destination);
 
       if (!manager.destinationExists(destination)) {
          throw BUNDLE.destinationNotExist(destination).setHandler(frameHandler);
       }
    }
 
-   public boolean autoCreateQueueIfPossible(String queue) throws ActiveMQStompException {
-      boolean autoCreated = false;
-
-      if (queue.startsWith(ResourceNames.JMS_QUEUE) && manager.getServer().getAddressSettingsRepository().getMatch(queue).isAutoCreateJmsQueues() && manager.getServer().locateQueue(new SimpleString(queue)) == null) {
-         SimpleString queueName = new SimpleString(queue);
-         try {
-            manager.getServer().createQueue(queueName, queueName, null, SimpleString.toSimpleString(this.getLogin()), true, false, true);
-         }
-         catch (Exception e) {
-            throw new ActiveMQStompException(e.getMessage(), e).setHandler(frameHandler);
+   public void autoCreateDestinationIfPossible(String queue) throws ActiveMQStompException {
+      try {
+         QueueCreator queueCreator = manager.getServer().getJMSDestinationCreator();
+         if (queueCreator != null) {
+            queueCreator.create(SimpleString.toSimpleString(queue));
          }
-         autoCreated = true;
       }
-
-      return autoCreated;
+      catch (Exception e) {
+         throw new ActiveMQStompException(e.getMessage(), e).setHandler(frameHandler);
+      }
    }
 
    @Override
@@ -618,7 +611,7 @@ public final class StompConnection implements RemotingConnection {
                   String id,
                   String durableSubscriptionName,
                   boolean noLocal) throws ActiveMQStompException {
-      autoCreateQueueIfPossible(destination);
+      autoCreateDestinationIfPossible(destination);
       if (noLocal) {
          String noLocalFilter = CONNECTION_ID_PROP + " <> '" + getID().toString() + "'";
          if (selector == null) {

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/e53649a6/artemis-server/src/main/java/org/apache/activemq/artemis/core/deployers/impl/FileConfigurationParser.java
----------------------------------------------------------------------
diff --git a/artemis-server/src/main/java/org/apache/activemq/artemis/core/deployers/impl/FileConfigurationParser.java b/artemis-server/src/main/java/org/apache/activemq/artemis/core/deployers/impl/FileConfigurationParser.java
index 0a6391b..b058aed 100644
--- a/artemis-server/src/main/java/org/apache/activemq/artemis/core/deployers/impl/FileConfigurationParser.java
+++ b/artemis-server/src/main/java/org/apache/activemq/artemis/core/deployers/impl/FileConfigurationParser.java
@@ -163,6 +163,10 @@ public final class FileConfigurationParser extends XMLConfigurationUtil {
 
    private static final String AUTO_DELETE_JMS_QUEUES = "auto-delete-jms-queues";
 
+   private static final String AUTO_CREATE_JMS_TOPICS = "auto-create-jms-topics";
+
+   private static final String AUTO_DELETE_JMS_TOPICS = "auto-delete-jms-topics";
+
    private static final String MANAGEMENT_BROWSE_PAGE_SIZE = "management-browse-page-size";
 
    private static final String MAX_CONNECTIONS_NODE_NAME = "max-connections";
@@ -796,6 +800,12 @@ public final class FileConfigurationParser extends XMLConfigurationUtil {
          else if (AUTO_DELETE_JMS_QUEUES.equalsIgnoreCase(name)) {
             addressSettings.setAutoDeleteJmsQueues(XMLUtil.parseBoolean(child));
          }
+         else if (AUTO_CREATE_JMS_TOPICS.equalsIgnoreCase(name)) {
+            addressSettings.setAutoCreateJmsTopics(XMLUtil.parseBoolean(child));
+         }
+         else if (AUTO_DELETE_JMS_TOPICS.equalsIgnoreCase(name)) {
+            addressSettings.setAutoDeleteJmsTopics(XMLUtil.parseBoolean(child));
+         }
          else if (MANAGEMENT_BROWSE_PAGE_SIZE.equalsIgnoreCase(name)) {
             addressSettings.setManagementBrowsePageSize(XMLUtil.parseInt(child));
          }

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/e53649a6/artemis-server/src/main/java/org/apache/activemq/artemis/core/management/impl/ActiveMQServerControlImpl.java
----------------------------------------------------------------------
diff --git a/artemis-server/src/main/java/org/apache/activemq/artemis/core/management/impl/ActiveMQServerControlImpl.java b/artemis-server/src/main/java/org/apache/activemq/artemis/core/management/impl/ActiveMQServerControlImpl.java
index df84c48..4c47e74 100644
--- a/artemis-server/src/main/java/org/apache/activemq/artemis/core/management/impl/ActiveMQServerControlImpl.java
+++ b/artemis-server/src/main/java/org/apache/activemq/artemis/core/management/impl/ActiveMQServerControlImpl.java
@@ -1519,6 +1519,8 @@ public class ActiveMQServerControlImpl extends AbstractControl implements Active
       settings.put("slowConsumerPolicy", policy);
       settings.put("autoCreateJmsQueues", addressSettings.isAutoCreateJmsQueues());
       settings.put("autoDeleteJmsQueues", addressSettings.isAutoDeleteJmsQueues());
+      settings.put("autoCreateJmsTopics", addressSettings.isAutoCreateJmsTopics());
+      settings.put("autoDeleteJmsTopics", addressSettings.isAutoDeleteJmsTopics());
 
       JSONObject jsonObject = new JSONObject(settings);
       return jsonObject.toString();
@@ -1544,7 +1546,9 @@ public class ActiveMQServerControlImpl extends AbstractControl implements Active
                                   final long slowConsumerCheckPeriod,
                                   final String slowConsumerPolicy,
                                   final boolean autoCreateJmsQueues,
-                                  final boolean autoDeleteJmsQueues) throws Exception {
+                                  final boolean autoDeleteJmsQueues,
+                                  final boolean autoCreateJmsTopics,
+                                  final boolean autoDeleteJmsTopics) throws Exception {
       checkStarted();
 
       // JBPAPP-6334 requested this to be pageSizeBytes > maxSizeBytes
@@ -1598,6 +1602,8 @@ public class ActiveMQServerControlImpl extends AbstractControl implements Active
       }
       addressSettings.setAutoCreateJmsQueues(autoCreateJmsQueues);
       addressSettings.setAutoDeleteJmsQueues(autoDeleteJmsQueues);
+      addressSettings.setAutoCreateJmsTopics(autoCreateJmsTopics);
+      addressSettings.setAutoDeleteJmsTopics(autoDeleteJmsTopics);
       server.getAddressSettingsRepository().addMatch(address, addressSettings);
 
       storageManager.storeAddressSetting(new PersistedAddressSetting(new SimpleString(address), addressSettings));

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/e53649a6/artemis-server/src/main/java/org/apache/activemq/artemis/core/protocol/core/ServerSessionPacketHandler.java
----------------------------------------------------------------------
diff --git a/artemis-server/src/main/java/org/apache/activemq/artemis/core/protocol/core/ServerSessionPacketHandler.java b/artemis-server/src/main/java/org/apache/activemq/artemis/core/protocol/core/ServerSessionPacketHandler.java
index 62c7d99..f12fd0b 100644
--- a/artemis-server/src/main/java/org/apache/activemq/artemis/core/protocol/core/ServerSessionPacketHandler.java
+++ b/artemis-server/src/main/java/org/apache/activemq/artemis/core/protocol/core/ServerSessionPacketHandler.java
@@ -39,6 +39,7 @@ import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SessionAdd
 import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SessionBindingQueryMessage;
 import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SessionBindingQueryResponseMessage;
 import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SessionBindingQueryResponseMessage_V2;
+import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SessionBindingQueryResponseMessage_V3;
 import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SessionConsumerCloseMessage;
 import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SessionConsumerFlowCreditMessage;
 import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SessionCreateConsumerMessage;
@@ -263,7 +264,10 @@ public class ServerSessionPacketHandler implements ChannelHandler {
                   requiresResponse = true;
                   SessionBindingQueryMessage request = (SessionBindingQueryMessage) packet;
                   BindingQueryResult result = session.executeBindingQuery(request.getAddress());
-                  if (channel.supports(PacketImpl.SESS_BINDINGQUERY_RESP_V2)) {
+                  if (channel.supports(PacketImpl.SESS_BINDINGQUERY_RESP_V3)) {
+                     response = new SessionBindingQueryResponseMessage_V3(result.isExists(), result.getQueueNames(), result.isAutoCreateJmsQueues(), result.isAutoCreateJmsTopics());
+                  }
+                  else if (channel.supports(PacketImpl.SESS_BINDINGQUERY_RESP_V2)) {
                      response = new SessionBindingQueryResponseMessage_V2(result.isExists(), result.getQueueNames(), result.isAutoCreateJmsQueues());
                   }
                   else {

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/e53649a6/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/ActiveMQServer.java
----------------------------------------------------------------------
diff --git a/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/ActiveMQServer.java b/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/ActiveMQServer.java
index d5786e6..e416205 100644
--- a/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/ActiveMQServer.java
+++ b/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/ActiveMQServer.java
@@ -126,6 +126,36 @@ public interface ActiveMQServer extends ActiveMQComponent {
     */
    void callActivationFailureListeners(Exception e);
 
+   /**
+    * @param callback {@link org.apache.activemq.artemis.core.server.PostQueueCreationCallback}
+    */
+   void registerPostQueueCreationCallback(PostQueueCreationCallback callback);
+
+   /**
+    * @param callback {@link org.apache.activemq.artemis.core.server.PostQueueCreationCallback}
+    */
+   void unregisterPostQueueCreationCallback(PostQueueCreationCallback callback);
+
+   /**
+    * @param queueName
+    */
+   void callPostQueueCreationCallbacks(SimpleString queueName) throws Exception;
+
+   /**
+    * @param callback {@link org.apache.activemq.artemis.core.server.PostQueueDeletionCallback}
+    */
+   void registerPostQueueDeletionCallback(PostQueueDeletionCallback callback);
+
+   /**
+    * @param callback {@link org.apache.activemq.artemis.core.server.PostQueueDeletionCallback}
+    */
+   void unregisterPostQueueDeletionCallback(PostQueueDeletionCallback callback);
+
+   /**
+    * @param queueName
+    */
+   void callPostQueueDeletionCallbacks(SimpleString address, SimpleString queueName) throws Exception;
+
    void checkQueueCreationLimit(String username) throws Exception;
 
    ServerSession createSession(String name,
@@ -196,7 +226,7 @@ public interface ActiveMQServer extends ActiveMQComponent {
    /**
     * @see org.apache.activemq.artemis.core.server.ActiveMQServer#setJMSQueueCreator(QueueCreator)
     */
-   QueueCreator getJMSQueueCreator();
+   QueueCreator getJMSDestinationCreator();
 
    /**
     * This is the queue deleter responsible for automatic JMS Queue deletions.

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/e53649a6/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/BindingQueryResult.java
----------------------------------------------------------------------
diff --git a/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/BindingQueryResult.java b/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/BindingQueryResult.java
index ba1c836..fce3a2e 100644
--- a/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/BindingQueryResult.java
+++ b/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/BindingQueryResult.java
@@ -28,14 +28,19 @@ public class BindingQueryResult {
 
    private boolean autoCreateJmsQueues;
 
+   private boolean autoCreateJmsTopics;
+
    public BindingQueryResult(final boolean exists,
                              final List<SimpleString> queueNames,
-                             final boolean autoCreateJmsQueues) {
+                             final boolean autoCreateJmsQueues,
+                             final boolean autoCreateJmsTopics) {
       this.exists = exists;
 
       this.queueNames = queueNames;
 
       this.autoCreateJmsQueues = autoCreateJmsQueues;
+
+      this.autoCreateJmsTopics = autoCreateJmsTopics;
    }
 
    public boolean isExists() {
@@ -46,6 +51,10 @@ public class BindingQueryResult {
       return autoCreateJmsQueues;
    }
 
+   public boolean isAutoCreateJmsTopics() {
+      return autoCreateJmsTopics;
+   }
+
    public List<SimpleString> getQueueNames() {
       return queueNames;
    }

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/e53649a6/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/PostQueueCreationCallback.java
----------------------------------------------------------------------
diff --git a/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/PostQueueCreationCallback.java b/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/PostQueueCreationCallback.java
new file mode 100644
index 0000000..2303c7b
--- /dev/null
+++ b/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/PostQueueCreationCallback.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.activemq.artemis.core.server;
+
+import org.apache.activemq.artemis.api.core.SimpleString;
+
+/**
+ * When a "core" queue is created this callback will be invoked
+ */
+public interface PostQueueCreationCallback {
+
+   void callback(SimpleString queueName) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/e53649a6/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/PostQueueDeletionCallback.java
----------------------------------------------------------------------
diff --git a/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/PostQueueDeletionCallback.java b/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/PostQueueDeletionCallback.java
new file mode 100644
index 0000000..dd4a95d
--- /dev/null
+++ b/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/PostQueueDeletionCallback.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.activemq.artemis.core.server;
+
+import org.apache.activemq.artemis.api.core.SimpleString;
+
+/**
+ * When a "core" queue is deleted this callback will be invoked
+ */
+public interface PostQueueDeletionCallback {
+
+   void callback(SimpleString address, SimpleString queueName) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/e53649a6/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/QueueDeleter.java
----------------------------------------------------------------------
diff --git a/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/QueueDeleter.java b/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/QueueDeleter.java
index 4bdb8a4..d062848 100644
--- a/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/QueueDeleter.java
+++ b/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/QueueDeleter.java
@@ -24,5 +24,5 @@ public interface QueueDeleter {
    /**
     * @return True if a queue was deleted.
     */
-   boolean delete(SimpleString address) throws Exception;
+   boolean delete(SimpleString queueName) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/e53649a6/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/ActiveMQServerImpl.java
----------------------------------------------------------------------
diff --git a/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/ActiveMQServerImpl.java b/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/ActiveMQServerImpl.java
index 50437a1..a5886d5 100644
--- a/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/ActiveMQServerImpl.java
+++ b/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/ActiveMQServerImpl.java
@@ -106,6 +106,8 @@ import org.apache.activemq.artemis.core.server.JournalType;
 import org.apache.activemq.artemis.core.server.LargeServerMessage;
 import org.apache.activemq.artemis.core.server.MemoryManager;
 import org.apache.activemq.artemis.core.server.NodeManager;
+import org.apache.activemq.artemis.core.server.PostQueueCreationCallback;
+import org.apache.activemq.artemis.core.server.PostQueueDeletionCallback;
 import org.apache.activemq.artemis.core.server.Queue;
 import org.apache.activemq.artemis.core.server.QueueCreator;
 import org.apache.activemq.artemis.core.server.QueueDeleter;
@@ -261,6 +263,10 @@ public class ActiveMQServerImpl implements ActiveMQServer {
 
    private final Set<ActivationFailureListener> activationFailureListeners = new ConcurrentHashSet<>();
 
+   private final Set<PostQueueCreationCallback> postQueueCreationCallbacks = new ConcurrentHashSet<>();
+
+   private final Set<PostQueueDeletionCallback> postQueueDeletionCallbacks = new ConcurrentHashSet<>();
+
    private volatile GroupingHandler groupingHandler;
 
    private NodeManager nodeManager;
@@ -564,6 +570,7 @@ public class ActiveMQServerImpl implements ActiveMQServer {
       }
 
       boolean autoCreateJmsQueues = address.toString().startsWith(ResourceNames.JMS_QUEUE) && getAddressSettingsRepository().getMatch(address.toString()).isAutoCreateJmsQueues();
+      boolean autoCreateJmsTopics = address.toString().startsWith(ResourceNames.JMS_TOPIC) && getAddressSettingsRepository().getMatch(address.toString()).isAutoCreateJmsTopics();
 
       List<SimpleString> names = new ArrayList<>();
 
@@ -571,7 +578,7 @@ public class ActiveMQServerImpl implements ActiveMQServer {
       ManagementService managementService = getManagementService();
       if (managementService != null) {
          if (address.equals(managementService.getManagementAddress())) {
-            return new BindingQueryResult(true, names, autoCreateJmsQueues);
+            return new BindingQueryResult(true, names, autoCreateJmsQueues, autoCreateJmsTopics);
          }
       }
 
@@ -583,7 +590,7 @@ public class ActiveMQServerImpl implements ActiveMQServer {
          }
       }
 
-      return new BindingQueryResult(!names.isEmpty(), names, autoCreateJmsQueues);
+      return new BindingQueryResult(!names.isEmpty(), names, autoCreateJmsQueues, autoCreateJmsTopics);
    }
 
    @Override
@@ -655,7 +662,7 @@ public class ActiveMQServerImpl implements ActiveMQServer {
    }
 
    @Override
-   public QueueCreator getJMSQueueCreator() {
+   public QueueCreator getJMSDestinationCreator() {
       return jmsQueueCreator;
    }
 
@@ -1496,6 +1503,8 @@ public class ActiveMQServerImpl implements ActiveMQServer {
          throw ActiveMQMessageBundle.BUNDLE.noSuchQueue(queueName);
       }
 
+      SimpleString address = binding.getAddress();
+
       Queue queue = (Queue) binding.getBindable();
 
       // This check is only valid if checkConsumerCount == true
@@ -1507,14 +1516,16 @@ public class ActiveMQServerImpl implements ActiveMQServer {
 
          if (queue.isDurable()) {
             // make sure the user has privileges to delete this queue
-            securityStore.check(binding.getAddress(), CheckType.DELETE_DURABLE_QUEUE, session);
+            securityStore.check(address, CheckType.DELETE_DURABLE_QUEUE, session);
          }
          else {
-            securityStore.check(binding.getAddress(), CheckType.DELETE_NON_DURABLE_QUEUE, session);
+            securityStore.check(address, CheckType.DELETE_NON_DURABLE_QUEUE, session);
          }
       }
 
       queue.deleteQueue(removeConsumers);
+
+      callPostQueueDeletionCallbacks(address, queueName);
    }
 
    @Override
@@ -1545,6 +1556,40 @@ public class ActiveMQServerImpl implements ActiveMQServer {
    }
 
    @Override
+   public void registerPostQueueCreationCallback(final PostQueueCreationCallback callback) {
+      postQueueCreationCallbacks.add(callback);
+   }
+
+   @Override
+   public void unregisterPostQueueCreationCallback(final PostQueueCreationCallback callback) {
+      postQueueCreationCallbacks.remove(callback);
+   }
+
+   @Override
+   public void callPostQueueCreationCallbacks(final SimpleString queueName) throws Exception {
+      for (PostQueueCreationCallback callback : postQueueCreationCallbacks) {
+         callback.callback(queueName);
+      }
+   }
+
+   @Override
+   public void registerPostQueueDeletionCallback(final PostQueueDeletionCallback callback) {
+      postQueueDeletionCallbacks.add(callback);
+   }
+
+   @Override
+   public void unregisterPostQueueDeletionCallback(final PostQueueDeletionCallback callback) {
+      postQueueDeletionCallbacks.remove(callback);
+   }
+
+   @Override
+   public void callPostQueueDeletionCallbacks(final SimpleString address, final SimpleString queueName) throws Exception {
+      for (PostQueueDeletionCallback callback : postQueueDeletionCallbacks) {
+         callback.callback(address, queueName);
+      }
+   }
+
+   @Override
    public ExecutorFactory getExecutorFactory() {
       return executorFactory;
    }
@@ -2091,7 +2136,7 @@ public class ActiveMQServerImpl implements ActiveMQServer {
          queue.setConsumersRefCount(new TransientQueueManagerImpl(this, queueName));
       }
       else if (autoCreated) {
-         queue.setConsumersRefCount(new AutoCreatedQueueManagerImpl(this, queueName));
+         queue.setConsumersRefCount(new AutoCreatedQueueManagerImpl(this.getJMSQueueDeleter(), queueName));
       }
 
       binding = new LocalQueueBinding(address, queue, nodeManager.getNodeId());
@@ -2127,6 +2172,8 @@ public class ActiveMQServerImpl implements ActiveMQServer {
       managementService.registerAddress(address);
       managementService.registerQueue(queue, address, storageManager);
 
+      callPostQueueCreationCallbacks(queueName);
+
       return queue;
    }
 

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/e53649a6/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/AutoCreatedQueueManagerImpl.java
----------------------------------------------------------------------
diff --git a/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/AutoCreatedQueueManagerImpl.java b/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/AutoCreatedQueueManagerImpl.java
index c39d269..10f9f56 100644
--- a/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/AutoCreatedQueueManagerImpl.java
+++ b/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/AutoCreatedQueueManagerImpl.java
@@ -17,41 +17,23 @@
 package org.apache.activemq.artemis.core.server.impl;
 
 import org.apache.activemq.artemis.api.core.SimpleString;
-import org.apache.activemq.artemis.core.server.ActiveMQServer;
 import org.apache.activemq.artemis.core.server.ActiveMQServerLogger;
 import org.apache.activemq.artemis.core.server.AutoCreatedQueueManager;
-import org.apache.activemq.artemis.core.server.Queue;
+import org.apache.activemq.artemis.core.server.QueueDeleter;
 import org.apache.activemq.artemis.utils.ReferenceCounterUtil;
-import org.jboss.logging.Logger;
 
 public class AutoCreatedQueueManagerImpl implements AutoCreatedQueueManager {
 
-   private static final Logger logger = Logger.getLogger(AutoCreatedQueueManagerImpl.class);
-
    private final SimpleString queueName;
 
-   private final ActiveMQServer server;
+   private final QueueDeleter deleter;
 
    private final Runnable runnable = new Runnable() {
       @Override
       public void run() {
          try {
-            Queue queue = server.locateQueue(queueName);
-            long consumerCount = queue.getConsumerCount();
-            long messageCount = queue.getMessageCount();
-            boolean isAutoDeleteJmsQueues = server.getAddressSettingsRepository().getMatch(queueName.toString()).isAutoDeleteJmsQueues();
-
-            if (server.locateQueue(queueName).getMessageCount() == 0 && isAutoDeleteJmsQueues) {
-               if (logger.isDebugEnabled()) {
-                  logger.debug("deleting auto-created queue \"" + queueName + ".\" consumerCount = " + consumerCount + "; messageCount = " + messageCount + "; isAutoDeleteJmsQueues = " + isAutoDeleteJmsQueues);
-               }
-
-               if (server.getJMSQueueDeleter() != null) {
-                  server.getJMSQueueDeleter().delete(queueName);
-               }
-            }
-            else if (logger.isDebugEnabled()) {
-               logger.debug("NOT deleting auto-created queue \"" + queueName + ".\" consumerCount = " + consumerCount + "; messageCount = " + messageCount + "; isAutoDeleteJmsQueues = " + isAutoDeleteJmsQueues);
+            if (deleter != null) {
+               deleter.delete(queueName);
             }
          }
          catch (Exception e) {
@@ -62,9 +44,8 @@ public class AutoCreatedQueueManagerImpl implements AutoCreatedQueueManager {
 
    private final ReferenceCounterUtil referenceCounterUtil = new ReferenceCounterUtil(runnable);
 
-   public AutoCreatedQueueManagerImpl(ActiveMQServer server, SimpleString queueName) {
-      this.server = server;
-
+   public AutoCreatedQueueManagerImpl(QueueDeleter deleter, SimpleString queueName) {
+      this.deleter = deleter;
       this.queueName = queueName;
    }
 

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/e53649a6/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/PostOfficeJournalLoader.java
----------------------------------------------------------------------
diff --git a/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/PostOfficeJournalLoader.java b/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/PostOfficeJournalLoader.java
index 353db3a..2ced2d8 100644
--- a/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/PostOfficeJournalLoader.java
+++ b/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/PostOfficeJournalLoader.java
@@ -146,7 +146,7 @@ public class PostOfficeJournalLoader implements JournalLoader {
          Queue queue = queueFactory.createQueue(queueBindingInfo.getId(), queueBindingInfo.getAddress(), queueBindingInfo.getQueueName(), filter, subscription, queueBindingInfo.getUser(), true, false, queueBindingInfo.isAutoCreated());
 
          if (queueBindingInfo.isAutoCreated()) {
-            queue.setConsumersRefCount(new AutoCreatedQueueManagerImpl(((PostOfficeImpl) postOffice).getServer(), queueBindingInfo.getQueueName()));
+            queue.setConsumersRefCount(new AutoCreatedQueueManagerImpl(((PostOfficeImpl) postOffice).getServer().getJMSQueueDeleter(), queueBindingInfo.getQueueName()));
          }
 
          Binding binding = new LocalQueueBinding(queueBindingInfo.getAddress(), queue, nodeManager.getNodeId());

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/e53649a6/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/ServerSessionImpl.java
----------------------------------------------------------------------
diff --git a/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/ServerSessionImpl.java b/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/ServerSessionImpl.java
index a4c1438..16cf238 100644
--- a/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/ServerSessionImpl.java
+++ b/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/ServerSessionImpl.java
@@ -487,8 +487,8 @@ public class ServerSessionImpl implements ServerSession, FailureListener {
 
       Queue queue;
 
-      // any non-temporary JMS queue created via this method should be marked as auto-created
-      if (!temporary && address.toString().startsWith(ResourceNames.JMS_QUEUE) && address.equals(name)) {
+      // any non-temporary JMS destination created via this method should be marked as auto-created
+      if (!temporary && ((address.toString().startsWith(ResourceNames.JMS_QUEUE) && address.equals(name)) || address.toString().startsWith(ResourceNames.JMS_TOPIC)) ) {
          queue = server.createQueue(address, name, filterString, SimpleString.toSimpleString(getUsername()), durable, temporary, true);
       }
       else {
@@ -1453,7 +1453,7 @@ public class ServerSessionImpl implements ServerSession, FailureListener {
    }
 
    private void installJMSHooks() {
-      this.queueCreator = server.getJMSQueueCreator();
+      this.queueCreator = server.getJMSDestinationCreator();
    }
 
    private Map<SimpleString, Pair<UUID, AtomicLong>> cloneTargetAddresses() {