You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@activemq.apache.org by ma...@apache.org on 2016/08/09 10:23:06 UTC

[3/5] activemq-artemis git commit: ARTEMIS-604 - Add checks for object messages in REST and AMQP - Rest interface fix - Doc fixes (Rest->REST) - JSON management and AMQP outbound

ARTEMIS-604 - Add checks for object messages in REST and AMQP
  - Rest interface fix
  - Doc fixes (Rest->REST)
  - JSON management and AMQP outbound


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

Branch: refs/heads/master
Commit: 2fb8341f8d05bb7b60391b0d06b32d18554a3e74
Parents: 0535218
Author: Howard Gao <ho...@gmail.com>
Authored: Thu Aug 4 09:39:20 2016 +0800
Committer: Martyn Taylor <mt...@redhat.com>
Committed: Tue Aug 9 11:22:48 2016 +0100

----------------------------------------------------------------------
 .../activemq/artemis/api/core/JsonUtil.java     |   5 +-
 .../proton/converter/ActiveMQJMSVendor.java     |  11 +-
 .../rest/MessageServiceConfiguration.java       |  19 ++
 .../artemis/rest/MessageServiceManager.java     |  23 +-
 .../rest/integration/EmbeddedRestActiveMQ.java  |   6 +-
 .../integration/EmbeddedRestActiveMQJMS.java    |   5 +
 .../RestMessagingBootstrapListener.java         |  32 +-
 .../artemis/rest/queue/ConsumedMessage.java     |   5 +-
 .../rest/queue/ConsumedObjectMessage.java       |  13 +-
 .../rest/queue/DestinationServiceManager.java   |  11 +
 .../artemis/rest/queue/QueueConsumer.java       |  11 +-
 .../rest/queue/QueueDestinationsResource.java   |   1 +
 .../artemis/rest/queue/QueueServiceManager.java |   5 +
 .../artemis/rest/queue/push/PushConsumer.java   |   8 +-
 .../rest/queue/push/PushConsumerResource.java   |  11 +-
 .../artemis/rest/queue/push/PushStrategy.java   |   3 +
 .../artemis/rest/queue/push/UriStrategy.java    |  10 +-
 .../artemis/rest/topic/PushSubscription.java    |   6 +-
 .../rest/topic/PushSubscriptionsResource.java   |  11 +-
 .../rest/topic/TopicDestinationsResource.java   |   2 +-
 .../artemis/rest/topic/TopicServiceManager.java |   5 +
 .../artemis/rest/util/HttpMessageHelper.java    |  45 +--
 .../activemq/artemis/rest/util/TimeoutTask.java |   4 +-
 .../activemq/artemis/rest/test/Embedded.java    |   2 +-
 .../artemis/rest/test/EmbeddedTest.java         |   2 +-
 .../test/PersistentPushQueueConsumerTest.java   |   2 +-
 .../test/PersistentPushTopicConsumerTest.java   |   2 +-
 docs/user-manual/en/security.md                 | 312 ++++++++++---------
 tests/integration-tests/pom.xml                 |  55 ++++
 .../artemis/tests/integration/rest/Order.java   |  79 +++++
 .../rest/RestDeserializationTest.java           | 192 ++++++++++++
 .../tests/integration/rest/RestTestBase.java    |  97 ++++++
 .../rest/util/QueueRestMessageContext.java      |  76 +++++
 .../integration/rest/util/ResponseUtil.java     |  34 ++
 .../rest/util/RestAMQConnection.java            | 104 +++++++
 .../rest/util/RestMessageContext.java           | 271 ++++++++++++++++
 .../rest/util/TopicRestMessageContext.java      |  83 +++++
 .../resources/rest/bwlist-rest-test-asm.xml     |  37 +++
 .../src/test/resources/rest/rest-test-asm.xml   |  37 +++
 .../rest-test-bwlist/webapp/WEB-INF/web.xml     |  56 ++++
 .../rest/rest-test/webapp/WEB-INF/web.xml       |  58 ++++
 41 files changed, 1528 insertions(+), 223 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2fb8341f/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/JsonUtil.java
----------------------------------------------------------------------
diff --git a/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/JsonUtil.java b/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/JsonUtil.java
index e50540d..a3da33b 100644
--- a/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/JsonUtil.java
+++ b/artemis-core-client/src/main/java/org/apache/activemq/artemis/api/core/JsonUtil.java
@@ -19,6 +19,7 @@ package org.apache.activemq.artemis.api.core;
 import org.apache.activemq.artemis.core.client.ActiveMQClientMessageBundle;
 import org.apache.activemq.artemis.utils.Base64;
 import org.apache.activemq.artemis.utils.JsonLoader;
+import org.apache.activemq.artemis.utils.ObjectInputStreamWithClassLoader;
 import org.apache.activemq.artemis.utils.StringEscapeUtils;
 
 import javax.json.Json;
@@ -32,7 +33,6 @@ import javax.json.JsonValue;
 import javax.management.openmbean.CompositeData;
 import javax.management.openmbean.CompositeDataSupport;
 import java.io.ByteArrayInputStream;
-import java.io.ObjectInputStream;
 import java.io.StringReader;
 import java.util.HashMap;
 import java.util.List;
