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 2017/12/19 14:34:27 UTC

[3/3] activemq-artemis git commit: ARTEMIS-1516 - Ensure JNDI via Tomcat Resource works

ARTEMIS-1516 - Ensure JNDI via Tomcat Resource works

Apply fix so that when using JNDI via tomcat resource it works. 
Replace original extract of JNDIStorable taken from Qpid, and use ActiveMQ5's as fits better to address this issue. (which primary use case is users migrating from 5.x)
Refactored ActiveMQConnectionFactory to externalise and turn into reference by StringRefAddr's instead of custom RefAddr which isnt standard.
Refactored ActiveMQDestinations similar
Refactored ActiveMQDestination to remove redundent and duplicated name field and ensured getters still behave the same


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

Branch: refs/heads/master
Commit: 3ef98044f88dffedce8a8a40b784eb4d4b6dff1f
Parents: 03ed49e
Author: Michael André Pearce <mi...@me.com>
Authored: Sat Nov 4 02:26:39 2017 +0000
Committer: Clebert Suconic <cl...@apache.org>
Committed: Tue Dec 19 09:20:35 2017 -0500

----------------------------------------------------------------------
 .../activemq/artemis/utils/uri/BeanSupport.java |  34 +++++
 .../jms/client/ActiveMQConnectionFactory.java   |  52 +++++--
 .../artemis/jms/client/ActiveMQDestination.java | 126 +++++++++--------
 .../artemis/jms/client/ActiveMQQueue.java       |  29 ++--
 .../jms/client/ActiveMQTemporaryQueue.java      |  10 +-
 .../jms/client/ActiveMQTemporaryTopic.java      |   7 +-
 .../artemis/jms/client/ActiveMQTopic.java       |  26 ++--
 .../ConnectionFactoryObjectFactory.java         |  44 ------
 .../referenceable/DestinationObjectFactory.java |  44 ------
 .../SerializableObjectRefAddr.java              |  71 ----------
 .../artemis/jndi/JNDIReferenceFactory.java      |  29 ++--
 .../activemq/artemis/jndi/JNDIStorable.java     |  60 ++++----
 .../amqp/converter/jms/ServerDestination.java   |   9 +-
 .../amqp/converter/jms/ServerJMSMessage.java    |   5 +-
 .../ConnectionFactoryObjectFactory.java         |  25 ++++
 .../SerializableObjectRefAddr.java              |  31 +++++
 .../ra/ActiveMQRAConnectionFactoryImpl.java     |   6 +-
 ...ctiveMQRAConnectionFactoryObjectFactory.java |  44 ++++++
 .../SerializableObjectRefAddr.java              |  71 ++++++++++
 docs/user-manual/en/SUMMARY.md                  |   1 +
 docs/user-manual/en/tomcat.md                   |  39 ++++++
 examples/features/sub-modules/pom.xml           |   3 +-
 examples/features/sub-modules/tomcat/README     |  21 +++
 examples/features/sub-modules/tomcat/pom.xml    | 127 +++++++++++++++++
 .../tomcat/sample/SendMessageController.java    |  48 +++++++
 .../sample/SystemOutPrintLnMessageListener.java |  28 ++++
 .../main/webapp/WEB-INF/application-context.xml |  50 +++++++
 .../main/webapp/WEB-INF/dispatcher-servlet.xml  |  28 ++++
 .../tomcat/src/main/webapp/WEB-INF/web.xml      |  41 ++++++
 .../tomcat7-maven-plugin/resources/context.xml  |  28 ++++
 .../artemis/jms/tests/ReferenceableTest.java    |   9 +-
 .../tests/unit/jms/jndi/ObjectFactoryTest.java  | 138 +++++++++++++++++++
 .../jms/jndi/StringRefAddrReferenceTest.java    | 119 ++++++++++++++++
 .../DestinationObjectFactoryTest.java           |  59 --------
 .../ra/ConnectionFactoryPropertiesTest.java     |   4 +
 .../DestinationObjectFactoryTest.java           |  60 ++++++++
 36 files changed, 1139 insertions(+), 387 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3ef98044/artemis-commons/src/main/java/org/apache/activemq/artemis/utils/uri/BeanSupport.java
----------------------------------------------------------------------
diff --git a/artemis-commons/src/main/java/org/apache/activemq/artemis/utils/uri/BeanSupport.java b/artemis-commons/src/main/java/org/apache/activemq/artemis/utils/uri/BeanSupport.java
index 70b36ab..94a589d 100644
--- a/artemis-commons/src/main/java/org/apache/activemq/artemis/utils/uri/BeanSupport.java
+++ b/artemis-commons/src/main/java/org/apache/activemq/artemis/utils/uri/BeanSupport.java
@@ -19,12 +19,14 @@ package org.apache.activemq.artemis.utils.uri;
 
 import java.beans.PropertyDescriptor;
 import java.io.UnsupportedEncodingException;
+import java.lang.reflect.InvocationTargetException;
 import java.net.URI;
 import java.net.URLDecoder;
 import java.net.URLEncoder;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Properties;
 import java.util.Set;
 
 import org.apache.commons.beanutils.BeanUtilsBean;
@@ -69,6 +71,38 @@ public class BeanSupport {
       return obj;
    }
 
