You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@qpid.apache.org by ta...@apache.org on 2016/10/21 15:48:16 UTC

[1/5] qpid-jms git commit: QPIDJMS-215 Perform Message encoding at send time

Repository: qpid-jms
Updated Branches:
  refs/heads/master 22a517d76 -> 657747b77


http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsStreamMessageFacadeTest.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsStreamMessageFacadeTest.java b/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsStreamMessageFacadeTest.java
index fa7272b..83ee4cd 100644
--- a/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsStreamMessageFacadeTest.java
+++ b/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsStreamMessageFacadeTest.java
@@ -48,8 +48,7 @@ public class AmqpJmsStreamMessageFacadeTest extends AmqpJmsMessageTypesTestCase
     public void testNewMessageToSendContainsMessageTypeAnnotation() throws Exception {
         AmqpJmsStreamMessageFacade amqpStreamMessageFacade = createNewStreamMessageFacade();
 
-        Message protonMessage = amqpStreamMessageFacade.getAmqpMessage();
-        MessageAnnotations annotations = protonMessage.getMessageAnnotations();
+        MessageAnnotations annotations = amqpStreamMessageFacade.getMessageAnnotations();
         Map<Symbol, Object> annotationsMap = annotations.getValue();
 
         assertNotNull("MessageAnnotations section was not present", annotations);
@@ -63,8 +62,7 @@ public class AmqpJmsStreamMessageFacadeTest extends AmqpJmsMessageTypesTestCase
     public void testNewMessageToSendContainsAmqpSequenceBody() throws Exception {
         AmqpJmsStreamMessageFacade amqpStreamMessageFacade = createNewStreamMessageFacade();
 
-        Message protonMessage = amqpStreamMessageFacade.getAmqpMessage();
-        Section body = protonMessage.getBody();
+        Section body = amqpStreamMessageFacade.getBody();
 
         assertNotNull("Body section was not present", body);
         assertTrue("Body section was not of expected type: " + body.getClass(), body instanceof AmqpSequence);

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsTextMessageFacadeTest.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsTextMessageFacadeTest.java b/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsTextMessageFacadeTest.java
index d1bfe0c..bd517ad 100644
--- a/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsTextMessageFacadeTest.java
+++ b/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsTextMessageFacadeTest.java
@@ -37,7 +37,6 @@ import org.apache.qpid.proton.amqp.Symbol;
 import org.apache.qpid.proton.amqp.messaging.AmqpSequence;
 import org.apache.qpid.proton.amqp.messaging.AmqpValue;
 import org.apache.qpid.proton.amqp.messaging.Data;
-import org.apache.qpid.proton.amqp.messaging.MessageAnnotations;
 import org.apache.qpid.proton.message.Message;
 import org.junit.Test;
 
@@ -52,11 +51,10 @@ public class AmqpJmsTextMessageFacadeTest extends AmqpJmsMessageTypesTestCase {
     public void testNewMessageToSendContainsMessageTypeAnnotation() throws Exception {
         AmqpJmsTextMessageFacade amqpTextMessageFacade = createNewTextMessageFacade();
 
-        Message protonMessage = amqpTextMessageFacade.getAmqpMessage();
-        MessageAnnotations annotations = protonMessage.getMessageAnnotations();
-        Map<Symbol, Object> annotationsMap = annotations.getValue();
+        assertNotNull("MessageAnnotations section was not present", amqpTextMessageFacade.getMessageAnnotations());
+
+        Map<Symbol, Object> annotationsMap = amqpTextMessageFacade.getMessageAnnotations().getValue();
 
-        assertNotNull("MessageAnnotations section was not present", annotations);
         assertNotNull("MessageAnnotations section value was not present", annotationsMap);
 
         assertTrue("expected message type annotation to be present", annotationsMap.containsKey(AmqpMessageSupport.getSymbol(JMS_MSG_TYPE)));
@@ -104,9 +102,9 @@ public class AmqpJmsTextMessageFacadeTest extends AmqpJmsMessageTypesTestCase {
         AmqpJmsTextMessageFacade amqpTextMessageFacade = createNewTextMessageFacade();
 
         amqpTextMessageFacade.setText(text);
-        assertNotNull(amqpTextMessageFacade.getAmqpMessage().getBody());
-        assertTrue(amqpTextMessageFacade.getAmqpMessage().getBody() instanceof AmqpValue);
-        assertEquals(text, ((AmqpValue) amqpTextMessageFacade.getAmqpMessage().getBody()).getValue());
+        assertNotNull(amqpTextMessageFacade.getBody());
+        assertTrue(amqpTextMessageFacade.getBody() instanceof AmqpValue);
+        assertEquals(text, ((AmqpValue) amqpTextMessageFacade.getBody()).getValue());
 
         assertEquals(text, amqpTextMessageFacade.getText());
     }

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpMessageSupportTest.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpMessageSupportTest.java b/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpMessageSupportTest.java
index 68babcb..5d285bf 100644
--- a/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpMessageSupportTest.java
+++ b/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpMessageSupportTest.java
@@ -24,10 +24,8 @@ import static org.junit.Assert.assertTrue;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.qpid.proton.Proton;
 import org.apache.qpid.proton.amqp.Symbol;
 import org.apache.qpid.proton.amqp.messaging.MessageAnnotations;
-import org.apache.qpid.proton.message.Message;
 import org.junit.Test;
 
 public class AmqpMessageSupportTest {
@@ -50,25 +48,18 @@ public class AmqpMessageSupportTest {
     public void testGetMessageAnnotationWhenMessageHasAnnotationsMap() {
         Map<Symbol, Object> messageAnnotationsMap = new HashMap<Symbol,Object>();
         messageAnnotationsMap.put(Symbol.valueOf("x-opt-test"), Boolean.TRUE);
-        Message message = Proton.message();
-        message.setMessageAnnotations(new MessageAnnotations(messageAnnotationsMap));
-
-        assertNotNull(AmqpMessageSupport.getMessageAnnotation("x-opt-test", message));
+        assertNotNull(AmqpMessageSupport.getMessageAnnotation("x-opt-test", new MessageAnnotations(messageAnnotationsMap)));
     }
 
     @Test
     public void testGetMessageAnnotationWhenMessageHasEmptyAnnotationsMap() {
         Map<Symbol, Object> messageAnnotationsMap = new HashMap<Symbol,Object>();
-        Message message = Proton.message();
-        message.setMessageAnnotations(new MessageAnnotations(messageAnnotationsMap));
-
-        assertNull(AmqpMessageSupport.getMessageAnnotation("x-opt-test", message));
+        assertNull(AmqpMessageSupport.getMessageAnnotation("x-opt-test", new MessageAnnotations(messageAnnotationsMap)));
     }
 
     @Test
-    public void testGetMessageAnnotationWhenMessageHasNoAnnotationsMap() {
-        Message message = Proton.message();
-        assertNull(AmqpMessageSupport.getMessageAnnotation("x-opt-test", message));
+    public void testGetMessageAnnotationWhenMessageAnnotationHasNoAnnotationsMap() {
+        assertNull(AmqpMessageSupport.getMessageAnnotation("x-opt-test", new MessageAnnotations(null)));
     }
 
     @Test
@@ -79,35 +70,27 @@ public class AmqpMessageSupportTest {
     //---------- isContentType -----------------------------------------------//
 
     @Test
-    public void testIsContentTypeWithNullStringValueAndNullMessageContentType() {
-        Message message = Proton.message();
-        assertTrue(AmqpMessageSupport.isContentType(null, message));
+    public void testIsContentTypeWithNullStringValueAndNullContentType() {
+        assertTrue(AmqpMessageSupport.isContentType(null, null));
     }
 
     @Test
-    public void testIsContentTypeWithNonNullStringValueAndNullMessageContentType() {
-        Message message = Proton.message();
-        assertFalse(AmqpMessageSupport.isContentType("test", message));
+    public void testIsContentTypeWithNonNullStringValueAndNullContentType() {
+        assertFalse(AmqpMessageSupport.isContentType("test", null));
     }
 
     @Test
-    public void testIsContentTypeWithNonNullStringValueAndNonNullMessageContentTypeNotEqual() {
-        Message message = Proton.message();
-        message.setContentType("fails");
-        assertFalse(AmqpMessageSupport.isContentType("test", message));
+    public void testIsContentTypeWithNonNullStringValueAndNonNullContentTypeNotEqual() {
+        assertFalse(AmqpMessageSupport.isContentType("test", Symbol.valueOf("fails")));
     }
 
     @Test
-    public void testIsContentTypeWithNonNullStringValueAndNonNullMessageContentTypeEqual() {
-        Message message = Proton.message();
-        message.setContentType("test");
-        assertTrue(AmqpMessageSupport.isContentType("test", message));
+    public void testIsContentTypeWithNonNullStringValueAndNonNullContentTypeEqual() {
+        assertTrue(AmqpMessageSupport.isContentType("test", Symbol.valueOf("test")));
     }
 
     @Test
-    public void testIsContentTypeWithNullStringValueAndNonNullMessageContentType() {
-        Message message = Proton.message();
-        message.setContentType("test");
-        assertFalse(AmqpMessageSupport.isContentType(null, message));
+    public void testIsContentTypeWithNullStringValueAndNonNullContentType() {
+        assertFalse(AmqpMessageSupport.isContentType(null, Symbol.valueOf("test")));
     }
 }

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/failover/FailoverIntegrationTest.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/failover/FailoverIntegrationTest.java b/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/failover/FailoverIntegrationTest.java
index 0ecf81b..a082395 100644
--- a/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/failover/FailoverIntegrationTest.java
+++ b/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/failover/FailoverIntegrationTest.java
@@ -1149,7 +1149,6 @@ public class FailoverIntegrationTest extends QpidJmsTestCase {
 
             final int MSG_COUNT = 5;
 
-            Message message = session.createTextMessage("content");
             for (int i = 0; i < MSG_COUNT; ++i) {
                 testPeer.expectTransferButDoNotRespond(new TransferPayloadCompositeMatcher());
             }
@@ -1161,9 +1160,11 @@ public class FailoverIntegrationTest extends QpidJmsTestCase {
             TestJmsCompletionListener listener = new TestJmsCompletionListener(MSG_COUNT + 1);
             try {
                 for (int i = 0; i < MSG_COUNT; ++i) {
+                    Message message = session.createTextMessage("content");
                     producer.send(message, listener);
                 }
 
+                Message message = session.createTextMessage("content");
                 producer.send(message, listener);
             } catch (JMSException e) {
                 LOG.warn("Caught unexpected error: {}", e.getMessage());

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-interop-tests/qpid-jms-activemq-tests/src/test/java/org/apache/qpid/jms/usecases/JmsLargeMessageSendRecvTimedTest.java
----------------------------------------------------------------------
diff --git a/qpid-jms-interop-tests/qpid-jms-activemq-tests/src/test/java/org/apache/qpid/jms/usecases/JmsLargeMessageSendRecvTimedTest.java b/qpid-jms-interop-tests/qpid-jms-activemq-tests/src/test/java/org/apache/qpid/jms/usecases/JmsLargeMessageSendRecvTimedTest.java
index 11aa6a9..fb4f986 100644
--- a/qpid-jms-interop-tests/qpid-jms-activemq-tests/src/test/java/org/apache/qpid/jms/usecases/JmsLargeMessageSendRecvTimedTest.java
+++ b/qpid-jms-interop-tests/qpid-jms-activemq-tests/src/test/java/org/apache/qpid/jms/usecases/JmsLargeMessageSendRecvTimedTest.java
@@ -73,7 +73,7 @@ public class JmsLargeMessageSendRecvTimedTest extends AmqpTestSupport {
 
     @Test(timeout = 5 * 60 * 1000)
     public void testSend50MBMessage() throws Exception {
-        doTestSendLargeMessage(1024 * 1024 * 50);
+        doTestSendLargeMessage(1024 * 1024 * 100);
     }
 
     public void doTestSendLargeMessage(int expectedSize) throws Exception{


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@qpid.apache.org
For additional commands, e-mail: commits-help@qpid.apache.org


[5/5] qpid-jms git commit: QPIDJMS-215 Perform Message encoding at send time

Posted by ta...@apache.org.
QPIDJMS-215 Perform Message encoding at send time

Remove the need for message copying on send by doing encoding at the
time the message is sent instead of copying and encoding the message
later.


Project: http://git-wip-us.apache.org/repos/asf/qpid-jms/repo
Commit: http://git-wip-us.apache.org/repos/asf/qpid-jms/commit/657747b7
Tree: http://git-wip-us.apache.org/repos/asf/qpid-jms/tree/657747b7
Diff: http://git-wip-us.apache.org/repos/asf/qpid-jms/diff/657747b7

Branch: refs/heads/master
Commit: 657747b776fe60068b1bc51e8327f48e44d1e685
Parents: 22a517d
Author: Timothy Bish <ta...@gmail.com>
Authored: Tue Oct 18 11:24:13 2016 -0400
Committer: Timothy Bish <ta...@gmail.com>
Committed: Fri Oct 21 11:46:27 2016 -0400

----------------------------------------------------------------------
 .../java/org/apache/qpid/jms/JmsSession.java    |  49 +-
 .../qpid/jms/message/JmsBytesMessage.java       |   6 +
 .../org/apache/qpid/jms/message/JmsMessage.java |  52 +-
 .../jms/message/JmsOutboundMessageDispatch.java |   9 +
 .../qpid/jms/message/JmsStreamMessage.java      |   3 +-
 .../jms/message/facade/JmsMessageFacade.java    |  14 +-
 .../qpid/jms/provider/amqp/AmqpConsumer.java    |   4 +-
 .../jms/provider/amqp/AmqpFixedProducer.java    |  42 +-
 .../jms/provider/amqp/message/AmqpCodec.java    | 379 ++++++++++
 .../amqp/message/AmqpJmsBytesMessageFacade.java |  51 +-
 .../amqp/message/AmqpJmsMapMessageFacade.java   |  80 +-
 .../amqp/message/AmqpJmsMessageBuilder.java     | 197 -----
 .../amqp/message/AmqpJmsMessageFacade.java      | 568 ++++++++++----
 .../amqp/message/AmqpJmsMessageFactory.java     |  30 +-
 .../message/AmqpJmsObjectMessageFacade.java     |  70 +-
 .../message/AmqpJmsStreamMessageFacade.java     | 103 ++-
 .../amqp/message/AmqpJmsTextMessageFacade.java  |  51 +-
 .../amqp/message/AmqpMessageSupport.java        |  23 +-
 .../message/AmqpSerializedObjectDelegate.java   |  64 +-
 .../amqp/message/AmqpTypedObjectDelegate.java   |  78 +-
 .../amqp/message/AmqpWritableBuffer.java        | 113 +++
 .../BytesMessageIntegrationTest.java            | 105 +++
 .../integration/MapMessageIntegrationTest.java  | 158 ++++
 .../jms/integration/MessageIntegrationTest.java |  99 +++
 .../ObjectMessageIntegrationTest.java           | 104 +++
 .../integration/ProducerIntegrationTest.java    |  21 +-
 .../jms/integration/SessionIntegrationTest.java |   3 +-
 .../StreamMessageIntegrationTest.java           | 154 +++-
 .../integration/TextMessageIntegrationTest.java | 106 +++
 .../apache/qpid/jms/message/JmsMessageTest.java |   3 +-
 .../facade/test/JmsTestBytesMessageFacade.java  |   7 +
 .../facade/test/JmsTestMessageFacade.java       |   5 +
 .../provider/amqp/message/AmqpCodecTest.java    | 732 ++++++++++++++++++
 .../message/AmqpJmsBytesMessageFacadeTest.java  |  49 +-
 .../message/AmqpJmsMapMessageFacadeTest.java    |   3 +-
 .../amqp/message/AmqpJmsMessageBuilderTest.java | 737 -------------------
 .../amqp/message/AmqpJmsMessageFacadeTest.java  | 184 +++--
 .../message/AmqpJmsMessageTypesTestCase.java    |  68 +-
 .../message/AmqpJmsObjectMessageFacadeTest.java |  32 +-
 .../message/AmqpJmsStreamMessageFacadeTest.java |   6 +-
 .../message/AmqpJmsTextMessageFacadeTest.java   |  14 +-
 .../amqp/message/AmqpMessageSupportTest.java    |  45 +-
 .../failover/FailoverIntegrationTest.java       |   3 +-
 .../JmsLargeMessageSendRecvTimedTest.java       |   2 +-
 44 files changed, 2883 insertions(+), 1743 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/main/java/org/apache/qpid/jms/JmsSession.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/main/java/org/apache/qpid/jms/JmsSession.java b/qpid-jms-client/src/main/java/org/apache/qpid/jms/JmsSession.java
index 39d8002..2595498 100644
--- a/qpid-jms-client/src/main/java/org/apache/qpid/jms/JmsSession.java
+++ b/qpid-jms-client/src/main/java/org/apache/qpid/jms/JmsSession.java
@@ -758,46 +758,39 @@ public class JmsSession implements AutoCloseable, Session, QueueSession, TopicSe
                 messageId = producer.getMessageIDBuilder().createMessageID(producer.getProducerId().toString(), messageSequence);
             }
 
-            JmsMessage copy = null;
+            JmsMessage outbound = null;
             if (isJmsMessage) {
-                JmsMessage jmsMessage = (JmsMessage) original;
-                jmsMessage.getFacade().setProviderMessageIdObject(messageId);
-
-                if (connection.isPopulateJMSXUserID()) {
-                    jmsMessage.getFacade().setUserIdBytes(connection.getEncodedUsername());
-                } else {
-                    // Prevent user spoofing the user ID value.
-                    jmsMessage.getFacade().setUserId(null);
-                }
-
-                copy = jmsMessage.copy();
+                outbound = (JmsMessage) original;
             } else {
-                copy = JmsMessageTransformation.transformMessage(connection, original);
-                copy.getFacade().setProviderMessageIdObject(messageId);
-                copy.setJMSDestination(destination);
-
-                if (connection.isPopulateJMSXUserID()) {
-                    copy.getFacade().setUserIdBytes(connection.getEncodedUsername());
-                } else {
-                    // Prevent user spoofing the user ID value.
-                    copy.getFacade().setUserId(null);
-                }
+                // Transform and assign the Destination as one of our own destination objects.
+                outbound = JmsMessageTransformation.transformMessage(connection, original);
+                outbound.setJMSDestination(destination);
+            }
 
+            outbound.getFacade().setProviderMessageIdObject(messageId);
+            if (!isJmsMessage) {
                 // If the original was a foreign message, we still need to update it
-                // with the properly encoded Message ID String, get it from the copy.
-                original.setJMSMessageID(copy.getJMSMessageID());
+                // with the properly encoded Message ID String, get it from the one
+                // we transformed from now that it is set.
+                original.setJMSMessageID(outbound.getJMSMessageID());
             }
 
-            // Update the JmsMessage based copy with the required values.
-            copy.setConnection(connection);
+            // If configured set the User ID using the value we have encoded and cached,
+            // otherwise clear to prevent caller from spoofing the user ID value.
+            if (connection.isPopulateJMSXUserID()) {
+                outbound.getFacade().setUserIdBytes(connection.getEncodedUsername());
+            } else {
+                outbound.getFacade().setUserId(null);
+            }
 
             boolean sync = connection.isForceSyncSend() ||
                            (!connection.isForceAsyncSend() && deliveryMode == DeliveryMode.PERSISTENT && !getTransacted());
 
-            copy.onSend(timeToLive);
+            outbound.onSend(timeToLive);
 
             JmsOutboundMessageDispatch envelope = new JmsOutboundMessageDispatch();
-            envelope.setMessage(copy);
+            envelope.setMessage(outbound);
+            envelope.setPayload(outbound.getFacade().encodeMessage());
             envelope.setProducerId(producer.getProducerId());
             envelope.setDestination(destination);
             envelope.setSendAsync(listener == null ? !sync : true);

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/main/java/org/apache/qpid/jms/message/JmsBytesMessage.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/main/java/org/apache/qpid/jms/message/JmsBytesMessage.java b/qpid-jms-client/src/main/java/org/apache/qpid/jms/message/JmsBytesMessage.java
index 75bc30c..887cada 100644
--- a/qpid-jms-client/src/main/java/org/apache/qpid/jms/message/JmsBytesMessage.java
+++ b/qpid-jms-client/src/main/java/org/apache/qpid/jms/message/JmsBytesMessage.java
@@ -393,6 +393,12 @@ public class JmsBytesMessage extends JmsMessage implements BytesMessage {
     }
 
     @Override
+    public void onSend(long producerTtl) throws JMSException {
+        reset();
+        super.onSend(producerTtl);
+    }
+
+    @Override
     public String toString() {
         return "JmsBytesMessage { " + facade + " }";
     }

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/main/java/org/apache/qpid/jms/message/JmsMessage.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/main/java/org/apache/qpid/jms/message/JmsMessage.java b/qpid-jms-client/src/main/java/org/apache/qpid/jms/message/JmsMessage.java
index 05143d8..cddf04f 100644
--- a/qpid-jms-client/src/main/java/org/apache/qpid/jms/message/JmsMessage.java
+++ b/qpid-jms-client/src/main/java/org/apache/qpid/jms/message/JmsMessage.java
@@ -42,6 +42,7 @@ public class JmsMessage implements javax.jms.Message {
     protected transient JmsConnection connection;
 
     protected final JmsMessageFacade facade;
+    protected boolean readOnly;
     protected boolean readOnlyBody;
     protected boolean readOnlyProperties;
     protected boolean validatePropertyNames = true;
@@ -123,6 +124,7 @@ public class JmsMessage implements javax.jms.Message {
 
     @Override
     public void clearBody() throws JMSException {
+        checkReadOnly();
         readOnlyBody = false;
         facade.clearBody();
     }
@@ -135,6 +137,14 @@ public class JmsMessage implements javax.jms.Message {
         this.validatePropertyNames = validatePropertyNames;
     }
 
+    public boolean isReadOnly() {
+        return this.readOnly;
+    }
+
+    public void setReadOnly(boolean readOnly) {
+        this.readOnly = readOnly;
+    }
+
     public boolean isReadOnlyBody() {
         return this.readOnlyBody;
     }
@@ -163,6 +173,7 @@ public class JmsMessage implements javax.jms.Message {
 
     @Override
     public void setJMSMessageID(String value) throws JMSException {
+        checkReadOnly();
         facade.setMessageId(value);
     }
 
@@ -173,6 +184,7 @@ public class JmsMessage implements javax.jms.Message {
 
     @Override
     public void setJMSTimestamp(long timestamp) throws JMSException {
+        checkReadOnly();
         facade.setTimestamp(timestamp);
     }
 
@@ -183,6 +195,7 @@ public class JmsMessage implements javax.jms.Message {
 
     @Override
     public void setJMSCorrelationID(String correlationId) throws JMSException {
+        checkReadOnly();
         facade.setCorrelationId(correlationId);
     }
 
@@ -193,6 +206,7 @@ public class JmsMessage implements javax.jms.Message {
 
     @Override
     public void setJMSCorrelationIDAsBytes(byte[] correlationId) throws JMSException {
+        checkReadOnly();
         facade.setCorrelationIdBytes(correlationId);
     }
 
@@ -203,6 +217,7 @@ public class JmsMessage implements javax.jms.Message {
 
     @Override
     public void setJMSReplyTo(Destination destination) throws JMSException {
+        checkReadOnly();
         facade.setReplyTo(JmsMessageTransformation.transformDestination(connection, destination));
     }
 
@@ -213,6 +228,7 @@ public class JmsMessage implements javax.jms.Message {
 
     @Override
     public void setJMSDestination(Destination destination) throws JMSException {
+        checkReadOnly();
         facade.setDestination(JmsMessageTransformation.transformDestination(connection, destination));
     }
 
@@ -223,6 +239,7 @@ public class JmsMessage implements javax.jms.Message {
 
     @Override
     public void setJMSDeliveryMode(int mode) throws JMSException {
+        checkReadOnly();
         facade.setPersistent(mode == DeliveryMode.PERSISTENT);
     }
 
@@ -233,6 +250,7 @@ public class JmsMessage implements javax.jms.Message {
 
     @Override
     public void setJMSRedelivered(boolean redelivered) throws JMSException {
+        checkReadOnly();
         facade.setRedelivered(redelivered);
     }
 
@@ -243,6 +261,7 @@ public class JmsMessage implements javax.jms.Message {
 
     @Override
     public void setJMSType(String type) throws JMSException {
+        checkReadOnly();
         facade.setType(type);
     }
 
@@ -253,6 +272,7 @@ public class JmsMessage implements javax.jms.Message {
 
     @Override
     public void setJMSExpiration(long expiration) throws JMSException {
+        checkReadOnly();
         facade.setExpiration(expiration);
     }
 
@@ -263,6 +283,7 @@ public class JmsMessage implements javax.jms.Message {
 
     @Override
     public void setJMSPriority(int priority) throws JMSException {
+        checkReadOnly();
         facade.setPriority(priority);
     }
 
@@ -273,11 +294,13 @@ public class JmsMessage implements javax.jms.Message {
 
     @Override
     public void setJMSDeliveryTime(long deliveryTime) throws JMSException {
+        checkReadOnly();
         facade.setDeliveryTime(deliveryTime);
     }
 
     @Override
     public void clearProperties() throws JMSException {
+        checkReadOnly();
         JmsMessagePropertyIntercepter.clearProperties(this, true);
     }
 
@@ -307,6 +330,7 @@ public class JmsMessage implements javax.jms.Message {
 
     @Override
     public void setObjectProperty(String name, Object value) throws JMSException {
+        checkReadOnly();
         JmsMessagePropertyIntercepter.setProperty(this, name, value);
     }
 
@@ -406,7 +430,9 @@ public class JmsMessage implements javax.jms.Message {
     /**
      * Used to trigger processing required to place the message in a state where it is
      * ready to be written to the wire.  This processing can include such tasks as ensuring
-     * that the proper message headers are set or compressing message bodies etc.
+     * that the proper message headers are set or compressing message bodies etc.  During this
+     * call the message is placed in a read-only mode and will not be returned to a writable
+     * state until send completion is triggered.
      *
      * @param producerTtl
      *        the time to live value that the producer was configured with at send time.
@@ -414,19 +440,27 @@ public class JmsMessage implements javax.jms.Message {
      * @throws JMSException if an error occurs while preparing the message for send.
      */
     public void onSend(long producerTtl) throws JMSException {
-        setReadOnlyBody(true);
-        setReadOnlyProperties(true);
+        setReadOnly(true);
         facade.onSend(producerTtl);
     }
 
     /**
+     * Used to trigger processing required to place the message into a writable state once
+     * again following completion of the send operation.
+     */
+    public void onSendComplete() {
+        setReadOnly(false);
+    }
+
+    /**
      * Used to trigger processing required before dispatch of a message to its intended
-     * consumer.  This method should perform any needed unmarshal or message property
+     * consumer.  This method should perform any needed decoding or message property
      * processing prior to the message arriving at a consumer.
      *
      * @throws JMSException if an error occurs while preparing the message for dispatch.
      */
     public void onDispatch() throws JMSException {
+        setReadOnly(false);
         setReadOnlyBody(true);
         setReadOnlyProperties(true);
         facade.onDispatch();
@@ -456,14 +490,20 @@ public class JmsMessage implements javax.jms.Message {
 
     //----- State validation methods -----------------------------------------//
 
+    protected void checkReadOnly() throws MessageNotWriteableException {
+        if (readOnly) {
+            throw new MessageNotWriteableException("Message is currently read-only");
+        }
+    }
+
     protected void checkReadOnlyProperties() throws MessageNotWriteableException {
-        if (readOnlyProperties) {
+        if (readOnly || readOnlyProperties) {
             throw new MessageNotWriteableException("Message properties are read-only");
         }
     }
 
     protected void checkReadOnlyBody() throws MessageNotWriteableException {
-        if (readOnlyBody) {
+        if (readOnly || readOnlyBody) {
             throw new MessageNotWriteableException("Message body is read-only");
         }
     }

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/main/java/org/apache/qpid/jms/message/JmsOutboundMessageDispatch.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/main/java/org/apache/qpid/jms/message/JmsOutboundMessageDispatch.java b/qpid-jms-client/src/main/java/org/apache/qpid/jms/message/JmsOutboundMessageDispatch.java
index a34768f..d676d69 100644
--- a/qpid-jms-client/src/main/java/org/apache/qpid/jms/message/JmsOutboundMessageDispatch.java
+++ b/qpid-jms-client/src/main/java/org/apache/qpid/jms/message/JmsOutboundMessageDispatch.java
@@ -31,6 +31,7 @@ public class JmsOutboundMessageDispatch {
     private boolean presettle;
     private boolean completionRequired;
     private long dispatchId;
+    private Object payload;
 
     private transient String stringView;
 
@@ -54,6 +55,14 @@ public class JmsOutboundMessageDispatch {
         this.message = message;
     }
 
+    public Object getPayload() {
+        return payload;
+    }
+
+    public void setPayload(Object payload) {
+        this.payload = payload;
+    }
+
     public JmsProducerId getProducerId() {
         return producerId;
     }

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/main/java/org/apache/qpid/jms/message/JmsStreamMessage.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/main/java/org/apache/qpid/jms/message/JmsStreamMessage.java b/qpid-jms-client/src/main/java/org/apache/qpid/jms/message/JmsStreamMessage.java
index c9765f3..e55f8a0 100644
--- a/qpid-jms-client/src/main/java/org/apache/qpid/jms/message/JmsStreamMessage.java
+++ b/qpid-jms-client/src/main/java/org/apache/qpid/jms/message/JmsStreamMessage.java
@@ -48,8 +48,8 @@ public class JmsStreamMessage extends JmsMessage implements StreamMessage {
 
     @Override
     public void onSend(long producerTtl) throws JMSException {
-        super.onSend(producerTtl);
         reset();
+        super.onSend(producerTtl);
     }
 
     @Override
@@ -464,6 +464,7 @@ public class JmsStreamMessage extends JmsMessage implements StreamMessage {
 
     @Override
     public void reset() throws JMSException {
+        checkReadOnly();
         bytes = null;
         remainingBytes = NO_BYTES_IN_FLIGHT;
         setReadOnlyBody(true);

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/main/java/org/apache/qpid/jms/message/facade/JmsMessageFacade.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/main/java/org/apache/qpid/jms/message/facade/JmsMessageFacade.java b/qpid-jms-client/src/main/java/org/apache/qpid/jms/message/facade/JmsMessageFacade.java
index d5616f5..048ff52 100644
--- a/qpid-jms-client/src/main/java/org/apache/qpid/jms/message/facade/JmsMessageFacade.java
+++ b/qpid-jms-client/src/main/java/org/apache/qpid/jms/message/facade/JmsMessageFacade.java
@@ -27,12 +27,6 @@ import org.apache.qpid.jms.JmsDestination;
  * own Message type and the JMS Message types.  A Provider can implement the Facade
  * interface and offer direct access to its message types without the need to
  * copy to / from a more generic JMS message instance.
- *
- * TODO - What exceptions if any do we really need to be throwing here.  For get methods
- *        we should synthesize an answer regardless and for set most of the checking for
- *        JMS compliance happens in the JMS message level.  Methods like setMessageId and
- *        setCorrelationId might need to although we should try and validate some at the
- *        upper level.
  */
 public interface JmsMessageFacade {
 
@@ -90,6 +84,7 @@ public interface JmsMessageFacade {
      * The method allows for passing through producer configuration details not
      * explicitly mapped into the JMS Message allowing the facade to create the
      * most correct and compact message on the wire.
+     *
      * @param producerTtl
      *        the time to live value configured on the producer when sent.
      *
@@ -455,4 +450,11 @@ public interface JmsMessageFacade {
      */
     boolean hasBody();
 
+    /**
+     * Encodes the protocol level Message instance for transmission.
+     *
+     * @return an Object that represents the encoded form of the message for the target provider.
+     */
+    Object encodeMessage();
+
 }

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/AmqpConsumer.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/AmqpConsumer.java b/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/AmqpConsumer.java
index 89586e3..0077478 100644
--- a/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/AmqpConsumer.java
+++ b/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/AmqpConsumer.java
@@ -40,7 +40,7 @@ import org.apache.qpid.jms.meta.JmsConsumerInfo;
 import org.apache.qpid.jms.provider.AsyncResult;
 import org.apache.qpid.jms.provider.ProviderConstants.ACK_TYPE;
 import org.apache.qpid.jms.provider.ProviderListener;
-import org.apache.qpid.jms.provider.amqp.message.AmqpJmsMessageBuilder;
+import org.apache.qpid.jms.provider.amqp.message.AmqpCodec;
 import org.apache.qpid.jms.util.IOExceptionSupport;
 import org.apache.qpid.proton.amqp.Binary;
 import org.apache.qpid.proton.amqp.messaging.Accepted;
@@ -420,7 +420,7 @@ public class AmqpConsumer extends AmqpAbstractResource<JmsConsumerInfo, Receiver
         incoming.setDefaultDeliveryState(Released.getInstance());
         JmsMessage message = null;
         try {
-            message = AmqpJmsMessageBuilder.createJmsMessage(this, unwrapIncomingMessage(incoming));
+            message = AmqpCodec.decodeMessage(this, unwrapIncomingMessage(incoming)).asJmsMessage();
         } catch (Exception e) {
             LOG.warn("Error on transform: {}", e.getMessage());
             // TODO - We could signal provider error but not sure we want to fail

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/AmqpFixedProducer.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/AmqpFixedProducer.java b/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/AmqpFixedProducer.java
index df39b78..07223d8 100644
--- a/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/AmqpFixedProducer.java
+++ b/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/AmqpFixedProducer.java
@@ -29,11 +29,9 @@ import javax.jms.JMSException;
 
 import org.apache.qpid.jms.JmsSendTimedOutException;
 import org.apache.qpid.jms.message.JmsOutboundMessageDispatch;
-import org.apache.qpid.jms.message.facade.JmsMessageFacade;
 import org.apache.qpid.jms.meta.JmsConnectionInfo;
 import org.apache.qpid.jms.meta.JmsProducerInfo;
 import org.apache.qpid.jms.provider.AsyncResult;
-import org.apache.qpid.jms.provider.amqp.message.AmqpJmsMessageFacade;
 import org.apache.qpid.jms.util.IOExceptionSupport;
 import org.apache.qpid.proton.amqp.Binary;
 import org.apache.qpid.proton.amqp.messaging.Accepted;
@@ -47,10 +45,11 @@ import org.apache.qpid.proton.amqp.transport.ErrorCondition;
 import org.apache.qpid.proton.amqp.transport.SenderSettleMode;
 import org.apache.qpid.proton.engine.Delivery;
 import org.apache.qpid.proton.engine.Sender;
-import org.apache.qpid.proton.message.Message;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import io.netty.buffer.ByteBuf;
+
 /**
  * AMQP Producer object that is used to manage JMS MessageProducer semantics.
  *
@@ -64,7 +63,6 @@ public class AmqpFixedProducer extends AmqpProducer {
     private final AmqpTransferTagGenerator tagGenerator = new AmqpTransferTagGenerator(true);
     private final Map<Object, InFlightSend> sent = new LinkedHashMap<Object, InFlightSend>();
     private final Map<Object, InFlightSend> blocked = new LinkedHashMap<Object, InFlightSend>();
-    private byte[] encodeBuffer = new byte[1024 * 8];
 
     private AsyncResult sendCompletionWatcher;
 
@@ -128,7 +126,6 @@ public class AmqpFixedProducer extends AmqpProducer {
 
         LOG.trace("Producer sending message: {}", envelope);
 
-        JmsMessageFacade facade = envelope.getMessage().getFacade();
         boolean presettle = envelope.isPresettle() || isPresettle();
         Delivery delivery = null;
 
@@ -148,8 +145,9 @@ public class AmqpFixedProducer extends AmqpProducer {
             context.registerTxProducer(this);
         }
 
-        AmqpJmsMessageFacade amqpMessageFacade = (AmqpJmsMessageFacade) facade;
-        encodeAndSend(amqpMessageFacade.getAmqpMessage(), delivery);
+        // Write the already encoded AMQP message into the Sender
+        ByteBuf encoded = (ByteBuf) envelope.getPayload();
+        getEndpoint().send(encoded.array(), encoded.arrayOffset() + encoded.readerIndex(), encoded.readableBytes());
 
         AmqpProvider provider = getParent().getProvider();
 
@@ -190,33 +188,6 @@ public class AmqpFixedProducer extends AmqpProducer {
         }
     }
 
-    private void encodeAndSend(Message message, Delivery delivery) throws IOException {
-
-        int encodedSize;
-        while (true) {
-            try {
-                encodedSize = message.encode(encodeBuffer, 0, encodeBuffer.length);
-                break;
-            } catch (java.nio.BufferOverflowException e) {
-                encodeBuffer = new byte[encodeBuffer.length * 2];
-            }
-        }
-
-        int sentSoFar = 0;
-
-        while (true) {
-            int sent = getEndpoint().send(encodeBuffer, sentSoFar, encodedSize - sentSoFar);
-            if (sent > 0) {
-                sentSoFar += sent;
-                if ((encodedSize - sentSoFar) == 0) {
-                    break;
-                }
-            } else {
-                LOG.warn("{} failed to send any data from current Message.", this);
-            }
-        }
-    }
-
     @Override
     public void processFlowUpdates(AmqpProvider provider) throws IOException {
         if (!blocked.isEmpty() && getEndpoint().getCredit() > 0) {
@@ -433,6 +404,9 @@ public class AmqpFixedProducer extends AmqpProducer {
                 blocked.remove(envelope.getMessageId());
             }
 
+            // Put the message back to usable state following send complete
+            envelope.getMessage().onSendComplete();
+
             // TODO - Should this take blocked sends into consideration.
             // Signal the watcher that all pending sends have completed if one is registered
             // and both the in-flight sends and blocked sends have completed.

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpCodec.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpCodec.java b/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpCodec.java
new file mode 100644
index 0000000..5e98d69
--- /dev/null
+++ b/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpCodec.java
@@ -0,0 +1,379 @@
+/*
+ * 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.qpid.jms.provider.amqp.message;
+
+import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.JMS_BYTES_MESSAGE;
+import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.JMS_MAP_MESSAGE;
+import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.JMS_MESSAGE;
+import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.JMS_MSG_TYPE;
+import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.JMS_OBJECT_MESSAGE;
+import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.JMS_STREAM_MESSAGE;
+import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.JMS_TEXT_MESSAGE;
+import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.OCTET_STREAM_CONTENT_TYPE;
+import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.SERIALIZED_JAVA_OBJECT_CONTENT_TYPE;
+import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.isContentType;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+
+import org.apache.qpid.jms.provider.amqp.AmqpConsumer;
+import org.apache.qpid.jms.util.ContentTypeSupport;
+import org.apache.qpid.jms.util.InvalidContentTypeException;
+import org.apache.qpid.proton.amqp.Binary;
+import org.apache.qpid.proton.amqp.Symbol;
+import org.apache.qpid.proton.amqp.messaging.AmqpSequence;
+import org.apache.qpid.proton.amqp.messaging.AmqpValue;
+import org.apache.qpid.proton.amqp.messaging.ApplicationProperties;
+import org.apache.qpid.proton.amqp.messaging.Data;
+import org.apache.qpid.proton.amqp.messaging.DeliveryAnnotations;
+import org.apache.qpid.proton.amqp.messaging.Footer;
+import org.apache.qpid.proton.amqp.messaging.Header;
+import org.apache.qpid.proton.amqp.messaging.MessageAnnotations;
+import org.apache.qpid.proton.amqp.messaging.Properties;
+import org.apache.qpid.proton.amqp.messaging.Section;
+import org.apache.qpid.proton.codec.AMQPDefinedTypes;
+import org.apache.qpid.proton.codec.DecoderImpl;
+import org.apache.qpid.proton.codec.EncoderImpl;
+import org.apache.qpid.proton.codec.WritableBuffer;
+
+import io.netty.buffer.ByteBuf;
+
+/**
+ * AMQP Codec class used to hide the details of encode / decode
+ */
+public final class AmqpCodec {
+
+    private static class EncoderDecoderPair {
+        DecoderImpl decoder = new DecoderImpl();
+        EncoderImpl encoder = new EncoderImpl(decoder);
+        {
+            AMQPDefinedTypes.registerAllTypes(decoder, encoder);
+        }
+    }
+
+    private static final ThreadLocal<EncoderDecoderPair> TLS_CODEC = new ThreadLocal<EncoderDecoderPair>() {
+        @Override
+        protected EncoderDecoderPair initialValue() {
+            return new EncoderDecoderPair();
+        }
+    };
+
+    /**
+     * @return a Encoder instance.
+     */
+    public static EncoderImpl getEncoder() {
+        return TLS_CODEC.get().encoder;
+    }
+
+    /**
+     * @return a Decoder instance.
+     */
+    public static DecoderImpl getDecoder() {
+        return TLS_CODEC.get().decoder;
+    }
+
+    /**
+     * Given an AMQP Section encode it and return the buffer holding the encoded value
+     *
+     * @param section
+     *      the AMQP Section value to encode.
+     *
+     * @return a buffer holding the encoded bytes of the given AMQP Section object.
+     */
+    public static ByteBuf encode(Section section) {
+        if (section == null) {
+            return null;
+        }
+
+        AmqpWritableBuffer buffer = new AmqpWritableBuffer();
+
+        EncoderImpl encoder = getEncoder();
+        encoder.setByteBuffer(buffer);
+        encoder.writeObject(section);
+        encoder.setByteBuffer((WritableBuffer) null);
+
+        return buffer.getBuffer();
+    }
+
+    /**
+     * Given an encoded AMQP Section, decode the value previously written there.
+     *
+     * @param encoded
+     *      the AMQP Section value to decode.
+     *
+     * @return a Section object read from its encoded form.
+     */
+    public static Section decode(ByteBuf encoded) {
+        if (encoded == null || !encoded.isReadable()) {
+            return null;
+        }
+
+        DecoderImpl decoder = TLS_CODEC.get().decoder;
+        decoder.setByteBuffer(encoded.nioBuffer());
+        Section result = (Section) decoder.readObject();
+        decoder.setByteBuffer(null);
+        encoded.resetReaderIndex();
+
+        return result;
+    }
+
+    /**
+     * Given a Message instance, encode the Message to the wire level representation
+     * of that Message.
+     *
+     * @param message
+     *      the Message that is to be encoded into the wire level representation.
+     *
+     * @return a buffer containing the wire level representation of the input Message.
+     */
+    public static ByteBuf encodeMessage(AmqpJmsMessageFacade message) {
+        AmqpWritableBuffer buffer = new AmqpWritableBuffer();
+
+        EncoderImpl encoder = getEncoder();
+        encoder.setByteBuffer(buffer);
+
+        if (message.getHeader() != null) {
+            encoder.writeObject(message.getHeader());
+        }
+        if (message.getDeliveryAnnotations() != null) {
+            encoder.writeObject(message.getDeliveryAnnotations());
+        }
+        if (message.getMessageAnnotations() != null) {
+            encoder.writeObject(message.getMessageAnnotations());
+        }
+        if (message.getProperties() != null) {
+            encoder.writeObject(message.getProperties());
+        }
+        if (message.getApplicationProperties() != null) {
+            encoder.writeObject(message.getApplicationProperties());
+        }
+        if (message.getBody() != null) {
+            encoder.writeObject(message.getBody());
+        }
+        if (message.getFooter() != null) {
+            encoder.writeObject(message.getFooter());
+        }
+
+        encoder.setByteBuffer((WritableBuffer) null);
+
+        return buffer.getBuffer();
+    }
+
+    /**
+     * Create a new JmsMessage and underlying JmsMessageFacade that represents the proper
+     * message type for the incoming AMQP message.
+     *
+     * @param consumer
+     *        The AmqpConsumer instance that will be linked to the decoded message.
+     * @param messageBytes
+     *        The the raw bytes that compose the incoming message. (Read-Only)
+     *
+     * @return a AmqpJmsMessageFacade instance decoded from the message bytes.
+     *
+     * @throws IOException if an error occurs while creating the message objects.
+     */
+    public static AmqpJmsMessageFacade decodeMessage(AmqpConsumer consumer, ByteBuf messageBytes) throws IOException {
+
+        DecoderImpl decoder = getDecoder();
+        ByteBuffer buffer = messageBytes.nioBuffer();
+        decoder.setByteBuffer(buffer);
+
+        Header header = null;
+        DeliveryAnnotations deliveryAnnotations = null;
+        MessageAnnotations messageAnnotations = null;
+        Properties properties = null;
+        ApplicationProperties applicationProperties = null;
+        Section body = null;
+        Footer footer = null;
+        Section section = null;
+
+        if (buffer.hasRemaining()) {
+            section = (Section) decoder.readObject();
+        }
+
+        if (section instanceof Header) {
+            header = (Header) section;
+            if (buffer.hasRemaining()) {
+                section = (Section) decoder.readObject();
+            } else {
+                section = null;
+            }
+
+        }
+        if (section instanceof DeliveryAnnotations) {
+            deliveryAnnotations = (DeliveryAnnotations) section;
+
+            if (buffer.hasRemaining()) {
+                section = (Section) decoder.readObject();
+            } else {
+                section = null;
+            }
+
+        }
+        if (section instanceof MessageAnnotations) {
+            messageAnnotations = (MessageAnnotations) section;
+
+            if (buffer.hasRemaining()) {
+                section = (Section) decoder.readObject();
+            } else {
+                section = null;
+            }
+
+        }
+        if (section instanceof Properties) {
+            properties = (Properties) section;
+
+            if (buffer.hasRemaining()) {
+                section = (Section) decoder.readObject();
+            } else {
+                section = null;
+            }
+
+        }
+        if (section instanceof ApplicationProperties) {
+            applicationProperties = (ApplicationProperties) section;
+
+            if (buffer.hasRemaining()) {
+                section = (Section) decoder.readObject();
+            } else {
+                section = null;
+            }
+
+        }
+        if (section != null && !(section instanceof Footer)) {
+            body = section;
+
+            if (buffer.hasRemaining()) {
+                section = (Section) decoder.readObject();
+            } else {
+                section = null;
+            }
+
+        }
+        if (section instanceof Footer) {
+            footer = (Footer) section;
+        }
+
+        decoder.setByteBuffer(null);
+        messageBytes.resetReaderIndex();
+
+        // First we try the easy way, if the annotation is there we don't have to work hard.
+        AmqpJmsMessageFacade result = createFromMsgAnnotation(messageAnnotations);
+        if (result == null) {
+            // Next, match specific section structures and content types
+            result = createWithoutAnnotation(body, properties);
+        }
+
+        if (result != null) {
+            result.setHeader(header);
+            result.setDeliveryAnnotations(deliveryAnnotations);
+            result.setMessageAnnotations(messageAnnotations);
+            result.setProperties(properties);
+            result.setApplicationProperties(applicationProperties);
+            result.setBody(body);
+            result.setFooter(footer);
+            result.initialize(consumer);
+
+            return result;
+        }
+
+        throw new IOException("Could not create a JMS message from incoming message");
+    }
+
+    private static AmqpJmsMessageFacade createFromMsgAnnotation(MessageAnnotations messageAnnotations) throws IOException {
+        Object annotation = AmqpMessageSupport.getMessageAnnotation(JMS_MSG_TYPE, messageAnnotations);
+        if (annotation != null) {
+            switch ((byte) annotation) {
+                case JMS_MESSAGE:
+                    return new AmqpJmsMessageFacade();
+                case JMS_BYTES_MESSAGE:
+                    return new AmqpJmsBytesMessageFacade();
+                case JMS_TEXT_MESSAGE:
+                    return new AmqpJmsTextMessageFacade(StandardCharsets.UTF_8);
+                case JMS_MAP_MESSAGE:
+                    return new AmqpJmsMapMessageFacade();
+                case JMS_STREAM_MESSAGE:
+                    return new AmqpJmsStreamMessageFacade();
+                case JMS_OBJECT_MESSAGE:
+                    return new AmqpJmsObjectMessageFacade();
+                default:
+                    throw new IOException("Invalid JMS Message Type annotation value found in message: " + annotation);
+            }
+        }
+
+        return null;
+    }
+
+    private static AmqpJmsMessageFacade createWithoutAnnotation(Section body, Properties properties) {
+        Symbol messageContentType = properties != null ? properties.getContentType() : null;
+
+        if (body == null) {
+            if (isContentType(SERIALIZED_JAVA_OBJECT_CONTENT_TYPE, messageContentType)) {
+                return new AmqpJmsObjectMessageFacade();
+            } else if (isContentType(OCTET_STREAM_CONTENT_TYPE, messageContentType) || isContentType(null, messageContentType)) {
+                return new AmqpJmsBytesMessageFacade();
+            } else {
+                Charset charset = getCharsetForTextualContent(messageContentType);
+                if (charset != null) {
+                    return new AmqpJmsTextMessageFacade(charset);
+                } else {
+                    return new AmqpJmsMessageFacade();
+                }
+            }
+        } else if (body instanceof Data) {
+            if (isContentType(OCTET_STREAM_CONTENT_TYPE, messageContentType) || isContentType(null, messageContentType)) {
+                return new AmqpJmsBytesMessageFacade();
+            } else if (isContentType(SERIALIZED_JAVA_OBJECT_CONTENT_TYPE, messageContentType)) {
+                return new AmqpJmsObjectMessageFacade();
+            } else {
+                Charset charset = getCharsetForTextualContent(messageContentType);
+                if (charset != null) {
+                    return new AmqpJmsTextMessageFacade(charset);
+                } else {
+                    return new AmqpJmsBytesMessageFacade();
+                }
+            }
+        } else if (body instanceof AmqpValue) {
+            Object value = ((AmqpValue) body).getValue();
+
+            if (value == null || value instanceof String) {
+                return new AmqpJmsTextMessageFacade(StandardCharsets.UTF_8);
+            } else if (value instanceof Binary) {
+                return new AmqpJmsBytesMessageFacade();
+            } else {
+                return new AmqpJmsObjectMessageFacade();
+            }
+        } else if (body instanceof AmqpSequence) {
+            return new AmqpJmsObjectMessageFacade();
+        }
+
+        return null;
+    }
+
+    private static Charset getCharsetForTextualContent(Symbol messageContentType) {
+        if (messageContentType != null) {
+            try {
+                return ContentTypeSupport.parseContentTypeForTextualCharset(messageContentType.toString());
+            } catch (InvalidContentTypeException e) {
+            }
+        }
+
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsBytesMessageFacade.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsBytesMessageFacade.java b/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsBytesMessageFacade.java
index 6b63fb7..30fc9c3 100644
--- a/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsBytesMessageFacade.java
+++ b/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsBytesMessageFacade.java
@@ -17,7 +17,6 @@
 package org.apache.qpid.jms.provider.amqp.message;
 
 import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.JMS_BYTES_MESSAGE;
-import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.JMS_MSG_TYPE;
 import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.OCTET_STREAM_CONTENT_TYPE;
 
 import java.io.IOException;
@@ -27,14 +26,12 @@ import java.io.OutputStream;
 import javax.jms.IllegalStateException;
 import javax.jms.JMSException;
 
+import org.apache.qpid.jms.message.JmsBytesMessage;
 import org.apache.qpid.jms.message.facade.JmsBytesMessageFacade;
-import org.apache.qpid.jms.provider.amqp.AmqpConnection;
-import org.apache.qpid.jms.provider.amqp.AmqpConsumer;
 import org.apache.qpid.proton.amqp.Binary;
 import org.apache.qpid.proton.amqp.messaging.AmqpValue;
 import org.apache.qpid.proton.amqp.messaging.Data;
 import org.apache.qpid.proton.amqp.messaging.Section;
-import org.apache.qpid.proton.message.Message;
 
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.ByteBufInputStream;
@@ -53,45 +50,26 @@ public class AmqpJmsBytesMessageFacade extends AmqpJmsMessageFacade implements J
     private transient ByteBufInputStream bytesIn;
     private transient ByteBufOutputStream bytesOut;
 
-    /**
-     * Creates a new facade instance
-     *
-     * @param connection
-     *        the AmqpConnection that under which this facade was created.
-     */
-    public AmqpJmsBytesMessageFacade(AmqpConnection connection) {
-        super(connection);
+    @Override
+    protected void initializeEmptyBody() {
         setContentType(OCTET_STREAM_CONTENT_TYPE);
-        setMessageAnnotation(JMS_MSG_TYPE, JMS_BYTES_MESSAGE);
-        getAmqpMessage().setBody(EMPTY_BODY);
-    }
-
-    /**
-     * Creates a new Facade around an incoming AMQP Message for dispatch to the
-     * JMS Consumer instance.
-     *
-     * @param consumer
-     *        the consumer that received this message.
-     * @param message
-     *        the incoming Message instance that is being wrapped.
-     */
-    public AmqpJmsBytesMessageFacade(AmqpConsumer consumer, Message message) {
-        super(consumer, message);
+        setBody(EMPTY_BODY);
     }
 
     @Override
     public AmqpJmsBytesMessageFacade copy() {
         reset();
-        AmqpJmsBytesMessageFacade copy = new AmqpJmsBytesMessageFacade(connection);
+        AmqpJmsBytesMessageFacade copy = new AmqpJmsBytesMessageFacade();
         copyInto(copy);
 
         Binary payload = getBinaryFromBody();
+        copy.setContentType(OCTET_STREAM_CONTENT_TYPE);
         if (payload.getLength() > 0) {
             byte[] result = new byte[payload.getLength()];
             System.arraycopy(payload.getArray(), payload.getArrayOffset(), result, 0, payload.getLength());
-            copy.message.setBody(new Data(new Binary(result)));
+            copy.setBody(new Data(new Binary(result)));
         } else {
-            copy.message.setBody(EMPTY_BODY);
+            copy.setBody(EMPTY_BODY);
         }
 
         return copy;
@@ -120,7 +98,7 @@ public class AmqpJmsBytesMessageFacade extends AmqpJmsMessageFacade implements J
             bytesOut = null;
         }
 
-        message.setBody(EMPTY_BODY);
+        setBody(EMPTY_BODY);
     }
 
     @Override
@@ -147,7 +125,7 @@ public class AmqpJmsBytesMessageFacade extends AmqpJmsMessageFacade implements J
 
         if (bytesOut == null) {
             bytesOut = new ByteBufOutputStream(Unpooled.buffer());
-            message.setBody(EMPTY_BODY);
+            setBody(EMPTY_BODY);
         }
 
         return bytesOut;
@@ -158,7 +136,7 @@ public class AmqpJmsBytesMessageFacade extends AmqpJmsMessageFacade implements J
         if (bytesOut != null) {
             ByteBuf writeBuf = bytesOut.buffer();
             Binary body = new Binary(writeBuf.array(), writeBuf.arrayOffset(), writeBuf.readableBytes());
-            message.setBody(new Data(body));
+            setBody(new Data(body));
             try {
                 bytesOut.close();
             } catch (IOException e) {
@@ -185,7 +163,7 @@ public class AmqpJmsBytesMessageFacade extends AmqpJmsMessageFacade implements J
      * @return the body binary, or empty substitute if there is none
      */
     private Binary getBinaryFromBody() {
-        Section body = getAmqpMessage().getBody();
+        Section body = getBody();
         Binary result = EMPTY_BINARY;
 
         if (body == null) {
@@ -222,6 +200,11 @@ public class AmqpJmsBytesMessageFacade extends AmqpJmsMessageFacade implements J
     }
 
     @Override
+    public JmsBytesMessage asJmsMessage() {
+        return new JmsBytesMessage(this);
+    }
+
+    @Override
     public byte[] copyBody() {
         Binary content = getBinaryFromBody();
         byte[] result = new byte[content.getLength()];

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMapMessageFacade.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMapMessageFacade.java b/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMapMessageFacade.java
index fe242f7..2f0a57a 100644
--- a/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMapMessageFacade.java
+++ b/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMapMessageFacade.java
@@ -17,7 +17,6 @@
 package org.apache.qpid.jms.provider.amqp.message;
 
 import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.JMS_MAP_MESSAGE;
-import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.JMS_MSG_TYPE;
 
 import java.util.Arrays;
 import java.util.Collections;
@@ -25,13 +24,11 @@ import java.util.Enumeration;
 import java.util.LinkedHashMap;
 import java.util.Map;
 
+import org.apache.qpid.jms.message.JmsMapMessage;
 import org.apache.qpid.jms.message.facade.JmsMapMessageFacade;
-import org.apache.qpid.jms.provider.amqp.AmqpConnection;
-import org.apache.qpid.jms.provider.amqp.AmqpConsumer;
 import org.apache.qpid.proton.amqp.Binary;
 import org.apache.qpid.proton.amqp.messaging.AmqpValue;
 import org.apache.qpid.proton.amqp.messaging.Section;
-import org.apache.qpid.proton.message.Message;
 
 /**
  * Wrapper around an AMQP Message instance that will be treated as a JMS MapMessage
@@ -42,48 +39,6 @@ public class AmqpJmsMapMessageFacade extends AmqpJmsMessageFacade implements Jms
     private Map<String,Object> messageBodyMap;
 
     /**
-     * Create a new facade ready for sending.
-     *
-     * @param connection
-     *        the AmqpConnection that under which this facade was created.
-     */
-    public AmqpJmsMapMessageFacade(AmqpConnection connection) {
-        super(connection);
-        initializeEmptyBody();
-        setMessageAnnotation(JMS_MSG_TYPE, JMS_MAP_MESSAGE);
-    }
-
-    /**
-     * Creates a new Facade around an incoming AMQP Message for dispatch to the
-     * JMS Consumer instance.
-     *
-     * @param consumer
-     *        the consumer that received this message.
-     * @param message
-     *        the incoming Message instance that is being wrapped.
-     */
-    @SuppressWarnings("unchecked")
-    public AmqpJmsMapMessageFacade(AmqpConsumer consumer, Message message) {
-        super(consumer, message);
-
-        Section body = getAmqpMessage().getBody();
-        if (body == null) {
-            initializeEmptyBody();
-        } else if (body instanceof AmqpValue) {
-            Object o = ((AmqpValue) body).getValue();
-            if (o == null) {
-                initializeEmptyBody();
-            } else if (o instanceof Map) {
-                messageBodyMap = (Map<String, Object>) o;
-            } else {
-                throw new IllegalStateException("Unexpected message body type: " + body.getClass().getSimpleName());
-            }
-        } else {
-            throw new IllegalStateException("Unexpected message body type: " + body.getClass().getSimpleName());
-        }
-    }
-
-    /**
      * @return the appropriate byte value that indicates the type of message this is.
      */
     @Override
@@ -93,8 +48,9 @@ public class AmqpJmsMapMessageFacade extends AmqpJmsMessageFacade implements Jms
 
     @Override
     public AmqpJmsMapMessageFacade copy() {
-        AmqpJmsMapMessageFacade copy = new AmqpJmsMapMessageFacade(connection);
+        AmqpJmsMapMessageFacade copy = new AmqpJmsMapMessageFacade();
         copyInto(copy);
+        copy.initializeEmptyBody();
         copy.messageBodyMap.putAll(messageBodyMap);
         return copy;
     }
@@ -146,10 +102,36 @@ public class AmqpJmsMapMessageFacade extends AmqpJmsMessageFacade implements Jms
         return !messageBodyMap.isEmpty();
     }
 
-    private void initializeEmptyBody() {
+    @Override
+    public JmsMapMessage asJmsMessage() {
+        return new JmsMapMessage(this);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    void setBody(Section body) {
+        if (body == null) {
+            initializeEmptyBody();
+        } else if (body instanceof AmqpValue) {
+            Object o = ((AmqpValue) body).getValue();
+            if (o == null) {
+                initializeEmptyBody();
+            } else if (o instanceof Map) {
+                messageBodyMap = (Map<String, Object>) o;
+                super.setBody(body);
+            } else {
+                throw new IllegalStateException("Unexpected message body type: " + body.getClass().getSimpleName());
+            }
+        } else {
+            throw new IllegalStateException("Unexpected message body type: " + body.getClass().getSimpleName());
+        }
+    }
+
+    @Override
+    protected void initializeEmptyBody() {
         // Using LinkedHashMap because AMQP map equality considers order,
         // so we should behave in as predictable a manner as possible
         messageBodyMap = new LinkedHashMap<String, Object>();
-        getAmqpMessage().setBody(new AmqpValue(messageBodyMap));
+        super.setBody(new AmqpValue(messageBodyMap));
     }
 }

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMessageBuilder.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMessageBuilder.java b/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMessageBuilder.java
deleted file mode 100644
index 4385246..0000000
--- a/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMessageBuilder.java
+++ /dev/null
@@ -1,197 +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.qpid.jms.provider.amqp.message;
-
-import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.JMS_BYTES_MESSAGE;
-import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.JMS_MAP_MESSAGE;
-import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.JMS_MESSAGE;
-import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.JMS_MSG_TYPE;
-import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.JMS_OBJECT_MESSAGE;
-import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.JMS_STREAM_MESSAGE;
-import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.JMS_TEXT_MESSAGE;
-import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.OCTET_STREAM_CONTENT_TYPE;
-import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.SERIALIZED_JAVA_OBJECT_CONTENT_TYPE;
-import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.decodeMessage;
-import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.isContentType;
-
-import java.io.IOException;
-import java.nio.charset.Charset;
-import java.nio.charset.StandardCharsets;
-
-import org.apache.qpid.jms.message.JmsBytesMessage;
-import org.apache.qpid.jms.message.JmsMapMessage;
-import org.apache.qpid.jms.message.JmsMessage;
-import org.apache.qpid.jms.message.JmsObjectMessage;
-import org.apache.qpid.jms.message.JmsStreamMessage;
-import org.apache.qpid.jms.message.JmsTextMessage;
-import org.apache.qpid.jms.provider.amqp.AmqpConsumer;
-import org.apache.qpid.jms.util.ContentTypeSupport;
-import org.apache.qpid.jms.util.InvalidContentTypeException;
-import org.apache.qpid.proton.amqp.Binary;
-import org.apache.qpid.proton.amqp.messaging.AmqpSequence;
-import org.apache.qpid.proton.amqp.messaging.AmqpValue;
-import org.apache.qpid.proton.amqp.messaging.Data;
-import org.apache.qpid.proton.amqp.messaging.Section;
-import org.apache.qpid.proton.message.Message;
-
-import io.netty.buffer.ByteBuf;
-
-/**
- * Builder class used to construct the appropriate JmsMessage / JmsMessageFacade
- * objects to wrap an incoming AMQP Message.
- */
-public class AmqpJmsMessageBuilder {
-
-    /**
-     * Create a new JmsMessage and underlying JmsMessageFacade that represents the proper
-     * message type for the incoming AMQP message.
-     *
-     * @param consumer
-     *        The provider AMQP Consumer instance where this message arrived at.
-     * @param messageBytes
-     *        The the raw bytes that compose the incoming message. (Read-Only)
-     *
-     * @return a JmsMessage instance properly configured for dispatch to the provider listener.
-     *
-     * @throws IOException if an error occurs while creating the message objects.
-     */
-    public static JmsMessage createJmsMessage(AmqpConsumer consumer, ByteBuf messageBytes) throws IOException {
-
-        Message amqpMessage = decodeMessage(messageBytes);
-
-        // First we try the easy way, if the annotation is there we don't have to work hard.
-        JmsMessage result = createFromMsgAnnotation(consumer, amqpMessage, messageBytes);
-        if (result != null) {
-            return result;
-        }
-
-        // Next, match specific section structures and content types
-        result = createWithoutAnnotation(consumer, amqpMessage, messageBytes);
-        if (result != null) {
-            return result;
-        }
-
-        throw new IOException("Could not create a JMS message from incoming message");
-    }
-
-    private static JmsMessage createFromMsgAnnotation(AmqpConsumer consumer, Message message, ByteBuf messageBytes) throws IOException {
-        Object annotation = AmqpMessageSupport.getMessageAnnotation(JMS_MSG_TYPE, message);
-        if (annotation != null) {
-
-            switch ((byte) annotation) {
-                case JMS_MESSAGE:
-                    return createMessage(consumer, message);
-                case JMS_BYTES_MESSAGE:
-                    return createBytesMessage(consumer, message);
-                case JMS_TEXT_MESSAGE:
-                    return createTextMessage(consumer, message, StandardCharsets.UTF_8);
-                case JMS_MAP_MESSAGE:
-                    return createMapMessage(consumer, message);
-                case JMS_STREAM_MESSAGE:
-                    return createStreamMessage(consumer, message);
-                case JMS_OBJECT_MESSAGE:
-                    return createObjectMessage(consumer, message, messageBytes);
-                default:
-                    throw new IOException("Invalid JMS Message Type annotation value found in message: " + annotation);
-            }
-        }
-
-        return null;
-    }
-
-    private static JmsMessage createWithoutAnnotation(AmqpConsumer consumer, Message message, ByteBuf messageBytes) {
-        Section body = message.getBody();
-
-        if (body == null) {
-            if (isContentType(SERIALIZED_JAVA_OBJECT_CONTENT_TYPE, message)) {
-                return createObjectMessage(consumer, message, messageBytes);
-            } else if (isContentType(OCTET_STREAM_CONTENT_TYPE, message) || isContentType(null, message)) {
-                return createBytesMessage(consumer, message);
-            } else {
-                Charset charset = getCharsetForTextualContent(message.getContentType());
-                if (charset != null) {
-                    return createTextMessage(consumer, message, charset);
-                } else {
-                    return createMessage(consumer, message);
-                }
-            }
-        } else if (body instanceof Data) {
-            if (isContentType(OCTET_STREAM_CONTENT_TYPE, message) || isContentType(null, message)) {
-                return createBytesMessage(consumer, message);
-            } else if (isContentType(SERIALIZED_JAVA_OBJECT_CONTENT_TYPE, message)) {
-                return createObjectMessage(consumer, message, messageBytes);
-            } else {
-                Charset charset = getCharsetForTextualContent(message.getContentType());
-                if (charset != null) {
-                    return createTextMessage(consumer, message, charset);
-                } else {
-                    return createBytesMessage(consumer, message);
-                }
-            }
-        } else if (body instanceof AmqpValue) {
-            Object value = ((AmqpValue) body).getValue();
-
-            if (value == null || value instanceof String) {
-                return createTextMessage(consumer, message, StandardCharsets.UTF_8);
-            } else if (value instanceof Binary) {
-                return createBytesMessage(consumer, message);
-            } else {
-                return createObjectMessage(consumer, message, messageBytes);
-            }
-        } else if (body instanceof AmqpSequence) {
-            return createObjectMessage(consumer, message, messageBytes);
-        }
-
-        return null;
-    }
-
-    private static JmsObjectMessage createObjectMessage(AmqpConsumer consumer, Message message, ByteBuf messageBytes) {
-        return new JmsObjectMessage(new AmqpJmsObjectMessageFacade(consumer, message, messageBytes.copy()));
-    }
-
-    private static JmsStreamMessage createStreamMessage(AmqpConsumer consumer, Message message) {
-        return new JmsStreamMessage(new AmqpJmsStreamMessageFacade(consumer, message));
-    }
-
-    private static JmsMapMessage createMapMessage(AmqpConsumer consumer, Message message) {
-        return new JmsMapMessage(new AmqpJmsMapMessageFacade(consumer, message));
-    }
-
-    private static JmsTextMessage createTextMessage(AmqpConsumer consumer, Message message, Charset charset) {
-        return new JmsTextMessage(new AmqpJmsTextMessageFacade(consumer, message, charset));
-    }
-
-    private static JmsBytesMessage createBytesMessage(AmqpConsumer consumer, Message message) {
-        return new JmsBytesMessage(new AmqpJmsBytesMessageFacade(consumer, message));
-    }
-
-    private static JmsMessage createMessage(AmqpConsumer consumer, Message message) {
-        return new JmsMessage(new AmqpJmsMessageFacade(consumer, message));
-    }
-
-    /**
-     * @param contentType the contentType of the received message
-     * @return the character set to use, or null if not to treat the message as text
-     */
-    private static Charset getCharsetForTextualContent(String contentType) {
-        try {
-            return ContentTypeSupport.parseContentTypeForTextualCharset(contentType);
-        } catch (InvalidContentTypeException e) {
-            return null;
-        }
-    }
-}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@qpid.apache.org
For additional commands, e-mail: commits-help@qpid.apache.org


[2/5] qpid-jms git commit: QPIDJMS-215 Perform Message encoding at send time

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMessageBuilderTest.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMessageBuilderTest.java b/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMessageBuilderTest.java
deleted file mode 100644
index eee5f8e..0000000
--- a/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMessageBuilderTest.java
+++ /dev/null
@@ -1,737 +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.qpid.jms.provider.amqp.message;
-
-import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.encodeMessage;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.nio.charset.Charset;
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-
-import org.apache.qpid.jms.message.JmsBytesMessage;
-import org.apache.qpid.jms.message.JmsMessage;
-import org.apache.qpid.jms.message.JmsObjectMessage;
-import org.apache.qpid.jms.message.JmsStreamMessage;
-import org.apache.qpid.jms.message.JmsTextMessage;
-import org.apache.qpid.jms.message.facade.JmsMessageFacade;
-import org.apache.qpid.jms.meta.JmsConsumerId;
-import org.apache.qpid.jms.meta.JmsConsumerInfo;
-import org.apache.qpid.jms.provider.amqp.AmqpConsumer;
-import org.apache.qpid.jms.test.QpidJmsTestCase;
-import org.apache.qpid.proton.Proton;
-import org.apache.qpid.proton.amqp.Binary;
-import org.apache.qpid.proton.amqp.Symbol;
-import org.apache.qpid.proton.amqp.messaging.AmqpSequence;
-import org.apache.qpid.proton.amqp.messaging.AmqpValue;
-import org.apache.qpid.proton.amqp.messaging.Data;
-import org.apache.qpid.proton.amqp.messaging.MessageAnnotations;
-import org.apache.qpid.proton.message.Message;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.Mockito;
-
-public class AmqpJmsMessageBuilderTest extends QpidJmsTestCase {
-    private AmqpConsumer mockConsumer;
-
-    @Before
-    @Override
-    public void setUp() throws Exception {
-        super.setUp();
-
-        JmsConsumerId consumerId = new JmsConsumerId("ID:MOCK:1", 1, 1);
-        mockConsumer = Mockito.mock(AmqpConsumer.class);
-        Mockito.when(mockConsumer.getResourceInfo()).thenReturn(new JmsConsumerInfo(consumerId));
-    }
-
-    @Test
-    public void testCreate() {
-        new AmqpJmsMessageBuilder();
-    }
-
-    // =============== With The Message Type Annotation =========
-    // ==========================================================
-
-    /**
-     * Test that a message with the {@value AmqpMessageSupport#JMS_MSG_TYPE}
-     * annotation set to  {@value AmqpMessageSupport#JMS_MESSAGE} is
-     * treated as a generic {@link JmsMessage} with {@link AmqpJmsMessageFacade}
-     *
-     * @throws Exception if an error occurs during the test.
-     */
-    @Test(expected = IOException.class)
-    public void testCreateMessageFromUnknownMessageTypeAnnotationValueThrows() throws Exception {
-        Message message = Proton.message();
-
-        Map<Symbol, Object> map = new HashMap<Symbol, Object>();
-        map.put(Symbol.valueOf(AmqpMessageSupport.JMS_MSG_TYPE), (byte) -1);
-
-        MessageAnnotations messageAnnotations = new MessageAnnotations(map);
-        message.setMessageAnnotations(messageAnnotations);
-
-        AmqpJmsMessageBuilder.createJmsMessage(mockConsumer, encodeMessage(message));
-    }
-
-    /**
-     * Test that a message with the {@value AmqpMessageSupport#JMS_MSG_TYPE}
-     * annotation set to  {@value AmqpMessageSupport#JMS_MESSAGE} is
-     * treated as a generic {@link JmsMessage} with {@link AmqpJmsMessageFacade}
-     *
-     * @throws Exception if an error occurs during the test.
-     */
-    @Test
-    public void testCreateGenericMessageFromMessageTypeAnnotation() throws Exception {
-        Message message = Proton.message();
-
-        Map<Symbol, Object> map = new HashMap<Symbol, Object>();
-        map.put(Symbol.valueOf(AmqpMessageSupport.JMS_MSG_TYPE), AmqpMessageSupport.JMS_MESSAGE);
-
-        MessageAnnotations messageAnnotations = new MessageAnnotations(map);
-        message.setMessageAnnotations(messageAnnotations);
-
-        JmsMessage jmsMessage = AmqpJmsMessageBuilder.createJmsMessage(mockConsumer, encodeMessage(message));
-        assertNotNull("Message should not be null", jmsMessage);
-        assertEquals("Unexpected message class type", JmsMessage.class, jmsMessage.getClass());
-
-        JmsMessageFacade facade = jmsMessage.getFacade();
-        assertNotNull("Facade should not be null", facade);
-        assertEquals("Unexpected facade class type", AmqpJmsMessageFacade.class, facade.getClass());
-    }
-
-    /**
-     * Test that a message with the {@value AmqpMessageSupport#JMS_MSG_TYPE}
-     * annotation set to  {@value AmqpMessageSupport#JMS_BYTES_MESSAGE} is
-     * treated as a {@link JmsBytesMessage} with {@link AmqpJmsBytesMessageFacade}
-     *
-     * @throws Exception if an error occurs during the test.
-     */
-    @Test
-    public void testCreateBytesMessageFromMessageTypeAnnotation() throws Exception {
-        Message message = Proton.message();
-
-        Map<Symbol, Object> map = new HashMap<Symbol, Object>();
-        map.put(Symbol.valueOf(AmqpMessageSupport.JMS_MSG_TYPE), AmqpMessageSupport.JMS_BYTES_MESSAGE);
-
-        MessageAnnotations messageAnnotations = new MessageAnnotations(map);
-        message.setMessageAnnotations(messageAnnotations);
-
-        JmsMessage jmsMessage = AmqpJmsMessageBuilder.createJmsMessage(mockConsumer, encodeMessage(message));
-        assertNotNull("Message should not be null", jmsMessage);
-        assertEquals("Unexpected message class type", JmsBytesMessage.class, jmsMessage.getClass());
-
-        JmsMessageFacade facade = jmsMessage.getFacade();
-        assertNotNull("Facade should not be null", facade);
-        assertEquals("Unexpected facade class type", AmqpJmsBytesMessageFacade.class, facade.getClass());
-    }
-
-    /**
-     * Test that a message with the {@value AmqpMessageSupport#JMS_MSG_TYPE}
-     * annotation set to  {@value AmqpMessageSupport#JMS_BYTES_MESSAGE} is
-     * treated as a {@link JmsTextMessage} with {@link AmqpJmsTextMessageFacade}
-     *
-     * @throws Exception if an error occurs during the test.
-     */
-    @Test
-    public void testCreateTextMessageFromMessageTypeAnnotation() throws Exception {
-        Message message = Proton.message();
-
-        Map<Symbol, Object> map = new HashMap<Symbol, Object>();
-        map.put(Symbol.valueOf(AmqpMessageSupport.JMS_MSG_TYPE), AmqpMessageSupport.JMS_TEXT_MESSAGE);
-
-        MessageAnnotations messageAnnotations = new MessageAnnotations(map);
-        message.setMessageAnnotations(messageAnnotations);
-
-        JmsMessage jmsMessage = AmqpJmsMessageBuilder.createJmsMessage(mockConsumer, encodeMessage(message));
-        assertNotNull("Message should not be null", jmsMessage);
-        assertEquals("Unexpected message class type", JmsTextMessage.class, jmsMessage.getClass());
-
-        JmsMessageFacade facade = jmsMessage.getFacade();
-        assertNotNull("Facade should not be null", facade);
-        assertEquals("Unexpected facade class type", AmqpJmsTextMessageFacade.class, facade.getClass());
-    }
-
-    /**
-     * Test that a message with the {@value AmqpMessageSupport#JMS_MSG_TYPE}
-     * annotation set to  {@value AmqpMessageSupport#JMS_OBJECT_MESSAGE} and
-     * content-type set to {@value AmqpMessageSupport#OCTET_STREAM_CONTENT_TYPE} is
-     * treated as a {@link JmsObjectMessage} with {@link AmqpJmsObjectMessageFacade}
-     * containing a {@link AmqpSerializedObjectDelegate}.
-     *
-     * @throws Exception if an error occurs during the test.
-     */
-    @Test
-    public void testCreateObjectMessageFromMessageTypeAnnotation() throws Exception {
-        createObjectMessageFromMessageTypeAnnotationTestImpl(true);
-    }
-
-    /**
-     * Test that a message with the {@value AmqpMessageSupport#JMS_MSG_TYPE}
-     * annotation set to  {@value AmqpMessageSupport#JMS_OBJECT_MESSAGE} and
-     * content-type not set is treated as a {@link JmsObjectMessage} with
-     * {@link AmqpJmsObjectMessageFacade} containing a {@link AmqpTypedObjectDelegate}.
-     *
-     * @throws Exception if an error occurs during the test.
-     */
-    @Test
-    public void testCreateObjectMessageFromMessageTypeAnnotationAnd() throws Exception {
-        createObjectMessageFromMessageTypeAnnotationTestImpl(false);
-    }
-
-    private void createObjectMessageFromMessageTypeAnnotationTestImpl(boolean setJavaSerializedContentType) throws Exception {
-        Message message = Proton.message();
-
-        Map<Symbol, Object> map = new HashMap<Symbol, Object>();
-        map.put(Symbol.valueOf(AmqpMessageSupport.JMS_MSG_TYPE), AmqpMessageSupport.JMS_OBJECT_MESSAGE);
-
-        MessageAnnotations messageAnnotations = new MessageAnnotations(map);
-        message.setMessageAnnotations(messageAnnotations);
-
-        if (setJavaSerializedContentType) {
-            message.setContentType(AmqpMessageSupport.SERIALIZED_JAVA_OBJECT_CONTENT_TYPE);
-        }
-
-        JmsMessage jmsMessage = AmqpJmsMessageBuilder.createJmsMessage(mockConsumer, encodeMessage(message));
-        assertNotNull("Message should not be null", jmsMessage);
-        assertEquals("Unexpected message class type", JmsObjectMessage.class, jmsMessage.getClass());
-
-        JmsMessageFacade facade = jmsMessage.getFacade();
-        assertNotNull("Facade should not be null", facade);
-        assertEquals("Unexpected facade class type", AmqpJmsObjectMessageFacade.class, facade.getClass());
-
-        AmqpObjectTypeDelegate delegate = ((AmqpJmsObjectMessageFacade) facade).getDelegate();
-        if (setJavaSerializedContentType) {
-            assertTrue("Unexpected delegate type: " + delegate, delegate instanceof AmqpSerializedObjectDelegate);
-        } else {
-            assertTrue("Unexpected delegate type: " + delegate, delegate instanceof AmqpTypedObjectDelegate);
-        }
-    }
-
-    /**
-     * Test that a message with the {@value AmqpMessageSupport#JMS_MSG_TYPE}
-     * annotation set to  {@value AmqpMessageSupport#JMS_STREAM_MESSAGE} is
-     * treated as a {@link JmsStreamMessage} with {@link AmqpJmsStreamMessageFacade}
-     *
-     * @throws Exception if an error occurs during the test.
-     */
-    @Test
-    public void testCreateStreamMessageFromMessageTypeAnnotation() throws Exception {
-
-        Message message = Proton.message();
-
-        Map<Symbol, Object> map = new HashMap<Symbol, Object>();
-        map.put(Symbol.valueOf(AmqpMessageSupport.JMS_MSG_TYPE), AmqpMessageSupport.JMS_STREAM_MESSAGE);
-
-        MessageAnnotations messageAnnotations = new MessageAnnotations(map);
-        message.setMessageAnnotations(messageAnnotations);
-
-        JmsMessage jmsMessage = AmqpJmsMessageBuilder.createJmsMessage(mockConsumer, encodeMessage(message));
-        assertNotNull("Message should not be null", jmsMessage);
-        assertEquals("Unexpected message class type", JmsStreamMessage.class, jmsMessage.getClass());
-
-        JmsMessageFacade facade = jmsMessage.getFacade();
-        assertNotNull("Facade should not be null", facade);
-        assertEquals("Unexpected facade class type", AmqpJmsStreamMessageFacade.class, facade.getClass());
-    }
-
-    // =============== Without The Message Type Annotation =========
-    // =============================================================
-
-    // --------- No Body Section ---------
-
-    /**
-     * Test that a message with no body section, but with the content type set to
-     * {@value AmqpMessageSupport#OCTET_STREAM_CONTENT_TYPE} results in a BytesMessage
-     * when not otherwise annotated to indicate the type of JMS message it is.
-     *
-     * @throws Exception if an error occurs during the test.
-     */
-    @Test
-    public void testCreateBytesMessageFromNoBodySectionAndContentType() throws Exception {
-        Message message = Proton.message();
-        message.setContentType(AmqpMessageSupport.OCTET_STREAM_CONTENT_TYPE);
-
-        JmsMessage jmsMessage = AmqpJmsMessageBuilder.createJmsMessage(mockConsumer, encodeMessage(message));
-        assertNotNull("Message should not be null", jmsMessage);
-        assertEquals("Unexpected message class type", JmsBytesMessage.class, jmsMessage.getClass());
-
-        JmsMessageFacade facade = jmsMessage.getFacade();
-        assertNotNull("Facade should not be null", facade);
-        assertEquals("Unexpected facade class type", AmqpJmsBytesMessageFacade.class, facade.getClass());
-    }
-
-    /**
-     * Test that a message with no body section, and no content-type results in a BytesMessage
-     * when not otherwise annotated to indicate the type of JMS message it is.
-     *
-     * @throws Exception if an error occurs during the test.
-     */
-    @Test
-    public void testCreateBytesMessageFromNoBodySectionAndNoContentType() throws Exception {
-        Message message = Proton.message();
-
-        assertNull(message.getContentType());
-
-        JmsMessage jmsMessage = AmqpJmsMessageBuilder.createJmsMessage(mockConsumer, encodeMessage(message));
-        assertNotNull("Message should not be null", jmsMessage);
-        assertEquals("Unexpected message class type", JmsBytesMessage.class, jmsMessage.getClass());
-
-        JmsMessageFacade facade = jmsMessage.getFacade();
-        assertNotNull("Facade should not be null", facade);
-        assertEquals("Unexpected facade class type", AmqpJmsBytesMessageFacade.class, facade.getClass());
-    }
-
-    /**
-    * Test that a message with no body section, but with the content type set to
-    * {@value AmqpMessageSupport#SERIALIZED_JAVA_OBJECT_CONTENT_TYPE} results in an ObjectMessage
-    * when not otherwise annotated to indicate the type of JMS message it is.
-     *
-     * @throws Exception if an error occurs during the test.
-    */
-    @Test
-    public void testCreateObjectMessageFromNoBodySectionAndContentType() throws Exception {
-        Message message = Proton.message();
-        message.setContentType(AmqpMessageSupport.SERIALIZED_JAVA_OBJECT_CONTENT_TYPE);
-
-        JmsMessage jmsMessage = AmqpJmsMessageBuilder.createJmsMessage(mockConsumer, encodeMessage(message));
-        assertNotNull("Message should not be null", jmsMessage);
-        assertEquals("Unexpected message class type", JmsObjectMessage.class, jmsMessage.getClass());
-
-        JmsMessageFacade facade = jmsMessage.getFacade();
-        assertNotNull("Facade should not be null", facade);
-        assertEquals("Unexpected facade class type", AmqpJmsObjectMessageFacade.class, facade.getClass());
-
-        AmqpObjectTypeDelegate delegate = ((AmqpJmsObjectMessageFacade) facade).getDelegate();
-        assertTrue("Unexpected delegate type: " + delegate, delegate instanceof AmqpSerializedObjectDelegate);
-    }
-
-    @Test
-    public void testCreateTextMessageFromNoBodySectionAndContentType() throws Exception {
-        Message message = Proton.message();
-        message.setContentType("text/plain");
-
-        JmsMessage jmsMessage = AmqpJmsMessageBuilder.createJmsMessage(mockConsumer, encodeMessage(message));
-        assertNotNull("Message should not be null", jmsMessage);
-        assertEquals("Unexpected message class type", JmsTextMessage.class, jmsMessage.getClass());
-
-        JmsMessageFacade facade = jmsMessage.getFacade();
-        assertNotNull("Facade should not be null", facade);
-        assertEquals("Unexpected facade class type", AmqpJmsTextMessageFacade.class, facade.getClass());
-    }
-
-    /**
-     * Test that a message with no body section, and with the content type set to
-     * an unknown value results in a plain Message when not otherwise annotated to
-     * indicate the type of JMS message it is.
-     *
-     * @throws Exception if an error occurs during the test.
-     */
-    public void testCreateGenericMessageFromNoBodySectionAndUnknownContentType() throws Exception {
-        Message message = Proton.message();
-        message.setContentType("unknown-content-type");
-
-        JmsMessage jmsMessage = AmqpJmsMessageBuilder.createJmsMessage(mockConsumer, encodeMessage(message));
-        assertNotNull("Message should not be null", jmsMessage);
-        assertEquals("Unexpected message class type", JmsMessage.class, jmsMessage.getClass());
-
-        JmsMessageFacade facade = jmsMessage.getFacade();
-        assertNotNull("Facade should not be null", facade);
-        assertEquals("Unexpected facade class type", AmqpJmsMessageFacade.class, facade.getClass());
-    }
-
-    // --------- Data Body Section ---------
-
-    /**
-     * Test that a data body containing nothing, but with the content type set to
-     * {@value AmqpMessageSupport#OCTET_STREAM_CONTENT_TYPE} results in a BytesMessage when not
-     * otherwise annotated to indicate the type of JMS message it is.
-     *
-     * @throws Exception if an error occurs during the test.
-     */
-    @Test
-    public void testCreateBytesMessageFromDataWithEmptyBinaryAndContentType() throws Exception {
-        Message message = Proton.message();
-        Binary binary = new Binary(new byte[0]);
-        message.setBody(new Data(binary));
-        message.setContentType(AmqpMessageSupport.OCTET_STREAM_CONTENT_TYPE);
-
-        JmsMessage jmsMessage = AmqpJmsMessageBuilder.createJmsMessage(mockConsumer, encodeMessage(message));
-        assertNotNull("Message should not be null", jmsMessage);
-        assertEquals("Unexpected message class type", JmsBytesMessage.class, jmsMessage.getClass());
-
-        JmsMessageFacade facade = jmsMessage.getFacade();
-        assertNotNull("Facade should not be null", facade);
-        assertEquals("Unexpected facade class type", AmqpJmsBytesMessageFacade.class, facade.getClass());
-    }
-
-    /**
-     * Test that a message with an empty data body section, and with the content type
-     * set to an unknown value results in a BytesMessage when not otherwise annotated
-     * to indicate the type of JMS message it is.
-     *
-     * @throws Exception if an error occurs during the test.
-     */
-    public void testCreateBytesMessageFromDataWithUnknownContentType() throws Exception {
-        Message message = Proton.message();
-        Binary binary = new Binary(new byte[0]);
-        message.setBody(new Data(binary));
-        message.setContentType("unknown-content-type");
-
-        JmsMessage jmsMessage = AmqpJmsMessageBuilder.createJmsMessage(mockConsumer, encodeMessage(message));
-        assertNotNull("Message should not be null", jmsMessage);
-        assertEquals("Unexpected message class type", JmsBytesMessage.class, jmsMessage.getClass());
-
-        JmsMessageFacade facade = jmsMessage.getFacade();
-        assertNotNull("Facade should not be null", facade);
-        assertEquals("Unexpected facade class type", AmqpJmsBytesMessageFacade.class, facade.getClass());
-    }
-
-    /**
-     * Test that a receiving a data body containing nothing and no content type being set
-     * results in a BytesMessage when not otherwise annotated to indicate the type of
-     * JMS message it is.
-     *
-     * @throws Exception if an error occurs during the test.
-     */
-    @Test
-    public void testCreateBytesMessageFromDataWithEmptyBinaryAndNoContentType() throws Exception {
-        Message message = Proton.message();
-        Binary binary = new Binary(new byte[0]);
-        message.setBody(new Data(binary));
-
-        assertNull(message.getContentType());
-
-        JmsMessage jmsMessage = AmqpJmsMessageBuilder.createJmsMessage(mockConsumer, encodeMessage(message));
-        assertNotNull("Message should not be null", jmsMessage);
-        assertEquals("Unexpected message class type", JmsBytesMessage.class, jmsMessage.getClass());
-
-        JmsMessageFacade facade = jmsMessage.getFacade();
-        assertNotNull("Facade should not be null", facade);
-        assertEquals("Unexpected facade class type", AmqpJmsBytesMessageFacade.class, facade.getClass());
-    }
-
-    /**
-     * Test that receiving a data body containing nothing, but with the content type set to
-     * {@value AmqpMessageSupport#SERIALIZED_JAVA_OBJECT_CONTENT_TYPE} results in an ObjectMessage
-     * when not otherwise annotated to indicate the type of JMS message it is.
-     *
-     * @throws Exception if an error occurs during the test.
-     */
-    @Test
-    public void testCreateObjectMessageFromDataWithContentTypeAndEmptyBinary() throws Exception {
-        Message message = Proton.message();
-        Binary binary = new Binary(new byte[0]);
-        message.setBody(new Data(binary));
-        message.setContentType(AmqpMessageSupport.SERIALIZED_JAVA_OBJECT_CONTENT_TYPE);
-
-        JmsMessage jmsMessage = AmqpJmsMessageBuilder.createJmsMessage(mockConsumer, encodeMessage(message));
-        assertNotNull("Message should not be null", jmsMessage);
-        assertEquals("Unexpected message class type", JmsObjectMessage.class, jmsMessage.getClass());
-
-        JmsMessageFacade facade = jmsMessage.getFacade();
-        assertNotNull("Facade should not be null", facade);
-        assertEquals("Unexpected facade class type", AmqpJmsObjectMessageFacade.class, facade.getClass());
-
-        AmqpObjectTypeDelegate delegate = ((AmqpJmsObjectMessageFacade) facade).getDelegate();
-        assertTrue("Unexpected delegate type: " + delegate, delegate instanceof AmqpSerializedObjectDelegate);
-    }
-
-    /**
-     * Test that receiving a Data body section with the content type set to
-     * 'text/plain' results in a TextMessage when not otherwise annotated to
-     * indicate the type of JMS message it is.
-     *
-     * @throws Exception if an error occurs during the test.
-     */
-    @Test
-    public void testCreateTextMessageFromDataWithContentTypeTextPlain() throws Exception {
-        doCreateTextMessageFromDataWithContentTypeTestImpl("text/plain;charset=iso-8859-1", StandardCharsets.ISO_8859_1);
-        doCreateTextMessageFromDataWithContentTypeTestImpl("text/plain;charset=us-ascii", StandardCharsets.US_ASCII);
-        doCreateTextMessageFromDataWithContentTypeTestImpl("text/plain;charset=utf-8", StandardCharsets.UTF_8);
-        doCreateTextMessageFromDataWithContentTypeTestImpl("text/plain", StandardCharsets.UTF_8);
-    }
-
-    @Test
-    public void testCreateTextMessageFromDataWithContentTypeTextJson() throws Exception {
-        doCreateTextMessageFromDataWithContentTypeTestImpl("text/json;charset=iso-8859-1", StandardCharsets.ISO_8859_1);
-        doCreateTextMessageFromDataWithContentTypeTestImpl("text/json;charset=us-ascii", StandardCharsets.US_ASCII);
-        doCreateTextMessageFromDataWithContentTypeTestImpl("text/json;charset=utf-8", StandardCharsets.UTF_8);
-        doCreateTextMessageFromDataWithContentTypeTestImpl("text/json", StandardCharsets.UTF_8);
-    }
-
-    @Test
-    public void testCreateTextMessageFromDataWithContentTypeTextHtml() throws Exception {
-        doCreateTextMessageFromDataWithContentTypeTestImpl("text/html;charset=iso-8859-1", StandardCharsets.ISO_8859_1);
-        doCreateTextMessageFromDataWithContentTypeTestImpl("text/html;charset=us-ascii", StandardCharsets.US_ASCII);
-        doCreateTextMessageFromDataWithContentTypeTestImpl("text/html;charset=utf-8", StandardCharsets.UTF_8);
-        doCreateTextMessageFromDataWithContentTypeTestImpl("text/html", StandardCharsets.UTF_8);
-    }
-
-    @Test
-    public void testCreateTextMessageFromDataWithContentTypeTextFoo() throws Exception {
-        doCreateTextMessageFromDataWithContentTypeTestImpl("text/foo;charset=iso-8859-1", StandardCharsets.ISO_8859_1);
-        doCreateTextMessageFromDataWithContentTypeTestImpl("text/foo;charset=us-ascii", StandardCharsets.US_ASCII);
-        doCreateTextMessageFromDataWithContentTypeTestImpl("text/foo;charset=utf-8", StandardCharsets.UTF_8);
-        doCreateTextMessageFromDataWithContentTypeTestImpl("text/foo", StandardCharsets.UTF_8);
-    }
-
-    @Test
-    public void testCreateTextMessageFromDataWithContentTypeApplicationJson() throws Exception {
-        doCreateTextMessageFromDataWithContentTypeTestImpl("application/json;charset=iso-8859-1", StandardCharsets.ISO_8859_1);
-        doCreateTextMessageFromDataWithContentTypeTestImpl("application/json;charset=us-ascii", StandardCharsets.US_ASCII);
-        doCreateTextMessageFromDataWithContentTypeTestImpl("application/json;charset=utf-8", StandardCharsets.UTF_8);
-        doCreateTextMessageFromDataWithContentTypeTestImpl("application/json", StandardCharsets.UTF_8);
-    }
-
-    @Test
-    public void testCreateTextMessageFromDataWithContentTypeApplicationJsonVariant() throws Exception {
-        doCreateTextMessageFromDataWithContentTypeTestImpl("application/something+json;charset=iso-8859-1", StandardCharsets.ISO_8859_1);
-        doCreateTextMessageFromDataWithContentTypeTestImpl("application/something+json;charset=us-ascii", StandardCharsets.US_ASCII);
-        doCreateTextMessageFromDataWithContentTypeTestImpl("application/something+json;charset=utf-8", StandardCharsets.UTF_8);
-        doCreateTextMessageFromDataWithContentTypeTestImpl("application/something+json", StandardCharsets.UTF_8);
-    }
-
-    @Test
-    public void testCreateTextMessageFromDataWithContentTypeApplicationJavascript() throws Exception {
-        doCreateTextMessageFromDataWithContentTypeTestImpl("application/javascript;charset=iso-8859-1", StandardCharsets.ISO_8859_1);
-        doCreateTextMessageFromDataWithContentTypeTestImpl("application/javascript;charset=us-ascii", StandardCharsets.US_ASCII);
-        doCreateTextMessageFromDataWithContentTypeTestImpl("application/javascript;charset=utf-8", StandardCharsets.UTF_8);
-        doCreateTextMessageFromDataWithContentTypeTestImpl("application/javascript", StandardCharsets.UTF_8);
-    }
-
-    @Test
-    public void testCreateTextMessageFromDataWithContentTypeApplicationEcmascript() throws Exception {
-        doCreateTextMessageFromDataWithContentTypeTestImpl("application/ecmascript;charset=iso-8859-1", StandardCharsets.ISO_8859_1);
-        doCreateTextMessageFromDataWithContentTypeTestImpl("application/ecmascript;charset=us-ascii", StandardCharsets.US_ASCII);
-        doCreateTextMessageFromDataWithContentTypeTestImpl("application/ecmascript;charset=utf-8", StandardCharsets.UTF_8);
-        doCreateTextMessageFromDataWithContentTypeTestImpl("application/ecmascript", StandardCharsets.UTF_8);
-    }
-
-    @Test
-    public void testCreateTextMessageFromDataWithContentTypeApplicationXml() throws Exception {
-        doCreateTextMessageFromDataWithContentTypeTestImpl("application/xml;charset=iso-8859-1", StandardCharsets.ISO_8859_1);
-        doCreateTextMessageFromDataWithContentTypeTestImpl("application/xml;charset=us-ascii", StandardCharsets.US_ASCII);
-        doCreateTextMessageFromDataWithContentTypeTestImpl("application/xml;charset=utf-8", StandardCharsets.UTF_8);
-        doCreateTextMessageFromDataWithContentTypeTestImpl("application/xml", StandardCharsets.UTF_8);
-    }
-
-    @Test
-    public void testCreateTextMessageFromDataWithContentTypeApplicationXmlVariant() throws Exception {
-        doCreateTextMessageFromDataWithContentTypeTestImpl("application/something+xml;charset=iso-8859-1", StandardCharsets.ISO_8859_1);
-        doCreateTextMessageFromDataWithContentTypeTestImpl("application/something+xml;charset=us-ascii", StandardCharsets.US_ASCII);
-        doCreateTextMessageFromDataWithContentTypeTestImpl("application/something+xml;charset=utf-8", StandardCharsets.UTF_8);
-        doCreateTextMessageFromDataWithContentTypeTestImpl("application/something+xml", StandardCharsets.UTF_8);
-    }
-
-    @Test
-    public void testCreateTextMessageFromDataWithContentTypeApplicationXmlDtd() throws Exception {
-        doCreateTextMessageFromDataWithContentTypeTestImpl("application/xml-dtd;charset=iso-8859-1", StandardCharsets.ISO_8859_1);
-        doCreateTextMessageFromDataWithContentTypeTestImpl("application/xml-dtd;charset=us-ascii", StandardCharsets.US_ASCII);
-        doCreateTextMessageFromDataWithContentTypeTestImpl("application/xml-dtd;charset=utf-8", StandardCharsets.UTF_8);
-        doCreateTextMessageFromDataWithContentTypeTestImpl("application/xml-dtd", StandardCharsets.UTF_8);
-    }
-
-    private void doCreateTextMessageFromDataWithContentTypeTestImpl(String contentType, Charset expectedCharset) throws IOException {
-        Message message = Proton.message();
-        Binary binary = new Binary(new byte[0]);
-        message.setBody(new Data(binary));
-        message.setContentType(contentType);
-
-        JmsMessage jmsMessage = AmqpJmsMessageBuilder.createJmsMessage(mockConsumer, encodeMessage(message));
-        assertNotNull("Message should not be null", jmsMessage);
-        assertEquals("Unexpected message class type", JmsTextMessage.class, jmsMessage.getClass());
-
-        JmsMessageFacade facade = jmsMessage.getFacade();
-        assertNotNull("Facade should not be null", facade);
-        assertEquals("Unexpected facade class type", AmqpJmsTextMessageFacade.class, facade.getClass());
-
-        AmqpJmsTextMessageFacade textFacade = (AmqpJmsTextMessageFacade) facade;
-        assertEquals("Unexpected character set", expectedCharset, textFacade.getCharset());
-    }
-
-    // --------- AmqpValue Body Section ---------
-
-    /**
-     * Test that an amqp-value body containing a string results in a TextMessage
-     * when not otherwise annotated to indicate the type of JMS message it is.
-     *
-     * @throws Exception if an error occurs during the test.
-     */
-    @Test
-    public void testCreateTextMessageFromAmqpValueWithString() throws Exception {
-        Message message = Proton.message();
-        message.setBody(new AmqpValue("content"));
-
-        JmsMessage jmsMessage = AmqpJmsMessageBuilder.createJmsMessage(mockConsumer, encodeMessage(message));
-        assertNotNull("Message should not be null", jmsMessage);
-        assertEquals("Unexpected message class type", JmsTextMessage.class, jmsMessage.getClass());
-
-        JmsMessageFacade facade = jmsMessage.getFacade();
-        assertNotNull("Facade should not be null", facade);
-        assertEquals("Unexpected facade class type", AmqpJmsTextMessageFacade.class, facade.getClass());
-    }
-
-    /**
-     * Test that an amqp-value body containing a null results in an TextMessage
-     * when not otherwise annotated to indicate the type of JMS message it is.
-     *
-     * @throws Exception if an error occurs during the test.
-     */
-    @Test
-    public void testCreateTextMessageFromAmqpValueWithNull() throws Exception {
-        Message message = Proton.message();
-        message.setBody(new AmqpValue(null));
-
-        JmsMessage jmsMessage = AmqpJmsMessageBuilder.createJmsMessage(mockConsumer, encodeMessage(message));
-        assertNotNull("Message should not be null", jmsMessage);
-        assertEquals("Unexpected message class type", JmsTextMessage.class, jmsMessage.getClass());
-
-        JmsMessageFacade facade = jmsMessage.getFacade();
-        assertNotNull("Facade should not be null", facade);
-        assertEquals("Unexpected facade class type", AmqpJmsTextMessageFacade.class, facade.getClass());
-    }
-
-    /**
-     * Test that an amqp-value body containing a map results in an ObjectMessage
-     * when not otherwise annotated to indicate the type of JMS message it is.
-     *
-     * @throws Exception if an error occurs during the test.
-     */
-    @Test
-    public void testCreateAmqpObjectMessageFromAmqpValueWithMap() throws Exception {
-        Message message = Proton.message();
-        Map<String, String> map = new HashMap<String,String>();
-        message.setBody(new AmqpValue(map));
-
-        JmsMessage jmsMessage = AmqpJmsMessageBuilder.createJmsMessage(mockConsumer, encodeMessage(message));
-        assertNotNull("Message should not be null", jmsMessage);
-        assertEquals("Unexpected message class type", JmsObjectMessage.class, jmsMessage.getClass());
-
-        JmsMessageFacade facade = jmsMessage.getFacade();
-        assertNotNull("Facade should not be null", facade);
-        assertEquals("Unexpected facade class type", AmqpJmsObjectMessageFacade.class, facade.getClass());
-
-        AmqpObjectTypeDelegate delegate = ((AmqpJmsObjectMessageFacade) facade).getDelegate();
-        assertTrue("Unexpected delegate type: " + delegate, delegate instanceof AmqpTypedObjectDelegate);
-    }
-
-    /**
-     * Test that an amqp-value body containing a list results in an ObjectMessage
-     * when not otherwise annotated to indicate the type of JMS message it is.
-     *
-     * @throws Exception if an error occurs during the test.
-     */
-    @Test
-    public void testCreateAmqpObjectMessageFromAmqpValueWithList() throws Exception {
-        Message message = Proton.message();
-        List<String> list = new ArrayList<String>();
-        message.setBody(new AmqpValue(list));
-
-        JmsMessage jmsMessage = AmqpJmsMessageBuilder.createJmsMessage(mockConsumer, encodeMessage(message));
-        assertNotNull("Message should not be null", jmsMessage);
-        assertEquals("Unexpected message class type", JmsObjectMessage.class, jmsMessage.getClass());
-
-        JmsMessageFacade facade = jmsMessage.getFacade();
-        assertNotNull("Facade should not be null", facade);
-        assertEquals("Unexpected facade class type", AmqpJmsObjectMessageFacade.class, facade.getClass());
-
-        AmqpObjectTypeDelegate delegate = ((AmqpJmsObjectMessageFacade) facade).getDelegate();
-        assertTrue("Unexpected delegate type: " + delegate, delegate instanceof AmqpTypedObjectDelegate);
-    }
-
-    /**
-     * Test that an amqp-value body containing a binary value results in BytesMessage
-     * when not otherwise annotated to indicate the type of JMS message it is.
-     *
-     * @throws Exception if an error occurs during the test.
-     */
-    @Test
-    public void testCreateAmqpBytesMessageFromAmqpValueWithBinary() throws Exception {
-        Message message = Proton.message();
-        Binary binary = new Binary(new byte[0]);
-        message.setBody(new AmqpValue(binary));
-
-        JmsMessage jmsMessage = AmqpJmsMessageBuilder.createJmsMessage(mockConsumer, encodeMessage(message));
-        assertNotNull("Message should not be null", jmsMessage);
-        assertEquals("Unexpected message class type", JmsBytesMessage.class, jmsMessage.getClass());
-
-        JmsMessageFacade facade = jmsMessage.getFacade();
-        assertNotNull("Facade should not be null", facade);
-        assertEquals("Unexpected facade class type", AmqpJmsBytesMessageFacade.class, facade.getClass());
-    }
-
-    /**
-     * Test that an amqp-value body containing a value which can't be categorised results in
-     * an ObjectMessage when not otherwise annotated to indicate the type of JMS message it is.
-     *
-     * @throws Exception if an error occurs during the test.
-     */
-    @Test
-    public void testCreateObjectMessageFromAmqpValueWithUncategorisedContent() throws Exception {
-        Message message = Proton.message();
-        message.setBody(new AmqpValue(UUID.randomUUID()));
-
-        JmsMessage jmsMessage = AmqpJmsMessageBuilder.createJmsMessage(mockConsumer, encodeMessage(message));
-        assertNotNull("Message should not be null", jmsMessage);
-        assertEquals("Unexpected message class type", JmsObjectMessage.class, jmsMessage.getClass());
-
-        JmsMessageFacade facade = jmsMessage.getFacade();
-        assertNotNull("Facade should not be null", facade);
-        assertEquals("Unexpected facade class type", AmqpJmsObjectMessageFacade.class, facade.getClass());
-
-        AmqpObjectTypeDelegate delegate = ((AmqpJmsObjectMessageFacade) facade).getDelegate();
-        assertTrue("Unexpected delegate type: " + delegate, delegate instanceof AmqpTypedObjectDelegate);
-    }
-
-    // --------- AmqpSequence Body Section ---------
-
-    /**
-     * Test that an amqp-sequence body containing a binary value results in an ObjectMessage
-     * when not otherwise annotated to indicate the type of JMS message it is.
-     *
-     * @throws Exception if an error occurs during the test.
-     */
-    @Test
-    public void testCreateObjectMessageMessageFromAmqpSequence() throws Exception
-    {
-        Message message = Proton.message();
-        List<String> list = new ArrayList<String>();
-        message.setBody(new AmqpSequence(list));
-
-        JmsMessage jmsMessage = AmqpJmsMessageBuilder.createJmsMessage(mockConsumer, encodeMessage(message));
-        assertNotNull("Message should not be null", jmsMessage);
-        assertEquals("Unexpected message class type", JmsObjectMessage.class, jmsMessage.getClass());
-
-        JmsMessageFacade facade = jmsMessage.getFacade();
-        assertNotNull("Facade should not be null", facade);
-        assertEquals("Unexpected facade class type", AmqpJmsObjectMessageFacade.class, facade.getClass());
-
-        AmqpObjectTypeDelegate delegate = ((AmqpJmsObjectMessageFacade) facade).getDelegate();
-        assertTrue("Unexpected delegate type: " + delegate, delegate instanceof AmqpTypedObjectDelegate);
-    }
-}

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMessageFacadeTest.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMessageFacadeTest.java b/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMessageFacadeTest.java
index bd50a67..544b70f 100644
--- a/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMessageFacadeTest.java
+++ b/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMessageFacadeTest.java
@@ -97,18 +97,17 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
     public void testNewMessageHasUnderlyingHeaderSectionWithDurableTrue() {
         AmqpJmsMessageFacade amqpMessageFacade = createNewMessageFacade();
 
-        Message underlying = amqpMessageFacade.getAmqpMessage();
-        assertNotNull("Expected message to have Header section", underlying.getHeader());
-        assertTrue("Durable not as expected", underlying.getHeader().getDurable());
+        assertNotNull("Expected message to have Header section", amqpMessageFacade.getHeader());
+        assertTrue("Durable not as expected", amqpMessageFacade.getHeader().getDurable());
     }
 
     @Test
     public void testNewMessageHasUnderlyingHeaderSectionWithNoTtlSet() {
         AmqpJmsMessageFacade amqpMessageFacade = createNewMessageFacade();
+        amqpMessageFacade.setHeader(new Header());
 
-        Message underlying = amqpMessageFacade.getAmqpMessage();
-        assertNotNull("Expected message to have Header section", underlying.getHeader());
-        assertNull("Ttl field should not be set", underlying.getHeader().getTtl());
+        assertNotNull("Expected message to have Header section", amqpMessageFacade.getHeader());
+        assertNull("Ttl field should not be set", amqpMessageFacade.getHeader().getTtl());
     }
 
     @Test
@@ -147,7 +146,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
         assertTrue("Should have a ttl override", amqpMessageFacade.hasAmqpTimeToLiveOverride());
         assertEquals(ttl, amqpMessageFacade.getAmqpTimeToLiveOverride());
         // check value on underlying TTL field is NOT set
-        assertNull("TTL field on underlying message should NOT be set", amqpMessageFacade.getAmqpMessage().getHeader().getTtl());
+        assertNull("TTL field on underlying message should NOT be set", amqpMessageFacade.getHeader().getTtl());
     }
 
     @Test
@@ -164,7 +163,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
 
         // check value on underlying TTL field is NOT set
         assertEquals("TTL has not been cleared", 0, message.getTtl());
-        assertNull("TTL field on underlying message should NOT be set", amqpMessageFacade.getAmqpMessage().getHeader().getTtl());
+        assertNull("TTL field on underlying message should NOT be set", amqpMessageFacade.getHeader().getTtl());
     }
 
     @Test
@@ -182,7 +181,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
 
         // check value on underlying TTL field is NOT set
         assertEquals("TTL has not been overriden", newTtl, message.getTtl());
-        assertEquals("TTL field on underlying message should be set", UnsignedInteger.valueOf(newTtl), amqpMessageFacade.getAmqpMessage().getHeader().getTtl());
+        assertEquals("TTL field on underlying message should be set", UnsignedInteger.valueOf(newTtl), amqpMessageFacade.getHeader().getTtl());
     }
 
     @Test
@@ -197,7 +196,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
         amqpMessageFacade.onSend(producerTtl);
 
         // check value on underlying TTL field is set to the override
-        assertEquals("TTL has not been overriden", overrideTtl, message.getTtl());
+        assertEquals("TTL has not been overriden", overrideTtl, amqpMessageFacade.getHeader().getTtl().intValue());
     }
 
     // --- delivery count  ---
@@ -247,12 +246,12 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
     @Test
     public void testGetDeliveryCountForReceivedMessageWithHeaderWithDeliveryCount() {
         Message message = Proton.message();
-        AmqpJmsMessageFacade amqpMessageFacade = createReceivedMessageFacade(createMockAmqpConsumer(), message);
-
         Header header = new Header();
         header.setDeliveryCount(new UnsignedInteger(1));
         message.setHeader(header);
 
+        AmqpJmsMessageFacade amqpMessageFacade = createReceivedMessageFacade(createMockAmqpConsumer(), message);
+
         // JMS delivery count starts at one.
         assertEquals("expected delivery count value not found", 2, amqpMessageFacade.getDeliveryCount());
 
@@ -278,12 +277,12 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
     @Test
     public void testSetRedeliveredWhenAlreadyRedeliveredDoesNotChangeDeliveryCount() {
         Message message = Proton.message();
-        AmqpJmsMessageFacade amqpMessageFacade = createReceivedMessageFacade(createMockAmqpConsumer(), message);
-
         Header header = new Header();
         header.setDeliveryCount(new UnsignedInteger(1));
         message.setHeader(header);
 
+        AmqpJmsMessageFacade amqpMessageFacade = createReceivedMessageFacade(createMockAmqpConsumer(), message);
+
         // Redelivered state inferred from delivery count
         assertTrue(amqpMessageFacade.isRedelivered());
         assertEquals(1, amqpMessageFacade.getRedeliveryCount());;
@@ -296,12 +295,12 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
     @Test
     public void testSetRedeliveredFalseClearsDeliveryCount() {
         Message message = Proton.message();
-        AmqpJmsMessageFacade amqpMessageFacade = createReceivedMessageFacade(createMockAmqpConsumer(), message);
-
         Header header = new Header();
         header.setDeliveryCount(new UnsignedInteger(1));
         message.setHeader(header);
 
+        AmqpJmsMessageFacade amqpMessageFacade = createReceivedMessageFacade(createMockAmqpConsumer(), message);
+
         // Redelivered state inferred from delivery count
         assertTrue(amqpMessageFacade.isRedelivered());
         assertEquals(1, amqpMessageFacade.getRedeliveryCount());;
@@ -424,9 +423,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
         amqpMessageFacade.setPriority(priority);
 
         assertEquals("expected priority value not found", priority, amqpMessageFacade.getPriority());
-
-        Message underlying = amqpMessageFacade.getAmqpMessage();
-        assertEquals("expected priority value not found", priority, underlying.getPriority());
+        assertEquals("expected priority value not found", priority, amqpMessageFacade.getHeader().getPriority().intValue());
     }
 
     /**
@@ -439,9 +436,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
         amqpMessageFacade.setPriority(-1);
 
         assertEquals("expected priority value not found", 0, amqpMessageFacade.getPriority());
-
-        Message underlying = amqpMessageFacade.getAmqpMessage();
-        assertEquals("expected priority value not found", 0, underlying.getPriority());
+        assertEquals("expected priority value not found", 0, amqpMessageFacade.getHeader().getPriority().intValue());
     }
 
     /**
@@ -454,9 +449,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
         amqpMessageFacade.setPriority(11);
 
         assertEquals("expected priority value not found", 9, amqpMessageFacade.getPriority());
-
-        Message underlying = amqpMessageFacade.getAmqpMessage();
-        assertEquals("expected priority value not found", 9, underlying.getPriority());
+        assertEquals("expected priority value not found", 9, amqpMessageFacade.getHeader().getPriority().intValue());
     }
 
     /**
@@ -497,8 +490,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
         assertEquals("expected priority value not returned", Message.DEFAULT_PRIORITY, amqpMessageFacade.getPriority());
 
         //check the underlying header field was actually cleared rather than set to Message.DEFAULT_PRIORITY
-        Message underlying = amqpMessageFacade.getAmqpMessage();
-        assertNull("underlying header priority field was not cleared", underlying.getHeader().getPriority());
+        assertNull("underlying header priority field was not cleared", amqpMessageFacade.getHeader().getPriority());
     }
 
     // ====== AMQP Properties Section =======
@@ -507,9 +499,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
     @Test
     public void testNewMessageHasNoUnderlyingPropertiesSection() {
         AmqpJmsMessageFacade amqpMessageFacade = createNewMessageFacade();
-
-        Message underlying = amqpMessageFacade.getAmqpMessage();
-        assertNull(underlying.getProperties());
+        assertNull(amqpMessageFacade.getProperties());
     }
 
     // --- group-id field ---
@@ -534,7 +524,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
 
         amqpMessageFacade.setGroupId(null);
 
-        assertNull("properties section was created", amqpMessageFacade.getAmqpMessage().getProperties());
+        assertNull("properties section was created", amqpMessageFacade.getProperties());
     }
 
     /**
@@ -551,8 +541,8 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
 
         amqpMessageFacade.setGroupId(groupId);
 
-        assertNotNull("properties section was not created", amqpMessageFacade.getAmqpMessage().getProperties());
-        assertEquals("value was not set for GroupId as expected", groupId, amqpMessageFacade.getAmqpMessage().getProperties().getGroupId());
+        assertNotNull("properties section was not created", amqpMessageFacade.getProperties());
+        assertEquals("value was not set for GroupId as expected", groupId, amqpMessageFacade.getProperties().getGroupId());
 
         assertEquals("value was not set for GroupId as expected", groupId, amqpMessageFacade.getGroupId());
     }
@@ -570,7 +560,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
         amqpMessageFacade.setGroupId(groupId);
         amqpMessageFacade.setGroupId(null);
 
-        assertNull("value was not cleared for GroupId as expected", amqpMessageFacade.getAmqpMessage().getProperties().getGroupId());
+        assertNull("value was not cleared for GroupId as expected", amqpMessageFacade.getProperties().getGroupId());
         assertNull("value was not cleared for GroupId as expected", amqpMessageFacade.getGroupId());
     }
 
@@ -613,7 +603,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
 
         amqpMessageFacade.setReplyToGroupId(null);
 
-        assertNull("properties section was created", amqpMessageFacade.getAmqpMessage().getProperties());
+        assertNull("properties section was created", amqpMessageFacade.getProperties());
     }
 
     /**
@@ -665,12 +655,10 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
 
         AmqpJmsMessageFacade amqpMessageFacade = createNewMessageFacade();
 
-        Message underlyingMessage = amqpMessageFacade.getAmqpMessage();
-
         amqpMessageFacade.setReplyToGroupId(replyToGroupId);
 
-        assertNotNull("expected ReplyToGroupId on message was not found", underlyingMessage.getReplyToGroupId());
-        assertEquals("expected ReplyToGroupId on message was not found", replyToGroupId, underlyingMessage.getReplyToGroupId());
+        assertNotNull("expected ReplyToGroupId on message was not found", amqpMessageFacade.getProperties().getReplyToGroupId());
+        assertEquals("expected ReplyToGroupId on message was not found", replyToGroupId, amqpMessageFacade.getProperties().getReplyToGroupId());
     }
 
     /**
@@ -699,7 +687,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
         int groupSequence = 5;
         amqpMessageFacade.setGroupSequence(groupSequence);
 
-        assertEquals("underlying message should have groupSequence field value", groupSequence, amqpMessageFacade.getAmqpMessage().getProperties().getGroupSequence().longValue());
+        assertEquals("underlying message should have groupSequence field value", groupSequence, amqpMessageFacade.getProperties().getGroupSequence().longValue());
         assertEquals("GroupSequence not as expected", groupSequence, amqpMessageFacade.getGroupSequence());
     }
 
@@ -717,7 +705,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
         UnsignedInteger mapped = UnsignedInteger.valueOf(MAX_UINT - delta);
         amqpMessageFacade.setGroupSequence(-1 - delta);
 
-        assertEquals("underlying message should have no groupSequence field value",mapped, amqpMessageFacade.getAmqpMessage().getProperties().getGroupSequence());
+        assertEquals("underlying message should have no groupSequence field value",mapped, amqpMessageFacade.getProperties().getGroupSequence());
         assertEquals("GroupSequence not as expected", -1 - delta, amqpMessageFacade.getGroupSequence());
     }
 
@@ -766,7 +754,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
 
         amqpMessageFacade.setGroupSequence(0);
 
-        assertNull("underlying message should still have no properties setion", amqpMessageFacade.getAmqpMessage().getProperties());
+        assertNull("underlying message should still have no properties setion", amqpMessageFacade.getProperties());
         assertEquals("GroupSequence should be 0", 0, amqpMessageFacade.getGroupSequence());
     }
 
@@ -782,13 +770,12 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
 
         AmqpJmsMessageFacade amqpMessageFacade = createNewMessageFacade();
 
-        Message underlyingMessage = amqpMessageFacade.getAmqpMessage();
-        assertNull(underlyingMessage.getAddress());
+        assertNull(amqpMessageFacade.getProperties());
 
         amqpMessageFacade.setDestination(dest);
 
-        assertNotNull(underlyingMessage.getAddress());
-        assertEquals(testToAddress, underlyingMessage.getAddress());
+        assertNotNull(amqpMessageFacade.getProperties().getTo());
+        assertEquals(testToAddress, amqpMessageFacade.getProperties().getTo());
         assertEquals(dest, amqpMessageFacade.getDestination());
     }
 
@@ -822,13 +809,12 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
 
         AmqpJmsMessageFacade amqpMessageFacade = createNewMessageFacade();
 
-        Message underlyingMessage = amqpMessageFacade.getAmqpMessage();
-        assertNull(underlyingMessage.getReplyTo());
+        assertNull(amqpMessageFacade.getProperties());
 
         amqpMessageFacade.setReplyTo(dest);
 
-        assertNotNull(underlyingMessage.getReplyTo());
-        assertEquals(testReplyToAddress, underlyingMessage.getReplyTo());
+        assertNotNull(amqpMessageFacade.getProperties().getReplyTo());
+        assertEquals(testReplyToAddress, amqpMessageFacade.getProperties().getReplyTo());
         assertEquals(dest, amqpMessageFacade.getReplyTo());
     }
 
@@ -871,10 +857,8 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
         AmqpJmsMessageFacade amqpMessageFacade = createNewMessageFacade();
         amqpMessageFacade.setCorrelationId(testCorrelationId);
 
-        Message amqpMessage = amqpMessageFacade.getAmqpMessage();
-        assertEquals("correlationId value on underlying AMQP message not as expected", testCorrelationId, amqpMessage.getCorrelationId());
+        assertEquals("correlationId value on underlying AMQP message not as expected", testCorrelationId, amqpMessageFacade.getProperties().getCorrelationId());
         assertEquals("Expected correlationId not returned", testCorrelationId, amqpMessageFacade.getCorrelationId());
-
     }
 
     /**
@@ -889,8 +873,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
         AmqpJmsMessageFacade amqpMessageFacade = createNewMessageFacade();
         amqpMessageFacade.setCorrelationId(testCorrelationId);
 
-        Message amqpMessage = amqpMessageFacade.getAmqpMessage();
-        assertEquals("correlationId value on underlying AMQP message not as expected", testCorrelationId, amqpMessage.getCorrelationId());
+        assertEquals("correlationId value on underlying AMQP message not as expected", testCorrelationId, amqpMessageFacade.getProperties().getCorrelationId());
         assertEquals("Expected correlationId not returned from facade", testCorrelationId, amqpMessageFacade.getCorrelationId());
     }
 
@@ -906,7 +889,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
         amqpMessageFacade.setCorrelationId("cid");
         amqpMessageFacade.setCorrelationId(null);
 
-        assertNull("Unexpected correlationId value on underlying AMQP message", amqpMessageFacade.getAmqpMessage().getCorrelationId());
+        assertNull("Unexpected correlationId value on underlying AMQP message", amqpMessageFacade.getCorrelationId());
         assertNull("Expected correlationId bytes to be null", amqpMessageFacade.getCorrelationId());
     }
 
@@ -944,7 +927,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
         AmqpJmsMessageFacade amqpMessageFacade = createNewMessageFacade();
         amqpMessageFacade.setCorrelationId(converted);
 
-        assertEquals("Unexpected correlationId value on underlying AMQP message", testCorrelationId, amqpMessageFacade.getAmqpMessage().getCorrelationId());
+        assertEquals("Unexpected correlationId value on underlying AMQP message", testCorrelationId, amqpMessageFacade.getProperties().getCorrelationId());
         assertEquals("Expected correlationId not returned", converted, amqpMessageFacade.getCorrelationId());
     }
 
@@ -972,7 +955,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
         AmqpJmsMessageFacade amqpMessageFacade = createNewMessageFacade();
         amqpMessageFacade.setCorrelationId(converted);
 
-        assertEquals("Unexpected correlationId value on underlying AMQP message", testCorrelationId, amqpMessageFacade.getAmqpMessage().getCorrelationId());
+        assertEquals("Unexpected correlationId value on underlying AMQP message", testCorrelationId, amqpMessageFacade.getProperties().getCorrelationId());
         assertEquals("Expected correlationId not returned", converted, amqpMessageFacade.getCorrelationId());
     }
 
@@ -1004,7 +987,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
         AmqpJmsMessageFacade amqpMessageFacade = createNewMessageFacade();
         amqpMessageFacade.setCorrelationId(converted);
 
-        assertEquals("Unexpected correlationId value on underlying AMQP message", testCorrelationId, amqpMessageFacade.getAmqpMessage().getCorrelationId());
+        assertEquals("Unexpected correlationId value on underlying AMQP message", testCorrelationId, amqpMessageFacade.getProperties().getCorrelationId());
         assertEquals("Expected correlationId not returned", converted, amqpMessageFacade.getCorrelationId());
     }
 
@@ -1021,7 +1004,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
         AmqpJmsMessageFacade amqpMessageFacade = createNewMessageFacade();
         amqpMessageFacade.setCorrelationIdBytes(bytes);
 
-        assertEquals("Unexpected correlationId value on underlying AMQP message", testCorrelationId, amqpMessageFacade.getAmqpMessage().getCorrelationId());
+        assertEquals("Unexpected correlationId value on underlying AMQP message", testCorrelationId, amqpMessageFacade.getProperties().getCorrelationId());
         assertArrayEquals("Expected correlationId bytes not returned", bytes, amqpMessageFacade.getCorrelationIdBytes());
     }
 
@@ -1039,7 +1022,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
         amqpMessageFacade.setCorrelationIdBytes(bytes);
         amqpMessageFacade.setCorrelationIdBytes(null);
 
-        assertNull("Unexpected correlationId value on underlying AMQP message", amqpMessageFacade.getAmqpMessage().getCorrelationId());
+        assertNull("Unexpected correlationId value on underlying AMQP message", amqpMessageFacade.getCorrelationId());
         assertNull("Expected correlationId bytes to be null", amqpMessageFacade.getCorrelationIdBytes());
     }
 
@@ -1069,7 +1052,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
 
         AmqpJmsMessageFacade amqpMessageFacade = createReceivedMessageFacade(createMockAmqpConsumer(), message);
 
-        assertEquals("Unexpected correlationId value on underlying AMQP message", testCorrelationId, amqpMessageFacade.getAmqpMessage().getCorrelationId());
+        assertEquals("Unexpected correlationId value on underlying AMQP message", testCorrelationId, amqpMessageFacade.getProperties().getCorrelationId());
         assertArrayEquals("Expected correlationId bytes not returned", bytes, amqpMessageFacade.getCorrelationIdBytes());
     }
 
@@ -1140,7 +1123,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
 
         amqpMessageFacade.setMessageId(testMessageId);
 
-        assertEquals("underlying messageId value not as expected", testMessageId, amqpMessageFacade.getAmqpMessage().getMessageId());
+        assertEquals("underlying messageId value not as expected", testMessageId, amqpMessageFacade.getMessageId());
     }
 
     /**
@@ -1157,11 +1140,11 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
 
         amqpMessageFacade.setMessageId(testMessageId);
 
-        assertNotNull("messageId should not be null", amqpMessageFacade.getAmqpMessage().getMessageId());
+        assertNotNull("messageId should not be null", amqpMessageFacade.getMessageId());
 
         amqpMessageFacade.setMessageId(null);
 
-        assertNull("Expected messageId to be null", amqpMessageFacade.getAmqpMessage().getMessageId());
+        assertNull("Expected messageId to be null", amqpMessageFacade.getMessageId());
         assertNull("ID was not null", amqpMessageFacade.getMessageId());
     }
 
@@ -1256,13 +1239,13 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
     public void testSetCreationTimeOnNewNewMessage() {
         AmqpJmsMessageFacade amqpMessageFacade = createNewMessageFacade();
 
-        assertNull("Expected null Properties section", amqpMessageFacade.getAmqpMessage().getProperties());
+        assertNull("Expected null Properties section", amqpMessageFacade.getProperties());
 
         long expected = 1;
         amqpMessageFacade.setTimestamp(expected);
 
         assertEquals("Unexpected timestamp value", expected, amqpMessageFacade.getTimestamp());
-        assertEquals("Expected creation-time field to be set on new Properties section", new Date(expected), amqpMessageFacade.getAmqpMessage().getProperties().getCreationTime());
+        assertEquals("Expected creation-time field to be set on new Properties section", new Date(expected), amqpMessageFacade.getProperties().getCreationTime());
     }
 
     @Test
@@ -1280,7 +1263,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
 
         amqpMessageFacade.setTimestamp(timestamp);
 
-        assertEquals("Expected creation-time field to be set", timestamp.longValue(), amqpMessageFacade.getAmqpMessage().getProperties().getCreationTime().getTime());
+        assertEquals("Expected creation-time field to be set", timestamp.longValue(), amqpMessageFacade.getProperties().getCreationTime().getTime());
         assertEquals("Expected timestamp", timestamp.longValue(), amqpMessageFacade.getTimestamp());
     }
 
@@ -1290,7 +1273,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
 
         amqpMessageFacade.setTimestamp(0);
 
-        assertNull("underlying message should have no properties section", amqpMessageFacade.getAmqpMessage().getProperties());
+        assertNull("underlying message should have no properties section", amqpMessageFacade.getProperties());
         assertEquals("Timestamp should not be set", 0, amqpMessageFacade.getTimestamp());
     }
 
@@ -1303,7 +1286,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
 
         amqpMessageFacade.setTimestamp(0);
 
-        assertNull("Expected creation-time to be null", amqpMessageFacade.getAmqpMessage().getProperties().getCreationTime());
+        assertNull("Expected creation-time to be null", amqpMessageFacade.getProperties().getCreationTime());
         assertEquals("Expected no timestamp", 0, amqpMessageFacade.getTimestamp());
     }
 
@@ -1324,7 +1307,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
 
         amqpMessageFacade.setExpiration(timestamp);
 
-        assertEquals("Expected absolute-expiry-time to be set", timestamp.longValue(), amqpMessageFacade.getAmqpMessage().getProperties().getAbsoluteExpiryTime().getTime());
+        assertEquals("Expected absolute-expiry-time to be set", timestamp.longValue(), amqpMessageFacade.getProperties().getAbsoluteExpiryTime().getTime());
         assertEquals("Expected expiration to be set", timestamp.longValue(), amqpMessageFacade.getExpiration());
     }
 
@@ -1332,11 +1315,11 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
     public void testSetExpirationZeroOnNewMessageDoesNotCreatePropertiesSection() {
         AmqpJmsMessageFacade amqpMessageFacade = createNewMessageFacade();
 
-        assertNull("Expected properties section not to exist", amqpMessageFacade.getAmqpMessage().getProperties());
+        assertNull("Expected properties section not to exist", amqpMessageFacade.getProperties());
 
         amqpMessageFacade.setExpiration(0);
 
-        assertNull("Expected properties section still not to exist", amqpMessageFacade.getAmqpMessage().getProperties());
+        assertNull("Expected properties section still not to exist", amqpMessageFacade.getProperties());
     }
 
     @Test
@@ -1348,7 +1331,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
 
         amqpMessageFacade.setExpiration(0);
 
-        assertNull("Expected absolute-expiry-time to be null", amqpMessageFacade.getAmqpMessage().getProperties().getAbsoluteExpiryTime());
+        assertNull("Expected absolute-expiry-time to be null", amqpMessageFacade.getProperties().getAbsoluteExpiryTime());
         assertEquals("Expected no expiration", 0, amqpMessageFacade.getExpiration());
     }
 
@@ -1393,8 +1376,8 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
 
         amqpMessageFacade.setUserId(userIdString);
 
-        assertNotNull("properties section was not created", amqpMessageFacade.getAmqpMessage().getProperties());
-        assertTrue("bytes were not set as expected for userid", Arrays.equals(bytes, amqpMessageFacade.getAmqpMessage().getProperties().getUserId().getArray()));
+        assertNotNull("properties section was not created", amqpMessageFacade.getProperties());
+        assertTrue("bytes were not set as expected for userid", Arrays.equals(bytes, amqpMessageFacade.getProperties().getUserId().getArray()));
         assertEquals("userid not as expected", userIdString, amqpMessageFacade.getUserId());
     }
 
@@ -1411,8 +1394,8 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
         amqpMessageFacade.setUserId(userIdString);
         amqpMessageFacade.setUserId(null);
 
-        assertNotNull("properties section was not created", amqpMessageFacade.getAmqpMessage().getProperties());
-        assertNull("bytes were not cleared as expected for userid", amqpMessageFacade.getAmqpMessage().getProperties().getUserId());
+        assertNotNull("properties section was not created", amqpMessageFacade.getProperties());
+        assertNull("bytes were not cleared as expected for userid", amqpMessageFacade.getProperties().getUserId());
         assertNull("userid not as expected", amqpMessageFacade.getUserId());
     }
 
@@ -1422,7 +1405,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
 
         amqpMessageFacade.setUserId(null);
 
-        assertNull("underlying message should still have no properties setion", amqpMessageFacade.getAmqpMessage().getProperties());
+        assertNull("underlying message should still have no properties setion", amqpMessageFacade.getProperties());
         assertEquals("UserId should be null", null, amqpMessageFacade.getUserId());
     }
 
@@ -1469,8 +1452,8 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
 
         amqpMessageFacade.setUserIdBytes(bytes);
 
-        assertNotNull("properties section was not created", amqpMessageFacade.getAmqpMessage().getProperties());
-        assertTrue("bytes were not set as expected for userid", Arrays.equals(bytes, amqpMessageFacade.getAmqpMessage().getProperties().getUserId().getArray()));
+        assertNotNull("properties section was not created", amqpMessageFacade.getProperties());
+        assertTrue("bytes were not set as expected for userid", Arrays.equals(bytes, amqpMessageFacade.getProperties().getUserId().getArray()));
         assertArrayEquals("userid bytes not as expected", bytes, amqpMessageFacade.getUserIdBytes());
     }
 
@@ -1489,8 +1472,8 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
         amqpMessageFacade.setUserIdBytes(bytes);
         amqpMessageFacade.setUserId(null);
 
-        assertNotNull("properties section was not created", amqpMessageFacade.getAmqpMessage().getProperties());
-        assertNull("bytes were not cleared as expected for userid", amqpMessageFacade.getAmqpMessage().getProperties().getUserId());
+        assertNotNull("properties section was not created", amqpMessageFacade.getProperties());
+        assertNull("bytes were not cleared as expected for userid", amqpMessageFacade.getProperties().getUserId());
         assertNull("userid bytes not as expected", amqpMessageFacade.getUserIdBytes());
     }
 
@@ -1500,7 +1483,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
 
         amqpMessageFacade.setUserIdBytes(null);
 
-        assertNull("underlying message should still have no properties setion", amqpMessageFacade.getAmqpMessage().getProperties());
+        assertNull("underlying message should still have no properties setion", amqpMessageFacade.getProperties());
         assertEquals("UserId should be null", null, amqpMessageFacade.getUserIdBytes());
     }
 
@@ -1511,20 +1494,18 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
     public void testNewMessageHasUnderlyingMessageAnnotationsSectionWithTypeAnnotation() {
         AmqpJmsMessageFacade amqpMessageFacade = createNewMessageFacade();;
 
-        Message underlying = amqpMessageFacade.getAmqpMessage();
-        assertNotNull(underlying.getMessageAnnotations());
+        assertNotNull(amqpMessageFacade.getMessageAnnotations());
         Symbol annotationKey = AmqpMessageSupport.getSymbol(AmqpMessageSupport.JMS_MSG_TYPE);
-        assertEquals(AmqpMessageSupport.JMS_MESSAGE, underlying.getMessageAnnotations().getValue().get(annotationKey));
+        assertEquals(AmqpMessageSupport.JMS_MESSAGE, amqpMessageFacade.getMessageAnnotations().getValue().get(annotationKey));
     }
 
     @Test
     public void testNewMessageDoesNotHaveUnderlyingMessageAnnotationsSectionWithDeliveryTime() {
         AmqpJmsMessageFacade amqpMessageFacade = createNewMessageFacade();;
 
-        Message underlying = amqpMessageFacade.getAmqpMessage();
-        assertNotNull(underlying.getMessageAnnotations());
+        assertNotNull(amqpMessageFacade.getMessageAnnotations());
         Symbol annotationKey = AmqpMessageSupport.getSymbol(AmqpMessageSupport.JMS_DELIVERY_TIME);
-        assertNull(underlying.getMessageAnnotations().getValue().get(annotationKey));
+        assertNull(amqpMessageFacade.getMessageAnnotations().getValue().get(annotationKey));
     }
 
     @Test
@@ -1594,7 +1575,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
         // check setting first annotation
         amqpMessageFacade.setMessageAnnotation(symbolKeyName, value);
 
-        MessageAnnotations underlyingAnnotations = amqpMessageFacade.getAmqpMessage().getMessageAnnotations();
+        MessageAnnotations underlyingAnnotations = amqpMessageFacade.getMessageAnnotations();
         assertNotNull(underlyingAnnotations);
 
         assertTrue(underlyingAnnotations.getValue().containsKey(Symbol.valueOf(symbolKeyName)));
@@ -1643,8 +1624,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
 
         amqpMessageFacade.clearMessageAnnotations();
 
-        Message underlying = amqpMessageFacade.getAmqpMessage();
-        assertNull(underlying.getMessageAnnotations());
+        assertNull(amqpMessageFacade.getMessageAnnotations());
     }
 
     @Test
@@ -1662,8 +1642,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
 
         amqpMessageFacade.clearMessageAnnotations();
 
-        Message underlying = amqpMessageFacade.getAmqpMessage();
-        assertNull(underlying.getMessageAnnotations());
+        assertNull(amqpMessageFacade.getMessageAnnotations());
     }
 
     // ====== Type =======
@@ -1681,7 +1660,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
         amqpMessageFacade.setType(jmsType);
 
         assertEquals("Subject should be set to the provded JMSType string", jmsType,
-                        amqpMessageFacade.getAmqpMessage().getSubject());
+                        amqpMessageFacade.getProperties().getSubject());
     }
 
     @Test
@@ -1691,9 +1670,9 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
 
         amqpMessageFacade.setType(jmsType);
         assertEquals("Subject should be set to the provded JMSType string", jmsType,
-                        amqpMessageFacade.getAmqpMessage().getSubject());
+                        amqpMessageFacade.getProperties().getSubject());
         amqpMessageFacade.setType(null);
-        assertNull("Subject should be clear", amqpMessageFacade.getAmqpMessage().getSubject());
+        assertNull("Subject should be clear", amqpMessageFacade.getProperties().getSubject());
     }
 
     /**
@@ -1838,7 +1817,7 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
         assertEquals(TEST_VALUE_STRING_A, amqpMessageFacade.getProperty(TEST_PROP_A));
 
         @SuppressWarnings("unchecked")
-        Map<String, Object> underlyingApplicationProps = amqpMessageFacade.getAmqpMessage().getApplicationProperties().getValue();
+        Map<String, Object> underlyingApplicationProps = amqpMessageFacade.getApplicationProperties().getValue();
         assertTrue(underlyingApplicationProps.containsKey(TEST_PROP_A));
         assertEquals(TEST_VALUE_STRING_A, underlyingApplicationProps.get(TEST_PROP_A));
     }
@@ -1947,9 +1926,10 @@ public class AmqpJmsMessageFacadeTest extends AmqpJmsMessageTypesTestCase  {
 
     @Test
     public void testClearBodyRemoveMessageBody() {
-        Message message = Mockito.mock(Message.class);
-        JmsMessageFacade amqpMessageFacade = createReceivedMessageFacade(createMockAmqpConsumer(), message);
+        Message message = Message.Factory.create();
+        AmqpJmsMessageFacade amqpMessageFacade = createReceivedMessageFacade(createMockAmqpConsumer(), message);
+        amqpMessageFacade = Mockito.spy(amqpMessageFacade);
         amqpMessageFacade.clearBody();
-        Mockito.verify(message).setBody(null);
+        Mockito.verify(amqpMessageFacade).setBody(null);
     }
 }

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMessageTypesTestCase.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMessageTypesTestCase.java b/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMessageTypesTestCase.java
index 48a78f6..813cc82 100644
--- a/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMessageTypesTestCase.java
+++ b/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMessageTypesTestCase.java
@@ -16,10 +16,6 @@
  */
 package org.apache.qpid.jms.provider.amqp.message;
 
-import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.encodeMessage;
-
-import java.nio.charset.StandardCharsets;
-
 import org.apache.qpid.jms.JmsDestination;
 import org.apache.qpid.jms.JmsTopic;
 import org.apache.qpid.jms.meta.JmsConnectionId;
@@ -47,51 +43,86 @@ public class AmqpJmsMessageTypesTestCase extends QpidJmsTestCase {
     //---------- Test Support Methods ----------------------------------------//
 
     protected AmqpJmsMessageFacade createNewMessageFacade() {
-        return new AmqpJmsMessageFacade(createMockAmqpConnection());
+        AmqpJmsMessageFacade facade = new AmqpJmsMessageFacade();
+        facade.initialize(createMockAmqpConnection());
+        return facade;
     }
 
     protected AmqpJmsMessageFacade createReceivedMessageFacade(AmqpConsumer amqpConsumer, Message message) {
-        return new AmqpJmsMessageFacade(amqpConsumer, message);
+        AmqpJmsMessageFacade facade = new AmqpJmsMessageFacade();
+        initializeReceivedMessage(facade, amqpConsumer, message);
+        return facade;
     }
 
     protected AmqpJmsTextMessageFacade createNewTextMessageFacade() {
-        return new AmqpJmsTextMessageFacade(createMockAmqpConnection());
+        AmqpJmsTextMessageFacade facade = new AmqpJmsTextMessageFacade();
+        facade.initialize(createMockAmqpConnection());
+        return facade;
     }
 
     protected AmqpJmsTextMessageFacade createReceivedTextMessageFacade(AmqpConsumer amqpConsumer, Message message) {
-        return new AmqpJmsTextMessageFacade(amqpConsumer, message, StandardCharsets.UTF_8);
+        AmqpJmsTextMessageFacade facade = new AmqpJmsTextMessageFacade();
+        initializeReceivedMessage(facade, amqpConsumer, message);
+        return facade;
     }
 
     protected AmqpJmsBytesMessageFacade createNewBytesMessageFacade() {
-        return new AmqpJmsBytesMessageFacade(createMockAmqpConnection());
+        AmqpJmsBytesMessageFacade facade = new AmqpJmsBytesMessageFacade();
+        facade.initialize(createMockAmqpConnection());
+        return facade;
     }
 
     protected AmqpJmsBytesMessageFacade createReceivedBytesMessageFacade(AmqpConsumer amqpConsumer, Message message) {
-        return new AmqpJmsBytesMessageFacade(amqpConsumer, message);
+        AmqpJmsBytesMessageFacade facade = new AmqpJmsBytesMessageFacade();
+        initializeReceivedMessage(facade, amqpConsumer, message);
+        return facade;
     }
 
     protected AmqpJmsMapMessageFacade createNewMapMessageFacade() {
-        return new AmqpJmsMapMessageFacade(createMockAmqpConnection());
+        AmqpJmsMapMessageFacade facade = new AmqpJmsMapMessageFacade();
+        facade.initialize(createMockAmqpConnection());
+        return facade;
     }
 
     protected AmqpJmsMapMessageFacade createReceivedMapMessageFacade(AmqpConsumer amqpConsumer, Message message) {
-        return new AmqpJmsMapMessageFacade(amqpConsumer, message);
+        AmqpJmsMapMessageFacade facade = new AmqpJmsMapMessageFacade();
+        initializeReceivedMessage(facade, amqpConsumer, message);
+        return facade;
     }
 
     protected AmqpJmsStreamMessageFacade createNewStreamMessageFacade() {
-        return new AmqpJmsStreamMessageFacade(createMockAmqpConnection());
+        AmqpJmsStreamMessageFacade facade = new AmqpJmsStreamMessageFacade();
+        facade.initialize(createMockAmqpConnection());
+        return facade;
     }
 
     protected AmqpJmsStreamMessageFacade createReceivedStreamMessageFacade(AmqpConsumer amqpConsumer, Message message) {
-        return new AmqpJmsStreamMessageFacade(amqpConsumer, message);
+        AmqpJmsStreamMessageFacade facade = new AmqpJmsStreamMessageFacade();
+        initializeReceivedMessage(facade, amqpConsumer, message);
+        return facade;
     }
 
     protected AmqpJmsObjectMessageFacade createNewObjectMessageFacade(boolean amqpTyped) {
-        return new AmqpJmsObjectMessageFacade(createMockAmqpConnection(), amqpTyped);
+        AmqpJmsObjectMessageFacade facade = new AmqpJmsObjectMessageFacade();
+        facade.initialize(createMockAmqpConnection(amqpTyped));
+        return facade;
     }
 
     protected AmqpJmsObjectMessageFacade createReceivedObjectMessageFacade(AmqpConsumer amqpConsumer, Message message) {
-        return new AmqpJmsObjectMessageFacade(amqpConsumer, message, encodeMessage(message));
+        AmqpJmsObjectMessageFacade facade = new AmqpJmsObjectMessageFacade();
+        initializeReceivedMessage(facade, amqpConsumer, message);
+        return facade;
+    }
+
+    protected void initializeReceivedMessage(AmqpJmsMessageFacade facade, AmqpConsumer amqpConsumer, Message message) {
+        facade.setHeader(message.getHeader());
+        facade.setDeliveryAnnotations(message.getDeliveryAnnotations());
+        facade.setMessageAnnotations(message.getMessageAnnotations());
+        facade.setProperties(message.getProperties());
+        facade.setApplicationProperties(message.getApplicationProperties());
+        facade.setBody(message.getBody());
+        facade.setFooter(message.getFooter());
+        facade.initialize(amqpConsumer);
     }
 
     protected AmqpConsumer createMockAmqpConsumer() {
@@ -105,9 +136,14 @@ public class AmqpJmsMessageTypesTestCase extends QpidJmsTestCase {
     }
 
     protected AmqpConnection createMockAmqpConnection() {
+        return createMockAmqpConnection(false);
+    }
+
+    protected AmqpConnection createMockAmqpConnection(boolean amqpTyped) {
         JmsConnectionId connectionId = new JmsConnectionId("ID:MOCK:1");
         AmqpConnection connection = Mockito.mock(AmqpConnection.class);
         Mockito.when(connection.getResourceInfo()).thenReturn(new JmsConnectionInfo(connectionId));
+        Mockito.when(connection.isObjectMessageUsesAmqpTypes()).thenReturn(amqpTyped);
 
         return connection;
     }

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsObjectMessageFacadeTest.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsObjectMessageFacadeTest.java b/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsObjectMessageFacadeTest.java
index f95c685..7a6f10f 100644
--- a/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsObjectMessageFacadeTest.java
+++ b/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsObjectMessageFacadeTest.java
@@ -56,8 +56,7 @@ public class AmqpJmsObjectMessageFacadeTest extends AmqpJmsMessageTypesTestCase
     public void testNewMessageToSendContainsMessageTypeAnnotation() throws Exception {
         AmqpJmsObjectMessageFacade amqpObjectMessageFacade = createNewObjectMessageFacade(false);
 
-        Message protonMessage = amqpObjectMessageFacade.getAmqpMessage();
-        MessageAnnotations annotations = protonMessage.getMessageAnnotations();
+        MessageAnnotations annotations = amqpObjectMessageFacade.getMessageAnnotations();
         Map<Symbol, Object> annotationsMap = annotations.getValue();
 
         assertNotNull("MessageAnnotations section was not present", annotations);
@@ -99,12 +98,11 @@ public class AmqpJmsObjectMessageFacadeTest extends AmqpJmsMessageTypesTestCase
         AmqpJmsObjectMessageFacade amqpObjectMessageFacade = createNewObjectMessageFacade(amqpTyped);
         amqpObjectMessageFacade.onSend(0);
 
-        Message protonMessage = amqpObjectMessageFacade.getAmqpMessage();
-        assertNotNull("Message body should be presents", protonMessage.getBody());
+        assertNotNull("Message body should be presents", amqpObjectMessageFacade.getBody());
         if(amqpTyped) {
-            assertSame("Expected existing body section to be replaced", AmqpTypedObjectDelegate.NULL_OBJECT_BODY, protonMessage.getBody());
+            assertSame("Expected existing body section to be replaced", AmqpTypedObjectDelegate.NULL_OBJECT_BODY, amqpObjectMessageFacade.getBody());
         } else {
-            assertSame("Expected existing body section to be replaced", AmqpSerializedObjectDelegate.NULL_OBJECT_BODY, protonMessage.getBody());
+            assertSame("Expected existing body section to be replaced", AmqpSerializedObjectDelegate.NULL_OBJECT_BODY, amqpObjectMessageFacade.getBody());
         }
     }
 
@@ -123,8 +121,6 @@ public class AmqpJmsObjectMessageFacadeTest extends AmqpJmsMessageTypesTestCase
         AmqpJmsObjectMessageFacade amqpObjectMessageFacade = createNewObjectMessageFacade(false);
         amqpObjectMessageFacade.setObject(content);
 
-        Message protonMessage = amqpObjectMessageFacade.getAmqpMessage();
-
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         ObjectOutputStream oos = new ObjectOutputStream(baos);
         oos.writeObject(content);
@@ -133,7 +129,7 @@ public class AmqpJmsObjectMessageFacadeTest extends AmqpJmsMessageTypesTestCase
         byte[] bytes = baos.toByteArray();
 
         // retrieve the bytes from the underlying message, check they match expectation
-        Section section = protonMessage.getBody();
+        Section section = amqpObjectMessageFacade.getBody();
         assertNotNull(section);
         assertEquals(Data.class, section.getClass());
         assertArrayEquals("Underlying message data section did not contain the expected bytes", bytes, ((Data) section).getValue().getArray());
@@ -152,10 +148,8 @@ public class AmqpJmsObjectMessageFacadeTest extends AmqpJmsMessageTypesTestCase
         AmqpJmsObjectMessageFacade amqpObjectMessageFacade = createNewObjectMessageFacade(true);
         amqpObjectMessageFacade.setObject(content);
 
-        Message protonMessage = amqpObjectMessageFacade.getAmqpMessage();
-
         // retrieve the body from the underlying message, check it matches expectation
-        Section section = protonMessage.getBody();
+        Section section = amqpObjectMessageFacade.getBody();
         assertNotNull(section);
         assertEquals(AmqpValue.class, section.getClass());
         assertEquals("Underlying message body did not contain the expected content", content, ((AmqpValue) section).getValue());
@@ -175,9 +169,9 @@ public class AmqpJmsObjectMessageFacadeTest extends AmqpJmsMessageTypesTestCase
 
         AmqpJmsObjectMessageFacade amqpObjectMessageFacade = createReceivedObjectMessageFacade(createMockAmqpConsumer(), protonMessage);
 
-        assertNotNull("Expected existing body section to be found", protonMessage.getBody());
+        assertNotNull("Expected existing body section to be found", amqpObjectMessageFacade.getBody());
         amqpObjectMessageFacade.setObject(null);
-        assertSame("Expected existing body section to be replaced", AmqpSerializedObjectDelegate.NULL_OBJECT_BODY, protonMessage.getBody());
+        assertSame("Expected existing body section to be replaced", AmqpSerializedObjectDelegate.NULL_OBJECT_BODY, amqpObjectMessageFacade.getBody());
         assertNull("Expected null object", amqpObjectMessageFacade.getObject());
     }
 
@@ -195,9 +189,9 @@ public class AmqpJmsObjectMessageFacadeTest extends AmqpJmsMessageTypesTestCase
 
         AmqpJmsObjectMessageFacade amqpObjectMessageFacade = createReceivedObjectMessageFacade(createMockAmqpConsumer(), protonMessage);
 
-        assertNotNull("Expected existing body section to be found", protonMessage.getBody());
+        assertNotNull("Expected existing body section to be found", amqpObjectMessageFacade.getBody());
         amqpObjectMessageFacade.clearBody();
-        assertSame("Expected existing body section to be replaced", AmqpSerializedObjectDelegate.NULL_OBJECT_BODY, protonMessage.getBody());
+        assertSame("Expected existing body section to be replaced", AmqpSerializedObjectDelegate.NULL_OBJECT_BODY, amqpObjectMessageFacade.getBody());
         assertNull("Expected null object", amqpObjectMessageFacade.getObject());
     }
 
@@ -321,11 +315,7 @@ public class AmqpJmsObjectMessageFacadeTest extends AmqpJmsMessageTypesTestCase
         Serializable serialized = amqpObjectMessageFacade.getObject();
         assertTrue("Unexpected object type returned", serialized instanceof Map<?, ?>);
         Map<String, String> returnedObject1 = (Map<String, String>) serialized;
-        if (contentType) {
-            assertNotSame("Expected different objects, due to snapshot being taken", origMap, returnedObject1);
-        } else {
-            assertSame("Expected same objects, due to initial snapshot of delivered value", origMap, returnedObject1);
-        }
+        assertNotSame("Expected different objects, due to snapshot being taken", origMap, returnedObject1);
         assertEquals("Expected equal objects, due to snapshot being taken", origMap, returnedObject1);
 
         // verify we get a different-but-equal object back when compared to the previously retrieved object


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@qpid.apache.org
For additional commands, e-mail: commits-help@qpid.apache.org


[3/5] qpid-jms git commit: QPIDJMS-215 Perform Message encoding at send time

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/MessageIntegrationTest.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/MessageIntegrationTest.java b/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/MessageIntegrationTest.java
index 48adaea..6cd1fef 100644
--- a/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/MessageIntegrationTest.java
+++ b/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/MessageIntegrationTest.java
@@ -38,11 +38,13 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.UUID;
 
+import javax.jms.CompletionListener;
 import javax.jms.Connection;
 import javax.jms.Destination;
 import javax.jms.JMSException;
 import javax.jms.Message;
 import javax.jms.MessageConsumer;
+import javax.jms.MessageNotWriteableException;
 import javax.jms.MessageProducer;
 import javax.jms.Queue;
 import javax.jms.Session;
@@ -1972,4 +1974,101 @@ public class MessageIntegrationTest extends QpidJmsTestCase
             testPeer.waitForAllHandlersToComplete(1000);
         }
     }
+
+    @Test(timeout = 20000)
+    public void testAsyncSendMarksMessageReadOnly() throws Exception {
+        try(TestAmqpPeer testPeer = new TestAmqpPeer();) {
+            JmsConnection connection = (JmsConnection) testFixture.establishConnecton(testPeer);
+            connection.setSendTimeout(15000);
+
+            testPeer.expectBegin();
+
+            Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
+            String queueName = "myQueue";
+            Queue queue = session.createQueue(queueName);
+
+            Message message = session.createMessage();
+            TransferPayloadCompositeMatcher messageMatcher = new TransferPayloadCompositeMatcher();
+
+            // Expect the producer to attach and grant it some credit, it should send
+            // a transfer which we will not send any response so that we can check that
+            // the inflight message is read-only
+            testPeer.expectSenderAttach();
+            testPeer.expectTransferButDoNotRespond(messageMatcher);
+            testPeer.expectClose();
+
+            MessageProducer producer = session.createProducer(queue);
+            TestJmsCompletionListener listener = new TestJmsCompletionListener();
+
+            try {
+                producer.send(message, listener);
+            } catch (Throwable error) {
+                fail("Send should not fail for async.");
+            }
+
+            try {
+                message.setJMSCorrelationID("test");
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSCorrelationIDAsBytes(new byte[]{});
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSDeliveryMode(0);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSDestination(queue);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSExpiration(0);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSMessageID(queueName);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSPriority(0);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSRedelivered(false);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSReplyTo(queue);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSTimestamp(0);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSType(queueName);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setStringProperty("test", "test");
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+
+            connection.close();
+
+            testPeer.waitForAllHandlersToComplete(1000);
+        }
+    }
+
+    private class TestJmsCompletionListener implements CompletionListener {
+
+        @Override
+        public void onCompletion(Message message) {
+        }
+
+        @Override
+        public void onException(Message message, Exception exception) {
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/ObjectMessageIntegrationTest.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/ObjectMessageIntegrationTest.java b/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/ObjectMessageIntegrationTest.java
index e382324..43a6640 100644
--- a/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/ObjectMessageIntegrationTest.java
+++ b/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/ObjectMessageIntegrationTest.java
@@ -35,16 +35,19 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.UUID;
 
+import javax.jms.CompletionListener;
 import javax.jms.Connection;
 import javax.jms.JMSException;
 import javax.jms.Message;
 import javax.jms.MessageConsumer;
 import javax.jms.MessageFormatException;
+import javax.jms.MessageNotWriteableException;
 import javax.jms.MessageProducer;
 import javax.jms.ObjectMessage;
 import javax.jms.Queue;
 import javax.jms.Session;
 
+import org.apache.qpid.jms.JmsConnection;
 import org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport;
 import org.apache.qpid.jms.test.QpidJmsTestCase;
 import org.apache.qpid.jms.test.testpeer.TestAmqpPeer;
@@ -532,4 +535,105 @@ public class ObjectMessageIntegrationTest extends QpidJmsTestCase {
             testPeer.waitForAllHandlersToComplete(3000);
         }
     }
+
+    @Test(timeout = 20000)
+    public void testAsyncSendMarksObjectMessageReadOnly() throws Exception {
+        try(TestAmqpPeer testPeer = new TestAmqpPeer();) {
+            JmsConnection connection = (JmsConnection) testFixture.establishConnecton(testPeer);
+            connection.setSendTimeout(15000);
+
+            testPeer.expectBegin();
+
+            Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
+            String queueName = "myQueue";
+            Queue queue = session.createQueue(queueName);
+
+            ObjectMessage message = session.createObjectMessage();
+            TransferPayloadCompositeMatcher messageMatcher = new TransferPayloadCompositeMatcher();
+
+            // Expect the producer to attach and grant it some credit, it should send
+            // a transfer which we will not send any response so that we can check that
+            // the inflight message is read-only
+            testPeer.expectSenderAttach();
+            testPeer.expectTransferButDoNotRespond(messageMatcher);
+            testPeer.expectClose();
+
+            MessageProducer producer = session.createProducer(queue);
+            TestJmsCompletionListener listener = new TestJmsCompletionListener();
+
+            try {
+                producer.send(message, listener);
+            } catch (Throwable error) {
+                fail("Send should not fail for async.");
+            }
+
+            try {
+                message.setJMSCorrelationID("test");
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSCorrelationIDAsBytes(new byte[]{});
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSDeliveryMode(0);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSDestination(queue);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSExpiration(0);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSMessageID(queueName);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSPriority(0);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSRedelivered(false);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSReplyTo(queue);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSTimestamp(0);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSType(queueName);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setStringProperty("test", "test");
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setObject("test");
+                fail("Message should not be writable after a send.");
+            } catch (MessageNotWriteableException mnwe) {}
+
+            connection.close();
+
+            testPeer.waitForAllHandlersToComplete(1000);
+        }
+    }
+
+    private class TestJmsCompletionListener implements CompletionListener {
+
+        @Override
+        public void onCompletion(Message message) {
+        }
+
+        @Override
+        public void onException(Message message, Exception exception) {
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/ProducerIntegrationTest.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/ProducerIntegrationTest.java b/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/ProducerIntegrationTest.java
index ea7654a..b5d6dd2 100644
--- a/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/ProducerIntegrationTest.java
+++ b/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/ProducerIntegrationTest.java
@@ -199,8 +199,7 @@ public class ProducerIntegrationTest extends QpidJmsTestCase {
             MessageProducer producer = session.createProducer(queue);
 
             // Create and transfer a new message
-            MessageHeaderSectionMatcher headersMatcher = new MessageHeaderSectionMatcher(true)
-                    .withDurable(equalTo(true));
+            MessageHeaderSectionMatcher headersMatcher = new MessageHeaderSectionMatcher(true).withDurable(equalTo(true));
             MessageAnnotationsSectionMatcher msgAnnotationsMatcher = new MessageAnnotationsSectionMatcher(true);
             TransferPayloadCompositeMatcher messageMatcher = new TransferPayloadCompositeMatcher();
             messageMatcher.setHeadersMatcher(headersMatcher);
@@ -233,8 +232,7 @@ public class ProducerIntegrationTest extends QpidJmsTestCase {
             // Create and transfer a new message, explicitly setting the deliveryMode on the
             // message (which applications shouldn't) to NON_PERSISTENT and sending it to check
             // that the producer ignores this value and sends the message as PERSISTENT(/durable)
-            MessageHeaderSectionMatcher headersMatcher = new MessageHeaderSectionMatcher(true)
-                    .withDurable(equalTo(true));
+            MessageHeaderSectionMatcher headersMatcher = new MessageHeaderSectionMatcher(true).withDurable(equalTo(true));
             MessageAnnotationsSectionMatcher msgAnnotationsMatcher = new MessageAnnotationsSectionMatcher(true);
             TransferPayloadCompositeMatcher messageMatcher = new TransferPayloadCompositeMatcher();
             messageMatcher.setHeadersMatcher(headersMatcher);
@@ -320,11 +318,9 @@ public class ProducerIntegrationTest extends QpidJmsTestCase {
             Matcher<Date> inRange = both(greaterThanOrEqualTo(creationLower)).and(lessThanOrEqualTo(creationUpper));
 
             String text = "myMessage";
-            MessageHeaderSectionMatcher headersMatcher = new MessageHeaderSectionMatcher(true)
-                    .withDurable(equalTo(true));
+            MessageHeaderSectionMatcher headersMatcher = new MessageHeaderSectionMatcher(true).withDurable(equalTo(true));
             MessageAnnotationsSectionMatcher msgAnnotationsMatcher = new MessageAnnotationsSectionMatcher(true);
-            MessagePropertiesSectionMatcher propsMatcher = new MessagePropertiesSectionMatcher(true)
-                    .withCreationTime(inRange);
+            MessagePropertiesSectionMatcher propsMatcher = new MessagePropertiesSectionMatcher(true).withCreationTime(inRange);
             TransferPayloadCompositeMatcher messageMatcher = new TransferPayloadCompositeMatcher();
             messageMatcher.setHeadersMatcher(headersMatcher);
             messageMatcher.setMessageAnnotationsMatcher(msgAnnotationsMatcher);
@@ -416,8 +412,7 @@ public class ProducerIntegrationTest extends QpidJmsTestCase {
             headersMatcher.withDurable(equalTo(true));
             headersMatcher.withTtl(equalTo(UnsignedInteger.valueOf(ttl)));
             MessageAnnotationsSectionMatcher msgAnnotationsMatcher = new MessageAnnotationsSectionMatcher(true);
-            MessagePropertiesSectionMatcher propsMatcher = new MessagePropertiesSectionMatcher(true)
-                    .withAbsoluteExpiryTime(inRange);
+            MessagePropertiesSectionMatcher propsMatcher = new MessagePropertiesSectionMatcher(true).withAbsoluteExpiryTime(inRange);
             TransferPayloadCompositeMatcher messageMatcher = new TransferPayloadCompositeMatcher();
             messageMatcher.setHeadersMatcher(headersMatcher);
             messageMatcher.setMessageAnnotationsMatcher(msgAnnotationsMatcher);
@@ -475,8 +470,7 @@ public class ProducerIntegrationTest extends QpidJmsTestCase {
                 headersMatcher.withTtl(equalTo(UnsignedInteger.valueOf(amqpTtl)));
             }
             MessageAnnotationsSectionMatcher msgAnnotationsMatcher = new MessageAnnotationsSectionMatcher(true);
-            MessagePropertiesSectionMatcher propsMatcher = new MessagePropertiesSectionMatcher(true)
-                    .withAbsoluteExpiryTime(inRange);
+            MessagePropertiesSectionMatcher propsMatcher = new MessagePropertiesSectionMatcher(true).withAbsoluteExpiryTime(inRange);
             TransferPayloadCompositeMatcher messageMatcher = new TransferPayloadCompositeMatcher();
             messageMatcher.setHeadersMatcher(headersMatcher);
             messageMatcher.setMessageAnnotationsMatcher(msgAnnotationsMatcher);
@@ -1031,8 +1025,6 @@ public class ProducerIntegrationTest extends QpidJmsTestCase {
             Queue queue = session.createQueue("myQueue");
             final MessageProducer producer = session.createProducer(queue);
 
-            Message message = session.createTextMessage("content");
-
             final int MSG_COUNT = 3;
 
             for (int i = 0; i < MSG_COUNT; ++i) {
@@ -1044,6 +1036,7 @@ public class ProducerIntegrationTest extends QpidJmsTestCase {
             TestJmsCompletionListener listener = new TestJmsCompletionListener(MSG_COUNT);
             try {
                 for (int i = 0; i < MSG_COUNT; ++i) {
+                    Message message = session.createTextMessage("content");
                     producer.send(message, listener);
                 }
             } catch (JMSException e) {

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/SessionIntegrationTest.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/SessionIntegrationTest.java b/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/SessionIntegrationTest.java
index e352b7d..bbb2ec7 100644
--- a/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/SessionIntegrationTest.java
+++ b/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/SessionIntegrationTest.java
@@ -1724,8 +1724,6 @@ public class SessionIntegrationTest extends QpidJmsTestCase {
             Queue queue = session.createQueue("myQueue");
             final MessageProducer producer = session.createProducer(queue);
 
-            Message message = session.createTextMessage("content");
-
             final int MSG_COUNT = 3;
 
             for (int i = 0; i < MSG_COUNT; ++i) {
@@ -1737,6 +1735,7 @@ public class SessionIntegrationTest extends QpidJmsTestCase {
             TestJmsCompletionListener listener = new TestJmsCompletionListener(MSG_COUNT);
             try {
                 for (int i = 0; i < MSG_COUNT; ++i) {
+                    Message message = session.createTextMessage("content");
                     producer.send(message, listener);
                 }
             } catch (JMSException e) {

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/StreamMessageIntegrationTest.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/StreamMessageIntegrationTest.java b/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/StreamMessageIntegrationTest.java
index 507d34e..2687a37 100644
--- a/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/StreamMessageIntegrationTest.java
+++ b/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/StreamMessageIntegrationTest.java
@@ -29,15 +29,18 @@ import static org.junit.Assert.fail;
 import java.util.ArrayList;
 import java.util.List;
 
+import javax.jms.CompletionListener;
 import javax.jms.Connection;
 import javax.jms.Message;
 import javax.jms.MessageConsumer;
 import javax.jms.MessageFormatException;
+import javax.jms.MessageNotWriteableException;
 import javax.jms.MessageProducer;
 import javax.jms.Queue;
 import javax.jms.Session;
 import javax.jms.StreamMessage;
 
+import org.apache.qpid.jms.JmsConnection;
 import org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport;
 import org.apache.qpid.jms.test.QpidJmsTestCase;
 import org.apache.qpid.jms.test.testpeer.TestAmqpPeer;
@@ -197,7 +200,7 @@ public class StreamMessageIntegrationTest extends QpidJmsTestCase {
             String myString = "myString";
             byte[] myBytes = "myBytes".getBytes();
 
-            //Prepare a MapMessage to send to the test peer to send
+            //Prepare a StreamMessage to send to the test peer to send
             StreamMessage streamMessage = session.createStreamMessage();
 
             streamMessage.writeBoolean(myBool);
@@ -268,4 +271,153 @@ public class StreamMessageIntegrationTest extends QpidJmsTestCase {
             testPeer.waitForAllHandlersToComplete(3000);
         }
     }
+
+    @Test(timeout = 20000)
+    public void testSentStreamMessageIsReadOnly() throws Exception {
+        try (TestAmqpPeer testPeer = new TestAmqpPeer();) {
+            Connection connection = testFixture.establishConnecton(testPeer);
+            testPeer.expectBegin();
+            testPeer.expectSenderAttach();
+
+            Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
+            Queue queue = session.createQueue("myQueue");
+            MessageProducer producer = session.createProducer(queue);
+
+            String myString = "myString";
+
+            // Prepare a StreamMessage to send to the test peer to send
+            StreamMessage streamMessage = session.createStreamMessage();
+
+            streamMessage.writeString(myString);
+
+            // prepare a matcher for the test peer to use to receive and verify the message
+            List<Object> list = new ArrayList<Object>();
+            list.add(myString);
+
+            MessageHeaderSectionMatcher headersMatcher = new MessageHeaderSectionMatcher(true).withDurable(equalTo(true));
+            MessageAnnotationsSectionMatcher msgAnnotationsMatcher = new MessageAnnotationsSectionMatcher(true);
+            msgAnnotationsMatcher.withEntry(Symbol.valueOf(AmqpMessageSupport.JMS_MSG_TYPE), equalTo(AmqpMessageSupport.JMS_STREAM_MESSAGE));
+            MessagePropertiesSectionMatcher propertiesMatcher = new MessagePropertiesSectionMatcher(true);
+            TransferPayloadCompositeMatcher messageMatcher = new TransferPayloadCompositeMatcher();
+            messageMatcher.setHeadersMatcher(headersMatcher);
+            messageMatcher.setMessageAnnotationsMatcher(msgAnnotationsMatcher);
+            messageMatcher.setPropertiesMatcher(propertiesMatcher);
+            messageMatcher.setMessageContentMatcher(new EncodedAmqpSequenceMatcher(list));
+
+            testPeer.expectTransfer(messageMatcher);
+            testPeer.expectClose();
+
+            producer.send(streamMessage);
+
+            try {
+                streamMessage.writeString(myString);
+                fail("Message should not be writable after a send.");
+            } catch (MessageNotWriteableException mnwe) {}
+
+            connection.close();
+
+            testPeer.waitForAllHandlersToComplete(3000);
+        }
+    }
+
+    @Test(timeout = 20000)
+    public void testAsyncSendMarksStreamMessageReadOnly() throws Exception {
+        try(TestAmqpPeer testPeer = new TestAmqpPeer();) {
+            JmsConnection connection = (JmsConnection) testFixture.establishConnecton(testPeer);
+            connection.setSendTimeout(15000);
+
+            testPeer.expectBegin();
+
+            Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
+            String queueName = "myQueue";
+            Queue queue = session.createQueue(queueName);
+
+            StreamMessage message = session.createStreamMessage();
+            TransferPayloadCompositeMatcher messageMatcher = new TransferPayloadCompositeMatcher();
+
+            // Expect the producer to attach and grant it some credit, it should send
+            // a transfer which we will not send any response so that we can check that
+            // the inflight message is read-only
+            testPeer.expectSenderAttach();
+            testPeer.expectTransferButDoNotRespond(messageMatcher);
+            testPeer.expectClose();
+
+            MessageProducer producer = session.createProducer(queue);
+            TestJmsCompletionListener listener = new TestJmsCompletionListener();
+
+            try {
+                producer.send(message, listener);
+            } catch (Throwable error) {
+                fail("Send should not fail for async.");
+            }
+
+            try {
+                message.setJMSCorrelationID("test");
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSCorrelationIDAsBytes(new byte[]{});
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSDeliveryMode(0);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSDestination(queue);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSExpiration(0);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSMessageID(queueName);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSPriority(0);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSRedelivered(false);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSReplyTo(queue);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSTimestamp(0);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSType(queueName);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setStringProperty("test", "test");
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.writeString("test");
+                fail("Message should not be writable after a send.");
+            } catch (MessageNotWriteableException mnwe) {}
+
+            connection.close();
+
+            testPeer.waitForAllHandlersToComplete(1000);
+        }
+    }
+
+    private class TestJmsCompletionListener implements CompletionListener {
+
+        @Override
+        public void onCompletion(Message message) {
+        }
+
+        @Override
+        public void onException(Message message, Exception exception) {
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/TextMessageIntegrationTest.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/TextMessageIntegrationTest.java b/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/TextMessageIntegrationTest.java
index 2869d03..5e593c4 100644
--- a/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/TextMessageIntegrationTest.java
+++ b/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/TextMessageIntegrationTest.java
@@ -28,16 +28,19 @@ import static org.junit.Assert.fail;
 
 import java.io.IOException;
 
+import javax.jms.CompletionListener;
 import javax.jms.Connection;
 import javax.jms.JMSException;
 import javax.jms.Message;
 import javax.jms.MessageConsumer;
 import javax.jms.MessageFormatException;
+import javax.jms.MessageNotWriteableException;
 import javax.jms.MessageProducer;
 import javax.jms.Queue;
 import javax.jms.Session;
 import javax.jms.TextMessage;
 
+import org.apache.qpid.jms.JmsConnection;
 import org.apache.qpid.jms.test.QpidJmsTestCase;
 import org.apache.qpid.jms.test.testpeer.TestAmqpPeer;
 import org.apache.qpid.jms.test.testpeer.describedtypes.sections.AmqpValueDescribedType;
@@ -319,4 +322,107 @@ public class TextMessageIntegrationTest extends QpidJmsTestCase {
             testPeer.waitForAllHandlersToComplete(3000);
         }
     }
+
+    @Test(timeout = 20000)
+    public void testAsyncSendMarksTextMessageReadOnly() throws Exception {
+        try(TestAmqpPeer testPeer = new TestAmqpPeer();) {
+            JmsConnection connection = (JmsConnection) testFixture.establishConnecton(testPeer);
+            connection.setSendTimeout(15000);
+
+            testPeer.expectBegin();
+
+            Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
+            String queueName = "myQueue";
+            Queue queue = session.createQueue(queueName);
+
+            TextMessage message = session.createTextMessage("text");
+            TransferPayloadCompositeMatcher messageMatcher = new TransferPayloadCompositeMatcher();
+
+            // Expect the producer to attach and grant it some credit, it should send
+            // a transfer which we will not send any response so that we can check that
+            // the inflight message is read-only
+            testPeer.expectSenderAttach();
+            testPeer.expectTransferButDoNotRespond(messageMatcher);
+            testPeer.expectClose();
+
+            MessageProducer producer = session.createProducer(queue);
+            TestJmsCompletionListener listener = new TestJmsCompletionListener();
+
+            try {
+                producer.send(message, listener);
+            } catch (Throwable error) {
+                fail("Send should not fail for async.");
+            }
+
+            try {
+                message.setJMSCorrelationID("test");
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSCorrelationIDAsBytes(new byte[]{});
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSDeliveryMode(0);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSDestination(queue);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSExpiration(0);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSMessageID(queueName);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSPriority(0);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSRedelivered(false);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSReplyTo(queue);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSTimestamp(0);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSType(queueName);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+
+            try {
+                message.setStringProperty("test", "test");
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+
+            try {
+                message.setText("test");
+                fail("Should not be able to set new body on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+
+            connection.close();
+
+            testPeer.waitForAllHandlersToComplete(1000);
+        }
+    }
+
+    private class TestJmsCompletionListener implements CompletionListener {
+
+        @Override
+        public void onCompletion(Message message) {
+        }
+
+        @Override
+        public void onException(Message message, Exception exception) {
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/test/java/org/apache/qpid/jms/message/JmsMessageTest.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/test/java/org/apache/qpid/jms/message/JmsMessageTest.java b/qpid-jms-client/src/test/java/org/apache/qpid/jms/message/JmsMessageTest.java
index 16cc9a7..7b05650 100644
--- a/qpid-jms-client/src/test/java/org/apache/qpid/jms/message/JmsMessageTest.java
+++ b/qpid-jms-client/src/test/java/org/apache/qpid/jms/message/JmsMessageTest.java
@@ -98,8 +98,7 @@ public class JmsMessageTest {
         assertFalse(msg.isReadOnlyBody());
         assertFalse(msg.isReadOnlyProperties());
         msg.onSend(0);
-        assertTrue(msg.isReadOnlyBody());
-        assertTrue(msg.isReadOnlyProperties());
+        assertTrue(msg.isReadOnly());
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/test/java/org/apache/qpid/jms/message/facade/test/JmsTestBytesMessageFacade.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/test/java/org/apache/qpid/jms/message/facade/test/JmsTestBytesMessageFacade.java b/qpid-jms-client/src/test/java/org/apache/qpid/jms/message/facade/test/JmsTestBytesMessageFacade.java
index 8b53651..94b8180 100644
--- a/qpid-jms-client/src/test/java/org/apache/qpid/jms/message/facade/test/JmsTestBytesMessageFacade.java
+++ b/qpid-jms-client/src/test/java/org/apache/qpid/jms/message/facade/test/JmsTestBytesMessageFacade.java
@@ -148,4 +148,11 @@ public final class JmsTestBytesMessageFacade extends JmsTestMessageFacade implem
 
         return result;
     }
+
+    @Override
+    public void onSend(long producerTtl) throws JMSException {
+        super.onSend(producerTtl);
+
+        reset();
+    }
 }

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/test/java/org/apache/qpid/jms/message/facade/test/JmsTestMessageFacade.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/test/java/org/apache/qpid/jms/message/facade/test/JmsTestMessageFacade.java b/qpid-jms-client/src/test/java/org/apache/qpid/jms/message/facade/test/JmsTestMessageFacade.java
index 9e182e5..fa06a68 100644
--- a/qpid-jms-client/src/test/java/org/apache/qpid/jms/message/facade/test/JmsTestMessageFacade.java
+++ b/qpid-jms-client/src/test/java/org/apache/qpid/jms/message/facade/test/JmsTestMessageFacade.java
@@ -362,4 +362,9 @@ public class JmsTestMessageFacade implements JmsMessageFacade {
     public boolean hasBody() {
         return false;
     }
+
+    @Override
+    public Object encodeMessage() {
+        return this;
+    }
 }

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpCodecTest.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpCodecTest.java b/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpCodecTest.java
new file mode 100644
index 0000000..2d7932b
--- /dev/null
+++ b/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpCodecTest.java
@@ -0,0 +1,732 @@
+/*
+ *
+ * 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.qpid.jms.provider.amqp.message;
+
+import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.encodeMessage;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+import org.apache.qpid.jms.message.JmsBytesMessage;
+import org.apache.qpid.jms.message.JmsMessage;
+import org.apache.qpid.jms.message.JmsObjectMessage;
+import org.apache.qpid.jms.message.JmsStreamMessage;
+import org.apache.qpid.jms.message.JmsTextMessage;
+import org.apache.qpid.jms.message.facade.JmsMessageFacade;
+import org.apache.qpid.jms.meta.JmsConsumerId;
+import org.apache.qpid.jms.meta.JmsConsumerInfo;
+import org.apache.qpid.jms.provider.amqp.AmqpConsumer;
+import org.apache.qpid.jms.test.QpidJmsTestCase;
+import org.apache.qpid.proton.Proton;
+import org.apache.qpid.proton.amqp.Binary;
+import org.apache.qpid.proton.amqp.Symbol;
+import org.apache.qpid.proton.amqp.messaging.AmqpSequence;
+import org.apache.qpid.proton.amqp.messaging.AmqpValue;
+import org.apache.qpid.proton.amqp.messaging.Data;
+import org.apache.qpid.proton.amqp.messaging.MessageAnnotations;
+import org.apache.qpid.proton.message.Message;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+public class AmqpCodecTest extends QpidJmsTestCase {
+    private AmqpConsumer mockConsumer;
+
+    @Before
+    @Override
+    public void setUp() throws Exception {
+        super.setUp();
+
+        JmsConsumerId consumerId = new JmsConsumerId("ID:MOCK:1", 1, 1);
+        mockConsumer = Mockito.mock(AmqpConsumer.class);
+        Mockito.when(mockConsumer.getResourceInfo()).thenReturn(new JmsConsumerInfo(consumerId));
+    }
+
+    // =============== With The Message Type Annotation =========
+    // ==========================================================
+
+    /**
+     * Test that a message with the {@value AmqpMessageSupport#JMS_MSG_TYPE}
+     * annotation set to  {@value AmqpMessageSupport#JMS_MESSAGE} is
+     * treated as a generic {@link JmsMessage} with {@link AmqpJmsMessageFacade}
+     *
+     * @throws Exception if an error occurs during the test.
+     */
+    @Test(expected = IOException.class)
+    public void testCreateMessageFromUnknownMessageTypeAnnotationValueThrows() throws Exception {
+        Message message = Proton.message();
+
+        Map<Symbol, Object> map = new HashMap<Symbol, Object>();
+        map.put(Symbol.valueOf(AmqpMessageSupport.JMS_MSG_TYPE), (byte) -1);
+
+        MessageAnnotations messageAnnotations = new MessageAnnotations(map);
+        message.setMessageAnnotations(messageAnnotations);
+
+        AmqpCodec.decodeMessage(mockConsumer, encodeMessage(message));
+    }
+
+    /**
+     * Test that a message with the {@value AmqpMessageSupport#JMS_MSG_TYPE}
+     * annotation set to  {@value AmqpMessageSupport#JMS_MESSAGE} is
+     * treated as a generic {@link JmsMessage} with {@link AmqpJmsMessageFacade}
+     *
+     * @throws Exception if an error occurs during the test.
+     */
+    @Test
+    public void testCreateGenericMessageFromMessageTypeAnnotation() throws Exception {
+        Message message = Proton.message();
+
+        Map<Symbol, Object> map = new HashMap<Symbol, Object>();
+        map.put(Symbol.valueOf(AmqpMessageSupport.JMS_MSG_TYPE), AmqpMessageSupport.JMS_MESSAGE);
+
+        MessageAnnotations messageAnnotations = new MessageAnnotations(map);
+        message.setMessageAnnotations(messageAnnotations);
+
+        JmsMessage jmsMessage = AmqpCodec.decodeMessage(mockConsumer, encodeMessage(message)).asJmsMessage();
+        assertNotNull("Message should not be null", jmsMessage);
+        assertEquals("Unexpected message class type", JmsMessage.class, jmsMessage.getClass());
+
+        JmsMessageFacade facade = jmsMessage.getFacade();
+        assertNotNull("Facade should not be null", facade);
+        assertEquals("Unexpected facade class type", AmqpJmsMessageFacade.class, facade.getClass());
+    }
+
+    /**
+     * Test that a message with the {@value AmqpMessageSupport#JMS_MSG_TYPE}
+     * annotation set to  {@value AmqpMessageSupport#JMS_BYTES_MESSAGE} is
+     * treated as a {@link JmsBytesMessage} with {@link AmqpJmsBytesMessageFacade}
+     *
+     * @throws Exception if an error occurs during the test.
+     */
+    @Test
+    public void testCreateBytesMessageFromMessageTypeAnnotation() throws Exception {
+        Message message = Proton.message();
+
+        Map<Symbol, Object> map = new HashMap<Symbol, Object>();
+        map.put(Symbol.valueOf(AmqpMessageSupport.JMS_MSG_TYPE), AmqpMessageSupport.JMS_BYTES_MESSAGE);
+
+        MessageAnnotations messageAnnotations = new MessageAnnotations(map);
+        message.setMessageAnnotations(messageAnnotations);
+
+        JmsMessage jmsMessage = AmqpCodec.decodeMessage(mockConsumer, encodeMessage(message)).asJmsMessage();
+        assertNotNull("Message should not be null", jmsMessage);
+        assertEquals("Unexpected message class type", JmsBytesMessage.class, jmsMessage.getClass());
+
+        JmsMessageFacade facade = jmsMessage.getFacade();
+        assertNotNull("Facade should not be null", facade);
+        assertEquals("Unexpected facade class type", AmqpJmsBytesMessageFacade.class, facade.getClass());
+    }
+
+    /**
+     * Test that a message with the {@value AmqpMessageSupport#JMS_MSG_TYPE}
+     * annotation set to  {@value AmqpMessageSupport#JMS_BYTES_MESSAGE} is
+     * treated as a {@link JmsTextMessage} with {@link AmqpJmsTextMessageFacade}
+     *
+     * @throws Exception if an error occurs during the test.
+     */
+    @Test
+    public void testCreateTextMessageFromMessageTypeAnnotation() throws Exception {
+        Message message = Proton.message();
+
+        Map<Symbol, Object> map = new HashMap<Symbol, Object>();
+        map.put(Symbol.valueOf(AmqpMessageSupport.JMS_MSG_TYPE), AmqpMessageSupport.JMS_TEXT_MESSAGE);
+
+        MessageAnnotations messageAnnotations = new MessageAnnotations(map);
+        message.setMessageAnnotations(messageAnnotations);
+
+        JmsMessage jmsMessage = AmqpCodec.decodeMessage(mockConsumer, encodeMessage(message)).asJmsMessage();
+        assertNotNull("Message should not be null", jmsMessage);
+        assertEquals("Unexpected message class type", JmsTextMessage.class, jmsMessage.getClass());
+
+        JmsMessageFacade facade = jmsMessage.getFacade();
+        assertNotNull("Facade should not be null", facade);
+        assertEquals("Unexpected facade class type", AmqpJmsTextMessageFacade.class, facade.getClass());
+    }
+
+    /**
+     * Test that a message with the {@value AmqpMessageSupport#JMS_MSG_TYPE}
+     * annotation set to  {@value AmqpMessageSupport#JMS_OBJECT_MESSAGE} and
+     * content-type set to {@value AmqpMessageSupport#OCTET_STREAM_CONTENT_TYPE} is
+     * treated as a {@link JmsObjectMessage} with {@link AmqpJmsObjectMessageFacade}
+     * containing a {@link AmqpSerializedObjectDelegate}.
+     *
+     * @throws Exception if an error occurs during the test.
+     */
+    @Test
+    public void testCreateObjectMessageFromMessageTypeAnnotation() throws Exception {
+        createObjectMessageFromMessageTypeAnnotationTestImpl(true);
+    }
+
+    /**
+     * Test that a message with the {@value AmqpMessageSupport#JMS_MSG_TYPE}
+     * annotation set to  {@value AmqpMessageSupport#JMS_OBJECT_MESSAGE} and
+     * content-type not set is treated as a {@link JmsObjectMessage} with
+     * {@link AmqpJmsObjectMessageFacade} containing a {@link AmqpTypedObjectDelegate}.
+     *
+     * @throws Exception if an error occurs during the test.
+     */
+    @Test
+    public void testCreateObjectMessageFromMessageTypeAnnotationAnd() throws Exception {
+        createObjectMessageFromMessageTypeAnnotationTestImpl(false);
+    }
+
+    private void createObjectMessageFromMessageTypeAnnotationTestImpl(boolean setJavaSerializedContentType) throws Exception {
+        Message message = Proton.message();
+
+        Map<Symbol, Object> map = new HashMap<Symbol, Object>();
+        map.put(Symbol.valueOf(AmqpMessageSupport.JMS_MSG_TYPE), AmqpMessageSupport.JMS_OBJECT_MESSAGE);
+
+        MessageAnnotations messageAnnotations = new MessageAnnotations(map);
+        message.setMessageAnnotations(messageAnnotations);
+
+        if (setJavaSerializedContentType) {
+            message.setContentType(AmqpMessageSupport.SERIALIZED_JAVA_OBJECT_CONTENT_TYPE);
+        }
+
+        JmsMessage jmsMessage = AmqpCodec.decodeMessage(mockConsumer, encodeMessage(message)).asJmsMessage();
+        assertNotNull("Message should not be null", jmsMessage);
+        assertEquals("Unexpected message class type", JmsObjectMessage.class, jmsMessage.getClass());
+
+        JmsMessageFacade facade = jmsMessage.getFacade();
+        assertNotNull("Facade should not be null", facade);
+        assertEquals("Unexpected facade class type", AmqpJmsObjectMessageFacade.class, facade.getClass());
+
+        AmqpObjectTypeDelegate delegate = ((AmqpJmsObjectMessageFacade) facade).getDelegate();
+        if (setJavaSerializedContentType) {
+            assertTrue("Unexpected delegate type: " + delegate, delegate instanceof AmqpSerializedObjectDelegate);
+        } else {
+            assertTrue("Unexpected delegate type: " + delegate, delegate instanceof AmqpTypedObjectDelegate);
+        }
+    }
+
+    /**
+     * Test that a message with the {@value AmqpMessageSupport#JMS_MSG_TYPE}
+     * annotation set to  {@value AmqpMessageSupport#JMS_STREAM_MESSAGE} is
+     * treated as a {@link JmsStreamMessage} with {@link AmqpJmsStreamMessageFacade}
+     *
+     * @throws Exception if an error occurs during the test.
+     */
+    @Test
+    public void testCreateStreamMessageFromMessageTypeAnnotation() throws Exception {
+
+        Message message = Proton.message();
+
+        Map<Symbol, Object> map = new HashMap<Symbol, Object>();
+        map.put(Symbol.valueOf(AmqpMessageSupport.JMS_MSG_TYPE), AmqpMessageSupport.JMS_STREAM_MESSAGE);
+
+        MessageAnnotations messageAnnotations = new MessageAnnotations(map);
+        message.setMessageAnnotations(messageAnnotations);
+
+        JmsMessage jmsMessage = AmqpCodec.decodeMessage(mockConsumer, encodeMessage(message)).asJmsMessage();
+        assertNotNull("Message should not be null", jmsMessage);
+        assertEquals("Unexpected message class type", JmsStreamMessage.class, jmsMessage.getClass());
+
+        JmsMessageFacade facade = jmsMessage.getFacade();
+        assertNotNull("Facade should not be null", facade);
+        assertEquals("Unexpected facade class type", AmqpJmsStreamMessageFacade.class, facade.getClass());
+    }
+
+    // =============== Without The Message Type Annotation =========
+    // =============================================================
+
+    // --------- No Body Section ---------
+
+    /**
+     * Test that a message with no body section, but with the content type set to
+     * {@value AmqpMessageSupport#OCTET_STREAM_CONTENT_TYPE} results in a BytesMessage
+     * when not otherwise annotated to indicate the type of JMS message it is.
+     *
+     * @throws Exception if an error occurs during the test.
+     */
+    @Test
+    public void testCreateBytesMessageFromNoBodySectionAndContentType() throws Exception {
+        Message message = Proton.message();
+        message.setContentType(AmqpMessageSupport.OCTET_STREAM_CONTENT_TYPE);
+
+        JmsMessage jmsMessage = AmqpCodec.decodeMessage(mockConsumer, encodeMessage(message)).asJmsMessage();
+        assertNotNull("Message should not be null", jmsMessage);
+        assertEquals("Unexpected message class type", JmsBytesMessage.class, jmsMessage.getClass());
+
+        JmsMessageFacade facade = jmsMessage.getFacade();
+        assertNotNull("Facade should not be null", facade);
+        assertEquals("Unexpected facade class type", AmqpJmsBytesMessageFacade.class, facade.getClass());
+    }
+
+    /**
+     * Test that a message with no body section, and no content-type results in a BytesMessage
+     * when not otherwise annotated to indicate the type of JMS message it is.
+     *
+     * @throws Exception if an error occurs during the test.
+     */
+    @Test
+    public void testCreateBytesMessageFromNoBodySectionAndNoContentType() throws Exception {
+        Message message = Proton.message();
+
+        assertNull(message.getContentType());
+
+        JmsMessage jmsMessage = AmqpCodec.decodeMessage(mockConsumer, encodeMessage(message)).asJmsMessage();
+        assertNotNull("Message should not be null", jmsMessage);
+        assertEquals("Unexpected message class type", JmsBytesMessage.class, jmsMessage.getClass());
+
+        JmsMessageFacade facade = jmsMessage.getFacade();
+        assertNotNull("Facade should not be null", facade);
+        assertEquals("Unexpected facade class type", AmqpJmsBytesMessageFacade.class, facade.getClass());
+    }
+
+    /**
+    * Test that a message with no body section, but with the content type set to
+    * {@value AmqpMessageSupport#SERIALIZED_JAVA_OBJECT_CONTENT_TYPE} results in an ObjectMessage
+    * when not otherwise annotated to indicate the type of JMS message it is.
+     *
+     * @throws Exception if an error occurs during the test.
+    */
+    @Test
+    public void testCreateObjectMessageFromNoBodySectionAndContentType() throws Exception {
+        Message message = Proton.message();
+        message.setContentType(AmqpMessageSupport.SERIALIZED_JAVA_OBJECT_CONTENT_TYPE);
+
+        JmsMessage jmsMessage = AmqpCodec.decodeMessage(mockConsumer, encodeMessage(message)).asJmsMessage();
+        assertNotNull("Message should not be null", jmsMessage);
+        assertEquals("Unexpected message class type", JmsObjectMessage.class, jmsMessage.getClass());
+
+        JmsMessageFacade facade = jmsMessage.getFacade();
+        assertNotNull("Facade should not be null", facade);
+        assertEquals("Unexpected facade class type", AmqpJmsObjectMessageFacade.class, facade.getClass());
+
+        AmqpObjectTypeDelegate delegate = ((AmqpJmsObjectMessageFacade) facade).getDelegate();
+        assertTrue("Unexpected delegate type: " + delegate, delegate instanceof AmqpSerializedObjectDelegate);
+    }
+
+    @Test
+    public void testCreateTextMessageFromNoBodySectionAndContentType() throws Exception {
+        Message message = Proton.message();
+        message.setContentType("text/plain");
+
+        JmsMessage jmsMessage = AmqpCodec.decodeMessage(mockConsumer, encodeMessage(message)).asJmsMessage();
+        assertNotNull("Message should not be null", jmsMessage);
+        assertEquals("Unexpected message class type", JmsTextMessage.class, jmsMessage.getClass());
+
+        JmsMessageFacade facade = jmsMessage.getFacade();
+        assertNotNull("Facade should not be null", facade);
+        assertEquals("Unexpected facade class type", AmqpJmsTextMessageFacade.class, facade.getClass());
+    }
+
+    /**
+     * Test that a message with no body section, and with the content type set to
+     * an unknown value results in a plain Message when not otherwise annotated to
+     * indicate the type of JMS message it is.
+     *
+     * @throws Exception if an error occurs during the test.
+     */
+    public void testCreateGenericMessageFromNoBodySectionAndUnknownContentType() throws Exception {
+        Message message = Proton.message();
+        message.setContentType("unknown-content-type");
+
+        JmsMessage jmsMessage = AmqpCodec.decodeMessage(mockConsumer, encodeMessage(message)).asJmsMessage();
+        assertNotNull("Message should not be null", jmsMessage);
+        assertEquals("Unexpected message class type", JmsMessage.class, jmsMessage.getClass());
+
+        JmsMessageFacade facade = jmsMessage.getFacade();
+        assertNotNull("Facade should not be null", facade);
+        assertEquals("Unexpected facade class type", AmqpJmsMessageFacade.class, facade.getClass());
+    }
+
+    // --------- Data Body Section ---------
+
+    /**
+     * Test that a data body containing nothing, but with the content type set to
+     * {@value AmqpMessageSupport#OCTET_STREAM_CONTENT_TYPE} results in a BytesMessage when not
+     * otherwise annotated to indicate the type of JMS message it is.
+     *
+     * @throws Exception if an error occurs during the test.
+     */
+    @Test
+    public void testCreateBytesMessageFromDataWithEmptyBinaryAndContentType() throws Exception {
+        Message message = Proton.message();
+        Binary binary = new Binary(new byte[0]);
+        message.setBody(new Data(binary));
+        message.setContentType(AmqpMessageSupport.OCTET_STREAM_CONTENT_TYPE);
+
+        JmsMessage jmsMessage = AmqpCodec.decodeMessage(mockConsumer, encodeMessage(message)).asJmsMessage();
+        assertNotNull("Message should not be null", jmsMessage);
+        assertEquals("Unexpected message class type", JmsBytesMessage.class, jmsMessage.getClass());
+
+        JmsMessageFacade facade = jmsMessage.getFacade();
+        assertNotNull("Facade should not be null", facade);
+        assertEquals("Unexpected facade class type", AmqpJmsBytesMessageFacade.class, facade.getClass());
+    }
+
+    /**
+     * Test that a message with an empty data body section, and with the content type
+     * set to an unknown value results in a BytesMessage when not otherwise annotated
+     * to indicate the type of JMS message it is.
+     *
+     * @throws Exception if an error occurs during the test.
+     */
+    public void testCreateBytesMessageFromDataWithUnknownContentType() throws Exception {
+        Message message = Proton.message();
+        Binary binary = new Binary(new byte[0]);
+        message.setBody(new Data(binary));
+        message.setContentType("unknown-content-type");
+
+        JmsMessage jmsMessage = AmqpCodec.decodeMessage(mockConsumer, encodeMessage(message)).asJmsMessage();
+        assertNotNull("Message should not be null", jmsMessage);
+        assertEquals("Unexpected message class type", JmsBytesMessage.class, jmsMessage.getClass());
+
+        JmsMessageFacade facade = jmsMessage.getFacade();
+        assertNotNull("Facade should not be null", facade);
+        assertEquals("Unexpected facade class type", AmqpJmsBytesMessageFacade.class, facade.getClass());
+    }
+
+    /**
+     * Test that a receiving a data body containing nothing and no content type being set
+     * results in a BytesMessage when not otherwise annotated to indicate the type of
+     * JMS message it is.
+     *
+     * @throws Exception if an error occurs during the test.
+     */
+    @Test
+    public void testCreateBytesMessageFromDataWithEmptyBinaryAndNoContentType() throws Exception {
+        Message message = Proton.message();
+        Binary binary = new Binary(new byte[0]);
+        message.setBody(new Data(binary));
+
+        assertNull(message.getContentType());
+
+        JmsMessage jmsMessage = AmqpCodec.decodeMessage(mockConsumer, encodeMessage(message)).asJmsMessage();
+        assertNotNull("Message should not be null", jmsMessage);
+        assertEquals("Unexpected message class type", JmsBytesMessage.class, jmsMessage.getClass());
+
+        JmsMessageFacade facade = jmsMessage.getFacade();
+        assertNotNull("Facade should not be null", facade);
+        assertEquals("Unexpected facade class type", AmqpJmsBytesMessageFacade.class, facade.getClass());
+    }
+
+    /**
+     * Test that receiving a data body containing nothing, but with the content type set to
+     * {@value AmqpMessageSupport#SERIALIZED_JAVA_OBJECT_CONTENT_TYPE} results in an ObjectMessage
+     * when not otherwise annotated to indicate the type of JMS message it is.
+     *
+     * @throws Exception if an error occurs during the test.
+     */
+    @Test
+    public void testCreateObjectMessageFromDataWithContentTypeAndEmptyBinary() throws Exception {
+        Message message = Proton.message();
+        Binary binary = new Binary(new byte[0]);
+        message.setBody(new Data(binary));
+        message.setContentType(AmqpMessageSupport.SERIALIZED_JAVA_OBJECT_CONTENT_TYPE);
+
+        JmsMessage jmsMessage = AmqpCodec.decodeMessage(mockConsumer, encodeMessage(message)).asJmsMessage();
+        assertNotNull("Message should not be null", jmsMessage);
+        assertEquals("Unexpected message class type", JmsObjectMessage.class, jmsMessage.getClass());
+
+        JmsMessageFacade facade = jmsMessage.getFacade();
+        assertNotNull("Facade should not be null", facade);
+        assertEquals("Unexpected facade class type", AmqpJmsObjectMessageFacade.class, facade.getClass());
+
+        AmqpObjectTypeDelegate delegate = ((AmqpJmsObjectMessageFacade) facade).getDelegate();
+        assertTrue("Unexpected delegate type: " + delegate, delegate instanceof AmqpSerializedObjectDelegate);
+    }
+
+    /**
+     * Test that receiving a Data body section with the content type set to
+     * 'text/plain' results in a TextMessage when not otherwise annotated to
+     * indicate the type of JMS message it is.
+     *
+     * @throws Exception if an error occurs during the test.
+     */
+    @Test
+    public void testCreateTextMessageFromDataWithContentTypeTextPlain() throws Exception {
+        doCreateTextMessageFromDataWithContentTypeTestImpl("text/plain;charset=iso-8859-1", StandardCharsets.ISO_8859_1);
+        doCreateTextMessageFromDataWithContentTypeTestImpl("text/plain;charset=us-ascii", StandardCharsets.US_ASCII);
+        doCreateTextMessageFromDataWithContentTypeTestImpl("text/plain;charset=utf-8", StandardCharsets.UTF_8);
+        doCreateTextMessageFromDataWithContentTypeTestImpl("text/plain", StandardCharsets.UTF_8);
+    }
+
+    @Test
+    public void testCreateTextMessageFromDataWithContentTypeTextJson() throws Exception {
+        doCreateTextMessageFromDataWithContentTypeTestImpl("text/json;charset=iso-8859-1", StandardCharsets.ISO_8859_1);
+        doCreateTextMessageFromDataWithContentTypeTestImpl("text/json;charset=us-ascii", StandardCharsets.US_ASCII);
+        doCreateTextMessageFromDataWithContentTypeTestImpl("text/json;charset=utf-8", StandardCharsets.UTF_8);
+        doCreateTextMessageFromDataWithContentTypeTestImpl("text/json", StandardCharsets.UTF_8);
+    }
+
+    @Test
+    public void testCreateTextMessageFromDataWithContentTypeTextHtml() throws Exception {
+        doCreateTextMessageFromDataWithContentTypeTestImpl("text/html;charset=iso-8859-1", StandardCharsets.ISO_8859_1);
+        doCreateTextMessageFromDataWithContentTypeTestImpl("text/html;charset=us-ascii", StandardCharsets.US_ASCII);
+        doCreateTextMessageFromDataWithContentTypeTestImpl("text/html;charset=utf-8", StandardCharsets.UTF_8);
+        doCreateTextMessageFromDataWithContentTypeTestImpl("text/html", StandardCharsets.UTF_8);
+    }
+
+    @Test
+    public void testCreateTextMessageFromDataWithContentTypeTextFoo() throws Exception {
+        doCreateTextMessageFromDataWithContentTypeTestImpl("text/foo;charset=iso-8859-1", StandardCharsets.ISO_8859_1);
+        doCreateTextMessageFromDataWithContentTypeTestImpl("text/foo;charset=us-ascii", StandardCharsets.US_ASCII);
+        doCreateTextMessageFromDataWithContentTypeTestImpl("text/foo;charset=utf-8", StandardCharsets.UTF_8);
+        doCreateTextMessageFromDataWithContentTypeTestImpl("text/foo", StandardCharsets.UTF_8);
+    }
+
+    @Test
+    public void testCreateTextMessageFromDataWithContentTypeApplicationJson() throws Exception {
+        doCreateTextMessageFromDataWithContentTypeTestImpl("application/json;charset=iso-8859-1", StandardCharsets.ISO_8859_1);
+        doCreateTextMessageFromDataWithContentTypeTestImpl("application/json;charset=us-ascii", StandardCharsets.US_ASCII);
+        doCreateTextMessageFromDataWithContentTypeTestImpl("application/json;charset=utf-8", StandardCharsets.UTF_8);
+        doCreateTextMessageFromDataWithContentTypeTestImpl("application/json", StandardCharsets.UTF_8);
+    }
+
+    @Test
+    public void testCreateTextMessageFromDataWithContentTypeApplicationJsonVariant() throws Exception {
+        doCreateTextMessageFromDataWithContentTypeTestImpl("application/something+json;charset=iso-8859-1", StandardCharsets.ISO_8859_1);
+        doCreateTextMessageFromDataWithContentTypeTestImpl("application/something+json;charset=us-ascii", StandardCharsets.US_ASCII);
+        doCreateTextMessageFromDataWithContentTypeTestImpl("application/something+json;charset=utf-8", StandardCharsets.UTF_8);
+        doCreateTextMessageFromDataWithContentTypeTestImpl("application/something+json", StandardCharsets.UTF_8);
+    }
+
+    @Test
+    public void testCreateTextMessageFromDataWithContentTypeApplicationJavascript() throws Exception {
+        doCreateTextMessageFromDataWithContentTypeTestImpl("application/javascript;charset=iso-8859-1", StandardCharsets.ISO_8859_1);
+        doCreateTextMessageFromDataWithContentTypeTestImpl("application/javascript;charset=us-ascii", StandardCharsets.US_ASCII);
+        doCreateTextMessageFromDataWithContentTypeTestImpl("application/javascript;charset=utf-8", StandardCharsets.UTF_8);
+        doCreateTextMessageFromDataWithContentTypeTestImpl("application/javascript", StandardCharsets.UTF_8);
+    }
+
+    @Test
+    public void testCreateTextMessageFromDataWithContentTypeApplicationEcmascript() throws Exception {
+        doCreateTextMessageFromDataWithContentTypeTestImpl("application/ecmascript;charset=iso-8859-1", StandardCharsets.ISO_8859_1);
+        doCreateTextMessageFromDataWithContentTypeTestImpl("application/ecmascript;charset=us-ascii", StandardCharsets.US_ASCII);
+        doCreateTextMessageFromDataWithContentTypeTestImpl("application/ecmascript;charset=utf-8", StandardCharsets.UTF_8);
+        doCreateTextMessageFromDataWithContentTypeTestImpl("application/ecmascript", StandardCharsets.UTF_8);
+    }
+
+    @Test
+    public void testCreateTextMessageFromDataWithContentTypeApplicationXml() throws Exception {
+        doCreateTextMessageFromDataWithContentTypeTestImpl("application/xml;charset=iso-8859-1", StandardCharsets.ISO_8859_1);
+        doCreateTextMessageFromDataWithContentTypeTestImpl("application/xml;charset=us-ascii", StandardCharsets.US_ASCII);
+        doCreateTextMessageFromDataWithContentTypeTestImpl("application/xml;charset=utf-8", StandardCharsets.UTF_8);
+        doCreateTextMessageFromDataWithContentTypeTestImpl("application/xml", StandardCharsets.UTF_8);
+    }
+
+    @Test
+    public void testCreateTextMessageFromDataWithContentTypeApplicationXmlVariant() throws Exception {
+        doCreateTextMessageFromDataWithContentTypeTestImpl("application/something+xml;charset=iso-8859-1", StandardCharsets.ISO_8859_1);
+        doCreateTextMessageFromDataWithContentTypeTestImpl("application/something+xml;charset=us-ascii", StandardCharsets.US_ASCII);
+        doCreateTextMessageFromDataWithContentTypeTestImpl("application/something+xml;charset=utf-8", StandardCharsets.UTF_8);
+        doCreateTextMessageFromDataWithContentTypeTestImpl("application/something+xml", StandardCharsets.UTF_8);
+    }
+
+    @Test
+    public void testCreateTextMessageFromDataWithContentTypeApplicationXmlDtd() throws Exception {
+        doCreateTextMessageFromDataWithContentTypeTestImpl("application/xml-dtd;charset=iso-8859-1", StandardCharsets.ISO_8859_1);
+        doCreateTextMessageFromDataWithContentTypeTestImpl("application/xml-dtd;charset=us-ascii", StandardCharsets.US_ASCII);
+        doCreateTextMessageFromDataWithContentTypeTestImpl("application/xml-dtd;charset=utf-8", StandardCharsets.UTF_8);
+        doCreateTextMessageFromDataWithContentTypeTestImpl("application/xml-dtd", StandardCharsets.UTF_8);
+    }
+
+    private void doCreateTextMessageFromDataWithContentTypeTestImpl(String contentType, Charset expectedCharset) throws IOException {
+        Message message = Proton.message();
+        Binary binary = new Binary(new byte[0]);
+        message.setBody(new Data(binary));
+        message.setContentType(contentType);
+
+        JmsMessage jmsMessage = AmqpCodec.decodeMessage(mockConsumer, encodeMessage(message)).asJmsMessage();
+        assertNotNull("Message should not be null", jmsMessage);
+        assertEquals("Unexpected message class type", JmsTextMessage.class, jmsMessage.getClass());
+
+        JmsMessageFacade facade = jmsMessage.getFacade();
+        assertNotNull("Facade should not be null", facade);
+        assertEquals("Unexpected facade class type", AmqpJmsTextMessageFacade.class, facade.getClass());
+
+        AmqpJmsTextMessageFacade textFacade = (AmqpJmsTextMessageFacade) facade;
+        assertEquals("Unexpected character set", expectedCharset, textFacade.getCharset());
+    }
+
+    // --------- AmqpValue Body Section ---------
+
+    /**
+     * Test that an amqp-value body containing a string results in a TextMessage
+     * when not otherwise annotated to indicate the type of JMS message it is.
+     *
+     * @throws Exception if an error occurs during the test.
+     */
+    @Test
+    public void testCreateTextMessageFromAmqpValueWithString() throws Exception {
+        Message message = Proton.message();
+        message.setBody(new AmqpValue("content"));
+
+        JmsMessage jmsMessage = AmqpCodec.decodeMessage(mockConsumer, encodeMessage(message)).asJmsMessage();
+        assertNotNull("Message should not be null", jmsMessage);
+        assertEquals("Unexpected message class type", JmsTextMessage.class, jmsMessage.getClass());
+
+        JmsMessageFacade facade = jmsMessage.getFacade();
+        assertNotNull("Facade should not be null", facade);
+        assertEquals("Unexpected facade class type", AmqpJmsTextMessageFacade.class, facade.getClass());
+    }
+
+    /**
+     * Test that an amqp-value body containing a null results in an TextMessage
+     * when not otherwise annotated to indicate the type of JMS message it is.
+     *
+     * @throws Exception if an error occurs during the test.
+     */
+    @Test
+    public void testCreateTextMessageFromAmqpValueWithNull() throws Exception {
+        Message message = Proton.message();
+        message.setBody(new AmqpValue(null));
+
+        JmsMessage jmsMessage = AmqpCodec.decodeMessage(mockConsumer, encodeMessage(message)).asJmsMessage();
+        assertNotNull("Message should not be null", jmsMessage);
+        assertEquals("Unexpected message class type", JmsTextMessage.class, jmsMessage.getClass());
+
+        JmsMessageFacade facade = jmsMessage.getFacade();
+        assertNotNull("Facade should not be null", facade);
+        assertEquals("Unexpected facade class type", AmqpJmsTextMessageFacade.class, facade.getClass());
+    }
+
+    /**
+     * Test that an amqp-value body containing a map results in an ObjectMessage
+     * when not otherwise annotated to indicate the type of JMS message it is.
+     *
+     * @throws Exception if an error occurs during the test.
+     */
+    @Test
+    public void testCreateAmqpObjectMessageFromAmqpValueWithMap() throws Exception {
+        Message message = Proton.message();
+        Map<String, String> map = new HashMap<String,String>();
+        message.setBody(new AmqpValue(map));
+
+        JmsMessage jmsMessage = AmqpCodec.decodeMessage(mockConsumer, encodeMessage(message)).asJmsMessage();
+        assertNotNull("Message should not be null", jmsMessage);
+        assertEquals("Unexpected message class type", JmsObjectMessage.class, jmsMessage.getClass());
+
+        JmsMessageFacade facade = jmsMessage.getFacade();
+        assertNotNull("Facade should not be null", facade);
+        assertEquals("Unexpected facade class type", AmqpJmsObjectMessageFacade.class, facade.getClass());
+
+        AmqpObjectTypeDelegate delegate = ((AmqpJmsObjectMessageFacade) facade).getDelegate();
+        assertTrue("Unexpected delegate type: " + delegate, delegate instanceof AmqpTypedObjectDelegate);
+    }
+
+    /**
+     * Test that an amqp-value body containing a list results in an ObjectMessage
+     * when not otherwise annotated to indicate the type of JMS message it is.
+     *
+     * @throws Exception if an error occurs during the test.
+     */
+    @Test
+    public void testCreateAmqpObjectMessageFromAmqpValueWithList() throws Exception {
+        Message message = Proton.message();
+        List<String> list = new ArrayList<String>();
+        message.setBody(new AmqpValue(list));
+
+        JmsMessage jmsMessage = AmqpCodec.decodeMessage(mockConsumer, encodeMessage(message)).asJmsMessage();
+        assertNotNull("Message should not be null", jmsMessage);
+        assertEquals("Unexpected message class type", JmsObjectMessage.class, jmsMessage.getClass());
+
+        JmsMessageFacade facade = jmsMessage.getFacade();
+        assertNotNull("Facade should not be null", facade);
+        assertEquals("Unexpected facade class type", AmqpJmsObjectMessageFacade.class, facade.getClass());
+
+        AmqpObjectTypeDelegate delegate = ((AmqpJmsObjectMessageFacade) facade).getDelegate();
+        assertTrue("Unexpected delegate type: " + delegate, delegate instanceof AmqpTypedObjectDelegate);
+    }
+
+    /**
+     * Test that an amqp-value body containing a binary value results in BytesMessage
+     * when not otherwise annotated to indicate the type of JMS message it is.
+     *
+     * @throws Exception if an error occurs during the test.
+     */
+    @Test
+    public void testCreateAmqpBytesMessageFromAmqpValueWithBinary() throws Exception {
+        Message message = Proton.message();
+        Binary binary = new Binary(new byte[0]);
+        message.setBody(new AmqpValue(binary));
+
+        JmsMessage jmsMessage = AmqpCodec.decodeMessage(mockConsumer, encodeMessage(message)).asJmsMessage();
+        assertNotNull("Message should not be null", jmsMessage);
+        assertEquals("Unexpected message class type", JmsBytesMessage.class, jmsMessage.getClass());
+
+        JmsMessageFacade facade = jmsMessage.getFacade();
+        assertNotNull("Facade should not be null", facade);
+        assertEquals("Unexpected facade class type", AmqpJmsBytesMessageFacade.class, facade.getClass());
+    }
+
+    /**
+     * Test that an amqp-value body containing a value which can't be categorised results in
+     * an ObjectMessage when not otherwise annotated to indicate the type of JMS message it is.
+     *
+     * @throws Exception if an error occurs during the test.
+     */
+    @Test
+    public void testCreateObjectMessageFromAmqpValueWithUncategorisedContent() throws Exception {
+        Message message = Proton.message();
+        message.setBody(new AmqpValue(UUID.randomUUID()));
+
+        JmsMessage jmsMessage = AmqpCodec.decodeMessage(mockConsumer, encodeMessage(message)).asJmsMessage();
+        assertNotNull("Message should not be null", jmsMessage);
+        assertEquals("Unexpected message class type", JmsObjectMessage.class, jmsMessage.getClass());
+
+        JmsMessageFacade facade = jmsMessage.getFacade();
+        assertNotNull("Facade should not be null", facade);
+        assertEquals("Unexpected facade class type", AmqpJmsObjectMessageFacade.class, facade.getClass());
+
+        AmqpObjectTypeDelegate delegate = ((AmqpJmsObjectMessageFacade) facade).getDelegate();
+        assertTrue("Unexpected delegate type: " + delegate, delegate instanceof AmqpTypedObjectDelegate);
+    }
+
+    // --------- AmqpSequence Body Section ---------
+
+    /**
+     * Test that an amqp-sequence body containing a binary value results in an ObjectMessage
+     * when not otherwise annotated to indicate the type of JMS message it is.
+     *
+     * @throws Exception if an error occurs during the test.
+     */
+    @Test
+    public void testCreateObjectMessageMessageFromAmqpSequence() throws Exception
+    {
+        Message message = Proton.message();
+        List<String> list = new ArrayList<String>();
+        message.setBody(new AmqpSequence(list));
+
+        JmsMessage jmsMessage = AmqpCodec.decodeMessage(mockConsumer, encodeMessage(message)).asJmsMessage();
+        assertNotNull("Message should not be null", jmsMessage);
+        assertEquals("Unexpected message class type", JmsObjectMessage.class, jmsMessage.getClass());
+
+        JmsMessageFacade facade = jmsMessage.getFacade();
+        assertNotNull("Facade should not be null", facade);
+        assertEquals("Unexpected facade class type", AmqpJmsObjectMessageFacade.class, facade.getClass());
+
+        AmqpObjectTypeDelegate delegate = ((AmqpJmsObjectMessageFacade) facade).getDelegate();
+        assertTrue("Unexpected delegate type: " + delegate, delegate instanceof AmqpTypedObjectDelegate);
+    }
+}

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsBytesMessageFacadeTest.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsBytesMessageFacadeTest.java b/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsBytesMessageFacadeTest.java
index 9afcc34..4ac9d6d 100644
--- a/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsBytesMessageFacadeTest.java
+++ b/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsBytesMessageFacadeTest.java
@@ -24,9 +24,6 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-import io.netty.buffer.ByteBufInputStream;
-import io.netty.buffer.ByteBufOutputStream;
-import io.netty.buffer.Unpooled;
 
 import java.io.IOException;
 import java.io.InputStream;
@@ -42,11 +39,16 @@ import org.apache.qpid.proton.amqp.messaging.AmqpSequence;
 import org.apache.qpid.proton.amqp.messaging.AmqpValue;
 import org.apache.qpid.proton.amqp.messaging.Data;
 import org.apache.qpid.proton.amqp.messaging.MessageAnnotations;
+import org.apache.qpid.proton.amqp.messaging.Properties;
 import org.apache.qpid.proton.amqp.messaging.Section;
 import org.apache.qpid.proton.message.Message;
 import org.junit.Test;
 import org.mockito.Mockito;
 
+import io.netty.buffer.ByteBufInputStream;
+import io.netty.buffer.ByteBufOutputStream;
+import io.netty.buffer.Unpooled;
+
 /**
  * Tests for class AmqpJmsBytesMessageFacade
  */
@@ -60,8 +62,7 @@ public class AmqpJmsBytesMessageFacadeTest extends AmqpJmsMessageTypesTestCase {
     public void testNewMessageContainsMessageTypeAnnotation() throws Exception {
         AmqpJmsBytesMessageFacade amqpBytesMessageFacade = createNewBytesMessageFacade();
 
-        Message protonMessage = amqpBytesMessageFacade.getAmqpMessage();
-        MessageAnnotations annotations = protonMessage.getMessageAnnotations();
+        MessageAnnotations annotations = amqpBytesMessageFacade.getMessageAnnotations();
         Map<Symbol, Object> annotationsMap = annotations.getValue();
 
         assertNotNull("MessageAnnotations section was not present", annotations);
@@ -93,10 +94,12 @@ public class AmqpJmsBytesMessageFacadeTest extends AmqpJmsMessageTypesTestCase {
     @Test
     public void testNewMessageHasContentType() throws Exception {
         AmqpJmsBytesMessageFacade amqpBytesMessageFacade = createNewBytesMessageFacade();
-        Message protonMessage = amqpBytesMessageFacade.getAmqpMessage();
 
-        assertNotNull(protonMessage);
-        String contentType = protonMessage.getContentType();
+        Properties properties = amqpBytesMessageFacade.getProperties();
+        assertNotNull(properties);
+        assertNotNull(properties.getContentType());
+
+        String contentType = properties.getContentType().toString();
         assertNotNull("content type should be set", contentType);
         assertEquals("application/octet-stream", contentType);
     }
@@ -143,10 +146,8 @@ public class AmqpJmsBytesMessageFacadeTest extends AmqpJmsMessageTypesTestCase {
     @Test
     public void testCopyOnPopulatedNewMessageCreatesDataSection() throws Exception {
         AmqpJmsBytesMessageFacade amqpBytesMessageFacade = createNewBytesMessageFacade();
-        Message protonMessage = amqpBytesMessageFacade.getAmqpMessage();
 
-        assertNotNull("underlying proton message was null", protonMessage);
-        assertDataBodyAsExpected(protonMessage, 0);
+        assertDataBodyAsExpected(amqpBytesMessageFacade.getBody(), 0);
 
         byte[] bytes = "myBytes".getBytes();
         OutputStream os = amqpBytesMessageFacade.getOutputStream();
@@ -154,8 +155,8 @@ public class AmqpJmsBytesMessageFacadeTest extends AmqpJmsMessageTypesTestCase {
 
         AmqpJmsBytesMessageFacade copy = amqpBytesMessageFacade.copy();
 
-        assertDataBodyAsExpected(protonMessage, bytes.length);
-        assertDataBodyAsExpected(copy.getAmqpMessage(), bytes.length);
+        assertDataBodyAsExpected(amqpBytesMessageFacade.getBody(), bytes.length);
+        assertDataBodyAsExpected(copy.getBody(), bytes.length);
     }
 
     /**
@@ -167,15 +168,13 @@ public class AmqpJmsBytesMessageFacadeTest extends AmqpJmsMessageTypesTestCase {
     @Test
     public void testCopyOfNewMessageDoesNotCreateDataSection() throws Exception {
         AmqpJmsBytesMessageFacade amqpBytesMessageFacade = createNewBytesMessageFacade();
-        Message origAmqpMessage = amqpBytesMessageFacade.getAmqpMessage();
 
-        assertNotNull("underlying proton message was null", origAmqpMessage);
-        assertDataBodyAsExpected(origAmqpMessage, 0);
+        assertDataBodyAsExpected(amqpBytesMessageFacade.getBody(), 0);
 
         AmqpJmsBytesMessageFacade copy = amqpBytesMessageFacade.copy();
 
-        assertDataBodyAsExpected(origAmqpMessage, 0);
-        assertDataBodyAsExpected(copy.getAmqpMessage(), 0);
+        assertDataBodyAsExpected(amqpBytesMessageFacade.getBody(), 0);
+        assertDataBodyAsExpected(copy.getBody(), 0);
     }
 
     @Test
@@ -191,7 +190,7 @@ public class AmqpJmsBytesMessageFacadeTest extends AmqpJmsMessageTypesTestCase {
         assertTrue("Expected no message content from facade", amqpBytesMessageFacade.getBodyLength() == 0);
         assertEquals("Expected no data from facade, but got some", END_OF_STREAM, amqpBytesMessageFacade.getInputStream().read(new byte[1]));
 
-        assertDataBodyAsExpected(amqpBytesMessageFacade.getAmqpMessage(), 0);
+        assertDataBodyAsExpected(amqpBytesMessageFacade.getBody(), 0);
     }
 
     @Test
@@ -209,7 +208,7 @@ public class AmqpJmsBytesMessageFacadeTest extends AmqpJmsMessageTypesTestCase {
 
         assertEquals("Expected no data from facade, but got some", END_OF_STREAM, amqpBytesMessageFacade.getInputStream().read(new byte[1]));
 
-        assertDataBodyAsExpected(amqpBytesMessageFacade.getAmqpMessage(), 0);
+        assertDataBodyAsExpected(amqpBytesMessageFacade.getBody(), 0);
     }
 
     @Test
@@ -225,7 +224,7 @@ public class AmqpJmsBytesMessageFacadeTest extends AmqpJmsMessageTypesTestCase {
 
         amqpBytesMessageFacade.clearBody();
 
-        assertDataBodyAsExpected(amqpBytesMessageFacade.getAmqpMessage(), 0);
+        assertDataBodyAsExpected(amqpBytesMessageFacade.getBody(), 0);
     }
 
     @Test
@@ -414,7 +413,6 @@ public class AmqpJmsBytesMessageFacadeTest extends AmqpJmsMessageTypesTestCase {
         Message message = Message.Factory.create();
         message.setBody(new Data(new Binary(orig)));
         AmqpJmsBytesMessageFacade amqpBytesMessageFacade = createReceivedBytesMessageFacade(createMockAmqpConsumer(), message);
-        Message protonMessage = amqpBytesMessageFacade.getAmqpMessage();
 
         OutputStream os = amqpBytesMessageFacade.getOutputStream();
         os.write(replacement);
@@ -423,7 +421,7 @@ public class AmqpJmsBytesMessageFacadeTest extends AmqpJmsMessageTypesTestCase {
 
         // Retrieve the new Binary from the underlying message, check they match
         // (the backing arrays may be different length so not checking arrayEquals)
-        Data body = (Data) protonMessage.getBody();
+        Data body = (Data) amqpBytesMessageFacade.getBody();
         assertEquals("Underlying message data section did not contain the expected bytes", new Binary(replacement), body.getValue());
 
         assertEquals("expected body length to match replacement bytes", replacement.length, amqpBytesMessageFacade.getBodyLength());
@@ -500,10 +498,9 @@ public class AmqpJmsBytesMessageFacadeTest extends AmqpJmsMessageTypesTestCase {
 
     //--------- utility methods ----------
 
-    private void assertDataBodyAsExpected(Message protonMessage, int length) {
-        Section body = protonMessage.getBody();
+    private void assertDataBodyAsExpected(Section body, int length) {
         assertNotNull("Expected body section to be present", body);
-        assertEquals("Unexpected body section type", Data.class, protonMessage.getBody().getClass());
+        assertEquals("Unexpected body section type", Data.class, body.getClass());
         Binary value = ((Data) body).getValue();
         assertNotNull(value);
         assertEquals("Unexpected body length", length, value.getLength());

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMapMessageFacadeTest.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMapMessageFacadeTest.java b/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMapMessageFacadeTest.java
index 9ae47cd..6716fb6 100644
--- a/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMapMessageFacadeTest.java
+++ b/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMapMessageFacadeTest.java
@@ -47,8 +47,7 @@ public class AmqpJmsMapMessageFacadeTest extends AmqpJmsMessageTypesTestCase {
     public void testNewMessageToSendContainsMessageTypeAnnotation() throws Exception {
         AmqpJmsMapMessageFacade amqpMapMessageFacade = createNewMapMessageFacade();
 
-        Message protonMessage = amqpMapMessageFacade.getAmqpMessage();
-        MessageAnnotations annotations = protonMessage.getMessageAnnotations();
+        MessageAnnotations annotations = amqpMapMessageFacade.getMessageAnnotations();
         Map<Symbol, Object> annotationsMap = annotations.getValue();
 
         assertNotNull("MessageAnnotations section was not present", annotations);


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@qpid.apache.org
For additional commands, e-mail: commits-help@qpid.apache.org


[4/5] qpid-jms git commit: QPIDJMS-215 Perform Message encoding at send time

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMessageFacade.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMessageFacade.java b/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMessageFacade.java
index 89094a1..9f90d87 100644
--- a/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMessageFacade.java
+++ b/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMessageFacade.java
@@ -34,20 +34,23 @@ import javax.jms.MessageFormatException;
 
 import org.apache.qpid.jms.JmsDestination;
 import org.apache.qpid.jms.exceptions.IdConversionException;
+import org.apache.qpid.jms.message.JmsMessage;
 import org.apache.qpid.jms.message.facade.JmsMessageFacade;
 import org.apache.qpid.jms.provider.amqp.AmqpConnection;
 import org.apache.qpid.jms.provider.amqp.AmqpConsumer;
-import org.apache.qpid.proton.Proton;
 import org.apache.qpid.proton.amqp.Binary;
 import org.apache.qpid.proton.amqp.Symbol;
 import org.apache.qpid.proton.amqp.UnsignedByte;
 import org.apache.qpid.proton.amqp.UnsignedInteger;
 import org.apache.qpid.proton.amqp.messaging.ApplicationProperties;
+import org.apache.qpid.proton.amqp.messaging.DeliveryAnnotations;
 import org.apache.qpid.proton.amqp.messaging.Footer;
 import org.apache.qpid.proton.amqp.messaging.Header;
 import org.apache.qpid.proton.amqp.messaging.MessageAnnotations;
 import org.apache.qpid.proton.amqp.messaging.Properties;
-import org.apache.qpid.proton.message.Message;
+import org.apache.qpid.proton.amqp.messaging.Section;
+
+import io.netty.buffer.ByteBuf;
 
 public class AmqpJmsMessageFacade implements JmsMessageFacade {
 
@@ -55,11 +58,15 @@ public class AmqpJmsMessageFacade implements JmsMessageFacade {
     private static final Charset UTF8 = Charset.forName("UTF-8");
     private static final long UINT_MAX = 0xFFFFFFFFL;
 
-    protected final Message message;
-    protected final AmqpConnection connection;
+    protected AmqpConnection connection;
 
-    private Map<Symbol,Object> messageAnnotationsMap;
-    private Map<String,Object> applicationPropertiesMap;
+    private Properties properties;
+    private Header header;
+    private Section body;
+    private Map<Symbol, Object> messageAnnotationsMap;
+    private Map<String, Object> applicationPropertiesMap;
+    private Map<Symbol, Object> deliveryAnnotationsMap;
+    private Map<Symbol, Object> footerMap;
 
     private JmsDestination replyTo;
     private JmsDestination destination;
@@ -74,42 +81,29 @@ public class AmqpJmsMessageFacade implements JmsMessageFacade {
     private Long userSpecifiedTTL = null;
 
     /**
-     * Create a new AMQP Message Facade with an empty message instance.
+     * Initialize the state of this message for send.
      *
      * @param connection
-     *        the AmqpConnection that under which this facade was created.
+     *      The connection that this message is linked to.
      */
-    public AmqpJmsMessageFacade(AmqpConnection connection) {
-        this.message = Proton.message();
-        this.message.setDurable(true);
-
+    public void initialize(AmqpConnection connection) {
         this.connection = connection;
-        setMessageAnnotation(JMS_MSG_TYPE, JMS_MESSAGE);
+
+        setMessageAnnotation(JMS_MSG_TYPE, getJmsMsgType());
+        setPersistent(true); // TODO - Remove to avoid default Header
+        initializeEmptyBody();
     }
 
     /**
-     * Creates a new Facade around an incoming AMQP Message for dispatch to the
-     * JMS Consumer instance.
+     * Initialize the state of this message for receive.
      *
      * @param consumer
-     *        the consumer that received this message.
-     * @param message
-     *        the incoming Message instance that is being wrapped.
+     *      The consumer that this message was read from.
      */
-    @SuppressWarnings("unchecked")
-    public AmqpJmsMessageFacade(AmqpConsumer consumer, Message message) {
-        this.message = message;
+    public void initialize(AmqpConsumer consumer) {
         this.connection = consumer.getConnection();
         this.consumerDestination = consumer.getDestination();
 
-        if (message.getMessageAnnotations() != null) {
-            messageAnnotationsMap = message.getMessageAnnotations().getValue();
-        }
-
-        if (message.getApplicationProperties() != null) {
-            applicationPropertiesMap = message.getApplicationProperties().getValue();
-        }
-
         Long ttl = getTtl();
         Long absoluteExpiryTime = getAbsoluteExpiryTime();
         if (absoluteExpiryTime == null && ttl != null) {
@@ -118,6 +112,13 @@ public class AmqpJmsMessageFacade implements JmsMessageFacade {
     }
 
     /**
+     * Used to indicate that a Message object should empty the body element and make
+     * any other internal updates to reflect the message now has no body value.
+     */
+    protected void initializeEmptyBody() {
+    }
+
+    /**
      * @return the appropriate byte value that indicates the type of message this is.
      */
     public byte getJmsMsgType() {
@@ -132,11 +133,22 @@ public class AmqpJmsMessageFacade implements JmsMessageFacade {
      * @return a String value indicating the message content type.
      */
     public String getContentType() {
-        return message.getContentType();
+        if (properties != null && properties.getContentType() != null) {
+            return properties.getContentType().toString();
+        }
+
+        return null;
     }
 
     public void setContentType(String value) {
-        message.setContentType(value);
+        if (properties == null) {
+            if (value == null) {
+                return;
+            }
+            lazyCreateProperties();
+        }
+
+        properties.setContentType(Symbol.valueOf(value));
     }
 
     @Override
@@ -212,11 +224,11 @@ public class AmqpJmsMessageFacade implements JmsMessageFacade {
         }
 
         if (ttl > 0 && ttl < UINT_MAX) {
-            message.setTtl(ttl);
+            lazyCreateHeader();
+            header.setTtl(UnsignedInteger.valueOf(ttl));
         } else {
-            Header hdr = message.getHeader();
-            if (hdr != null) {
-                hdr.setTtl(null);
+            if (header != null) {
+                header.setTtl(null);
             }
         }
 
@@ -229,7 +241,7 @@ public class AmqpJmsMessageFacade implements JmsMessageFacade {
 
     @Override
     public void clearBody() {
-        message.setBody(null);
+        setBody(null);
     }
 
     @Override
@@ -239,13 +251,14 @@ public class AmqpJmsMessageFacade implements JmsMessageFacade {
 
     @Override
     public AmqpJmsMessageFacade copy() throws JMSException {
-        AmqpJmsMessageFacade copy = new AmqpJmsMessageFacade(connection);
+        AmqpJmsMessageFacade copy = new AmqpJmsMessageFacade();
         copyInto(copy);
         return copy;
     }
 
-    @SuppressWarnings("unchecked")
     protected void copyInto(AmqpJmsMessageFacade target) {
+        target.connection = connection;
+
         if (consumerDestination != null) {
             target.consumerDestination = consumerDestination;
         }
@@ -266,48 +279,42 @@ public class AmqpJmsMessageFacade implements JmsMessageFacade {
             target.userSpecifiedTTL = userSpecifiedTTL;
         }
 
-        Message targetMsg = target.getAmqpMessage();
-
-        if (message.getHeader() != null) {
+        if (header != null) {
             Header headers = new Header();
-            headers.setDurable(message.getHeader().getDurable());
-            headers.setPriority(message.getHeader().getPriority());
-            headers.setTtl(message.getHeader().getTtl());
-            headers.setFirstAcquirer(message.getHeader().getFirstAcquirer());
-            headers.setDeliveryCount(message.getHeader().getDeliveryCount());
-            targetMsg.setHeader(headers);
-        }
+            headers.setDurable(header.getDurable());
+            headers.setPriority(header.getPriority());
+            headers.setTtl(header.getTtl());
+            headers.setFirstAcquirer(header.getFirstAcquirer());
+            headers.setDeliveryCount(header.getDeliveryCount());
 
-        if (message.getFooter() != null && message.getFooter().getValue() != null) {
-            Map<Object, Object> newFooterMap = new HashMap<Object, Object>();
-            newFooterMap.putAll(message.getFooter().getValue());
-            targetMsg.setFooter(new Footer(newFooterMap));
+            target.setHeader(headers);
         }
 
-        if (message.getProperties() != null) {
+        if (properties != null) {
             Properties properties = new Properties();
 
-            properties.setMessageId(message.getProperties().getMessageId());
-            properties.setUserId(message.getProperties().getUserId());
-            properties.setTo(message.getProperties().getTo());
-            properties.setSubject(message.getProperties().getSubject());
-            properties.setReplyTo(message.getProperties().getReplyTo());
-            properties.setCorrelationId(message.getProperties().getCorrelationId());
-            properties.setContentType(message.getProperties().getContentType());
-            properties.setContentEncoding(message.getProperties().getContentEncoding());
-            properties.setAbsoluteExpiryTime(message.getProperties().getAbsoluteExpiryTime());
-            properties.setCreationTime(message.getProperties().getCreationTime());
-            properties.setGroupId(message.getProperties().getGroupId());
-            properties.setGroupSequence(message.getProperties().getGroupSequence());
-            properties.setReplyToGroupId(message.getProperties().getReplyToGroupId());
-
-            targetMsg.setProperties(properties);
+            properties.setMessageId(getProperties().getMessageId());
+            properties.setUserId(getProperties().getUserId());
+            properties.setTo(getProperties().getTo());
+            properties.setSubject(getProperties().getSubject());
+            properties.setReplyTo(getProperties().getReplyTo());
+            properties.setCorrelationId(getProperties().getCorrelationId());
+            properties.setContentType(getProperties().getContentType());
+            properties.setContentEncoding(getProperties().getContentEncoding());
+            properties.setAbsoluteExpiryTime(getProperties().getAbsoluteExpiryTime());
+            properties.setCreationTime(getProperties().getCreationTime());
+            properties.setGroupId(getProperties().getGroupId());
+            properties.setGroupSequence(getProperties().getGroupSequence());
+            properties.setReplyToGroupId(getProperties().getReplyToGroupId());
+
+            target.setProperties(properties);
         }
 
-        if (message.getDeliveryAnnotations() != null && message.getDeliveryAnnotations().getValue() != null) {
-            Map<Symbol, Object> newDeliveryAnnotations = new HashMap<Symbol, Object>();
-            newDeliveryAnnotations.putAll(message.getDeliveryAnnotations().getValue());
-            targetMsg.setFooter(new Footer(newDeliveryAnnotations));
+        target.setBody(body);
+
+        if (deliveryAnnotationsMap != null) {
+            target.lazyCreateDeliveryAnnotations();
+            target.deliveryAnnotationsMap.putAll(deliveryAnnotationsMap);
         }
 
         if (applicationPropertiesMap != null) {
@@ -319,33 +326,61 @@ public class AmqpJmsMessageFacade implements JmsMessageFacade {
             target.lazyCreateMessageAnnotations();
             target.messageAnnotationsMap.putAll(messageAnnotationsMap);
         }
+
+        if (footerMap != null) {
+            target.lazyCreateFooter();
+            target.footerMap.putAll(footerMap);
+        }
     }
 
     @Override
     public String getMessageId() {
-        Object underlying = message.getMessageId();
+        Object underlying = null;
+
+        if (properties != null) {
+            underlying = properties.getMessageId();
+        }
+
         return AmqpMessageIdHelper.INSTANCE.toMessageIdString(underlying);
     }
 
     @Override
     public Object getProviderMessageIdObject() {
-        return message.getMessageId();
+        return properties == null ? null : properties.getMessageId();
     }
 
     @Override
     public void setProviderMessageIdObject(Object messageId) {
-        message.setMessageId(messageId);
+        if (properties == null) {
+            if (messageId == null) {
+                return;
+            }
+
+            lazyCreateProperties();
+        }
+
+        properties.setMessageId(messageId);
     }
 
     @Override
     public void setMessageId(String messageId) throws IdConversionException {
-        message.setMessageId(AmqpMessageIdHelper.INSTANCE.toIdObject(messageId));
+        Object value = AmqpMessageIdHelper.INSTANCE.toIdObject(messageId);
+
+        if (properties == null) {
+            if (value == null) {
+                return;
+            }
+
+            lazyCreateProperties();
+        }
+
+        properties.setMessageId(value);
     }
 
     @Override
     public long getTimestamp() {
-        if (message.getProperties() != null) {
-            Date timestamp = message.getProperties().getCreationTime();
+        if (properties != null) {
+            Date timestamp = properties.getCreationTime();
             if (timestamp != null) {
                 return timestamp.getTime();
             }
@@ -356,39 +391,62 @@ public class AmqpJmsMessageFacade implements JmsMessageFacade {
 
     @Override
     public void setTimestamp(long timestamp) {
-        if (timestamp != 0) {
-            message.setCreationTime(timestamp);
-        } else {
-            if (message.getProperties() != null) {
-                message.getProperties().setCreationTime(null);
+        if (properties == null) {
+            if (timestamp == 0) {
+                return;
             }
+
+            lazyCreateProperties();
+        }
+
+        if (timestamp == 0) {
+            properties.setCreationTime(null);
+        } else {
+            properties.setCreationTime(new Date(timestamp));
         }
     }
 
     @Override
     public String getCorrelationId() {
-        return AmqpMessageIdHelper.INSTANCE.toCorrelationIdString(message.getCorrelationId());
+        if (properties == null) {
+            return null;
+        }
+
+        return AmqpMessageIdHelper.INSTANCE.toCorrelationIdString(properties.getCorrelationId());
     }
 
     @Override
     public void setCorrelationId(String correlationId) throws IdConversionException {
-        if (correlationId == null) {
-            message.setCorrelationId(null);
-        } else {
+        Object idObject = null;
+
+        if (correlationId != null) {
             if (AmqpMessageIdHelper.INSTANCE.hasMessageIdPrefix(correlationId)) {
                 // JMSMessageID value, process it for possible type conversion
-                Object idObject = AmqpMessageIdHelper.INSTANCE.toIdObject(correlationId);
-                message.setCorrelationId(idObject);
+                idObject = AmqpMessageIdHelper.INSTANCE.toIdObject(correlationId);
             } else {
-                // application-specific value, send as-is
-                message.setCorrelationId(correlationId);
+                idObject = correlationId;
             }
         }
+
+        if (properties == null) {
+            if (idObject == null) {
+                return;
+            }
+
+            lazyCreateProperties();
+        }
+
+        properties.setCorrelationId(idObject);
     }
 
     @Override
     public byte[] getCorrelationIdBytes() throws JMSException {
-        Object correlationId = message.getCorrelationId();
+        Object correlationId = null;
+
+        if (properties != null) {
+            correlationId = properties.getCorrelationId();
+        }
+
         if (correlationId == null) {
             return null;
         } else if (correlationId instanceof Binary) {
@@ -412,17 +470,37 @@ public class AmqpJmsMessageFacade implements JmsMessageFacade {
             binaryIdValue = new Binary(Arrays.copyOf(correlationId, correlationId.length));
         }
 
-        message.setCorrelationId(binaryIdValue);
+        if (properties == null) {
+            if (binaryIdValue == null) {
+                return;
+            }
+
+            lazyCreateProperties();
+        }
+
+        properties.setCorrelationId(binaryIdValue);
     }
 
     @Override
     public boolean isPersistent() {
-        return message.isDurable();
+        if (header != null && header.getDurable() != null) {
+            return header.getDurable();
+        }
+
+        return false;
     }
 
     @Override
     public void setPersistent(boolean value) {
-        this.message.setDurable(value);
+        if (header == null) {
+            if (value == false) {
+                return;
+            } else {
+                lazyCreateHeader();
+            }
+        }
+
+        header.setDurable(value);
     }
 
     @Override
@@ -437,8 +515,8 @@ public class AmqpJmsMessageFacade implements JmsMessageFacade {
 
     @Override
     public int getRedeliveryCount() {
-        if (message.getHeader() != null) {
-            UnsignedInteger count = message.getHeader().getDeliveryCount();
+        if (header != null) {
+            UnsignedInteger count = header.getDeliveryCount();
             if (count != null) {
                 return count.intValue();
             }
@@ -450,11 +528,12 @@ public class AmqpJmsMessageFacade implements JmsMessageFacade {
     @Override
     public void setRedeliveryCount(int redeliveryCount) {
         if (redeliveryCount == 0) {
-            if (message.getHeader() != null) {
-                message.getHeader().setDeliveryCount(null);
+            if (header != null) {
+                header.setDeliveryCount(null);
             }
         } else {
-            message.setDeliveryCount(redeliveryCount);
+            lazyCreateHeader();
+            header.setDeliveryCount(UnsignedInteger.valueOf(redeliveryCount));
         }
     }
 
@@ -478,24 +557,29 @@ public class AmqpJmsMessageFacade implements JmsMessageFacade {
 
     @Override
     public String getType() {
-        return message.getSubject();
+        if (properties != null) {
+            return properties.getSubject();
+        }
+
+        return null;
     }
 
     @Override
     public void setType(String type) {
         if (type != null) {
-            message.setSubject(type);
+            lazyCreateProperties();
+            properties.setSubject(type);
         } else {
-            if (message.getProperties() != null) {
-                message.getProperties().setSubject(null);
+            if (properties != null) {
+                properties.setSubject(null);
             }
         }
     }
 
     @Override
     public int getPriority() {
-        if (message.getHeader() != null) {
-            UnsignedByte priority = message.getHeader().getPriority();
+        if (header != null) {
+            UnsignedByte priority = header.getPriority();
             if (priority != null) {
                 int scaled = priority.intValue();
                 if (scaled > 9) {
@@ -512,10 +596,8 @@ public class AmqpJmsMessageFacade implements JmsMessageFacade {
     @Override
     public void setPriority(int priority) {
         if (priority == DEFAULT_PRIORITY) {
-            if (message.getHeader() == null) {
-                return;
-            } else {
-                message.getHeader().setPriority(null);
+            if (header != null) {
+                header.setPriority(null);
             }
         } else {
             byte scaled = (byte) priority;
@@ -525,7 +607,8 @@ public class AmqpJmsMessageFacade implements JmsMessageFacade {
                 scaled = 9;
             }
 
-            message.setPriority(scaled);
+            lazyCreateHeader();
+            header.setPriority(UnsignedByte.valueOf(scaled));
         }
     }
 
@@ -636,20 +719,33 @@ public class AmqpJmsMessageFacade implements JmsMessageFacade {
     }
 
     public void setReplyToGroupId(String replyToGroupId) {
-        message.setReplyToGroupId(replyToGroupId);
+        if (replyToGroupId != null) {
+            lazyCreateProperties();
+            properties.setReplyToGroupId(replyToGroupId);
+        } else {
+            if (properties != null) {
+                properties.setReplyToGroupId(null);
+            }
+        }
     }
 
     public String getReplyToGroupId() {
-        return message.getReplyToGroupId();
+        if (properties != null) {
+            return properties.getReplyToGroupId();
+        }
+
+        return null;
     }
 
     @Override
     public String getUserId() {
         String userId = null;
-        byte[] userIdBytes = message.getUserId();
 
-        if (userIdBytes != null) {
-            userId = new String(userIdBytes, UTF8);
+        if (properties != null && properties.getUserId() != null) {
+            Binary userIdBytes = properties.getUserId();
+            if (userIdBytes.getLength() != 0) {
+                userId = new String(userIdBytes.getArray(), userIdBytes.getArrayOffset(), userIdBytes.getLength(), UTF8);
+            }
         }
 
         return userId;
@@ -663,44 +759,66 @@ public class AmqpJmsMessageFacade implements JmsMessageFacade {
         }
 
         if (bytes == null) {
-            if (message.getProperties() != null) {
-                message.getProperties().setUserId(null);
+            if (properties != null) {
+                properties.setUserId(null);
             }
         } else {
-            message.setUserId(bytes);
+            lazyCreateProperties();
+            properties.setUserId(new Binary(bytes));
         }
     }
 
     @Override
     public byte[] getUserIdBytes() {
-        return message.getUserId();
+        if(properties == null || properties.getUserId() == null) {
+            return null;
+        } else {
+            final Binary userId = properties.getUserId();
+            byte[] id = new byte[userId.getLength()];
+            System.arraycopy(userId.getArray(), userId.getArrayOffset(), id, 0, userId.getLength());
+            return id;
+        }
     }
 
     @Override
     public void setUserIdBytes(byte[] userId) {
         if (userId == null || userId.length == 0) {
-            if (message.getProperties() != null) {
-                message.getProperties().setUserId(null);
+            if (properties != null) {
+                properties.setUserId(null);
             }
         } else {
-            message.setUserId(userId);
+            lazyCreateProperties();
+            byte[] id = new byte[userId.length];
+            System.arraycopy(userId, 0, id, 0, userId.length);
+            properties.setUserId(new Binary(id));
         }
     }
 
     @Override
     public String getGroupId() {
-        return message.getGroupId();
+        if (properties != null) {
+            return properties.getGroupId();
+        }
+
+        return null;
     }
 
     @Override
     public void setGroupId(String groupId) {
-        message.setGroupId(groupId);
+        if (groupId != null) {
+            lazyCreateProperties();
+            properties.setGroupId(groupId);
+        } else {
+            if (properties != null) {
+                properties.setGroupId(null);
+            }
+        }
     }
 
     @Override
     public int getGroupSequence() {
-        if (message.getProperties() != null) {
-            UnsignedInteger groupSeqUint = message.getProperties().getGroupSequence();
+        if (properties != null) {
+            UnsignedInteger groupSeqUint = properties.getGroupSequence();
             if (groupSeqUint != null) {
                 // This wraps it into the negative int range if uint is over 2^31-1
                 return groupSeqUint.intValue();
@@ -713,32 +831,26 @@ public class AmqpJmsMessageFacade implements JmsMessageFacade {
     @Override
     public void setGroupSequence(int groupSequence) {
         // This wraps it into the upper uint range if a negative was provided
-        if (groupSequence == 0) {
-            if (message.getProperties() != null) {
-                message.getProperties().setGroupSequence(null);
-            }
+        if (groupSequence != 0) {
+            lazyCreateProperties();
+            properties.setGroupSequence(UnsignedInteger.valueOf(groupSequence));
         } else {
-            message.setGroupSequence(groupSequence);
+            if (properties != null) {
+                properties.setGroupSequence(null);
+            }
         }
     }
 
     @Override
     public boolean hasBody() {
-        return message.getBody() == null;
-    }
-
-    /**
-     * @return the true AMQP Message instance wrapped by this Facade.
-     */
-    public Message getAmqpMessage() {
-        return this.message;
+        return body == null;
     }
 
     /**
      * The AmqpConnection instance that is associated with this Message.
      * @return the connection
      */
-    public AmqpConnection getConnection() {
+    AmqpConnection getConnection() {
         return connection;
     }
 
@@ -813,7 +925,6 @@ public class AmqpJmsMessageFacade implements JmsMessageFacade {
      */
     void clearMessageAnnotations() {
         messageAnnotationsMap = null;
-        message.setMessageAnnotations(null);
     }
 
     /**
@@ -821,33 +932,151 @@ public class AmqpJmsMessageFacade implements JmsMessageFacade {
      */
     void clearAllApplicationProperties() {
         applicationPropertiesMap = null;
-        message.setApplicationProperties(null);
     }
 
     String getToAddress() {
-        return message.getAddress();
+        if (properties != null) {
+            return properties.getTo();
+        }
+
+        return null;
     }
 
     void setToAddress(String address) {
-        message.setAddress(address);
+        if (address != null) {
+            lazyCreateProperties();
+            properties.setTo(address);
+        } else {
+            if (properties != null) {
+                properties.setTo(null);
+            }
+        }
     }
 
     String getReplyToAddress() {
-        return message.getReplyTo();
+        if (properties != null) {
+            return properties.getReplyTo();
+        }
+
+        return null;
     }
 
     void setReplyToAddress(String address) {
-        this.message.setReplyTo(address);
+        if (address != null) {
+            lazyCreateProperties();
+            properties.setReplyTo(address);
+        } else {
+            if (properties != null) {
+                properties.setReplyTo(null);
+            }
+        }
     }
 
     JmsDestination getConsumerDestination() {
         return this.consumerDestination;
     }
 
+    public JmsMessage asJmsMessage() {
+        return new JmsMessage(this);
+    }
+
+    @Override
+    public ByteBuf encodeMessage() {
+        return AmqpCodec.encodeMessage(this);
+    }
+
+    //----- Access to AMQP Message Values ------------------------------------//
+
+    Header getHeader() {
+        return header;
+    }
+
+    void setHeader(Header header) {
+        this.header = header;
+    }
+
+    Properties getProperties() {
+        return properties;
+    }
+
+    void setProperties(Properties properties) {
+        this.properties = properties;
+    }
+
+    Section getBody() {
+        return body;
+    }
+
+    void setBody(Section body) {
+        this.body = body;
+    }
+
+    MessageAnnotations getMessageAnnotations() {
+        MessageAnnotations result = null;
+        if (messageAnnotationsMap != null && !messageAnnotationsMap.isEmpty()) {
+            result = new MessageAnnotations(messageAnnotationsMap);
+        }
+
+        return result;
+    }
+
+    void setMessageAnnotations(MessageAnnotations messageAnnotations) {
+        if (messageAnnotations != null) {
+            this.messageAnnotationsMap = messageAnnotations.getValue();
+        }
+    }
+
+    DeliveryAnnotations getDeliveryAnnotations() {
+        DeliveryAnnotations result = null;
+        if (deliveryAnnotationsMap != null && !deliveryAnnotationsMap.isEmpty()) {
+            result = new DeliveryAnnotations(deliveryAnnotationsMap);
+        }
+
+        return result;
+    }
+
+    void setDeliveryAnnotations(DeliveryAnnotations deliveryAnnotations) {
+        if (deliveryAnnotations != null) {
+            this.deliveryAnnotationsMap = deliveryAnnotations.getValue();
+        }
+    }
+
+    ApplicationProperties getApplicationProperties() {
+        ApplicationProperties result = null;
+        if (applicationPropertiesMap != null && !applicationPropertiesMap.isEmpty()) {
+            result = new ApplicationProperties(applicationPropertiesMap);
+        }
+        return result;
+    }
+
+    @SuppressWarnings("unchecked")
+    void setApplicationProperties(ApplicationProperties applicationProperties) {
+        if (applicationProperties != null) {
+            this.applicationPropertiesMap = applicationProperties.getValue();
+        }
+    }
+
+    Footer getFooter() {
+        Footer result = null;
+        if (footerMap != null && footerMap.isEmpty()) {
+            result = new Footer(footerMap);
+        }
+        return result;
+    }
+
+    @SuppressWarnings("unchecked")
+    void setFooter(Footer footer) {
+        if (footer != null) {
+            this.footerMap = footer.getValue();
+        }
+    }
+
+    //----- Internal Message Utility Methods ---------------------------------//
+
     private Long getAbsoluteExpiryTime() {
         Long result = null;
-        if (message.getProperties() != null) {
-            Date date = message.getProperties().getAbsoluteExpiryTime();
+        if (properties != null) {
+            Date date = properties.getAbsoluteExpiryTime();
             if (date != null) {
                 result = date.getTime();
             }
@@ -858,8 +1087,8 @@ public class AmqpJmsMessageFacade implements JmsMessageFacade {
 
     private Long getTtl() {
         Long result = null;
-        if (message.getHeader() != null) {
-            UnsignedInteger ttl = message.getHeader().getTtl();
+        if (header != null) {
+            UnsignedInteger ttl = header.getTtl();
             if (ttl != null) {
                 result = ttl.longValue();
             }
@@ -869,26 +1098,49 @@ public class AmqpJmsMessageFacade implements JmsMessageFacade {
     }
 
     private void setAbsoluteExpiryTime(Long expiration) {
-        if (expiration == null) {
-            if (message.getProperties() != null) {
-                message.getProperties().setAbsoluteExpiryTime(null);
+        if (expiration == null || expiration == 0l) {
+            if (properties != null) {
+                properties.setAbsoluteExpiryTime(null);
             }
         } else {
-            message.setExpiryTime(expiration);
+            lazyCreateProperties();
+            properties.setAbsoluteExpiryTime(new Date(expiration));
+        }
+    }
+
+    private void lazyCreateProperties() {
+        if (properties == null) {
+            properties = new Properties();
+        }
+    }
+
+    private void lazyCreateHeader() {
+        if (header == null) {
+            header = new Header();
         }
     }
 
     private void lazyCreateMessageAnnotations() {
         if (messageAnnotationsMap == null) {
-            messageAnnotationsMap = new HashMap<Symbol,Object>();
-            message.setMessageAnnotations(new MessageAnnotations(messageAnnotationsMap));
+            messageAnnotationsMap = new HashMap<Symbol, Object>();
+        }
+    }
+
+    private void lazyCreateDeliveryAnnotations() {
+        if (deliveryAnnotationsMap == null) {
+            deliveryAnnotationsMap = new HashMap<Symbol, Object>();
         }
     }
 
     private void lazyCreateApplicationProperties() {
         if (applicationPropertiesMap == null) {
             applicationPropertiesMap = new HashMap<String, Object>();
-            message.setApplicationProperties(new ApplicationProperties(applicationPropertiesMap));
+        }
+    }
+
+    private void lazyCreateFooter() {
+        if (footerMap == null) {
+            footerMap = new HashMap<Symbol, Object>();
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMessageFactory.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMessageFactory.java b/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMessageFactory.java
index 5b78556..25e1489 100644
--- a/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMessageFactory.java
+++ b/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsMessageFactory.java
@@ -29,8 +29,6 @@ import org.apache.qpid.jms.message.JmsMessageFactory;
 import org.apache.qpid.jms.message.JmsObjectMessage;
 import org.apache.qpid.jms.message.JmsStreamMessage;
 import org.apache.qpid.jms.message.JmsTextMessage;
-import org.apache.qpid.jms.message.facade.JmsObjectMessageFacade;
-import org.apache.qpid.jms.message.facade.JmsTextMessageFacade;
 import org.apache.qpid.jms.provider.amqp.AmqpConnection;
 
 /**
@@ -53,7 +51,9 @@ public class AmqpJmsMessageFactory implements JmsMessageFactory {
 
     @Override
     public JmsMessage createMessage() throws JMSException {
-        return new JmsMessage(new AmqpJmsMessageFacade(connection));
+        AmqpJmsMessageFacade facade = new AmqpJmsMessageFacade();
+        facade.initialize(connection);
+        return facade.asJmsMessage();
     }
 
     @Override
@@ -63,29 +63,35 @@ public class AmqpJmsMessageFactory implements JmsMessageFactory {
 
     @Override
     public JmsTextMessage createTextMessage(String payload) throws JMSException {
-
-        JmsTextMessageFacade facade = new AmqpJmsTextMessageFacade(connection);
+        AmqpJmsTextMessageFacade facade = new AmqpJmsTextMessageFacade();
+        facade.initialize(connection);
 
         if (payload != null) {
             facade.setText(payload);
         }
 
-        return new JmsTextMessage(facade);
+        return facade.asJmsMessage();
     }
 
     @Override
     public JmsBytesMessage createBytesMessage() throws JMSException {
-        return new JmsBytesMessage(new AmqpJmsBytesMessageFacade(connection));
+        AmqpJmsBytesMessageFacade facade = new AmqpJmsBytesMessageFacade();
+        facade.initialize(connection);
+        return facade.asJmsMessage();
     }
 
     @Override
     public JmsMapMessage createMapMessage() throws JMSException {
-        return new JmsMapMessage(new AmqpJmsMapMessageFacade(connection));
+        AmqpJmsMapMessageFacade facade = new AmqpJmsMapMessageFacade();
+        facade.initialize(connection);
+        return facade.asJmsMessage();
     }
 
     @Override
     public JmsStreamMessage createStreamMessage() throws JMSException {
-        return new JmsStreamMessage(new AmqpJmsStreamMessageFacade(connection));
+        AmqpJmsStreamMessageFacade facade = new AmqpJmsStreamMessageFacade();
+        facade.initialize(connection);
+        return facade.asJmsMessage();
     }
 
     @Override
@@ -95,9 +101,9 @@ public class AmqpJmsMessageFactory implements JmsMessageFactory {
 
     @Override
     public JmsObjectMessage createObjectMessage(Serializable payload) throws JMSException {
-        JmsObjectMessageFacade facade = new AmqpJmsObjectMessageFacade(
-            connection, connection.isObjectMessageUsesAmqpTypes());
+        AmqpJmsObjectMessageFacade facade = new AmqpJmsObjectMessageFacade();
 
+        facade.initialize(connection);
         if (payload != null) {
             try {
                 facade.setObject(payload);
@@ -106,6 +112,6 @@ public class AmqpJmsMessageFactory implements JmsMessageFactory {
             }
         }
 
-        return new JmsObjectMessage(facade);
+        return facade.asJmsMessage();
     }
 }

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsObjectMessageFacade.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsObjectMessageFacade.java b/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsObjectMessageFacade.java
index f4b541f..e0f026f 100644
--- a/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsObjectMessageFacade.java
+++ b/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsObjectMessageFacade.java
@@ -16,7 +16,6 @@
  */
 package org.apache.qpid.jms.provider.amqp.message;
 
-import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.JMS_MSG_TYPE;
 import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.JMS_OBJECT_MESSAGE;
 
 import java.io.IOException;
@@ -25,13 +24,11 @@ import java.io.Serializable;
 import javax.jms.JMSException;
 
 import org.apache.qpid.jms.exceptions.JmsExceptionSupport;
+import org.apache.qpid.jms.message.JmsObjectMessage;
 import org.apache.qpid.jms.message.facade.JmsObjectMessageFacade;
 import org.apache.qpid.jms.policy.JmsDeserializationPolicy;
 import org.apache.qpid.jms.provider.amqp.AmqpConnection;
 import org.apache.qpid.jms.provider.amqp.AmqpConsumer;
-import org.apache.qpid.proton.message.Message;
-
-import io.netty.buffer.ByteBuf;
 
 /**
  * Wrapper around an AMQP Message instance that will be treated as a JMS ObjectMessage
@@ -40,46 +37,20 @@ import io.netty.buffer.ByteBuf;
 public class AmqpJmsObjectMessageFacade extends AmqpJmsMessageFacade implements JmsObjectMessageFacade {
 
     private AmqpObjectTypeDelegate delegate;
+    private JmsDeserializationPolicy deserializationPolicy;
 
-    private final JmsDeserializationPolicy deserializationPolicy;
-
-    /**
-     * Creates a new facade instance for outgoing message
-     *
-     * @param connection
-     *        the AmqpConnection that under which this facade was created.
-     * @param isAmqpTypeEncoded
-     *        controls the type used to encode the body.
-     */
-    public AmqpJmsObjectMessageFacade(AmqpConnection connection, boolean isAmqpTypeEncoded) {
-        this(connection, isAmqpTypeEncoded, null);
-    }
-
-    private AmqpJmsObjectMessageFacade(AmqpConnection connection, boolean isAmqpTypeEncoded, JmsDeserializationPolicy deserializationPolicy) {
-        super(connection);
-        this.deserializationPolicy = deserializationPolicy;
-
-        setMessageAnnotation(JMS_MSG_TYPE, JMS_OBJECT_MESSAGE);
-        initDelegate(isAmqpTypeEncoded, null);
+    @Override
+    public void initialize(AmqpConnection connection) {
+        super.initialize(connection);
+        initDelegate(connection.isObjectMessageUsesAmqpTypes());
     }
 
-    /**
-     * Creates a new Facade around an incoming AMQP Message for dispatch to the
-     * JMS Consumer instance.
-     *
-     * @param consumer
-     *        the consumer that received this message.
-     * @param message
-     *        the incoming Message instance that is being wrapped.
-     * @param messageBytes
-     *        a copy of the raw bytes of the incoming message.
-     */
-    public AmqpJmsObjectMessageFacade(AmqpConsumer consumer, Message message, ByteBuf messageBytes) {
-        super(consumer, message);
+    @Override
+    public void initialize(AmqpConsumer consumer) {
+        super.initialize(consumer);
         deserializationPolicy = consumer.getResourceInfo().getDeserializationPolicy();
-
-        boolean javaSerialized = AmqpMessageSupport.SERIALIZED_JAVA_OBJECT_CONTENT_TYPE.equals(message.getContentType());
-        initDelegate(!javaSerialized, messageBytes);
+        boolean javaSerialized = AmqpMessageSupport.SERIALIZED_JAVA_OBJECT_CONTENT_TYPE.equals(getContentType());
+        initDelegate(!javaSerialized);
     }
 
     /**
@@ -96,7 +67,9 @@ public class AmqpJmsObjectMessageFacade extends AmqpJmsMessageFacade implements
 
     @Override
     public AmqpJmsObjectMessageFacade copy() throws JMSException {
-        AmqpJmsObjectMessageFacade copy = new AmqpJmsObjectMessageFacade(connection, isAmqpTypedEncoding(), deserializationPolicy);
+        AmqpJmsObjectMessageFacade copy = new AmqpJmsObjectMessageFacade();
+        copy.deserializationPolicy = deserializationPolicy;
+        copy.initDelegate(isAmqpTypedEncoding());
         copyInto(copy);
         try {
             delegate.copyInto(copy.delegate);
@@ -135,6 +108,11 @@ public class AmqpJmsObjectMessageFacade extends AmqpJmsMessageFacade implements
         delegate.onSend();
     }
 
+    @Override
+    public JmsObjectMessage asJmsMessage() {
+        return new JmsObjectMessage(this);
+    }
+
     void setUseAmqpTypedEncoding(boolean useAmqpTypedEncoding) throws JMSException {
         if (useAmqpTypedEncoding != delegate.isAmqpTypeEncoded()) {
             try {
@@ -142,9 +120,9 @@ public class AmqpJmsObjectMessageFacade extends AmqpJmsMessageFacade implements
 
                 AmqpObjectTypeDelegate newDelegate = null;
                 if (useAmqpTypedEncoding) {
-                    newDelegate = new AmqpTypedObjectDelegate(this, null);
+                    newDelegate = new AmqpTypedObjectDelegate(this);
                 } else {
-                    newDelegate = new AmqpSerializedObjectDelegate(this, null, deserializationPolicy);
+                    newDelegate = new AmqpSerializedObjectDelegate(this, deserializationPolicy);
                 }
 
                 newDelegate.setObject(existingObject);
@@ -156,11 +134,11 @@ public class AmqpJmsObjectMessageFacade extends AmqpJmsMessageFacade implements
         }
     }
 
-    private void initDelegate(boolean useAmqpTypes, ByteBuf messageBytes) {
+    private void initDelegate(boolean useAmqpTypes) {
         if (!useAmqpTypes) {
-            delegate = new AmqpSerializedObjectDelegate(this, messageBytes, deserializationPolicy);
+            delegate = new AmqpSerializedObjectDelegate(this, deserializationPolicy);
         } else {
-            delegate = new AmqpTypedObjectDelegate(this, messageBytes);
+            delegate = new AmqpTypedObjectDelegate(this);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsStreamMessageFacade.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsStreamMessageFacade.java b/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsStreamMessageFacade.java
index 64f1fbd..4843065 100644
--- a/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsStreamMessageFacade.java
+++ b/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsStreamMessageFacade.java
@@ -16,7 +16,6 @@
  */
 package org.apache.qpid.jms.provider.amqp.message;
 
-import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.JMS_MSG_TYPE;
 import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.JMS_STREAM_MESSAGE;
 
 import java.util.ArrayList;
@@ -25,14 +24,12 @@ import java.util.List;
 
 import javax.jms.MessageEOFException;
 
+import org.apache.qpid.jms.message.JmsStreamMessage;
 import org.apache.qpid.jms.message.facade.JmsStreamMessageFacade;
-import org.apache.qpid.jms.provider.amqp.AmqpConnection;
-import org.apache.qpid.jms.provider.amqp.AmqpConsumer;
 import org.apache.qpid.proton.amqp.Binary;
 import org.apache.qpid.proton.amqp.messaging.AmqpSequence;
 import org.apache.qpid.proton.amqp.messaging.AmqpValue;
 import org.apache.qpid.proton.amqp.messaging.Section;
-import org.apache.qpid.proton.message.Message;
 
 /**
  * Wrapper around an AMQP Message instance that will be treated as a JMS StreamMessage
@@ -43,61 +40,11 @@ public class AmqpJmsStreamMessageFacade extends AmqpJmsMessageFacade implements
     private List<Object> list;
     private int position = 0;
 
-    /**
-     * Create a new facade ready for sending.
-     *
-     * @param connection
-     *        the AmqpConnection that under which this facade was created.
-     */
-    public AmqpJmsStreamMessageFacade(AmqpConnection connection) {
-        super(connection);
-        list = initializeEmptyBodyList(true);
-        setMessageAnnotation(JMS_MSG_TYPE, JMS_STREAM_MESSAGE);
-    }
-
-    /**
-     * Creates a new Facade around an incoming AMQP Message for dispatch to the
-     * JMS Consumer instance.
-     *
-     * @param consumer
-     *        the consumer that received this message.
-     * @param message
-     *        the incoming Message instance that is being wrapped.
-     */
-    @SuppressWarnings("unchecked")
-    public AmqpJmsStreamMessageFacade(AmqpConsumer consumer, Message message) {
-        super(consumer, message);
-
-        Section body = getAmqpMessage().getBody();
-        if (body == null) {
-            list = initializeEmptyBodyList(true);
-        } else if (body instanceof AmqpValue) {
-            Object value = ((AmqpValue) body).getValue();
-
-            if (value == null) {
-                list = initializeEmptyBodyList(false);
-            } else if (value instanceof List) {
-                list = (List<Object>) value;
-            } else {
-                throw new IllegalStateException("Unexpected amqp-value body content type: " + value.getClass().getSimpleName());
-            }
-        } else if (body instanceof AmqpSequence) {
-            List<?> value = ((AmqpSequence) body).getValue();
-
-            if (value == null) {
-                list = initializeEmptyBodyList(true);
-            } else {
-                list = (List<Object>) value;
-            }
-        } else {
-            throw new IllegalStateException("Unexpected message body type: " + body.getClass().getSimpleName());
-        }
-    }
-
     @Override
     public AmqpJmsStreamMessageFacade copy() {
-        AmqpJmsStreamMessageFacade copy = new AmqpJmsStreamMessageFacade(connection);
+        AmqpJmsStreamMessageFacade copy = new AmqpJmsStreamMessageFacade();
         copyInto(copy);
+        copy.initializeEmptyBodyList(getBody() instanceof AmqpSequence);
         copy.list.addAll(list);
         return copy;
     }
@@ -166,13 +113,53 @@ public class AmqpJmsStreamMessageFacade extends AmqpJmsMessageFacade implements
         return !list.isEmpty();
     }
 
+    @Override
+    public JmsStreamMessage asJmsMessage() {
+        return new JmsStreamMessage(this);
+    }
+
+    @Override
+    protected void initializeEmptyBody() {
+        list = initializeEmptyBodyList(true);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    void setBody(Section body) {
+        if (body == null) {
+            list = initializeEmptyBodyList(true);
+        } else if (body instanceof AmqpValue) {
+            Object value = ((AmqpValue) body).getValue();
+
+            if (value == null) {
+                list = initializeEmptyBodyList(false);
+            } else if (value instanceof List) {
+                list = (List<Object>) value;
+                super.setBody(body);
+            } else {
+                throw new IllegalStateException("Unexpected amqp-value body content type: " + value.getClass().getSimpleName());
+            }
+        } else if (body instanceof AmqpSequence) {
+            List<?> value = ((AmqpSequence) body).getValue();
+
+            if (value == null) {
+                list = initializeEmptyBodyList(true);
+            } else {
+                list = (List<Object>) value;
+                super.setBody(body);
+            }
+        } else {
+            throw new IllegalStateException("Unexpected message body type: " + body.getClass().getSimpleName());
+        }
+    }
+
     private List<Object> initializeEmptyBodyList(boolean useSequenceBody) {
         List<Object> emptyList = new ArrayList<Object>();
 
         if (useSequenceBody) {
-            message.setBody(new AmqpSequence(emptyList));
+            setBody(new AmqpSequence(emptyList));
         } else {
-            message.setBody(new AmqpValue(emptyList));
+            setBody(new AmqpValue(emptyList));
         }
 
         return emptyList;

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsTextMessageFacade.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsTextMessageFacade.java b/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsTextMessageFacade.java
index 44ed9e6..5595362 100644
--- a/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsTextMessageFacade.java
+++ b/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpJmsTextMessageFacade.java
@@ -16,7 +16,6 @@
  */
 package org.apache.qpid.jms.provider.amqp.message;
 
-import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.JMS_MSG_TYPE;
 import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.JMS_TEXT_MESSAGE;
 
 import java.nio.ByteBuffer;
@@ -28,14 +27,12 @@ import java.nio.charset.StandardCharsets;
 import javax.jms.JMSException;
 
 import org.apache.qpid.jms.exceptions.JmsExceptionSupport;
+import org.apache.qpid.jms.message.JmsTextMessage;
 import org.apache.qpid.jms.message.facade.JmsTextMessageFacade;
-import org.apache.qpid.jms.provider.amqp.AmqpConnection;
-import org.apache.qpid.jms.provider.amqp.AmqpConsumer;
 import org.apache.qpid.proton.amqp.Binary;
 import org.apache.qpid.proton.amqp.messaging.AmqpValue;
 import org.apache.qpid.proton.amqp.messaging.Data;
 import org.apache.qpid.proton.amqp.messaging.Section;
-import org.apache.qpid.proton.message.Message;
 
 /**
  * Wrapper around an AMQP Message instance that will be treated as a JMS TextMessage
@@ -45,32 +42,11 @@ public class AmqpJmsTextMessageFacade extends AmqpJmsMessageFacade implements Jm
 
     private final Charset charset;
 
-    /**
-     * Create a new AMQP Message facade ready for sending.
-     *
-     * @param connection
-     *        the AmqpConnection that under which this facade was created.
-     */
-    public AmqpJmsTextMessageFacade(AmqpConnection connection) {
-        super(connection);
-        setMessageAnnotation(JMS_MSG_TYPE, JMS_TEXT_MESSAGE);
-        setText(null);
-        charset = StandardCharsets.UTF_8;
+    public AmqpJmsTextMessageFacade() {
+        this(StandardCharsets.UTF_8);
     }
 
-    /**
-     * Creates a new Facade around an incoming AMQP Message for dispatch to the
-     * JMS Consumer instance.
-     *
-     * @param consumer
-     *        the consumer that received this message.
-     * @param message
-     *        the incoming Message instance that is being wrapped.
-     * @param charset
-     *        the character set to use when decoding the text when the body is a Data section
-     */
-    public AmqpJmsTextMessageFacade(AmqpConsumer consumer, Message message, Charset charset) {
-        super(consumer, message);
+    AmqpJmsTextMessageFacade(Charset charset) {
         this.charset = charset;
     }
 
@@ -84,7 +60,7 @@ public class AmqpJmsTextMessageFacade extends AmqpJmsMessageFacade implements Jm
 
     @Override
     public AmqpJmsTextMessageFacade copy() throws JMSException {
-        AmqpJmsTextMessageFacade copy = new AmqpJmsTextMessageFacade(connection);
+        AmqpJmsTextMessageFacade copy = new AmqpJmsTextMessageFacade();
         copyInto(copy);
         copy.setText(getText());
         return copy;
@@ -92,7 +68,7 @@ public class AmqpJmsTextMessageFacade extends AmqpJmsMessageFacade implements Jm
 
     @Override
     public String getText() throws JMSException {
-        Section body = getAmqpMessage().getBody();
+        Section body = getBody();
 
         if (body == null) {
             return null;
@@ -126,13 +102,12 @@ public class AmqpJmsTextMessageFacade extends AmqpJmsMessageFacade implements Jm
 
     @Override
     public void setText(String value) {
-        AmqpValue body = new AmqpValue(value);
-        getAmqpMessage().setBody(body);
+        setBody(new AmqpValue(value));
     }
 
     @Override
     public void clearBody() {
-        setText(null);
+        setBody(new AmqpValue(null));
     }
 
     @Override
@@ -144,7 +119,17 @@ public class AmqpJmsTextMessageFacade extends AmqpJmsMessageFacade implements Jm
         }
     }
 
+    @Override
+    public JmsTextMessage asJmsMessage() {
+        return new JmsTextMessage(this);
+    }
+
     Charset getCharset() {
         return charset;
     }
+
+    @Override
+    protected void initializeEmptyBody() {
+        setBody(new AmqpValue(null));
+    }
 }

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpMessageSupport.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpMessageSupport.java b/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpMessageSupport.java
index 40987e1..702870b 100644
--- a/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpMessageSupport.java
+++ b/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpMessageSupport.java
@@ -19,6 +19,7 @@ package org.apache.qpid.jms.provider.amqp.message;
 import java.util.Map;
 
 import org.apache.qpid.proton.amqp.Symbol;
+import org.apache.qpid.proton.amqp.messaging.MessageAnnotations;
 import org.apache.qpid.proton.message.Message;
 
 import io.netty.buffer.ByteBuf;
@@ -117,14 +118,14 @@ public final class AmqpMessageSupport {
      *
      * @param key
      *        the String key to use to lookup an annotation.
-     * @param message
-     *        the AMQP message object that is being examined.
+     * @param messageAnnotations
+     *        the AMQP message annotations object that is being examined.
      *
      * @return the given annotation value or null if not present in the message.
      */
-    public static Object getMessageAnnotation(String key, Message message) {
-        if (message != null && message.getMessageAnnotations() != null) {
-            Map<Symbol, Object> annotations = message.getMessageAnnotations().getValue();
+    public static Object getMessageAnnotation(String key, MessageAnnotations messageAnnotations) {
+        if (messageAnnotations != null && messageAnnotations.getValue() != null) {
+            Map<Symbol, Object> annotations = messageAnnotations.getValue();
             return annotations.get(AmqpMessageSupport.getSymbol(key));
         }
 
@@ -138,16 +139,18 @@ public final class AmqpMessageSupport {
      *
      * @param contentType
      *        content type string to compare against, or null if none
-     * @param message
-     *        the AMQP message object that is being examined.
+     * @param messageContentType
+     *        the content type value read from an AMQP message object.
      *
      * @return true if content type matches
      */
-    public static boolean isContentType(String contentType, Message message) {
+    public static boolean isContentType(String contentType, Symbol messageContentType) {
         if (contentType == null) {
-            return message.getContentType() == null;
+            return messageContentType == null;
+        } else if (messageContentType == null) {
+            return false;
         } else {
-            return contentType.equals(message.getContentType());
+            return contentType.equals(messageContentType.toString());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpSerializedObjectDelegate.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpSerializedObjectDelegate.java b/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpSerializedObjectDelegate.java
index ec73ba9..b8cdbb4 100644
--- a/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpSerializedObjectDelegate.java
+++ b/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpSerializedObjectDelegate.java
@@ -17,14 +17,12 @@
 package org.apache.qpid.jms.provider.amqp.message;
 
 import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.SERIALIZED_JAVA_OBJECT_CONTENT_TYPE;
-import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.decodeMessage;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.ObjectOutputStream;
 import java.io.Serializable;
-import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.qpid.jms.policy.JmsDeserializationPolicy;
 import org.apache.qpid.jms.util.ClassLoadingAwareObjectInputStream;
@@ -32,9 +30,6 @@ import org.apache.qpid.jms.util.ClassLoadingAwareObjectInputStream.TrustedClassF
 import org.apache.qpid.proton.amqp.Binary;
 import org.apache.qpid.proton.amqp.messaging.Data;
 import org.apache.qpid.proton.amqp.messaging.Section;
-import org.apache.qpid.proton.message.Message;
-
-import io.netty.buffer.ByteBuf;
 
 /**
  * Wrapper around an AMQP Message instance that will be treated as a JMS ObjectMessage
@@ -56,10 +51,7 @@ public class AmqpSerializedObjectDelegate implements AmqpObjectTypeDelegate, Tru
     }
 
     private final AmqpJmsMessageFacade parent;
-    private final Message message;
-    private final AtomicReference<Section> cachedReceivedBody = new AtomicReference<Section>();
     private final JmsDeserializationPolicy deserializationPolicy;
-    private ByteBuf messageBytes;
     private boolean localContent;
 
     /**
@@ -67,23 +59,14 @@ public class AmqpSerializedObjectDelegate implements AmqpObjectTypeDelegate, Tru
      *
      * @param parent
      *        the AMQP message facade instance where the object is to be stored / read.
-     * @param messageBytes
-     *        the raw bytes that comprise the message when it was received.
      * @param deserializationPolicy
      *        the JmsDeserializationPolicy that is used to validate the security of message
      *        content, may be null (e.g on new outgoing messages).
      */
-    public AmqpSerializedObjectDelegate(AmqpJmsMessageFacade parent, ByteBuf messageBytes, JmsDeserializationPolicy deserializationPolicy) {
+    public AmqpSerializedObjectDelegate(AmqpJmsMessageFacade parent, JmsDeserializationPolicy deserializationPolicy) {
         this.parent = parent;
-        this.message = parent.getAmqpMessage();
-        this.message.setContentType(SERIALIZED_JAVA_OBJECT_CONTENT_TYPE);
-        this.messageBytes = messageBytes;
+        this.parent.setContentType(SERIALIZED_JAVA_OBJECT_CONTENT_TYPE);
         this.deserializationPolicy = deserializationPolicy;
-
-        // Cache the body so the first access can grab it without extra work.
-        if (messageBytes != null) {
-            cachedReceivedBody.set(message.getBody());
-        }
     }
 
     private static byte[] getSerializedBytes(Serializable value) throws IOException {
@@ -100,31 +83,24 @@ public class AmqpSerializedObjectDelegate implements AmqpObjectTypeDelegate, Tru
 
     @Override
     public Serializable getObject() throws IOException, ClassNotFoundException {
-        Binary bin = null;
-
-        Section body = cachedReceivedBody.getAndSet(null);
-        if (body == null) {
-            if (messageBytes != null) {
-                body = decodeMessage(messageBytes).getBody();
-            } else {
-                body = message.getBody();
-            }
-        }
+        Binary binary = null;
+
+        Section body = parent.getBody();
 
         if (body == null || body == NULL_OBJECT_BODY) {
             return null;
         } else if (body instanceof Data) {
-            bin = ((Data) body).getValue();
+            binary = ((Data) body).getValue();
         } else {
             throw new IllegalStateException("Unexpected body type: " + body.getClass().getSimpleName());
         }
 
-        if (bin == null) {
+        if (binary == null) {
             return null;
         } else {
             Serializable serialized = null;
 
-            try (ByteArrayInputStream bais = new ByteArrayInputStream(bin.getArray(), bin.getArrayOffset(), bin.getLength());
+            try (ByteArrayInputStream bais = new ByteArrayInputStream(binary.getArray(), binary.getArrayOffset(), binary.getLength());
                  ClassLoadingAwareObjectInputStream objIn = new ClassLoadingAwareObjectInputStream(bais, this)) {
 
                 serialized = (Serializable) objIn.readObject();
@@ -136,24 +112,21 @@ public class AmqpSerializedObjectDelegate implements AmqpObjectTypeDelegate, Tru
 
     @Override
     public void setObject(Serializable value) throws IOException {
-        cachedReceivedBody.set(null);
-
         if (value == null) {
-            message.setBody(NULL_OBJECT_BODY);
+            parent.setBody(NULL_OBJECT_BODY);
         } else {
             byte[] bytes = getSerializedBytes(value);
-            message.setBody(new Data(new Binary(bytes)));
+            parent.setBody(new Data(new Binary(bytes)));
         }
 
-        messageBytes = null;
         localContent = true;
     }
 
     @Override
     public void onSend() {
-        message.setContentType(SERIALIZED_JAVA_OBJECT_CONTENT_TYPE);
-        if (message.getBody() == null) {
-            message.setBody(NULL_OBJECT_BODY);
+        parent.setContentType(SERIALIZED_JAVA_OBJECT_CONTENT_TYPE);
+        if (parent.getBody() == null) {
+            parent.setBody(NULL_OBJECT_BODY);
         }
     }
 
@@ -164,20 +137,11 @@ public class AmqpSerializedObjectDelegate implements AmqpObjectTypeDelegate, Tru
         } else {
             AmqpSerializedObjectDelegate target = (AmqpSerializedObjectDelegate) copy;
 
-            // Swap our cached value to the copy, we will just decode it if we need it.
-            target.cachedReceivedBody.set(cachedReceivedBody.getAndSet(null));
-
-            // If we have the original bytes just copy those and let the next get
-            // decode them into the payload, otherwise we need to do a deep copy.
-            if (messageBytes != null) {
-                target.messageBytes = messageBytes.copy();
-            }
-
             target.localContent = localContent;
 
             // Copy the already encoded message body if it exists, subsequent gets
             // will deserialize the data so no mutations can occur.
-            target.message.setBody(message.getBody());
+            target.parent.setBody(parent.getBody());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpTypedObjectDelegate.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpTypedObjectDelegate.java b/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpTypedObjectDelegate.java
index 1296eaa..88b7b9d 100644
--- a/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpTypedObjectDelegate.java
+++ b/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpTypedObjectDelegate.java
@@ -16,20 +16,15 @@
  */
 package org.apache.qpid.jms.provider.amqp.message;
 
-import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.decodeMessage;
-import static org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport.encodeMessage;
-
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.qpid.proton.amqp.messaging.AmqpSequence;
 import org.apache.qpid.proton.amqp.messaging.AmqpValue;
 import org.apache.qpid.proton.amqp.messaging.Data;
 import org.apache.qpid.proton.amqp.messaging.Section;
-import org.apache.qpid.proton.message.Message;
 
 import io.netty.buffer.ByteBuf;
 
@@ -41,39 +36,31 @@ public class AmqpTypedObjectDelegate implements AmqpObjectTypeDelegate {
 
     static final AmqpValue NULL_OBJECT_BODY = new AmqpValue(null);
 
-    private final Message message;
-    private final AtomicReference<Section> cachedReceivedBody = new AtomicReference<Section>();
-    private ByteBuf messageBytes;
+    private ByteBuf encodedBody;
+    private final AmqpJmsMessageFacade parent;
 
     /**
      * Create a new delegate that uses Java serialization to store the message content.
      *
      * @param parent
      *        the AMQP message facade instance where the object is to be stored / read.
-     * @param messageBytes
-     *        the raw bytes that comprise the AMQP message that was received.
      */
-    public AmqpTypedObjectDelegate(AmqpJmsMessageFacade parent, ByteBuf messageBytes) {
-        this.message = parent.getAmqpMessage();
-        this.message.setContentType(null);
-        this.messageBytes = messageBytes;
-
-        // Cache the body so the first access can grab it without extra work.
-        if (messageBytes != null) {
-            cachedReceivedBody.set(message.getBody());
+    public AmqpTypedObjectDelegate(AmqpJmsMessageFacade parent) {
+        this.parent = parent;
+        this.parent.setContentType(null);
+
+        // Create a duplicate of the message body for decode on read attempts
+        if (parent.getBody() != null) {
+            encodedBody = AmqpCodec.encode(parent.getBody());
         }
     }
 
     @Override
     public Serializable getObject() throws IOException, ClassNotFoundException {
-        Section body = cachedReceivedBody.getAndSet(null);
+        Section body = null;
 
-        if (body == null) {
-            if (messageBytes != null) {
-                body = decodeMessage(messageBytes).getBody();
-            } else {
-                body = message.getBody();
-            }
+        if (encodedBody != null) {
+            body = AmqpCodec.decode(encodedBody);
         }
 
         if (body == null) {
@@ -98,20 +85,15 @@ public class AmqpTypedObjectDelegate implements AmqpObjectTypeDelegate {
 
     @Override
     public void setObject(Serializable value) throws IOException {
-        cachedReceivedBody.set(null);
-
         if (value == null) {
-            message.setBody(NULL_OBJECT_BODY);
-            messageBytes = null;
+            parent.setBody(NULL_OBJECT_BODY);
+            encodedBody = null;
         } else if (isSupportedAmqpValueObjectType(value)) {
-            Message transfer = Message.Factory.create();
-
             // Exchange the incoming body value for one that is created from encoding
             // and decoding the value. Save the bytes for subsequent getObject and
             // copyInto calls to use.
-            transfer.setBody(new AmqpValue(value));
-            messageBytes = encodeMessage(transfer);
-            transfer = decodeMessage(messageBytes);
+            encodedBody = AmqpCodec.encode(new AmqpValue(value));
+            Section decodedBody = AmqpCodec.decode(encodedBody);
 
             // This step requires a heavy-weight operation of both encoding and decoding the
             // incoming body value in order to create a copy such that changes to the original
@@ -120,7 +102,7 @@ public class AmqpTypedObjectDelegate implements AmqpObjectTypeDelegate {
             // proton such that we can encode the body and use those bytes directly on the
             // message as it is being sent.
 
-            message.setBody(transfer.getBody());
+            parent.setBody(decodedBody);
         } else {
             // TODO: Data and AmqpSequence?
             throw new IllegalArgumentException("Encoding this object type with the AMQP type system is not supported: " + value.getClass().getName());
@@ -129,9 +111,9 @@ public class AmqpTypedObjectDelegate implements AmqpObjectTypeDelegate {
 
     @Override
     public void onSend() {
-        message.setContentType(null);
-        if (message.getBody() == null) {
-            message.setBody(NULL_OBJECT_BODY);
+        parent.setContentType(null);
+        if (parent.getBody() == null) {
+            parent.setBody(NULL_OBJECT_BODY);
         }
     }
 
@@ -142,23 +124,17 @@ public class AmqpTypedObjectDelegate implements AmqpObjectTypeDelegate {
         } else {
             AmqpTypedObjectDelegate target = (AmqpTypedObjectDelegate) copy;
 
-            // Swap our cached value (if any) to the copy, we will just decode it if we need it later.
-            target.cachedReceivedBody.set(cachedReceivedBody.getAndSet(null));
-
-            if (messageBytes != null) {
-                // If we have the original bytes just copy those and let the next get
-                // decode them into the payload (or for the copy, use the cached
-                // body if it was swapped above).
-                target.messageBytes = messageBytes.copy();
+            // If there ever was a body then we will have a snapshot of it and we can
+            // be sure that our state is correct.
+            if (encodedBody != null) {
+                // If we have any body bytes just duplicate those and let the next get
+                // decode them into the returned object payload value.
+                target.encodedBody = encodedBody.duplicate();
 
                 // Internal message body copy to satisfy sends. This is safe since the body was set
                 // from a copy (decoded from the bytes) to ensure it is a snapshot. Also safe for
                 // gets as they will use the message bytes (or cached body if set) to return the object.
-                target.message.setBody(message.getBody());
-            } else {
-                // We have to deep get/set copy here, otherwise a get might return
-                // the object value carried by the original version.
-                copy.setObject(getObject());
+                target.parent.setBody(parent.getBody());
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpWritableBuffer.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpWritableBuffer.java b/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpWritableBuffer.java
new file mode 100644
index 0000000..7088b5f
--- /dev/null
+++ b/qpid-jms-client/src/main/java/org/apache/qpid/jms/provider/amqp/message/AmqpWritableBuffer.java
@@ -0,0 +1,113 @@
+/*
+ * 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.qpid.jms.provider.amqp.message;
+
+import java.nio.ByteBuffer;
+
+import org.apache.qpid.proton.codec.WritableBuffer;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+
+/**
+ * Writable Buffer implementation based on a Netty ByteBuf
+ */
+public class AmqpWritableBuffer implements WritableBuffer {
+
+    public ByteBuf nettyBuffer;
+
+    public AmqpWritableBuffer() {
+        nettyBuffer = Unpooled.buffer(1024);
+    }
+
+    public AmqpWritableBuffer(ByteBuf buffer) {
+        nettyBuffer = buffer;
+    }
+
+    public ByteBuf getBuffer() {
+        return nettyBuffer;
+    }
+
+    @Override
+    public void put(byte b) {
+        nettyBuffer.writeByte(b);
+    }
+
+    @Override
+    public void putFloat(float f) {
+        nettyBuffer.writeFloat(f);
+    }
+
+    @Override
+    public void putDouble(double d) {
+        nettyBuffer.writeDouble(d);
+    }
+
+    @Override
+    public void put(byte[] src, int offset, int length) {
+        nettyBuffer.writeBytes(src, offset, length);
+    }
+
+    @Override
+    public void put(ByteBuffer payload) {
+        nettyBuffer.writeBytes(payload);
+    }
+
+    public void put(ByteBuf payload) {
+        nettyBuffer.writeBytes(payload);
+    }
+
+    @Override
+    public void putShort(short s) {
+        nettyBuffer.writeShort(s);
+    }
+
+    @Override
+    public void putInt(int i) {
+        nettyBuffer.writeInt(i);
+    }
+
+    @Override
+    public void putLong(long l) {
+        nettyBuffer.writeLong(l);
+    }
+
+    @Override
+    public boolean hasRemaining() {
+        return nettyBuffer.writerIndex() < nettyBuffer.capacity();
+    }
+
+    @Override
+    public int remaining() {
+        return nettyBuffer.capacity() - nettyBuffer.writerIndex();
+    }
+
+    @Override
+    public int position() {
+        return nettyBuffer.writerIndex();
+    }
+
+    @Override
+    public void position(int position) {
+        nettyBuffer.writerIndex(position);
+    }
+
+    @Override
+    public int limit() {
+        return nettyBuffer.capacity();
+    }
+}

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/BytesMessageIntegrationTest.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/BytesMessageIntegrationTest.java b/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/BytesMessageIntegrationTest.java
index f9188c5..3254c87 100644
--- a/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/BytesMessageIntegrationTest.java
+++ b/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/BytesMessageIntegrationTest.java
@@ -22,6 +22,7 @@ import static org.hamcrest.Matchers.equalTo;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.ByteArrayOutputStream;
 import java.io.DataOutputStream;
@@ -29,14 +30,17 @@ import java.io.IOException;
 import java.util.Arrays;
 
 import javax.jms.BytesMessage;
+import javax.jms.CompletionListener;
 import javax.jms.Connection;
 import javax.jms.JMSException;
 import javax.jms.Message;
 import javax.jms.MessageConsumer;
+import javax.jms.MessageNotWriteableException;
 import javax.jms.MessageProducer;
 import javax.jms.Queue;
 import javax.jms.Session;
 
+import org.apache.qpid.jms.JmsConnection;
 import org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport;
 import org.apache.qpid.jms.test.QpidJmsTestCase;
 import org.apache.qpid.jms.test.testpeer.TestAmqpPeer;
@@ -383,4 +387,105 @@ public class BytesMessageIntegrationTest extends QpidJmsTestCase {
             testPeer.waitForAllHandlersToComplete(3000);
         }
     }
+
+    @Test(timeout = 20000)
+    public void testAsyncSendMarksBytesMessageReadOnly() throws Exception {
+        try(TestAmqpPeer testPeer = new TestAmqpPeer();) {
+            JmsConnection connection = (JmsConnection) testFixture.establishConnecton(testPeer);
+            connection.setSendTimeout(15000);
+
+            testPeer.expectBegin();
+
+            Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
+            String queueName = "myQueue";
+            Queue queue = session.createQueue(queueName);
+
+            BytesMessage message = session.createBytesMessage();
+            TransferPayloadCompositeMatcher messageMatcher = new TransferPayloadCompositeMatcher();
+
+            // Expect the producer to attach and grant it some credit, it should send
+            // a transfer which we will not send any response so that we can check that
+            // the inflight message is read-only
+            testPeer.expectSenderAttach();
+            testPeer.expectTransferButDoNotRespond(messageMatcher);
+            testPeer.expectClose();
+
+            MessageProducer producer = session.createProducer(queue);
+            TestJmsCompletionListener listener = new TestJmsCompletionListener();
+
+            try {
+                producer.send(message, listener);
+            } catch (Throwable error) {
+                fail("Send should not fail for async.");
+            }
+
+            try {
+                message.setJMSCorrelationID("test");
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSCorrelationIDAsBytes(new byte[]{});
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSDeliveryMode(0);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSDestination(queue);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSExpiration(0);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSMessageID(queueName);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSPriority(0);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSRedelivered(false);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSReplyTo(queue);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSTimestamp(0);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSType(queueName);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setStringProperty("test", "test");
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.writeBoolean(true);
+                fail("Message should not be writable after a send.");
+            } catch (MessageNotWriteableException mnwe) {}
+
+            connection.close();
+
+            testPeer.waitForAllHandlersToComplete(1000);
+        }
+    }
+
+    private class TestJmsCompletionListener implements CompletionListener {
+
+        @Override
+        public void onCompletion(Message message) {
+        }
+
+        @Override
+        public void onException(Message message, Exception exception) {
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/657747b7/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/MapMessageIntegrationTest.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/MapMessageIntegrationTest.java b/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/MapMessageIntegrationTest.java
index 10bcffd..02ca520 100644
--- a/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/MapMessageIntegrationTest.java
+++ b/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/MapMessageIntegrationTest.java
@@ -29,15 +29,18 @@ import java.util.Arrays;
 import java.util.LinkedHashMap;
 import java.util.Map;
 
+import javax.jms.CompletionListener;
 import javax.jms.Connection;
 import javax.jms.MapMessage;
 import javax.jms.Message;
 import javax.jms.MessageConsumer;
 import javax.jms.MessageFormatException;
+import javax.jms.MessageNotWriteableException;
 import javax.jms.MessageProducer;
 import javax.jms.Queue;
 import javax.jms.Session;
 
+import org.apache.qpid.jms.JmsConnection;
 import org.apache.qpid.jms.provider.amqp.message.AmqpMessageSupport;
 import org.apache.qpid.jms.test.QpidJmsTestCase;
 import org.apache.qpid.jms.test.testpeer.TestAmqpPeer;
@@ -263,4 +266,159 @@ public class MapMessageIntegrationTest extends QpidJmsTestCase {
             testPeer.waitForAllHandlersToComplete(3000);
         }
     }
+
+    @Test(timeout = 20000)
+    public void testSendMapMessageIsWritable() throws Exception {
+        try (TestAmqpPeer testPeer = new TestAmqpPeer();) {
+            Connection connection = testFixture.establishConnecton(testPeer);
+            testPeer.expectBegin();
+            testPeer.expectSenderAttach();
+
+            Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
+            Queue queue = session.createQueue("myQueue");
+            MessageProducer producer = session.createProducer(queue);
+
+            String myIntKey = "myInt";
+            int myInt = Integer.MAX_VALUE;
+            String myStringKey = "myString";
+            String myString = myStringKey;
+
+            // Prepare a MapMessage to send to the test peer to send
+            MapMessage mapMessage = session.createMapMessage();
+
+            mapMessage.setString(myStringKey, myString);
+
+            // prepare a matcher for the test peer to use to receive and verify the message
+            Map<String, Object> map = new LinkedHashMap<String, Object>();
+            map.put(myStringKey, myString);
+
+            MessageHeaderSectionMatcher headersMatcher = new MessageHeaderSectionMatcher(true).withDurable(equalTo(true));
+            MessageAnnotationsSectionMatcher msgAnnotationsMatcher = new MessageAnnotationsSectionMatcher(true);
+            msgAnnotationsMatcher.withEntry(Symbol.valueOf(AmqpMessageSupport.JMS_MSG_TYPE), equalTo(AmqpMessageSupport.JMS_MAP_MESSAGE));
+            MessagePropertiesSectionMatcher propertiesMatcher = new MessagePropertiesSectionMatcher(true);
+            TransferPayloadCompositeMatcher messageMatcher = new TransferPayloadCompositeMatcher();
+            messageMatcher.setHeadersMatcher(headersMatcher);
+            messageMatcher.setMessageAnnotationsMatcher(msgAnnotationsMatcher);
+            messageMatcher.setPropertiesMatcher(propertiesMatcher);
+            messageMatcher.setMessageContentMatcher(new EncodedAmqpValueMatcher(map));
+
+            testPeer.expectTransfer(messageMatcher);
+
+            // send the message
+            producer.send(mapMessage);
+
+            // Update the message and matcher and send again
+            mapMessage.setInt(myIntKey, myInt);
+            map.put(myIntKey, myInt);
+            testPeer.expectTransfer(messageMatcher);
+            testPeer.expectClose();
+
+            producer.send(mapMessage);
+
+            connection.close();
+
+            testPeer.waitForAllHandlersToComplete(3000);
+        }
+    }
+
+    @Test(timeout = 20000)
+    public void testAsyncSendMarksMapMessageReadOnly() throws Exception {
+        try(TestAmqpPeer testPeer = new TestAmqpPeer();) {
+            JmsConnection connection = (JmsConnection) testFixture.establishConnecton(testPeer);
+            connection.setSendTimeout(15000);
+
+            testPeer.expectBegin();
+
+            Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
+            String queueName = "myQueue";
+            Queue queue = session.createQueue(queueName);
+
+            MapMessage message = session.createMapMessage();
+            TransferPayloadCompositeMatcher messageMatcher = new TransferPayloadCompositeMatcher();
+
+            // Expect the producer to attach and grant it some credit, it should send
+            // a transfer which we will not send any response so that we can check that
+            // the inflight message is read-only
+            testPeer.expectSenderAttach();
+            testPeer.expectTransferButDoNotRespond(messageMatcher);
+            testPeer.expectClose();
+
+            MessageProducer producer = session.createProducer(queue);
+            TestJmsCompletionListener listener = new TestJmsCompletionListener();
+
+            try {
+                producer.send(message, listener);
+            } catch (Throwable error) {
+                fail("Send should not fail for async.");
+            }
+
+            try {
+                message.setJMSCorrelationID("test");
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSCorrelationIDAsBytes(new byte[]{});
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSDeliveryMode(0);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSDestination(queue);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSExpiration(0);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSMessageID(queueName);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSPriority(0);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSRedelivered(false);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSReplyTo(queue);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSTimestamp(0);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setJMSType(queueName);
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setStringProperty("test", "test");
+                fail("Should not be able to set properties on inflight message");
+            } catch (MessageNotWriteableException mnwe) {}
+            try {
+                message.setString("test", "test");
+                fail("Message should not be writable after a send.");
+            } catch (MessageNotWriteableException mnwe) {}
+
+            connection.close();
+
+            testPeer.waitForAllHandlersToComplete(1000);
+        }
+    }
+
+    private class TestJmsCompletionListener implements CompletionListener {
+
+        @Override
+        public void onCompletion(Message message) {
+        }
+
+        @Override
+        public void onException(Message message, Exception exception) {
+        }
+    }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@qpid.apache.org
For additional commands, e-mail: commits-help@qpid.apache.org