@@ -155,7 +155,8 @@ public final class JsonUtil {
                   CompositeData[] cds = new CompositeData[data.length];
                   for (int i1 = 0; i1 < data.length; i1++) {
                      String dataConverted  = convertJsonValue(data[i1], String.class).toString();
-                     ObjectInputStream ois = new ObjectInputStream(new ByteArrayInputStream(Base64.decode(dataConverted)));
+                     ObjectInputStreamWithClassLoader ois = new ObjectInputStreamWithClassLoader(new ByteArrayInputStream(Base64.decode(dataConverted)));
+                     ois.setWhiteList("java.util,java.lang,javax.management");
                      cds[i1] = (CompositeDataSupport) ois.readObject();
                   }
                   innerVal = cds;

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2fb8341f/artemis-protocols/artemis-amqp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/proton/converter/ActiveMQJMSVendor.java
----------------------------------------------------------------------
diff --git a/artemis-protocols/artemis-amqp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/proton/converter/ActiveMQJMSVendor.java b/artemis-protocols/artemis-amqp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/proton/converter/ActiveMQJMSVendor.java
index 169c8e4..ed887cf 100644
--- a/artemis-protocols/artemis-amqp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/proton/converter/ActiveMQJMSVendor.java
+++ b/artemis-protocols/artemis-amqp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/proton/converter/ActiveMQJMSVendor.java
@@ -24,18 +24,18 @@ import javax.jms.ObjectMessage;
 import javax.jms.StreamMessage;
 import javax.jms.TextMessage;
 
-import org.apache.activemq.artemis.core.protocol.proton.converter.jms.ServerDestination;
-import org.apache.activemq.artemis.core.protocol.proton.converter.jms.ServerJMSObjectMessage;
-import org.apache.activemq.artemis.core.protocol.proton.converter.message.JMSVendor;
-import org.apache.activemq.artemis.jms.client.ActiveMQDestination;
 import org.apache.activemq.artemis.core.buffers.impl.ResetLimitWrappedActiveMQBuffer;
+import org.apache.activemq.artemis.core.protocol.proton.converter.jms.ServerDestination;
 import org.apache.activemq.artemis.core.protocol.proton.converter.jms.ServerJMSBytesMessage;
 import org.apache.activemq.artemis.core.protocol.proton.converter.jms.ServerJMSMapMessage;
 import org.apache.activemq.artemis.core.protocol.proton.converter.jms.ServerJMSMessage;
+import org.apache.activemq.artemis.core.protocol.proton.converter.jms.ServerJMSObjectMessage;
 import org.apache.activemq.artemis.core.protocol.proton.converter.jms.ServerJMSStreamMessage;
 import org.apache.activemq.artemis.core.protocol.proton.converter.jms.ServerJMSTextMessage;
+import org.apache.activemq.artemis.core.protocol.proton.converter.message.JMSVendor;
 import org.apache.activemq.artemis.core.server.ServerMessage;
 import org.apache.activemq.artemis.core.server.impl.ServerMessageImpl;
+import org.apache.activemq.artemis.jms.client.ActiveMQDestination;
 import org.apache.activemq.artemis.utils.IDGenerator;
 
 public class ActiveMQJMSVendor implements JMSVendor {
@@ -116,12 +116,9 @@ public class ActiveMQJMSVendor implements JMSVendor {
             return new ServerJMSMapMessage(wrapped, deliveryCount);
          case org.apache.activemq.artemis.api.core.Message.TEXT_TYPE:
             return new ServerJMSTextMessage(wrapped, deliveryCount);
-         case org.apache.activemq.artemis.api.core.Message.OBJECT_TYPE:
-            return new ServerJMSObjectMessage(wrapped, deliveryCount);
          default:
             return new ServerJMSMessage(wrapped, deliveryCount);
       }
-
    }
 
    @Override

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2fb8341f/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/MessageServiceConfiguration.java
----------------------------------------------------------------------
diff --git a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/MessageServiceConfiguration.java b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/MessageServiceConfiguration.java
index ff9e8e7..177a7f6 100644
--- a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/MessageServiceConfiguration.java
+++ b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/MessageServiceConfiguration.java
@@ -34,6 +34,9 @@ public class MessageServiceConfiguration {
    private String inVmId = "0";
    private boolean useLinkHeaders = false;
 
+   private String deserializationWhiteList;
+   private String deserializationBlackList;
+
    @XmlElement(name = "server-in-vm-id")
    public String getInVmId() {
       return inVmId;
@@ -132,4 +135,20 @@ public class MessageServiceConfiguration {
    public void setConsumerWindowSize(int consumerWindowSize) {
       this.consumerWindowSize = consumerWindowSize;
    }
+
+   public String getDeserializationWhiteList() {
+      return deserializationWhiteList;
+   }
+
+   public void setDeserializationWhiteList(String deserializationWhiteList) {
+      this.deserializationWhiteList = deserializationWhiteList;
+   }
+
+   public String getDeserializationBlackList() {
+      return deserializationBlackList;
+   }
+
+   public void setDeserializationBlackList(String deserializationBlackList) {
+      this.deserializationBlackList = deserializationBlackList;
+   }
 }

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2fb8341f/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/MessageServiceManager.java
----------------------------------------------------------------------
diff --git a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/MessageServiceManager.java b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/MessageServiceManager.java
index e9c853b..4aaa95d 100644
--- a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/MessageServiceManager.java
+++ b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/MessageServiceManager.java
@@ -23,6 +23,7 @@ import java.net.URL;
 import java.util.HashMap;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
 
 import javax.xml.bind.JAXBContext;
 
@@ -32,6 +33,7 @@ import org.apache.activemq.artemis.api.core.client.ServerLocator;
 import org.apache.activemq.artemis.core.client.impl.ServerLocatorImpl;
 import org.apache.activemq.artemis.core.remoting.impl.invm.InVMConnectorFactory;
 import org.apache.activemq.artemis.core.remoting.impl.invm.TransportConstants;
+import org.apache.activemq.artemis.jms.client.ConnectionFactoryOptions;
 import org.apache.activemq.artemis.rest.queue.DestinationSettings;
 import org.apache.activemq.artemis.rest.queue.QueueServiceManager;
 import org.apache.activemq.artemis.rest.topic.TopicServiceManager;
@@ -46,8 +48,8 @@ import org.apache.activemq.artemis.utils.XMLUtil;
 public class MessageServiceManager {
 
    protected ExecutorService threadPool;
-   protected QueueServiceManager queueManager = new QueueServiceManager();
-   protected TopicServiceManager topicManager = new TopicServiceManager();
+   protected QueueServiceManager queueManager;
+   protected TopicServiceManager topicManager;
    protected TimeoutTask timeoutTask;
    protected int timeoutTaskInterval = 1;
    protected MessageServiceConfiguration configuration = new MessageServiceConfiguration();
@@ -55,6 +57,13 @@ public class MessageServiceManager {
    protected String configResourcePath;
    protected BindingRegistry registry;
 
+   private ClientSessionFactory consumerSessionFactory;
+
+   public MessageServiceManager(ConnectionFactoryOptions jmsOptions) {
+      queueManager = new QueueServiceManager(jmsOptions);
+      topicManager = new TopicServiceManager(jmsOptions);
+   }
+
    public BindingRegistry getRegistry() {
       return registry;
    }
@@ -147,7 +156,7 @@ public class MessageServiceManager {
          consumerLocator.setConsumerWindowSize(configuration.getConsumerWindowSize());
       }
 
-      ClientSessionFactory consumerSessionFactory = consumerLocator.createSessionFactory();
+      consumerSessionFactory = consumerLocator.createSessionFactory();
       ActiveMQRestLogger.LOGGER.debug("Created ClientSessionFactory: " + consumerSessionFactory);
 
       ServerLocator defaultLocator = new ServerLocatorImpl(false, new TransportConfiguration(InVMConnectorFactory.class.getName(), transportConfig));
@@ -197,5 +206,13 @@ public class MessageServiceManager {
       if (topicManager != null)
          topicManager.stop();
       topicManager = null;
+      this.timeoutTask.stop();
+      threadPool.shutdown();
+      try {
+         threadPool.awaitTermination(5000, TimeUnit.SECONDS);
+      }
+      catch (InterruptedException e) {
+      }
+      this.consumerSessionFactory.close();
    }
 }

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2fb8341f/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/integration/EmbeddedRestActiveMQ.java
----------------------------------------------------------------------
diff --git a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/integration/EmbeddedRestActiveMQ.java b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/integration/EmbeddedRestActiveMQ.java
index 9ac9b06..fc8ae4e 100644
--- a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/integration/EmbeddedRestActiveMQ.java
+++ b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/integration/EmbeddedRestActiveMQ.java
@@ -17,6 +17,7 @@
 package org.apache.activemq.artemis.rest.integration;
 
 import org.apache.activemq.artemis.core.server.embedded.EmbeddedActiveMQ;
+import org.apache.activemq.artemis.jms.client.ConnectionFactoryOptions;
 import org.jboss.resteasy.plugins.server.tjws.TJWSEmbeddedJaxrsServer;
 import org.apache.activemq.artemis.rest.MessageServiceManager;
 import org.jboss.resteasy.test.TestPortProvider;
@@ -25,13 +26,14 @@ public class EmbeddedRestActiveMQ {
 
    protected TJWSEmbeddedJaxrsServer tjws = new TJWSEmbeddedJaxrsServer();
    protected EmbeddedActiveMQ embeddedActiveMQ;
-   protected MessageServiceManager manager = new MessageServiceManager();
+   protected MessageServiceManager manager = new MessageServiceManager(null);
 
-   public EmbeddedRestActiveMQ() {
+   public EmbeddedRestActiveMQ(ConnectionFactoryOptions jmsOptions) {
       int port = TestPortProvider.getPort();
       tjws.setPort(port);
       tjws.setRootResourcePath("");
       tjws.setSecurityDomain(null);
+      manager = new MessageServiceManager(jmsOptions);
       initEmbeddedActiveMQ();
    }
 

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2fb8341f/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/integration/EmbeddedRestActiveMQJMS.java
----------------------------------------------------------------------
diff --git a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/integration/EmbeddedRestActiveMQJMS.java b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/integration/EmbeddedRestActiveMQJMS.java
index 75fd361..3b14437 100644
--- a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/integration/EmbeddedRestActiveMQJMS.java
+++ b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/integration/EmbeddedRestActiveMQJMS.java
@@ -16,11 +16,16 @@
  */
 package org.apache.activemq.artemis.rest.integration;
 
+import org.apache.activemq.artemis.jms.client.ConnectionFactoryOptions;
 import org.apache.activemq.artemis.jms.server.embedded.EmbeddedJMS;
 import org.apache.activemq.artemis.spi.core.naming.BindingRegistry;
 
 public class EmbeddedRestActiveMQJMS extends EmbeddedRestActiveMQ {
 
+   public EmbeddedRestActiveMQJMS(ConnectionFactoryOptions jmsOptions) {
+      super(jmsOptions);
+   }
+
    @Override
    protected void initEmbeddedActiveMQ() {
       embeddedActiveMQ = new EmbeddedJMS();

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2fb8341f/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/integration/RestMessagingBootstrapListener.java
----------------------------------------------------------------------
diff --git a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/integration/RestMessagingBootstrapListener.java b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/integration/RestMessagingBootstrapListener.java
index 20ce51d..60cea03 100644
--- a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/integration/RestMessagingBootstrapListener.java
+++ b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/integration/RestMessagingBootstrapListener.java
@@ -20,22 +20,28 @@ import javax.servlet.ServletContext;
 import javax.servlet.ServletContextEvent;
 import javax.servlet.ServletContextListener;
 
+import org.apache.activemq.artemis.jms.client.ConnectionFactoryOptions;
 import org.apache.activemq.artemis.rest.MessageServiceManager;
+import org.apache.activemq.artemis.utils.ObjectInputStreamWithClassLoader;
 import org.jboss.resteasy.spi.Registry;
 
-public class RestMessagingBootstrapListener implements ServletContextListener {
+public class RestMessagingBootstrapListener implements ServletContextListener, ConnectionFactoryOptions {
 
    MessageServiceManager manager;
+   private String deserializationBlackList;
+   private String deserializationWhiteList;
 
    @Override
    public void contextInitialized(ServletContextEvent contextEvent) {
       ServletContext context = contextEvent.getServletContext();
-      String configfile = context.getInitParameter("rest.messaging.config.file");
       Registry registry = (Registry) context.getAttribute(Registry.class.getName());
       if (registry == null) {
          throw new RuntimeException("You must install RESTEasy as a Bootstrap Listener and it must be listed before this class");
       }
-      manager = new MessageServiceManager();
+      String configfile = context.getInitParameter("rest.messaging.config.file");
+      deserializationBlackList = context.getInitParameter(ObjectInputStreamWithClassLoader.BLACKLIST_PROPERTY);
+      deserializationWhiteList = context.getInitParameter(ObjectInputStreamWithClassLoader.WHITELIST_PROPERTY);
+      manager = new MessageServiceManager(this);
 
       if (configfile != null) {
          manager.setConfigResourcePath(configfile);
@@ -56,4 +62,24 @@ public class RestMessagingBootstrapListener implements ServletContextListener {
          manager.stop();
       }
    }
+
+   @Override
+   public String getDeserializationBlackList() {
+      return deserializationBlackList;
+   }
+
+   @Override
+   public void setDeserializationBlackList(String blackList) {
+      deserializationBlackList = blackList;
+   }
+
+   @Override
+   public String getDeserializationWhiteList() {
+      return deserializationWhiteList;
+   }
+
+   @Override
+   public void setDeserializationWhiteList(String whiteList) {
+      deserializationWhiteList = whiteList;
+   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2fb8341f/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/ConsumedMessage.java
----------------------------------------------------------------------
diff --git a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/ConsumedMessage.java b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/ConsumedMessage.java
index 3c49912..22b8f7f 100644
--- a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/ConsumedMessage.java
+++ b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/ConsumedMessage.java
@@ -19,6 +19,7 @@ package org.apache.activemq.artemis.rest.queue;
 import org.apache.activemq.artemis.api.core.Message;
 import org.apache.activemq.artemis.api.core.SimpleString;
 import org.apache.activemq.artemis.api.core.client.ClientMessage;
+import org.apache.activemq.artemis.jms.client.ConnectionFactoryOptions;
 import org.apache.activemq.artemis.rest.HttpHeaderProperty;
 import org.apache.activemq.artemis.rest.ActiveMQRestLogger;
 
@@ -51,13 +52,13 @@ public abstract class ConsumedMessage {
       }
    }
 
-   public static ConsumedMessage createConsumedMessage(ClientMessage message) {
+   public static ConsumedMessage createConsumedMessage(ClientMessage message, ConnectionFactoryOptions options) {
       Boolean aBoolean = message.getBooleanProperty(POSTED_AS_HTTP_MESSAGE);
       if (aBoolean != null && aBoolean.booleanValue()) {
          return new ConsumedHttpMessage(message);
       }
       else if (message.getType() == Message.OBJECT_TYPE) {
-         return new ConsumedObjectMessage(message);
+         return new ConsumedObjectMessage(message, options);
       }
       else {
          throw new IllegalArgumentException("ClientMessage must be an HTTP message or an Object message: " + message + " type: " + message.getType());

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2fb8341f/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/ConsumedObjectMessage.java
----------------------------------------------------------------------
diff --git a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/ConsumedObjectMessage.java b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/ConsumedObjectMessage.java
index b03e84f..ddd7a81 100644
--- a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/ConsumedObjectMessage.java
+++ b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/ConsumedObjectMessage.java
@@ -18,19 +18,22 @@ package org.apache.activemq.artemis.rest.queue;
 
 import org.apache.activemq.artemis.api.core.Message;
 import org.apache.activemq.artemis.api.core.client.ClientMessage;
+import org.apache.activemq.artemis.jms.client.ConnectionFactoryOptions;
+import org.apache.activemq.artemis.utils.ObjectInputStreamWithClassLoader;
 
 import javax.ws.rs.core.Response;
 import java.io.ByteArrayInputStream;
-import java.io.ObjectInputStream;
 
 public class ConsumedObjectMessage extends ConsumedMessage {
 
    protected Object readObject;
+   private ConnectionFactoryOptions options;
 
-   public ConsumedObjectMessage(ClientMessage message) {
+   public ConsumedObjectMessage(ClientMessage message, ConnectionFactoryOptions options) {
       super(message);
       if (message.getType() != Message.OBJECT_TYPE)
          throw new IllegalArgumentException("Client message must be an OBJECT_TYPE");
+      this.options = options;
    }
 
    @Override
@@ -43,7 +46,11 @@ public class ConsumedObjectMessage extends ConsumedMessage {
             message.getBodyBuffer().readBytes(body);
             ByteArrayInputStream bais = new ByteArrayInputStream(body);
             try {
-               ObjectInputStream ois = new ObjectInputStream(bais);
+               ObjectInputStreamWithClassLoader ois = new ObjectInputStreamWithClassLoader(bais);
+               if (options != null) {
+                  ois.setWhiteList(options.getDeserializationWhiteList());
+                  ois.setBlackList(options.getDeserializationBlackList());
+               }
                readObject = ois.readObject();
             }
             catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2fb8341f/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/DestinationServiceManager.java
----------------------------------------------------------------------
diff --git a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/DestinationServiceManager.java b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/DestinationServiceManager.java
index 7d69bd9..93512d9 100644
--- a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/DestinationServiceManager.java
+++ b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/DestinationServiceManager.java
@@ -21,6 +21,7 @@ import org.apache.activemq.artemis.api.core.client.ClientSessionFactory;
 import org.apache.activemq.artemis.api.core.client.ActiveMQClient;
 import org.apache.activemq.artemis.api.core.client.ServerLocator;
 import org.apache.activemq.artemis.core.remoting.impl.invm.InVMConnectorFactory;
+import org.apache.activemq.artemis.jms.client.ConnectionFactoryOptions;
 import org.apache.activemq.artemis.rest.util.LinkStrategy;
 import org.apache.activemq.artemis.rest.util.TimeoutTask;
 import org.apache.activemq.artemis.spi.core.naming.BindingRegistry;
@@ -40,6 +41,12 @@ public abstract class DestinationServiceManager {
    protected LinkStrategy linkStrategy;
    protected BindingRegistry registry;
 
+   protected ConnectionFactoryOptions jmsOptions;
+
+   public DestinationServiceManager(ConnectionFactoryOptions jmsOptions) {
+      this.jmsOptions = jmsOptions;
+   }
+
    public BindingRegistry getRegistry() {
       return registry;
    }
@@ -157,4 +164,8 @@ public abstract class DestinationServiceManager {
    public abstract void start() throws Exception;
 
    public abstract void stop();
+
+   public ConnectionFactoryOptions getJmsOptions() {
+      return jmsOptions;
+   }
 }

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2fb8341f/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/QueueConsumer.java
----------------------------------------------------------------------
diff --git a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/QueueConsumer.java b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/QueueConsumer.java
index 96a1ae9..1ebe786 100644
--- a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/QueueConsumer.java
+++ b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/QueueConsumer.java
@@ -33,6 +33,7 @@ import org.apache.activemq.artemis.api.core.client.ClientConsumer;
 import org.apache.activemq.artemis.api.core.client.ClientMessage;
 import org.apache.activemq.artemis.api.core.client.ClientSession;
 import org.apache.activemq.artemis.api.core.client.ClientSessionFactory;
+import org.apache.activemq.artemis.jms.client.ConnectionFactoryOptions;
 import org.apache.activemq.artemis.rest.ActiveMQRestLogger;
 import org.apache.activemq.artemis.rest.util.Constants;
 import org.apache.activemq.artemis.rest.util.LinkStrategy;
@@ -179,7 +180,7 @@ public class QueueConsumer {
             return builder.build();
          }
          previousIndex = index;
-         lastConsumed = ConsumedMessage.createConsumedMessage(message);
+         lastConsumed = ConsumedMessage.createConsumedMessage(message, this.getJmsOptions());
          String token = Long.toString(lastConsumed.getMessageID());
          Response response = getMessageResponse(lastConsumed, info, basePath, token).build();
          if (autoAck)
@@ -187,7 +188,9 @@ public class QueueConsumer {
          return response;
       }
       catch (Exception e) {
-         throw new RuntimeException(e);
+         Response errorResponse = Response.serverError().entity(e.getMessage())
+                 .status(Response.Status.INTERNAL_SERVER_ERROR).build();
+         return errorResponse;
       }
    }
 
@@ -264,4 +267,8 @@ public class QueueConsumer {
       String uri = builder.build().toString();
       serviceManager.getLinkStrategy().setLinkHeader(response, "consumer", "consumer", uri, MediaType.APPLICATION_XML);
    }
+
+   public ConnectionFactoryOptions getJmsOptions() {
+      return serviceManager.getJmsOptions();
+   }
 }

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2fb8341f/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/QueueDestinationsResource.java
----------------------------------------------------------------------
diff --git a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/QueueDestinationsResource.java b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/QueueDestinationsResource.java
index 7f7ce09..4f66bb9 100644
--- a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/QueueDestinationsResource.java
+++ b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/QueueDestinationsResource.java
@@ -145,6 +145,7 @@ public class QueueDestinationsResource {
       PushConsumerResource push = new PushConsumerResource();
       push.setDestination(queueName);
       push.setSessionFactory(manager.getConsumerSessionFactory());
+      push.setJmsOptions(manager.getJmsOptions());
       queueResource.setPushConsumers(push);
 
       PostMessage sender = null;

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2fb8341f/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/QueueServiceManager.java
----------------------------------------------------------------------
diff --git a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/QueueServiceManager.java b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/QueueServiceManager.java
index 4c61b4d..7779248 100644
--- a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/QueueServiceManager.java
+++ b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/QueueServiceManager.java
@@ -21,6 +21,7 @@ import java.util.List;
 
 import org.apache.activemq.artemis.api.core.SimpleString;
 import org.apache.activemq.artemis.api.core.client.ClientSession;
+import org.apache.activemq.artemis.jms.client.ConnectionFactoryOptions;
 import org.apache.activemq.artemis.rest.queue.push.PushStore;
 import org.apache.activemq.artemis.rest.queue.push.FilePushStore;
 
@@ -30,6 +31,10 @@ public class QueueServiceManager extends DestinationServiceManager {
    protected List<QueueDeployment> queues = new ArrayList<>();
    protected QueueDestinationsResource destination;
 
+   public QueueServiceManager(ConnectionFactoryOptions jmsOptions) {
+      super(jmsOptions);
+   }
+
    public List<QueueDeployment> getQueues() {
       return queues;
    }

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2fb8341f/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/push/PushConsumer.java
----------------------------------------------------------------------
diff --git a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/push/PushConsumer.java b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/push/PushConsumer.java
index c8f17b3..023e3f0 100644
--- a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/push/PushConsumer.java
+++ b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/push/PushConsumer.java
@@ -20,6 +20,7 @@ import org.apache.activemq.artemis.api.core.ActiveMQException;
 import org.apache.activemq.artemis.api.core.client.ClientConsumer;
 import org.apache.activemq.artemis.api.core.client.ClientSession;
 import org.apache.activemq.artemis.api.core.client.ClientSessionFactory;
+import org.apache.activemq.artemis.jms.client.ConnectionFactoryOptions;
 import org.apache.activemq.artemis.rest.ActiveMQRestLogger;
 import org.apache.activemq.artemis.rest.queue.push.xml.PushRegistration;
 import org.apache.activemq.artemis.utils.SelectorTranslator;
@@ -38,16 +39,20 @@ public class PushConsumer {
    protected PushStrategy strategy;
    protected PushStore store;
 
+   private ConnectionFactoryOptions jmsOptions;
+
    public PushConsumer(ClientSessionFactory factory,
                        String destination,
                        String id,
                        PushRegistration registration,
-                       PushStore store) {
+                       PushStore store,
+                       ConnectionFactoryOptions jmsOptions) {
       this.factory = factory;
       this.destination = destination;
       this.id = id;
       this.registration = registration;
       this.store = store;
+      this.jmsOptions = jmsOptions;
    }
 
    public PushStrategy getStrategy() {
@@ -79,6 +84,7 @@ public class PushConsumer {
          strategy = new UriStrategy();
       }
       strategy.setRegistration(registration);
+      strategy.setJmsOptions(jmsOptions);
       strategy.start();
 
       sessions = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2fb8341f/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/push/PushConsumerResource.java
----------------------------------------------------------------------
diff --git a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/push/PushConsumerResource.java b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/push/PushConsumerResource.java
index 76dcf5e..ef183e5 100644
--- a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/push/PushConsumerResource.java
+++ b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/push/PushConsumerResource.java
@@ -33,6 +33,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.activemq.artemis.api.core.client.ClientSessionFactory;
+import org.apache.activemq.artemis.jms.client.ConnectionFactoryOptions;
 import org.apache.activemq.artemis.rest.queue.push.xml.PushRegistration;
 import org.apache.activemq.artemis.rest.ActiveMQRestLogger;
 
@@ -45,6 +46,8 @@ public class PushConsumerResource {
    protected final AtomicLong sessionCounter = new AtomicLong(1);
    protected PushStore pushStore;
 
+   private ConnectionFactoryOptions jmsOptions;
+
    public void start() {
 
    }
@@ -66,7 +69,7 @@ public class PushConsumerResource {
    public void addRegistration(PushRegistration reg) throws Exception {
       if (reg.isEnabled() == false)
          return;
-      PushConsumer consumer = new PushConsumer(sessionFactory, destination, reg.getId(), reg, pushStore);
+      PushConsumer consumer = new PushConsumer(sessionFactory, destination, reg.getId(), reg, pushStore, jmsOptions);
       consumer.start();
       consumers.put(reg.getId(), consumer);
    }
@@ -80,7 +83,7 @@ public class PushConsumerResource {
       String genId = sessionCounter.getAndIncrement() + "-" + startup;
       registration.setId(genId);
       registration.setDestination(destination);
-      PushConsumer consumer = new PushConsumer(sessionFactory, destination, genId, registration, pushStore);
+      PushConsumer consumer = new PushConsumer(sessionFactory, destination, genId, registration, pushStore, jmsOptions);
       try {
          consumer.start();
          if (registration.isDurable() && pushStore != null) {
@@ -142,4 +145,8 @@ public class PushConsumerResource {
    public void setDestination(String destination) {
       this.destination = destination;
    }
+
+   public void setJmsOptions(ConnectionFactoryOptions jmsOptions) {
+      this.jmsOptions = jmsOptions;
+   }
 }

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2fb8341f/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/push/PushStrategy.java
----------------------------------------------------------------------
diff --git a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/push/PushStrategy.java b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/push/PushStrategy.java
index 8812223..6f291d3 100644
--- a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/push/PushStrategy.java
+++ b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/push/PushStrategy.java
@@ -17,6 +17,7 @@
 package org.apache.activemq.artemis.rest.queue.push;
 
 import org.apache.activemq.artemis.api.core.client.ClientMessage;
+import org.apache.activemq.artemis.jms.client.ConnectionFactoryOptions;
 import org.apache.activemq.artemis.rest.queue.push.xml.PushRegistration;
 
 public interface PushStrategy {
@@ -36,4 +37,6 @@ public interface PushStrategy {
    void start() throws Exception;
 
    void stop() throws Exception;
+
+   void setJmsOptions(ConnectionFactoryOptions jmsOptions);
 }

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2fb8341f/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/push/UriStrategy.java
----------------------------------------------------------------------
diff --git a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/push/UriStrategy.java b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/push/UriStrategy.java
index 5bca167..2df104b 100644
--- a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/push/UriStrategy.java
+++ b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/queue/push/UriStrategy.java
@@ -19,6 +19,7 @@ package org.apache.activemq.artemis.rest.queue.push;
 import javax.ws.rs.core.UriBuilder;
 import java.io.IOException;
 
+import org.apache.activemq.artemis.jms.client.ConnectionFactoryOptions;
 import org.apache.activemq.artemis.rest.queue.push.xml.BasicAuth;
 import org.apache.activemq.artemis.rest.queue.push.xml.PushRegistration;
 import org.apache.activemq.artemis.rest.util.HttpMessageHelper;
@@ -59,6 +60,8 @@ public class UriStrategy implements PushStrategy {
    protected String method;
    protected String contentType;
 
+   protected ConnectionFactoryOptions jmsOptions;
+
    UriStrategy() {
       connManager.setDefaultMaxPerRoute(100);
       connManager.setMaxTotal(1000);
@@ -106,6 +109,11 @@ public class UriStrategy implements PushStrategy {
    }
 
    @Override
+   public void setJmsOptions(ConnectionFactoryOptions jmsOptions) {
+      this.jmsOptions = jmsOptions;
+   }
+
+   @Override
    public boolean push(ClientMessage message) {
       ActiveMQRestLogger.LOGGER.debug("Pushing " + message);
       String uri = createUri(message);
@@ -120,7 +128,7 @@ public class UriStrategy implements PushStrategy {
             ActiveMQRestLogger.LOGGER.debug("Setting XmlHttpHeader: " + header.getName() + "=" + header.getValue());
             request.header(header.getName(), header.getValue());
          }
-         HttpMessageHelper.buildMessage(message, request, contentType);
+         HttpMessageHelper.buildMessage(message, request, contentType, jmsOptions);
          ClientResponse<?> res = null;
          try {
             ActiveMQRestLogger.LOGGER.debug(method + " " + uri);

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2fb8341f/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/topic/PushSubscription.java
----------------------------------------------------------------------
diff --git a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/topic/PushSubscription.java b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/topic/PushSubscription.java
index da1cf34..1f5789f 100644
--- a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/topic/PushSubscription.java
+++ b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/topic/PushSubscription.java
@@ -19,6 +19,7 @@ package org.apache.activemq.artemis.rest.topic;
 import org.apache.activemq.artemis.api.core.ActiveMQException;
 import org.apache.activemq.artemis.api.core.client.ClientSession;
 import org.apache.activemq.artemis.api.core.client.ClientSessionFactory;
+import org.apache.activemq.artemis.jms.client.ConnectionFactoryOptions;
 import org.apache.activemq.artemis.rest.queue.push.PushStore;
 import org.apache.activemq.artemis.rest.ActiveMQRestLogger;
 import org.apache.activemq.artemis.rest.queue.push.PushConsumer;
@@ -30,8 +31,9 @@ public class PushSubscription extends PushConsumer {
                            String destination,
                            String id,
                            PushRegistration registration,
-                           PushStore store) {
-      super(factory, destination, id, registration, store);
+                           PushStore store,
+                           ConnectionFactoryOptions jmsOptions) {
+      super(factory, destination, id, registration, store, jmsOptions);
    }
 
    @Override

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2fb8341f/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/topic/PushSubscriptionsResource.java
----------------------------------------------------------------------
diff --git a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/topic/PushSubscriptionsResource.java b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/topic/PushSubscriptionsResource.java
index c01459b..6027007 100644
--- a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/topic/PushSubscriptionsResource.java
+++ b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/topic/PushSubscriptionsResource.java
@@ -20,6 +20,7 @@ import org.apache.activemq.artemis.api.core.ActiveMQException;
 import org.apache.activemq.artemis.api.core.SimpleString;
 import org.apache.activemq.artemis.api.core.client.ClientSession;
 import org.apache.activemq.artemis.api.core.client.ClientSessionFactory;
+import org.apache.activemq.artemis.jms.client.ConnectionFactoryOptions;
 import org.apache.activemq.artemis.rest.queue.push.PushConsumer;
 import org.apache.activemq.artemis.rest.ActiveMQRestLogger;
 
@@ -47,6 +48,12 @@ public class PushSubscriptionsResource {
    protected final AtomicLong sessionCounter = new AtomicLong(1);
    protected TopicPushStore pushStore;
 
+   private ConnectionFactoryOptions jmsOptions;
+
+   public PushSubscriptionsResource(ConnectionFactoryOptions jmsOptions) {
+      this.jmsOptions = jmsOptions;
+   }
+
    public void stop() {
       for (PushConsumer consumer : consumers.values()) {
          consumer.stop();
@@ -92,7 +99,7 @@ public class PushSubscriptionsResource {
       if (!query.isExists()) {
          createSession = createSubscription(destination, reg.isDurable());
       }
-      PushSubscription consumer = new PushSubscription(sessionFactory, reg.getDestination(), reg.getId(), reg, pushStore);
+      PushSubscription consumer = new PushSubscription(sessionFactory, reg.getDestination(), reg.getId(), reg, pushStore, jmsOptions);
       try {
          consumer.start();
       }
@@ -133,7 +140,7 @@ public class PushSubscriptionsResource {
       registration.setTopic(destination);
       ClientSession createSession = createSubscription(genId, registration.isDurable());
       try {
-         PushSubscription consumer = new PushSubscription(sessionFactory, genId, genId, registration, pushStore);
+         PushSubscription consumer = new PushSubscription(sessionFactory, genId, genId, registration, pushStore, jmsOptions);
          try {
             consumer.start();
             if (registration.isDurable() && pushStore != null) {

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2fb8341f/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/topic/TopicDestinationsResource.java
----------------------------------------------------------------------
diff --git a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/topic/TopicDestinationsResource.java b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/topic/TopicDestinationsResource.java
index 8c72e49..dda2a0b 100644
--- a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/topic/TopicDestinationsResource.java
+++ b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/topic/TopicDestinationsResource.java
@@ -137,7 +137,7 @@ public class TopicDestinationsResource {
 
       subscriptionsResource.setDestination(topicName);
       subscriptionsResource.setSessionFactory(manager.getConsumerSessionFactory());
-      PushSubscriptionsResource push = new PushSubscriptionsResource();
+      PushSubscriptionsResource push = new PushSubscriptionsResource(manager.getJmsOptions());
       push.setDestination(topicName);
       push.setSessionFactory(manager.getConsumerSessionFactory());
       topicResource.setPushSubscriptions(push);

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2fb8341f/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/topic/TopicServiceManager.java
----------------------------------------------------------------------
diff --git a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/topic/TopicServiceManager.java b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/topic/TopicServiceManager.java
index 3cb7b39..7dbdd23 100644
--- a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/topic/TopicServiceManager.java
+++ b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/topic/TopicServiceManager.java
@@ -18,6 +18,7 @@ package org.apache.activemq.artemis.rest.topic;
 
 import org.apache.activemq.artemis.api.core.SimpleString;
 import org.apache.activemq.artemis.api.core.client.ClientSession;
+import org.apache.activemq.artemis.jms.client.ConnectionFactoryOptions;
 import org.apache.activemq.artemis.rest.queue.DestinationServiceManager;
 
 import java.util.ArrayList;
@@ -29,6 +30,10 @@ public class TopicServiceManager extends DestinationServiceManager {
    protected List<TopicDeployment> topics = new ArrayList<>();
    protected TopicDestinationsResource destination;
 
+   public TopicServiceManager(ConnectionFactoryOptions jmsOptions) {
+      super(jmsOptions);
+   }
+
    public TopicPushStore getPushStore() {
       return pushStore;
    }

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2fb8341f/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/util/HttpMessageHelper.java
----------------------------------------------------------------------
diff --git a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/util/HttpMessageHelper.java b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/util/HttpMessageHelper.java
index 989453d..7d97eef 100644
--- a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/util/HttpMessageHelper.java
+++ b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/util/HttpMessageHelper.java
@@ -18,15 +18,15 @@ package org.apache.activemq.artemis.rest.util;
 
 import org.apache.activemq.artemis.api.core.SimpleString;
 import org.apache.activemq.artemis.api.core.client.ClientMessage;
+import org.apache.activemq.artemis.jms.client.ConnectionFactoryOptions;
 import org.apache.activemq.artemis.rest.HttpHeaderProperty;
 import org.apache.activemq.artemis.rest.ActiveMQRestLogger;
+import org.apache.activemq.artemis.utils.ObjectInputStreamWithClassLoader;
 import org.jboss.resteasy.client.ClientRequest;
 
 import javax.ws.rs.core.HttpHeaders;
 import javax.ws.rs.core.MultivaluedMap;
-import javax.ws.rs.core.Response;
 import java.io.ByteArrayInputStream;
-import java.io.ObjectInputStream;
 import java.util.List;
 import java.util.Map.Entry;
 
@@ -39,39 +39,7 @@ public class HttpMessageHelper {
       return lowerKey.toLowerCase().startsWith("content") || lowerKey.toLowerCase().equals("link");
    }
 
-   public static void buildMessage(ClientMessage message, Response.ResponseBuilder builder) {
-      for (SimpleString key : message.getPropertyNames()) {
-         String k = key.toString();
-         String headerName = HttpHeaderProperty.fromPropertyName(k);
-         if (headerName == null) {
-            continue;
-         }
-         builder.header(headerName, message.getStringProperty(k));
-      }
-      int size = message.getBodySize();
-      if (size > 0) {
-         byte[] body = new byte[size];
-         message.getBodyBuffer().readBytes(body);
-         Boolean aBoolean = message.getBooleanProperty(POSTED_AS_HTTP_MESSAGE);
-         if (aBoolean != null && aBoolean.booleanValue()) {
-            builder.entity(body);
-         }
-         else {
-            ByteArrayInputStream bais = new ByteArrayInputStream(body);
-            Object obj = null;
-            try {
-               ObjectInputStream ois = new ObjectInputStream(bais);
-               obj = ois.readObject();
-            }
-            catch (Exception e) {
-               throw new RuntimeException(e);
-            }
-            builder.entity(obj);
-         }
-      }
-   }
-
-   public static void buildMessage(ClientMessage message, ClientRequest request, String contentType) {
+   public static void buildMessage(ClientMessage message, ClientRequest request, String contentType, ConnectionFactoryOptions jmsOptions) {
       for (SimpleString key : message.getPropertyNames()) {
          String k = key.toString();
          String headerName = HttpHeaderProperty.fromPropertyName(k);
@@ -105,12 +73,17 @@ public class HttpMessageHelper {
             ByteArrayInputStream bais = new ByteArrayInputStream(body);
             Object obj = null;
             try {
-               ObjectInputStream ois = new ObjectInputStream(bais);
+               ObjectInputStreamWithClassLoader ois = new ObjectInputStreamWithClassLoader(bais);
+               if (jmsOptions != null) {
+                  ois.setBlackList(jmsOptions.getDeserializationBlackList());
+                  ois.setWhiteList(jmsOptions.getDeserializationWhiteList());
+               }
                obj = ois.readObject();
                ActiveMQRestLogger.LOGGER.debug("**** Building Message from object: " + obj.toString());
                request.body(contentType, obj);
             }
             catch (Exception e) {
+               e.printStackTrace();
                throw new RuntimeException(e);
             }
          }

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2fb8341f/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/util/TimeoutTask.java
----------------------------------------------------------------------
diff --git a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/util/TimeoutTask.java b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/util/TimeoutTask.java
index 381a47c..233a533 100644
--- a/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/util/TimeoutTask.java
+++ b/artemis-rest/src/main/java/org/apache/activemq/artemis/rest/util/TimeoutTask.java
@@ -78,7 +78,9 @@ public class TimeoutTask implements Runnable {
 
    public synchronized void stop() {
       running = false;
-      thread.interrupt();
+      if (thread != null) {
+         thread.interrupt();
+      }
    }
 
    public synchronized int getInterval() {

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2fb8341f/artemis-rest/src/test/java/org/apache/activemq/artemis/rest/test/Embedded.java
----------------------------------------------------------------------
diff --git a/artemis-rest/src/test/java/org/apache/activemq/artemis/rest/test/Embedded.java b/artemis-rest/src/test/java/org/apache/activemq/artemis/rest/test/Embedded.java
index 36ca637..67a7a53 100644
--- a/artemis-rest/src/test/java/org/apache/activemq/artemis/rest/test/Embedded.java
+++ b/artemis-rest/src/test/java/org/apache/activemq/artemis/rest/test/Embedded.java
@@ -29,7 +29,7 @@ import org.jboss.resteasy.test.TestPortProvider;
 
 public class Embedded {
 
-   protected MessageServiceManager manager = new MessageServiceManager();
+   protected MessageServiceManager manager = new MessageServiceManager(null);
    protected MessageServiceConfiguration config = new MessageServiceConfiguration();
    protected ActiveMQServer activeMQServer;
    protected TJWSEmbeddedJaxrsServer tjws = new TJWSEmbeddedJaxrsServer();

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2fb8341f/artemis-rest/src/test/java/org/apache/activemq/artemis/rest/test/EmbeddedTest.java
----------------------------------------------------------------------
diff --git a/artemis-rest/src/test/java/org/apache/activemq/artemis/rest/test/EmbeddedTest.java b/artemis-rest/src/test/java/org/apache/activemq/artemis/rest/test/EmbeddedTest.java
index 91f927c..cf4e81e 100644
--- a/artemis-rest/src/test/java/org/apache/activemq/artemis/rest/test/EmbeddedTest.java
+++ b/artemis-rest/src/test/java/org/apache/activemq/artemis/rest/test/EmbeddedTest.java
@@ -48,7 +48,7 @@ public class EmbeddedTest {
 
    @BeforeClass
    public static void startEmbedded() throws Exception {
-      server = new EmbeddedRestActiveMQJMS();
+      server = new EmbeddedRestActiveMQJMS(null);
       server.getManager().setConfigResourcePath("activemq-rest.xml");
       SecurityConfiguration securityConfiguration = new SecurityConfiguration();
       securityConfiguration.addUser("guest", "guest");

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2fb8341f/artemis-rest/src/test/java/org/apache/activemq/artemis/rest/test/PersistentPushQueueConsumerTest.java
----------------------------------------------------------------------
diff --git a/artemis-rest/src/test/java/org/apache/activemq/artemis/rest/test/PersistentPushQueueConsumerTest.java b/artemis-rest/src/test/java/org/apache/activemq/artemis/rest/test/PersistentPushQueueConsumerTest.java
index d0e6c4f..bc15969 100644
--- a/artemis-rest/src/test/java/org/apache/activemq/artemis/rest/test/PersistentPushQueueConsumerTest.java
+++ b/artemis-rest/src/test/java/org/apache/activemq/artemis/rest/test/PersistentPushQueueConsumerTest.java
@@ -52,7 +52,7 @@ public class PersistentPushQueueConsumerTest {
       activeMQServer.start();
 
       deployment = EmbeddedContainer.start();
-      manager = new MessageServiceManager();
+      manager = new MessageServiceManager(null);
       manager.start();
       deployment.getRegistry().addSingletonResource(manager.getQueueManager().getDestination());
       deployment.getRegistry().addSingletonResource(manager.getTopicManager().getDestination());

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/2fb8341f/artemis-rest/src/test/java/org/apache/activemq/artemis/rest/test/PersistentPushTopicConsumerTest.java
----------------------------------------------------------------------
diff --git a/artemis-rest/src/test/java/org/apache/activemq/artemis/rest/test/PersistentPushTopicConsumerTest.java b/artemis-rest/src/test/java/org/apache/activemq/artemis/rest/test/PersistentPushTopicConsumerTest.java
index d44cea9..571bc92 100644
--- a/artemis-rest/src/test/java/org/apache/activemq/artemis/rest/test/PersistentPushTopicConsumerTest.java
+++ b/artemis-rest/src/test/java/org/apache/activemq/artemis/rest/test/PersistentPushTopicConsumerTest.java
@@ -72,7 +72,7 @@ public class PersistentPushTopicConsumerTest {
    public static void startup() throws Exception {
       deployment = EmbeddedContainer.start();
 
-      manager = new MessageServiceManager();
+      manager = new MessageServiceManager(null);
       manager.start();
       deployment.getRegistry().addSingletonResource(manager.getQueueManager().getDestination());
       deployment.getRegistry().addSingletonResource(manager.getTopicManager().getDestination());