+   public static <P> P setProperties(P bean, Properties properties)
+      throws IllegalAccessException, NoSuchMethodException, InvocationTargetException {
+      synchronized (beanUtils) {
+         PropertyDescriptor[] descriptors = beanUtils.getPropertyUtils().getPropertyDescriptors(bean);
+         for (PropertyDescriptor descriptor : descriptors) {
+            if (descriptor.getReadMethod() != null && isWriteable(descriptor, null)) {
+               String value = properties.getProperty(descriptor.getName());
+               if (value != null) {
+                  beanUtils.setProperty(bean, descriptor.getName(), value);
+               }
+            }
+         }
+      }
+      return bean;
+   }
+
+   public static <P> Properties getProperties(P bean, Properties properties)
+      throws IllegalAccessException, NoSuchMethodException, InvocationTargetException {
+      synchronized (beanUtils) {
+         PropertyDescriptor[] descriptors = beanUtils.getPropertyUtils().getPropertyDescriptors(bean);
+         for (PropertyDescriptor descriptor : descriptors) {
+            if (descriptor.getReadMethod() != null && isWriteable(descriptor, null)) {
+               String value = beanUtils.getProperty(bean, descriptor.getName());
+               if (value != null) {
+                  properties.put(descriptor.getName(), value);
+               }
+            }
+         }
+      }
+      return properties;
+   }
+
    public static void setData(URI uri,
                               HashMap<String, Object> properties,
                               Set<String> allowableProperties,

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3ef98044/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQConnectionFactory.java
----------------------------------------------------------------------
diff --git a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQConnectionFactory.java b/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQConnectionFactory.java
index ba5359a..bb729f6 100644
--- a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQConnectionFactory.java
+++ b/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQConnectionFactory.java
@@ -30,17 +30,17 @@ import javax.jms.XAConnectionFactory;
 import javax.jms.XAJMSContext;
 import javax.jms.XAQueueConnection;
 import javax.jms.XATopicConnection;
-import javax.naming.NamingException;
-import javax.naming.Reference;
-import javax.naming.Referenceable;
+import javax.naming.Context;
 import java.io.Externalizable;
 import java.io.IOException;
 import java.io.InvalidObjectException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
+import java.lang.reflect.InvocationTargetException;
 import java.net.URI;
 import java.security.AccessController;
 import java.security.PrivilegedAction;
+import java.util.Properties;
 
 import org.apache.activemq.artemis.api.core.DiscoveryGroupConfiguration;
 import org.apache.activemq.artemis.api.core.TransportConfiguration;
@@ -51,18 +51,18 @@ import org.apache.activemq.artemis.api.core.client.ServerLocator;
 import org.apache.activemq.artemis.api.jms.ActiveMQJMSConstants;
 import org.apache.activemq.artemis.api.jms.JMSFactoryType;
 import org.apache.activemq.artemis.core.client.impl.ServerLocatorImpl;
-import org.apache.activemq.artemis.jms.referenceable.ConnectionFactoryObjectFactory;
-import org.apache.activemq.artemis.jms.referenceable.SerializableObjectRefAddr;
+import org.apache.activemq.artemis.jndi.JNDIStorable;
 import org.apache.activemq.artemis.spi.core.remoting.ClientProtocolManagerFactory;
 import org.apache.activemq.artemis.uri.ConnectionFactoryParser;
 import org.apache.activemq.artemis.uri.ServerLocatorParser;
 import org.apache.activemq.artemis.utils.ClassloadingUtil;
+import org.apache.activemq.artemis.utils.uri.BeanSupport;
 
 /**
  * <p>ActiveMQ Artemis implementation of a JMS ConnectionFactory.</p>
  * <p>This connection factory will use defaults defined by {@link DefaultConnectionProperties}.
  */
-public class ActiveMQConnectionFactory implements ConnectionFactoryOptions, Externalizable, Referenceable, ConnectionFactory, XAConnectionFactory, AutoCloseable {
+public class ActiveMQConnectionFactory extends JNDIStorable implements ConnectionFactoryOptions, Externalizable, ConnectionFactory, XAConnectionFactory, AutoCloseable {
 
    private static final long serialVersionUID = -7554006056207377105L;
 
@@ -206,10 +206,14 @@ public class ActiveMQConnectionFactory implements ConnectionFactoryOptions, Exte
       this(DefaultConnectionProperties.DEFAULT_BROKER_URL);
    }
 
-   public ActiveMQConnectionFactory(String url) {
+   public ActiveMQConnectionFactory(String brokerURL) {
+      setBrokerURL(brokerURL);
+   }
+
+   private void setBrokerURL(String brokerURL) {
       ConnectionFactoryParser cfParser = new ConnectionFactoryParser();
       try {
-         URI uri = cfParser.expandURI(url);
+         URI uri = cfParser.expandURI(brokerURL);
          serverLocator = ServerLocatorImpl.newLocator(uri);
          cfParser.populateObject(uri, this);
       } catch (Exception e) {
@@ -380,8 +384,36 @@ public class ActiveMQConnectionFactory implements ConnectionFactoryOptions, Exte
    }
 
    @Override
-   public Reference getReference() throws NamingException {
-      return new Reference(this.getClass().getCanonicalName(), new SerializableObjectRefAddr("ActiveMQ-CF", this), ConnectionFactoryObjectFactory.class.getCanonicalName(), null);
+   protected void buildFromProperties(Properties props) {
+      String url = props.getProperty(Context.PROVIDER_URL);
+      if (url == null || url.isEmpty()) {
+         url = props.getProperty("brokerURL");
+      }
+      if (url != null && url.length() > 0) {
+         setBrokerURL(url);
+      }
+      if (url == null || url.isEmpty()) {
+         throw new IllegalArgumentException(Context.PROVIDER_URL + " or " + "brokerURL is required");
+      }
+      try {
+         BeanSupport.setProperties(this, props);
+      } catch (IllegalAccessException | NoSuchMethodException | InvocationTargetException e) {
+         throw new RuntimeException(e);
+      }
+   }
+
+   @Override
+   protected void populateProperties(Properties props) {
+      try {
+         URI uri = toURI();
+         if (uri != null) {
+            props.put(Context.PROVIDER_URL, uri.toASCIIString());
+            props.put("brokerURL", uri.toASCIIString());
+         }
+         BeanSupport.getProperties(this, props);
+      } catch (IOException | IllegalAccessException | NoSuchMethodException | InvocationTargetException e) {
+         throw new RuntimeException(e);
+      }
    }
 
    public boolean isHA() {

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3ef98044/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQDestination.java
----------------------------------------------------------------------
diff --git a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQDestination.java b/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQDestination.java
index c0ab4b9..d374265 100644
--- a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQDestination.java
+++ b/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQDestination.java
@@ -19,21 +19,18 @@ package org.apache.activemq.artemis.jms.client;
 import javax.jms.Destination;
 import javax.jms.JMSException;
 import javax.jms.JMSRuntimeException;
-import javax.naming.NamingException;
-import javax.naming.Reference;
-import javax.naming.Referenceable;
 import java.io.Serializable;
+import java.util.Properties;
 import java.util.UUID;
 
 import org.apache.activemq.artemis.api.core.Pair;
 import org.apache.activemq.artemis.api.core.SimpleString;
-import org.apache.activemq.artemis.jms.referenceable.DestinationObjectFactory;
-import org.apache.activemq.artemis.jms.referenceable.SerializableObjectRefAddr;
+import org.apache.activemq.artemis.jndi.JNDIStorable;
 
 /**
  * ActiveMQ Artemis implementation of a JMS Destination.
  */
-public class ActiveMQDestination implements Destination, Serializable, Referenceable {
+public class ActiveMQDestination extends JNDIStorable implements Destination, Serializable {
    // Constants -----------------------------------------------------
 
    // Static --------------------------------------------------------
@@ -78,27 +75,27 @@ public class ActiveMQDestination implements Destination, Serializable, Reference
          case TEMP_TOPIC:
             return new ActiveMQTopic(name, true);
          case DESTINATION:
-            return new ActiveMQDestination(name, name, TYPE.DESTINATION, null);
+            return new ActiveMQDestination(name, TYPE.DESTINATION, null);
          default:
             throw new IllegalArgumentException("Invalid default destination type: " + defaultType);
       }
    }
 
-   public static Destination fromPrefixedName(final String address) {
-      if (address.startsWith(ActiveMQDestination.QUEUE_QUALIFIED_PREFIX)) {
-         String name = address.substring(ActiveMQDestination.QUEUE_QUALIFIED_PREFIX.length());
-         return createQueue(name);
-      } else if (address.startsWith(ActiveMQDestination.TOPIC_QUALIFIED_PREFIX)) {
-         String name = address.substring(ActiveMQDestination.TOPIC_QUALIFIED_PREFIX.length());
-         return createTopic(name);
-      } else if (address.startsWith(ActiveMQDestination.TEMP_QUEUE_QUALIFED_PREFIX)) {
-         String name = address.substring(ActiveMQDestination.TEMP_QUEUE_QUALIFED_PREFIX.length());
-         return new ActiveMQTemporaryQueue(name, name, null);
-      } else if (address.startsWith(ActiveMQDestination.TEMP_TOPIC_QUALIFED_PREFIX)) {
-         String name = address.substring(ActiveMQDestination.TEMP_TOPIC_QUALIFED_PREFIX.length());
-         return new ActiveMQTemporaryTopic(name, name, null);
+   public static Destination fromPrefixedName(final String name) {
+      if (name.startsWith(ActiveMQDestination.QUEUE_QUALIFIED_PREFIX)) {
+         String address = name.substring(ActiveMQDestination.QUEUE_QUALIFIED_PREFIX.length());
+         return createQueue(address);
+      } else if (name.startsWith(ActiveMQDestination.TOPIC_QUALIFIED_PREFIX)) {
+         String address = name.substring(ActiveMQDestination.TOPIC_QUALIFIED_PREFIX.length());
+         return createTopic(address);
+      } else if (name.startsWith(ActiveMQDestination.TEMP_QUEUE_QUALIFED_PREFIX)) {
+         String address = name.substring(ActiveMQDestination.TEMP_QUEUE_QUALIFED_PREFIX.length());
+         return new ActiveMQTemporaryQueue(address, null);
+      } else if (name.startsWith(ActiveMQDestination.TEMP_TOPIC_QUALIFED_PREFIX)) {
+         String address = name.substring(ActiveMQDestination.TEMP_TOPIC_QUALIFED_PREFIX.length());
+         return new ActiveMQTemporaryTopic(address, null);
       } else {
-         return new ActiveMQDestination(address, address, TYPE.DESTINATION, null);
+         return new ActiveMQDestination(name, TYPE.DESTINATION, null);
       }
    }
 
@@ -191,58 +188,48 @@ public class ActiveMQDestination implements Destination, Serializable, Reference
       return new SimpleString(TOPIC_QUALIFIED_PREFIX + name);
    }
 
-   public static ActiveMQQueue createQueue(final String name) {
-      return new ActiveMQQueue(name);
+   public static ActiveMQQueue createQueue(final String address) {
+      return new ActiveMQQueue(address);
    }
 
-   public static ActiveMQTopic createTopic(final String name) {
-      return new ActiveMQTopic(name);
+   public static ActiveMQTopic createTopic(final String address) {
+      return new ActiveMQTopic(address);
    }
 
-   public static ActiveMQTemporaryQueue createTemporaryQueue(final String name, final ActiveMQSession session) {
-      return new ActiveMQTemporaryQueue(name, name, session);
+   public static ActiveMQTemporaryQueue createTemporaryQueue(final String address, final ActiveMQSession session) {
+      return new ActiveMQTemporaryQueue(address, session);
    }
 
-   public static ActiveMQTemporaryQueue createTemporaryQueue(final String name) {
-      return createTemporaryQueue(name, null);
+   public static ActiveMQTemporaryQueue createTemporaryQueue(final String address) {
+      return createTemporaryQueue(address, null);
    }
 
    public static ActiveMQTemporaryQueue createTemporaryQueue(final ActiveMQSession session) {
-      String name = UUID.randomUUID().toString();
+      String address = UUID.randomUUID().toString();
 
-      return createTemporaryQueue(name, session);
+      return createTemporaryQueue(address, session);
    }
 
    public static ActiveMQTemporaryTopic createTemporaryTopic(final ActiveMQSession session) {
-      String name = UUID.randomUUID().toString();
+      String address = UUID.randomUUID().toString();
 
-      return createTemporaryTopic(name, session);
+      return createTemporaryTopic(address, session);
    }
 
-   public static ActiveMQTemporaryTopic createTemporaryTopic(String name, final ActiveMQSession session) {
-      return new ActiveMQTemporaryTopic(name, name, session);
+   public static ActiveMQTemporaryTopic createTemporaryTopic(String address, final ActiveMQSession session) {
+      return new ActiveMQTemporaryTopic(address, session);
    }
 
-   public static ActiveMQTemporaryTopic createTemporaryTopic(String name) {
-      return createTemporaryTopic(name, null);
+   public static ActiveMQTemporaryTopic createTemporaryTopic(String address) {
+      return createTemporaryTopic(address, null);
    }
 
    // Attributes ----------------------------------------------------
 
    /**
-    * The JMS name
-    */
-   protected final String name;
-
-   /**
     * The core address
     */
-   private final String address;
-
-   /**
-    * SimpleString version of address
-    */
-   private final SimpleString simpleAddress;
+   private SimpleString simpleAddress;
 
    private final TYPE type;
 
@@ -251,25 +238,34 @@ public class ActiveMQDestination implements Destination, Serializable, Reference
    // Constructors --------------------------------------------------
 
    protected ActiveMQDestination(final String address,
-                                 final String name,
                                  final TYPE type,
                                  final ActiveMQSession session) {
-      this.address = address;
+      this.simpleAddress = SimpleString.toSimpleString(address);
 
-      this.name = name;
+      this.type = type;
 
-      simpleAddress = new SimpleString(address);
+      this.session = session;
+   }
+
+   protected ActiveMQDestination(final SimpleString address,
+                                 final TYPE type,
+                                 final ActiveMQSession session) {
+      this.simpleAddress = address;
 
       this.type = type;
 
       this.session = session;
    }
 
-   // Referenceable implementation ---------------------------------------
+   public void setAddress(String address) {
+      setSimpleAddress(SimpleString.toSimpleString(address));
+   }
 
-   @Override
-   public Reference getReference() throws NamingException {
-      return new Reference(this.getClass().getCanonicalName(), new SerializableObjectRefAddr("ActiveMQ-DEST", this), DestinationObjectFactory.class.getCanonicalName(), null);
+   public void setSimpleAddress(SimpleString address) {
+      if (address == null) {
+         throw new IllegalArgumentException("address cannot be null");
+      }
+      this.simpleAddress = address;
    }
 
    public void delete() throws JMSException {
@@ -293,7 +289,7 @@ public class ActiveMQDestination implements Destination, Serializable, Reference
    // Public --------------------------------------------------------
 
    public String getAddress() {
-      return address;
+      return simpleAddress.toString();
    }
 
    public SimpleString getSimpleAddress() {
@@ -301,7 +297,7 @@ public class ActiveMQDestination implements Destination, Serializable, Reference
    }
 
    public String getName() {
-      return name;
+      return simpleAddress.toString();
    }
 
    public boolean isTemporary() {
@@ -324,12 +320,22 @@ public class ActiveMQDestination implements Destination, Serializable, Reference
 
       ActiveMQDestination that = (ActiveMQDestination) o;
 
-      return address.equals(that.address);
+      return simpleAddress.equals(that.simpleAddress);
    }
 
    @Override
    public int hashCode() {
-      return address.hashCode();
+      return simpleAddress.hashCode();
+   }
+
+   @Override
+   protected void buildFromProperties(Properties props) {
+      setAddress(props.getProperty("address"));
+   }
+
+   @Override
+   protected void populateProperties(Properties props) {
+      props.put("address", getAddress());
    }
 
    // Package protected ---------------------------------------------

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3ef98044/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQQueue.java
----------------------------------------------------------------------
diff --git a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQQueue.java b/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQQueue.java
index 2f9a47b..2deefa9 100644
--- a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQQueue.java
+++ b/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQQueue.java
@@ -18,8 +18,6 @@ package org.apache.activemq.artemis.jms.client;
 
 import javax.jms.Queue;
 
-import org.apache.activemq.artemis.api.core.SimpleString;
-
 /**
  * ActiveMQ Artemis implementation of a JMS Queue.
  * <br>
@@ -32,35 +30,30 @@ public class ActiveMQQueue extends ActiveMQDestination implements Queue {
 
    // Static --------------------------------------------------------
 
-   public static SimpleString createAddressFromName(final String name) {
-      return new SimpleString(name);
-   }
-
    // Attributes ----------------------------------------------------
 
    // Constructors --------------------------------------------------
+   public ActiveMQQueue() {
+      this(null);
+   }
 
-   public ActiveMQQueue(final String name) {
-      super(name, name, TYPE.QUEUE, null);
+   public ActiveMQQueue(final String address) {
+      super(address, TYPE.QUEUE, null);
    }
 
-   public ActiveMQQueue(final String name, boolean temporary) {
-      super(name, name, temporary ? TYPE.TEMP_QUEUE : TYPE.QUEUE, null);
+   public ActiveMQQueue(final String address, boolean temporary) {
+      super(address, temporary ? TYPE.TEMP_QUEUE : TYPE.QUEUE, null);
    }
 
    /**
     * @param address
-    * @param name
     * @param temporary
     * @param session
     */
-   public ActiveMQQueue(String address, String name, boolean temporary, ActiveMQSession session) {
-      super(address, name, temporary ? TYPE.TEMP_QUEUE : TYPE.QUEUE, session);
+   public ActiveMQQueue(String address, boolean temporary, ActiveMQSession session) {
+      super(address, temporary ? TYPE.TEMP_QUEUE : TYPE.QUEUE, session);
    }
 
-   public ActiveMQQueue(final String address, final String name) {
-      super(address, name, TYPE.QUEUE, null);
-   }
 
    // Queue implementation ------------------------------------------
 
@@ -68,12 +61,12 @@ public class ActiveMQQueue extends ActiveMQDestination implements Queue {
 
    @Override
    public String getQueueName() {
-      return name;
+      return getAddress();
    }
 
    @Override
    public String toString() {
-      return "ActiveMQQueue[" + name + "]";
+      return "ActiveMQQueue[" + getAddress() + "]";
    }
 
    @Override

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3ef98044/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQTemporaryQueue.java
----------------------------------------------------------------------
diff --git a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQTemporaryQueue.java b/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQTemporaryQueue.java
index 88a822a..b79c36a 100644
--- a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQTemporaryQueue.java
+++ b/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQTemporaryQueue.java
@@ -37,19 +37,21 @@ public class ActiveMQTemporaryQueue extends ActiveMQQueue implements TemporaryQu
    // TemporaryQueue implementation ------------------------------------------
 
    // Public --------------------------------------------------------
+   public ActiveMQTemporaryQueue() {
+      this(null, null);
+   }
 
    /**
     * @param address
-    * @param name
     * @param session
     */
-   public ActiveMQTemporaryQueue(String address, String name, ActiveMQSession session) {
-      super(address, name, true, session);
+   public ActiveMQTemporaryQueue(String address, ActiveMQSession session) {
+      super(address, true, session);
    }
 
    @Override
    public String toString() {
-      return "ActiveMQTemporaryQueue[" + name + "]";
+      return "ActiveMQTemporaryQueue[" + getAddress() + "]";
    }
 
    @Override

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3ef98044/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQTemporaryTopic.java
----------------------------------------------------------------------
diff --git a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQTemporaryTopic.java b/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQTemporaryTopic.java
index 98b5ba6..457663d 100644
--- a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQTemporaryTopic.java
+++ b/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQTemporaryTopic.java
@@ -29,9 +29,12 @@ public class ActiveMQTemporaryTopic extends ActiveMQTopic implements TemporaryTo
    // Static --------------------------------------------------------
 
    // Constructors --------------------------------------------------
+   public ActiveMQTemporaryTopic() {
+      this(null, null);
+   }
 
-   protected ActiveMQTemporaryTopic(final String address, final String name, final ActiveMQSession session) {
-      super(address, name, true, session);
+   public ActiveMQTemporaryTopic(final String address, final ActiveMQSession session) {
+      super(address, true, session);
    }
 
    // Public --------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3ef98044/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQTopic.java
----------------------------------------------------------------------
diff --git a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQTopic.java b/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQTopic.java
index 94bdd25..e22e67b 100644
--- a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQTopic.java
+++ b/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/client/ActiveMQTopic.java
@@ -18,8 +18,6 @@ package org.apache.activemq.artemis.jms.client;
 
 import javax.jms.Topic;
 
-import org.apache.activemq.artemis.api.core.SimpleString;
-
 /**
  * ActiveMQ Artemis implementation of a JMS Topic.
  * <br>
@@ -31,44 +29,42 @@ public class ActiveMQTopic extends ActiveMQDestination implements Topic {
    private static final long serialVersionUID = 7873614001276404156L;
    // Static --------------------------------------------------------
 
-   public static SimpleString createAddressFromName(final String name) {
-      return new SimpleString(name);
-   }
-
    // Attributes ----------------------------------------------------
 
    // Constructors --------------------------------------------------
+   public ActiveMQTopic() {
+      this(null);
+   }
 
-   public ActiveMQTopic(final String name) {
-      this(name, false);
+   public ActiveMQTopic(final String address) {
+      this(address, false);
    }
 
-   public ActiveMQTopic(final String name, boolean temporary) {
-      super(name, name, TYPE.TOPIC, null);
+   public ActiveMQTopic(final String address, boolean temporary) {
+      super(address, TYPE.TOPIC, null);
    }
 
    /**
     * @param address
-    * @param name
     * @param temporary
     * @param session
     */
-   protected ActiveMQTopic(String address, String name, boolean temporary, ActiveMQSession session) {
-      super(address, name, temporary ? TYPE.TEMP_TOPIC : TYPE.TOPIC, session);
+   protected ActiveMQTopic(String address, boolean temporary, ActiveMQSession session) {
+      super(address, temporary ? TYPE.TEMP_TOPIC : TYPE.TOPIC, session);
    }
 
    // Topic implementation ------------------------------------------
 
    @Override
    public String getTopicName() {
-      return name;
+      return getName();
    }
 
    // Public --------------------------------------------------------
 
    @Override
    public String toString() {
-      return "ActiveMQTopic[" + name + "]";
+      return "ActiveMQTopic[" + getName() + "]";
    }
 
    @Override

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3ef98044/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/referenceable/ConnectionFactoryObjectFactory.java
----------------------------------------------------------------------
diff --git a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/referenceable/ConnectionFactoryObjectFactory.java b/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/referenceable/ConnectionFactoryObjectFactory.java
deleted file mode 100644
index ad125f0..0000000
--- a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/referenceable/ConnectionFactoryObjectFactory.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.jms.referenceable;
-
-import javax.naming.Context;
-import javax.naming.Name;
-import javax.naming.Reference;
-import javax.naming.spi.ObjectFactory;
-import java.util.Hashtable;
-
-/**
- * A ConnectionFactoryObjectFactory.
- *
- * Given a reference - reconstructs an ActiveMQRAConnectionFactory
- */
-public class ConnectionFactoryObjectFactory implements ObjectFactory {
-
-   @Override
-   public Object getObjectInstance(final Object ref,
-                                   final Name name,
-                                   final Context ctx,
-                                   final Hashtable<?, ?> props) throws Exception {
-      Reference r = (Reference) ref;
-
-      byte[] bytes = (byte[]) r.get("ActiveMQ-CF").getContent();
-
-      // Deserialize
-      return SerializableObjectRefAddr.deserialize(bytes);
-   }
-}

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3ef98044/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/referenceable/DestinationObjectFactory.java
----------------------------------------------------------------------
diff --git a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/referenceable/DestinationObjectFactory.java b/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/referenceable/DestinationObjectFactory.java
deleted file mode 100644
index 896e815..0000000
--- a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/referenceable/DestinationObjectFactory.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.jms.referenceable;
-
-import javax.naming.Context;
-import javax.naming.Name;
-import javax.naming.Reference;
-import javax.naming.spi.ObjectFactory;
-import java.util.Hashtable;
-
-/**
- * A DestinationObjectFactory.
- *
- * Given a Reference - reconstructs an ActiveMQDestination
- */
-public class DestinationObjectFactory implements ObjectFactory {
-
-   @Override
-   public Object getObjectInstance(final Object ref,
-                                   final Name name,
-                                   final Context ctx,
-                                   final Hashtable<?, ?> props) throws Exception {
-      Reference r = (Reference) ref;
-
-      byte[] bytes = (byte[]) r.get("ActiveMQ-DEST").getContent();
-
-      // Deserialize
-      return SerializableObjectRefAddr.deserialize(bytes);
-   }
-}

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3ef98044/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/referenceable/SerializableObjectRefAddr.java
----------------------------------------------------------------------
diff --git a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/referenceable/SerializableObjectRefAddr.java b/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/referenceable/SerializableObjectRefAddr.java
deleted file mode 100644
index 09c971d..0000000
--- a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jms/referenceable/SerializableObjectRefAddr.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * 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.jms.referenceable;
-
-import javax.naming.NamingException;
-import javax.naming.RefAddr;
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-
-/**
- * A SerializableObjectRefAddr.
- *
- * A RefAddr that can be used for any serializable object.
- *
- * Basically the address is the serialized form of the object as a byte[]
- */
-public class SerializableObjectRefAddr extends RefAddr {
-
-   private static final long serialVersionUID = 9158134548376171898L;
-
-   private final byte[] bytes;
-
-   public SerializableObjectRefAddr(final String type, final Object content) throws NamingException {
-      super(type);
-
-      try {
-         // Serialize the object
-         ByteArrayOutputStream bos = new ByteArrayOutputStream();
-
-         ObjectOutputStream oos = new ObjectOutputStream(bos);
-
-         oos.writeObject(content);
-
-         oos.flush();
-
-         bytes = bos.toByteArray();
-      } catch (IOException e) {
-         throw new NamingException("Failed to serialize object:" + content + ", " + e.getMessage());
-      }
-   }
-
-   @Override
-   public Object getContent() {
-      return bytes;
-   }
-
-   public static Object deserialize(final byte[] bytes) throws IOException, ClassNotFoundException {
-      ByteArrayInputStream bis = new ByteArrayInputStream(bytes);
-
-      ObjectInputStream ois = new ObjectInputStream(bis);
-
-      return ois.readObject();
-   }
-}

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3ef98044/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jndi/JNDIReferenceFactory.java
----------------------------------------------------------------------
diff --git a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jndi/JNDIReferenceFactory.java b/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jndi/JNDIReferenceFactory.java
index c226760..0b4b0c2 100644
--- a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jndi/JNDIReferenceFactory.java
+++ b/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jndi/JNDIReferenceFactory.java
@@ -19,14 +19,13 @@ package org.apache.activemq.artemis.jndi;
 import javax.naming.Context;
 import javax.naming.Name;
 import javax.naming.NamingException;
-import javax.naming.RefAddr;
 import javax.naming.Reference;
 import javax.naming.StringRefAddr;
 import javax.naming.spi.ObjectFactory;
 import java.util.Enumeration;
-import java.util.HashMap;
 import java.util.Hashtable;
-import java.util.Map;
+import java.util.Properties;
+
 
 /**
  * Converts objects implementing JNDIStorable into a property fields so they can be
@@ -63,12 +62,7 @@ public class JNDIReferenceFactory implements ObjectFactory {
          Class<?> theClass = loadClass(this, reference.getClassName());
          if (JNDIStorable.class.isAssignableFrom(theClass)) {
             JNDIStorable store = (JNDIStorable) theClass.newInstance();
-            Map<String, String> properties = new HashMap<>();
-            for (Enumeration<RefAddr> iter = reference.getAll(); iter.hasMoreElements();) {
-               StringRefAddr addr = (StringRefAddr) iter.nextElement();
-               properties.put(addr.getType(), (addr.getContent() == null) ? "" : addr.getContent().toString());
-            }
-            store.setProperties(properties);
+            store.setProperties(getProperties(reference));
             result = store;
          }
       } else {
@@ -77,6 +71,15 @@ public class JNDIReferenceFactory implements ObjectFactory {
       return result;
    }
 
+   public static Properties getProperties(Reference reference) {
+      Properties properties = new Properties();
+      for (Enumeration iter = reference.getAll(); iter.hasMoreElements();) {
+         StringRefAddr addr = (StringRefAddr)iter.nextElement();
+         properties.put(addr.getType(), (addr.getContent() == null) ? "" : addr.getContent());
+      }
+      return properties;
+   }
+
    /**
     * Create a Reference instance from a JNDIStorable object
     *
@@ -92,10 +95,10 @@ public class JNDIReferenceFactory implements ObjectFactory {
    public static Reference createReference(String instanceClassName, JNDIStorable po) throws NamingException {
       Reference result = new Reference(instanceClassName, JNDIReferenceFactory.class.getName(), null);
       try {
-         Map<String, String> props = po.getProperties();
-         for (Map.Entry<String, String> entry : props.entrySet()) {
-            StringRefAddr addr = new StringRefAddr(entry.getKey(), entry.getValue());
-            result.add(addr);
+         Properties props = po.getProperties();
+         for (Enumeration iter = props.propertyNames(); iter.hasMoreElements();) {
+            String key = (String)iter.nextElement();
+            result.add(new StringRefAddr(key, props.getProperty(key)));
          }
       } catch (Exception e) {
          throw new NamingException(e.getMessage());

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3ef98044/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jndi/JNDIStorable.java
----------------------------------------------------------------------
diff --git a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jndi/JNDIStorable.java b/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jndi/JNDIStorable.java
index 2c26b0a..b25dcf2 100644
--- a/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jndi/JNDIStorable.java
+++ b/artemis-jms-client/src/main/java/org/apache/activemq/artemis/jndi/JNDIStorable.java
@@ -16,32 +16,27 @@
  */
 package org.apache.activemq.artemis.jndi;
 
-import javax.naming.NamingException;
-import javax.naming.Reference;
-import javax.naming.Referenceable;
-import java.io.Externalizable;
 import java.io.IOException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
-import java.util.LinkedHashMap;
-import java.util.Map;
+import java.util.Properties;
+
+import javax.naming.NamingException;
+import javax.naming.Reference;
+import javax.naming.Referenceable;
 
 /**
  * Facilitates objects to be stored in JNDI as properties
- *
- * @since 1.0
  */
-public abstract class JNDIStorable implements Referenceable, Externalizable {
+public abstract class JNDIStorable implements Referenceable {
 
    /**
     * Set the properties that will represent the instance in JNDI
     *
     * @param props
     *     The properties to use when building the new isntance.
-    *
-    * @return a new, unmodifiable, map containing any unused properties, or empty if none were.
     */
-   protected abstract Map<String, String> buildFromProperties(Map<String, String> props);
+   protected abstract void buildFromProperties(Properties props);
 
    /**
     * Initialize the instance from properties stored in JNDI
@@ -49,7 +44,7 @@ public abstract class JNDIStorable implements Referenceable, Externalizable {
     * @param props
     *     The properties to use when initializing the new instance.
     */
-   protected abstract void populateProperties(Map<String, String> props);
+   protected abstract void populateProperties(Properties props);
 
    /**
     * set the properties for this instance as retrieved from JNDI
@@ -59,8 +54,8 @@ public abstract class JNDIStorable implements Referenceable, Externalizable {
     *
     * @return a new, unmodifiable, map containing any unused properties, or empty if none were.
     */
-   public synchronized Map<String, String> setProperties(Map<String, String> props) {
-      return buildFromProperties(props);
+   synchronized void setProperties(Properties props) {
+      buildFromProperties(props);
    }
 
    /**
@@ -68,8 +63,8 @@ public abstract class JNDIStorable implements Referenceable, Externalizable {
     *
     * @return the properties
     */
-   public synchronized Map<String, String> getProperties() {
-      Map<String, String> properties = new LinkedHashMap<>();
+   synchronized Properties getProperties() {
+      Properties properties = new Properties();
       populateProperties(properties);
       return properties;
    }
@@ -87,30 +82,29 @@ public abstract class JNDIStorable implements Referenceable, Externalizable {
    }
 
    /**
-    * @see Externalizable#readExternal(ObjectInput)
+    * Method for class's implementing externalizable to delegate to if not custom implementing.
+    *
+    * @param in
+    * @throws IOException
+    * @throws ClassNotFoundException
+    * @see java.io.Externalizable#readExternal(java.io.ObjectInput)
     */
-   @SuppressWarnings("unchecked")
-   @Override
-   public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-      Map<String, String> props = (Map<String, String>) in.readObject();
+   public void readObject(ObjectInput in) throws IOException, ClassNotFoundException {
+      Properties props = (Properties)in.readObject();
       if (props != null) {
          setProperties(props);
       }
    }
 
    /**
-    * @see Externalizable#writeExternal(ObjectOutput)
+    * Method for class's implementing externalizable to delegate to if not custom implementing.
+    *
+    * @param out
+    * @throws IOException
+    * @see java.io.Externalizable#writeExternal(java.io.ObjectOutput)
     */
-   @Override
-   public void writeExternal(ObjectOutput out) throws IOException {
+   public void writeObject(ObjectOutput out) throws IOException {
       out.writeObject(getProperties());
    }
 
-   protected String getProperty(Map<String, String> map, String key, String defaultValue) {
-      String value = map.get(key);
-      if (value != null) {
-         return value;
-      }
-      return defaultValue;
-   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3ef98044/artemis-protocols/artemis-amqp-protocol/src/main/java/org/apache/activemq/artemis/protocol/amqp/converter/jms/ServerDestination.java
----------------------------------------------------------------------
diff --git a/artemis-protocols/artemis-amqp-protocol/src/main/java/org/apache/activemq/artemis/protocol/amqp/converter/jms/ServerDestination.java b/artemis-protocols/artemis-amqp-protocol/src/main/java/org/apache/activemq/artemis/protocol/amqp/converter/jms/ServerDestination.java
index cee9ee5..5a2f55b 100644
--- a/artemis-protocols/artemis-amqp-protocol/src/main/java/org/apache/activemq/artemis/protocol/amqp/converter/jms/ServerDestination.java
+++ b/artemis-protocols/artemis-amqp-protocol/src/main/java/org/apache/activemq/artemis/protocol/amqp/converter/jms/ServerDestination.java
@@ -19,6 +19,7 @@ package org.apache.activemq.artemis.protocol.amqp.converter.jms;
 import javax.jms.JMSException;
 import javax.jms.Queue;
 
+import org.apache.activemq.artemis.api.core.SimpleString;
 import org.apache.activemq.artemis.jms.client.ActiveMQDestination;
 
 /**
@@ -27,8 +28,12 @@ import org.apache.activemq.artemis.jms.client.ActiveMQDestination;
  */
 public class ServerDestination extends ActiveMQDestination implements Queue {
 
-   public ServerDestination(String name) {
-      super(name, name, TYPE.DESTINATION, null);
+   public ServerDestination(String address) {
+      super(address, TYPE.DESTINATION, null);
+   }
+
+   public ServerDestination(SimpleString address) {
+      super(address, TYPE.DESTINATION, null);
    }
 
    @Override

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3ef98044/artemis-protocols/artemis-amqp-protocol/src/main/java/org/apache/activemq/artemis/protocol/amqp/converter/jms/ServerJMSMessage.java
----------------------------------------------------------------------
diff --git a/artemis-protocols/artemis-amqp-protocol/src/main/java/org/apache/activemq/artemis/protocol/amqp/converter/jms/ServerJMSMessage.java b/artemis-protocols/artemis-amqp-protocol/src/main/java/org/apache/activemq/artemis/protocol/amqp/converter/jms/ServerJMSMessage.java
index 5962e39..b070591 100644
--- a/artemis-protocols/artemis-amqp-protocol/src/main/java/org/apache/activemq/artemis/protocol/amqp/converter/jms/ServerJMSMessage.java
+++ b/artemis-protocols/artemis-amqp-protocol/src/main/java/org/apache/activemq/artemis/protocol/amqp/converter/jms/ServerJMSMessage.java
@@ -139,7 +139,7 @@ public class ServerJMSMessage implements Message {
    public final Destination getJMSReplyTo() throws JMSException {
       SimpleString reply = MessageUtil.getJMSReplyTo(message);
       if (reply != null) {
-         return new ServerDestination(reply.toString());
+         return new ServerDestination(reply);
       } else {
          return null;
       }
@@ -148,7 +148,6 @@ public class ServerJMSMessage implements Message {
    @Override
    public final void setJMSReplyTo(Destination replyTo) throws JMSException {
       MessageUtil.setJMSReplyTo(message, replyTo == null ? null : ((ActiveMQDestination) replyTo).getSimpleAddress());
-
    }
 
    @Override
@@ -158,7 +157,7 @@ public class ServerJMSMessage implements Message {
       if (sdest == null) {
          return null;
       } else {
-         return new ServerDestination(sdest.toString());
+         return new ServerDestination(sdest);
       }
    }
 

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3ef98044/artemis-ra/src/main/java/org/apache/activemq/artemis/jms/referenceable/ConnectionFactoryObjectFactory.java
----------------------------------------------------------------------
diff --git a/artemis-ra/src/main/java/org/apache/activemq/artemis/jms/referenceable/ConnectionFactoryObjectFactory.java b/artemis-ra/src/main/java/org/apache/activemq/artemis/jms/referenceable/ConnectionFactoryObjectFactory.java
new file mode 100644
index 0000000..4c7a68a
--- /dev/null
+++ b/artemis-ra/src/main/java/org/apache/activemq/artemis/jms/referenceable/ConnectionFactoryObjectFactory.java
@@ -0,0 +1,25 @@
+/*
+ * 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.jms.referenceable;
+
+/**
+ * Done for back compatibility with the package/class move.
+ *
+ * Should be removed on next major version change.
+ */
+public class ConnectionFactoryObjectFactory extends org.apache.activemq.artemis.ra.referenceable.ActiveMQRAConnectionFactoryObjectFactory {
+}

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3ef98044/artemis-ra/src/main/java/org/apache/activemq/artemis/jms/referenceable/SerializableObjectRefAddr.java
----------------------------------------------------------------------
diff --git a/artemis-ra/src/main/java/org/apache/activemq/artemis/jms/referenceable/SerializableObjectRefAddr.java b/artemis-ra/src/main/java/org/apache/activemq/artemis/jms/referenceable/SerializableObjectRefAddr.java
new file mode 100644
index 0000000..92aaf9e
--- /dev/null
+++ b/artemis-ra/src/main/java/org/apache/activemq/artemis/jms/referenceable/SerializableObjectRefAddr.java
@@ -0,0 +1,31 @@
+/*
+ * 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.jms.referenceable;
+
+import javax.naming.NamingException;
+
+/**
+ * Done for back compatibility with the package/class move.
+ *
+ * Should be removed on next major version change.
+ */
+public class SerializableObjectRefAddr extends org.apache.activemq.artemis.ra.referenceable.SerializableObjectRefAddr {
+
+   public SerializableObjectRefAddr(final String type, final Object content) throws NamingException {
+      super(type, content);
+   }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3ef98044/artemis-ra/src/main/java/org/apache/activemq/artemis/ra/ActiveMQRAConnectionFactoryImpl.java
----------------------------------------------------------------------
diff --git a/artemis-ra/src/main/java/org/apache/activemq/artemis/ra/ActiveMQRAConnectionFactoryImpl.java b/artemis-ra/src/main/java/org/apache/activemq/artemis/ra/ActiveMQRAConnectionFactoryImpl.java
index faf111e..e3c4c9d 100644
--- a/artemis-ra/src/main/java/org/apache/activemq/artemis/ra/ActiveMQRAConnectionFactoryImpl.java
+++ b/artemis-ra/src/main/java/org/apache/activemq/artemis/ra/ActiveMQRAConnectionFactoryImpl.java
@@ -35,8 +35,8 @@ import javax.resource.ResourceException;
 import javax.resource.spi.ConnectionManager;
 
 import org.apache.activemq.artemis.jms.client.ActiveMQConnectionFactory;
-import org.apache.activemq.artemis.jms.referenceable.ConnectionFactoryObjectFactory;
-import org.apache.activemq.artemis.jms.referenceable.SerializableObjectRefAddr;
+import org.apache.activemq.artemis.ra.referenceable.ActiveMQRAConnectionFactoryObjectFactory;
+import org.apache.activemq.artemis.ra.referenceable.SerializableObjectRefAddr;
 
 /**
  * The connection factory
@@ -118,7 +118,7 @@ public class ActiveMQRAConnectionFactoryImpl implements ActiveMQRAConnectionFact
       }
       if (reference == null) {
          try {
-            reference = new Reference(this.getClass().getCanonicalName(), new SerializableObjectRefAddr("ActiveMQ-CF", this), ConnectionFactoryObjectFactory.class.getCanonicalName(), null);
+            reference = new Reference(this.getClass().getCanonicalName(), new SerializableObjectRefAddr("ActiveMQ-CF", this), ActiveMQRAConnectionFactoryObjectFactory.class.getCanonicalName(), null);
          } catch (NamingException e) {
             ActiveMQRALogger.LOGGER.errorCreatingReference(e);
          }

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3ef98044/artemis-ra/src/main/java/org/apache/activemq/artemis/ra/referenceable/ActiveMQRAConnectionFactoryObjectFactory.java
----------------------------------------------------------------------
diff --git a/artemis-ra/src/main/java/org/apache/activemq/artemis/ra/referenceable/ActiveMQRAConnectionFactoryObjectFactory.java b/artemis-ra/src/main/java/org/apache/activemq/artemis/ra/referenceable/ActiveMQRAConnectionFactoryObjectFactory.java
new file mode 100644
index 0000000..ab4e99d
--- /dev/null
+++ b/artemis-ra/src/main/java/org/apache/activemq/artemis/ra/referenceable/ActiveMQRAConnectionFactoryObjectFactory.java
@@ -0,0 +1,44 @@
+/*
+ * 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.ra.referenceable;
+
+import javax.naming.Context;
+import javax.naming.Name;
+import javax.naming.Reference;
+import javax.naming.spi.ObjectFactory;
+import java.util.Hashtable;
+
+/**
+ * A ConnectionFactoryObjectFactory.
+ *
+ * Given a reference - reconstructs an ActiveMQRAConnectionFactory
+ */
+public class ActiveMQRAConnectionFactoryObjectFactory implements ObjectFactory {
+
+   @Override
+   public Object getObjectInstance(final Object ref,
+                                   final Name name,
+                                   final Context ctx,
+                                   final Hashtable<?, ?> props) throws Exception {
+      Reference r = (Reference) ref;
+
+      byte[] bytes = (byte[]) r.get("ActiveMQ-CF").getContent();
+
+      // Deserialize
+      return SerializableObjectRefAddr.deserialize(bytes);
+   }
+}

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3ef98044/artemis-ra/src/main/java/org/apache/activemq/artemis/ra/referenceable/SerializableObjectRefAddr.java
----------------------------------------------------------------------
diff --git a/artemis-ra/src/main/java/org/apache/activemq/artemis/ra/referenceable/SerializableObjectRefAddr.java b/artemis-ra/src/main/java/org/apache/activemq/artemis/ra/referenceable/SerializableObjectRefAddr.java
new file mode 100644
index 0000000..b4fce9f
--- /dev/null
+++ b/artemis-ra/src/main/java/org/apache/activemq/artemis/ra/referenceable/SerializableObjectRefAddr.java
@@ -0,0 +1,71 @@
+/*
+ * 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.ra.referenceable;
+
+import javax.naming.NamingException;
+import javax.naming.RefAddr;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+
+/**
+ * A SerializableObjectRefAddr.
+ *
+ * A RefAddr that can be used for any serializable object.
+ *
+ * Basically the address is the serialized form of the object as a byte[]
+ */
+public class SerializableObjectRefAddr extends RefAddr {
+
+   private static final long serialVersionUID = 9158134548376171898L;
+
+   private final byte[] bytes;
+
+   public SerializableObjectRefAddr(final String type, final Object content) throws NamingException {
+      super(type);
+
+      try {
+         // Serialize the object
+         ByteArrayOutputStream bos = new ByteArrayOutputStream();
+
+         ObjectOutputStream oos = new ObjectOutputStream(bos);
+
+         oos.writeObject(content);
+
+         oos.flush();
+
+         bytes = bos.toByteArray();
+      } catch (IOException e) {
+         throw new NamingException("Failed to serialize object:" + content + ", " + e.getMessage());
+      }
+   }
+
+   @Override
+   public Object getContent() {
+      return bytes;
+   }
+
+   public static Object deserialize(final byte[] bytes) throws IOException, ClassNotFoundException {
+      ByteArrayInputStream bis = new ByteArrayInputStream(bytes);
+
+      ObjectInputStream ois = new ObjectInputStream(bis);
+
+      return ois.readObject();
+   }
+}

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3ef98044/docs/user-manual/en/SUMMARY.md
----------------------------------------------------------------------
diff --git a/docs/user-manual/en/SUMMARY.md b/docs/user-manual/en/SUMMARY.md
index 1a2f531..ba488f3 100644
--- a/docs/user-manual/en/SUMMARY.md
+++ b/docs/user-manual/en/SUMMARY.md
@@ -56,6 +56,7 @@
 * [REST Interface](rest.md)
 * [Embedding Apache ActiveMQ Artemis](embedding-activemq.md)
 * [Apache Karaf](karaf.md)
+* [Apache Tomcat](tomcat.md)
 * [Spring Integration](spring-integration.md)
 * [CDI Integration](cdi-integration.md)
 * [Intercepting Operations](intercepting-operations.md)

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3ef98044/docs/user-manual/en/tomcat.md
----------------------------------------------------------------------
diff --git a/docs/user-manual/en/tomcat.md b/docs/user-manual/en/tomcat.md
new file mode 100644
index 0000000..ac6befa
--- /dev/null
+++ b/docs/user-manual/en/tomcat.md
@@ -0,0 +1,39 @@
+# Apache ActiveMQ Artemis - Apache Tomcat Support
+
+
+##Apache Tomcat resource context client configuration
+
+Apache ActiveMQ Artemis provides support for configuring the client, in the tomcat resource context.xml of Tomcat container.
+
+This is very similar to the way this is done in ActiveMQ 5.x so anyone migrating should find this familiar.
+Please note though the connection url and properties that can be set for ActiveMQ Artemis are different please see [Migration Documentation](https://activemq.apache.org/artemis/migration/)
+
+#### Example of Connection Factory
+````
+<Context>
+    ...
+  <Resource name="jms/ConnectionFactory" auth="Container" type="org.apache.activemq.artemis.jms.client.ActiveMQConnectionFactory" description="JMS Connection Factory"
+        factory="org.apache.activemq.artemis.jndi.JNDIReferenceFactory" brokerURL="tcp://localhost:61616" />
+    ...
+</Context>
+````
+
+#### Example of Destination (Queue and Topic)
+
+````
+<Context>
+  ...
+  <Resource name="jms/ExampleQueue" auth="Container" type="org.apache.activemq.artemis.jms.client.ActiveMQQueue" description="JMS Queue"
+        factory="org.apache.activemq.artemis.jndi.JNDIReferenceFactory" address="ExampleQueue" />
+  ...
+  <Resource name="jms/ExampleTopic" auth="Container" type="org.apache.activemq.artemis.jms.client.ActiveMQTopic" description="JMS Topic"
+         factory="org.apache.activemq.artemis.jndi.JNDIReferenceFactory" address="ExampleTopic" />
+  ...
+</Context>
+````
+
+#### Example Tomcat App
+
+A sample tomcat app with the container context configured as an example can be seen here: 
+
+/examples/features/sub-modules/tomcat
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3ef98044/examples/features/sub-modules/pom.xml
----------------------------------------------------------------------
diff --git a/examples/features/sub-modules/pom.xml b/examples/features/sub-modules/pom.xml
index ce20c04..bb6aeaa 100644
--- a/examples/features/sub-modules/pom.xml
+++ b/examples/features/sub-modules/pom.xml
@@ -50,7 +50,8 @@ under the License.
          <id>release</id>
          <modules>
             <module>artemis-ra-rar</module>
-	         <module>inter-broker-bridge</module>
+            <module>inter-broker-bridge</module>
+            <module>tomcat</module>
          </modules>
       </profile>
    </profiles>

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3ef98044/examples/features/sub-modules/tomcat/README
----------------------------------------------------------------------
diff --git a/examples/features/sub-modules/tomcat/README b/examples/features/sub-modules/tomcat/README
new file mode 100644
index 0000000..dfc8462
--- /dev/null
+++ b/examples/features/sub-modules/tomcat/README
@@ -0,0 +1,21 @@
+# Apache ActiveMQ Artemis Tomcat Integration JNDI Resources Sample
+
+This is a Sample Tomcat application showing JNDI resource in tomcat context.xml for ConnectionFactory 
+and Destination for Apache ActiveMQ Artemis.
+
+The sample context.xml used by the tomcat in this example can be found under:
+/src/tomcat7-maven-plugin/resources/context.xml
+
+To run 
+
+start Apache ActiveMQ Artemis on port 61616
+ 
+then
+
+mvn clean install tomcat7:exec-war
+
+then to send message 
+
+http://localhost:8080/tomcat-sample/send
+
+this will cause a "hello world" message to send, and the consumer should consume and output it to sysout.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3ef98044/examples/features/sub-modules/tomcat/pom.xml
----------------------------------------------------------------------
diff --git a/examples/features/sub-modules/tomcat/pom.xml b/examples/features/sub-modules/tomcat/pom.xml
new file mode 100644
index 0000000..8773f3b
--- /dev/null
+++ b/examples/features/sub-modules/tomcat/pom.xml
@@ -0,0 +1,127 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.activemq.examples.modules</groupId>
+        <artifactId>broker-modules</artifactId>
+        <version>2.5.0-SNAPSHOT</version>
+    </parent>
+
+
+    <artifactId>artemis-tomcat-jndi-resources-sample</artifactId>
+    <packaging>war</packaging>
+    <name>Artemis Tomcat JNDI Resources Example</name>
+
+    <properties>
+        <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+        <projectBaseUri>${project.baseUri}</projectBaseUri>
+        <java.version>1.6</java.version>
+
+        <servlet-api.version>3.0-alpha-1</servlet-api.version>
+    </properties>
+
+    <dependencies>
+
+        <!-- Spring Dependencies -->
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-context</artifactId>
+            <version>${spring.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-webmvc</artifactId>
+            <version>${spring.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-jms</artifactId>
+            <version>${spring.version}</version>
+        </dependency>
+        
+        
+        <dependency>
+            <groupId>org.apache.activemq</groupId>
+            <artifactId>artemis-jms-client</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>javax.servlet</groupId>
+            <artifactId>servlet-api</artifactId>
+            <version>${servlet-api.version}</version>
+            <scope>provided</scope>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-compiler-plugin</artifactId>
+                <version>2.3.2</version>
+                <configuration>
+                    <source>${java.version}</source>
+                    <target>${java.version}</target>
+                </configuration>
+            </plugin>
+
+            <plugin>
+                <groupId>org.apache.tomcat.maven</groupId>
+                <artifactId>tomcat7-maven-plugin</artifactId>
+                <version>2.1</version>
+
+                <configuration>
+                    <contextFile>${project.basedir}\src\tomcat7-maven-plugin\resources\context.xml</contextFile>
+                </configuration>
+                <executions>
+                    <execution>
+                        <id>tomcat-war-exec</id>
+                        <goals>
+                            <goal>exec-war</goal>
+                        </goals>
+                        <phase>package</phase>
+                        <configuration>
+                            <warRunDependencies>
+                                <warRunDependency>
+                                    <dependency>
+                                        <groupId>org.apache.activemq.examples.modules</groupId>
+                                        <artifactId>artemis-tomcat-jndi-resources-sample</artifactId>
+                                        <version>${project.version}</version>
+                                        <type>war</type>
+                                    </dependency>
+                                </warRunDependency>
+                            </warRunDependencies>
+                        </configuration>
+                    </execution>
+                </executions>
+
+                <!--<configuration>-->
+                <!--<port>9090</port>-->
+                <!--<path>/</path>-->
+                <!--</configuration>-->
+            </plugin>
+        </plugins>
+    </build>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3ef98044/examples/features/sub-modules/tomcat/src/main/java/org/apache/activemq/artemis/example/tomcat/sample/SendMessageController.java
----------------------------------------------------------------------
diff --git a/examples/features/sub-modules/tomcat/src/main/java/org/apache/activemq/artemis/example/tomcat/sample/SendMessageController.java b/examples/features/sub-modules/tomcat/src/main/java/org/apache/activemq/artemis/example/tomcat/sample/SendMessageController.java
new file mode 100644
index 0000000..eb36179
--- /dev/null
+++ b/examples/features/sub-modules/tomcat/src/main/java/org/apache/activemq/artemis/example/tomcat/sample/SendMessageController.java
@@ -0,0 +1,48 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.example.tomcat.sample;
+
+import javax.jms.JMSException;
+import javax.jms.Message;
+import javax.jms.Session;
+
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.jms.core.JmsTemplate;
+import org.springframework.jms.core.MessageCreator;
+import org.springframework.web.bind.annotation.RequestMapping;
+import org.springframework.web.bind.annotation.RequestParam;
+import org.springframework.web.bind.annotation.ResponseBody;
+import org.springframework.web.bind.annotation.RestController;
+
+@RestController
+public class SendMessageController {
+    
+    @Autowired
+    private JmsTemplate jmsTemplate;
+    
+    
+    @RequestMapping("/send")
+    public @ResponseBody String send(@RequestParam(value="text", defaultValue="hello world") final String text) {
+        jmsTemplate.send(new MessageCreator() {
+            @Override
+            public Message createMessage(Session session) throws JMSException {
+                return session.createTextMessage(text);
+            }
+        });
+        return "sent: " + text;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3ef98044/examples/features/sub-modules/tomcat/src/main/java/org/apache/activemq/artemis/example/tomcat/sample/SystemOutPrintLnMessageListener.java
----------------------------------------------------------------------
diff --git a/examples/features/sub-modules/tomcat/src/main/java/org/apache/activemq/artemis/example/tomcat/sample/SystemOutPrintLnMessageListener.java b/examples/features/sub-modules/tomcat/src/main/java/org/apache/activemq/artemis/example/tomcat/sample/SystemOutPrintLnMessageListener.java
new file mode 100644
index 0000000..1f57c13
--- /dev/null
+++ b/examples/features/sub-modules/tomcat/src/main/java/org/apache/activemq/artemis/example/tomcat/sample/SystemOutPrintLnMessageListener.java
@@ -0,0 +1,28 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.example.tomcat.sample;
+
+import javax.jms.Message;
+import javax.jms.MessageListener;
+
+
+public class SystemOutPrintLnMessageListener implements MessageListener {
+   @Override
+   public void onMessage(Message message) {
+      System.out.println(message);
+   }
+}

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3ef98044/examples/features/sub-modules/tomcat/src/main/webapp/WEB-INF/application-context.xml
----------------------------------------------------------------------
diff --git a/examples/features/sub-modules/tomcat/src/main/webapp/WEB-INF/application-context.xml b/examples/features/sub-modules/tomcat/src/main/webapp/WEB-INF/application-context.xml
new file mode 100644
index 0000000..ffd4728
--- /dev/null
+++ b/examples/features/sub-modules/tomcat/src/main/webapp/WEB-INF/application-context.xml
@@ -0,0 +1,50 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed 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.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:context="http://www.springframework.org/schema/context"
+       xmlns:jee="http://www.springframework.org/schema/jee"
+       xmlns:mvc="http://www.springframework.org/schema/mvc"
+       xsi:schemaLocation="http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/context
+        http://www.springframework.org/schema/context/spring-context.xsd
+        http://www.springframework.org/schema/mvc
+        http://www.springframework.org/schema/mvc/spring-mvc.xsd
+        http://www.springframework.org/schema/jee
+        http://www.springframework.org/schema/jee/spring-jee.xsd">
+
+
+    <jee:jndi-lookup id="connectionFactory" jndi-name="java:comp/env/jms/connectionFactory" />
+    <jee:jndi-lookup id="InputQueue" jndi-name="java:comp/env/jms/InputQueue" />
+
+    <bean id="InputListener" class="org.apache.activemq.artemis.example.tomcat.sample.SystemOutPrintLnMessageListener" />
+    
+    <bean id="jmsTemplate" class="org.springframework.jms.core.JmsTemplate" >
+        <property name="connectionFactory">
+            <bean class="org.springframework.jms.connection.CachingConnectionFactory" >
+                <property name="targetConnectionFactory" ref="connectionFactory" />
+            </bean>
+        </property>
+        <property name="defaultDestination" ref="InputQueue" />
+    </bean>
+    
+    <bean id="messageListenerContainer"
+          class="org.springframework.jms.listener.DefaultMessageListenerContainer">
+        <property name="connectionFactory" ref="connectionFactory" />
+        <property name="destination" ref="InputQueue" />
+        <property name="messageListener" ref="InputListener" />
+    </bean>
+</beans>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3ef98044/examples/features/sub-modules/tomcat/src/main/webapp/WEB-INF/dispatcher-servlet.xml
----------------------------------------------------------------------
diff --git a/examples/features/sub-modules/tomcat/src/main/webapp/WEB-INF/dispatcher-servlet.xml b/examples/features/sub-modules/tomcat/src/main/webapp/WEB-INF/dispatcher-servlet.xml
new file mode 100644
index 0000000..b1da14d
--- /dev/null
+++ b/examples/features/sub-modules/tomcat/src/main/webapp/WEB-INF/dispatcher-servlet.xml
@@ -0,0 +1,28 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed 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.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:context="http://www.springframework.org/schema/context"
+       xmlns:mvc="http://www.springframework.org/schema/mvc"
+       xsi:schemaLocation="http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/context
+        http://www.springframework.org/schema/context/spring-context.xsd
+        http://www.springframework.org/schema/mvc
+    	http://www.springframework.org/schema/mvc/spring-mvc.xsd">
+    
+    <context:component-scan base-package="org.apache.activemq.artemis.example.tomcat.sample" />
+    <mvc:annotation-driven />
+</beans>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3ef98044/examples/features/sub-modules/tomcat/src/main/webapp/WEB-INF/web.xml
----------------------------------------------------------------------
diff --git a/examples/features/sub-modules/tomcat/src/main/webapp/WEB-INF/web.xml b/examples/features/sub-modules/tomcat/src/main/webapp/WEB-INF/web.xml
new file mode 100644
index 0000000..974c3e4
--- /dev/null
+++ b/examples/features/sub-modules/tomcat/src/main/webapp/WEB-INF/web.xml
@@ -0,0 +1,41 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed 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.
+-->
+<web-app xmlns="http://java.sun.com/xml/ns/javaee" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://java.sun.com/xml/ns/javaee http://java.sun.com/xml/ns/javaee/web-app_3_0.xsd"
+         version="3.0">
+
+
+    <listener>
+        <listener-class>org.springframework.web.context.ContextLoaderListener</listener-class>
+    </listener>
+
+
+    <context-param>
+        <param-name>contextConfigLocation</param-name>
+        <param-value>WEB-INF/application-context.xml</param-value>
+    </context-param>
+
+    <servlet>
+        <servlet-name>dispatcher</servlet-name>
+        <servlet-class>org.springframework.web.servlet.DispatcherServlet</servlet-class>
+    </servlet>
+
+    <servlet-mapping>
+        <servlet-name>dispatcher</servlet-name>
+        <url-pattern>/*</url-pattern>
+    </servlet-mapping>
+    
+    
+</web-app>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3ef98044/examples/features/sub-modules/tomcat/src/tomcat7-maven-plugin/resources/context.xml
----------------------------------------------------------------------
diff --git a/examples/features/sub-modules/tomcat/src/tomcat7-maven-plugin/resources/context.xml b/examples/features/sub-modules/tomcat/src/tomcat7-maven-plugin/resources/context.xml
new file mode 100644
index 0000000..9e78cca
--- /dev/null
+++ b/examples/features/sub-modules/tomcat/src/tomcat7-maven-plugin/resources/context.xml
@@ -0,0 +1,28 @@
+<!--
+Licensed 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.
+-->
+<Context>
+    <Resource name="jms/connectionFactory"
+              auth="Container"
+              type="org.apache.activemq.artemis.jms.client.ActiveMQConnectionFactory"
+              description="JMS Connection Factory"
+              factory="org.apache.activemq.artemis.jndi.JNDIReferenceFactory"
+              brokerURL="(tcp://localhost:61616)?ha=true&amp;useKQueue=false&amp;useEpoll=false"/>
+    
+    <Resource name="jms/InputQueue"
+              auth="Container"
+              type="org.apache.activemq.artemis.jms.client.ActiveMQQueue"
+              factory="org.apache.activemq.artemis.jndi.JNDIReferenceFactory"
+              address="InputQueue"/>
+
+</Context>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3ef98044/tests/jms-tests/src/test/java/org/apache/activemq/artemis/jms/tests/ReferenceableTest.java
----------------------------------------------------------------------
diff --git a/tests/jms-tests/src/test/java/org/apache/activemq/artemis/jms/tests/ReferenceableTest.java b/tests/jms-tests/src/test/java/org/apache/activemq/artemis/jms/tests/ReferenceableTest.java
index 2cebf04..1795193 100644
--- a/tests/jms-tests/src/test/java/org/apache/activemq/artemis/jms/tests/ReferenceableTest.java
+++ b/tests/jms-tests/src/test/java/org/apache/activemq/artemis/jms/tests/ReferenceableTest.java
@@ -25,6 +25,7 @@ import javax.jms.Session;
 import javax.jms.TextMessage;
 import javax.naming.Reference;
 import javax.naming.Referenceable;
+import javax.naming.spi.ObjectFactory;
 import java.io.Serializable;
 
 import org.apache.activemq.artemis.jms.client.ActiveMQConnectionFactory;
@@ -32,8 +33,6 @@ import org.apache.activemq.artemis.jms.client.ActiveMQDestination;
 import org.apache.activemq.artemis.jms.client.ActiveMQJMSConnectionFactory;
 import org.apache.activemq.artemis.jms.client.ActiveMQQueue;
 import org.apache.activemq.artemis.jms.client.ActiveMQTopic;
-import org.apache.activemq.artemis.jms.referenceable.ConnectionFactoryObjectFactory;
-import org.apache.activemq.artemis.jms.referenceable.DestinationObjectFactory;
 import org.apache.activemq.artemis.jms.tests.util.ProxyAssertSupport;
 import org.junit.Test;
 
@@ -81,7 +80,7 @@ public class ReferenceableTest extends JMSTestCase {
 
       Class<?> factoryClass = Class.forName(factoryName);
 
-      ConnectionFactoryObjectFactory factory = (ConnectionFactoryObjectFactory) factoryClass.newInstance();
+      ObjectFactory factory = (ObjectFactory) factoryClass.newInstance();
 
       Object instance = factory.getObjectInstance(cfRef, null, null, null);
 
@@ -100,7 +99,7 @@ public class ReferenceableTest extends JMSTestCase {
 
       Class<?> factoryClass = Class.forName(factoryName);
 
-      DestinationObjectFactory factory = (DestinationObjectFactory) factoryClass.newInstance();
+      ObjectFactory factory = (ObjectFactory) factoryClass.newInstance();
 
       Object instance = factory.getObjectInstance(queueRef, null, null, null);
 
@@ -121,7 +120,7 @@ public class ReferenceableTest extends JMSTestCase {
 
       Class factoryClass = Class.forName(factoryName);
 
-      DestinationObjectFactory factory = (DestinationObjectFactory) factoryClass.newInstance();
+      ObjectFactory factory = (ObjectFactory) factoryClass.newInstance();
 
       Object instance = factory.getObjectInstance(topicRef, null, null, null);