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 2017/12/01 21:11:57 UTC

[4/4] qpid-proton-j git commit: PROTON-1708 Optimizations for the EncoderImpl and DecoderImpl

PROTON-1708 Optimizations for the EncoderImpl and DecoderImpl

Provides several performance enhancements for the AMQP codec EncoderImpl
and DecoderImpl classes.

* Reduce instanceof and other type checks fro known types or by looking
ahead in the working buffer
* Reduce the number of throw away objects created during encode and
decode phase such as dynamic type constructor and char decoders etc.
* Provide fast write paths for both primitive types and known Described
types that are most often used
* Provide fast write path for most primitive write operations and for
Described types that are most often used
* Ensure the hot path for most read and write operations can be inlined
for the most common types. 
* Adds some look ahead methods for users of the decoder that allow for
skipping types that aren't needed 
* Allow Map types to be given fixed Type handlers for the Map keys for
those AMQP Map types whose Key types are specified to avoid the overhead
of looking them up on each Key read / write. 

Project: http://git-wip-us.apache.org/repos/asf/qpid-proton-j/repo
Commit: http://git-wip-us.apache.org/repos/asf/qpid-proton-j/commit/2a2d3ff2
Tree: http://git-wip-us.apache.org/repos/asf/qpid-proton-j/tree/2a2d3ff2
Diff: http://git-wip-us.apache.org/repos/asf/qpid-proton-j/diff/2a2d3ff2

Branch: refs/heads/master
Commit: 2a2d3ff27bd606af3e3c3c93705efff65c7a6302
Parents: cb3b9f4
Author: Timothy Bish <ta...@gmail.com>
Authored: Fri Dec 1 16:11:34 2017 -0500
Committer: Timothy Bish <ta...@gmail.com>
Committed: Fri Dec 1 16:11:34 2017 -0500

----------------------------------------------------------------------
 .../org/apache/qpid/proton/amqp/Symbol.java     |  15 +
 .../qpid/proton/codec/AMQPDefinedTypes.java     |  30 +-
 .../proton/codec/AbstractDescribedType.java     |  12 +-
 .../org/apache/qpid/proton/codec/ArrayType.java |  21 +-
 .../apache/qpid/proton/codec/BinaryType.java    |  33 +-
 .../org/apache/qpid/proton/codec/Decoder.java   |   1 +
 .../apache/qpid/proton/codec/DecoderImpl.java   | 577 +++++++++++--------
 .../proton/codec/DynamicTypeConstructor.java    |   5 +
 .../apache/qpid/proton/codec/EncoderImpl.java   | 196 ++++---
 .../codec/FastPathDescribedTypeConstructor.java |  28 +
 .../codec/FixedSizePrimitiveTypeEncoding.java   |   5 +
 .../org/apache/qpid/proton/codec/ListType.java  | 157 ++++-
 .../org/apache/qpid/proton/codec/MapType.java   | 246 ++++++--
 .../apache/qpid/proton/codec/StringType.java    |  50 +-
 .../apache/qpid/proton/codec/SymbolType.java    |  57 +-
 .../apache/qpid/proton/codec/TimestampType.java |  11 +-
 .../qpid/proton/codec/TypeConstructor.java      |   2 +
 .../org/apache/qpid/proton/codec/UUIDType.java  |   6 +
 .../qpid/proton/codec/UnsignedByteType.java     |   5 +
 .../qpid/proton/codec/UnsignedIntegerType.java  |  22 +-
 .../qpid/proton/codec/UnsignedLongType.java     |  26 +-
 .../qpid/proton/codec/UnsignedShortType.java    |   5 +
 .../proton/codec/messaging/AcceptedType.java    |   2 +-
 .../codec/messaging/AmqpSequenceType.java       |   5 +-
 .../proton/codec/messaging/AmqpValueType.java   |   5 +-
 .../qpid/proton/codec/messaging/DataType.java   |   5 +-
 .../codec/messaging/FastPathAcceptedType.java   | 123 ++++
 .../messaging/FastPathAmqpSequenceType.java     | 104 ++++
 .../codec/messaging/FastPathAmqpValueType.java  | 104 ++++
 .../FastPathApplicationPropertiesType.java      | 112 ++++
 .../codec/messaging/FastPathDataType.java       | 104 ++++
 .../FastPathDeliveryAnnotationsType.java        | 113 ++++
 .../codec/messaging/FastPathFooterType.java     | 110 ++++
 .../codec/messaging/FastPathHeaderType.java     | 244 ++++++++
 .../FastPathMessageAnnotationsType.java         | 113 ++++
 .../codec/messaging/FastPathPropertiesType.java | 308 ++++++++++
 .../qpid/proton/codec/messaging/HeaderType.java |   2 -
 .../proton/codec/messaging/PropertiesType.java  |   6 +-
 .../proton/codec/transport/DispositionType.java |   2 +-
 .../transport/FastPathDispositionType.java      | 248 ++++++++
 .../codec/transport/FastPathFlowType.java       | 279 +++++++++
 .../codec/transport/FastPathTransferType.java   | 295 ++++++++++
 .../qpid/proton/codec/transport/FlowType.java   |   6 +-
 .../proton/codec/transport/TransferType.java    |   8 +-
 .../proton/codec/AmqpValueTypeCodecTest.java    |  81 +++
 .../codec/ApplicationPropertiesCodecTest.java   |  77 +++
 .../codec/ApplicationPropertiesTypeTest.java    |  89 +++
 .../qpid/proton/codec/ArrayTypeCodecTest.java   | 277 +++++++++
 .../org/apache/qpid/proton/codec/Benchmark.java | 420 ++++++++++++++
 .../qpid/proton/codec/CodecTestSupport.java     |  48 ++
 .../qpid/proton/codec/HeaderTypeCodecTest.java  | 145 +++++
 .../qpid/proton/codec/ListTypeCodecTest.java    | 120 ++++
 .../qpid/proton/codec/MapTypeCodecTest.java     |  98 ++++
 .../codec/MessageAnnotationsTypeCodecTest.java  |  88 +++
 .../apache/qpid/proton/codec/NoLocalType.java   |  46 ++
 .../qpid/proton/codec/PropertiesCodecTest.java  |  99 ++++
 .../qpid/proton/codec/StringTypeTest.java       |  24 +
 .../qpid/proton/codec/TransferTypeTest.java     |  82 +++
 .../codec/UnknownDescribedTypeCodecTest.java    | 131 +++++
 .../proton/codec/UnsignedIntegerTypeTest.java   |  26 +
 .../qpid/proton/codec/UnsignedLongTypeTest.java |  25 +
 61 files changed, 5199 insertions(+), 485 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/qpid-proton-j/blob/2a2d3ff2/proton-j/src/main/java/org/apache/qpid/proton/amqp/Symbol.java
----------------------------------------------------------------------
diff --git a/proton-j/src/main/java/org/apache/qpid/proton/amqp/Symbol.java b/proton-j/src/main/java/org/apache/qpid/proton/amqp/Symbol.java
index 17e6177..4e3d476 100644
--- a/proton-j/src/main/java/org/apache/qpid/proton/amqp/Symbol.java
+++ b/proton-j/src/main/java/org/apache/qpid/proton/amqp/Symbol.java
@@ -21,16 +21,23 @@
 
 package org.apache.qpid.proton.amqp;
 
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
 import java.util.concurrent.ConcurrentHashMap;
 
+import org.apache.qpid.proton.codec.WritableBuffer;
+
 public final class Symbol implements Comparable<Symbol>, CharSequence
 {
     private final String _underlying;
+    private final byte[] _underlyingBytes;
+
     private static final ConcurrentHashMap<String, Symbol> _symbols = new ConcurrentHashMap<String, Symbol>(2048);
 
     private Symbol(String underlying)
     {
         _underlying = underlying;
+        _underlyingBytes = underlying.getBytes(StandardCharsets.US_ASCII);
     }
 
     public int length()
@@ -90,5 +97,13 @@ public final class Symbol implements Comparable<Symbol>, CharSequence
         return symbol;
     }
 
+    public void writeTo(WritableBuffer buffer)
+    {
+        buffer.put(_underlyingBytes, 0, _underlyingBytes.length);
+    }
 
+    public void writeTo(ByteBuffer buffer)
+    {
+        buffer.put(_underlyingBytes, 0, _underlyingBytes.length);
+    }
 }

http://git-wip-us.apache.org/repos/asf/qpid-proton-j/blob/2a2d3ff2/proton-j/src/main/java/org/apache/qpid/proton/codec/AMQPDefinedTypes.java
----------------------------------------------------------------------
diff --git a/proton-j/src/main/java/org/apache/qpid/proton/codec/AMQPDefinedTypes.java b/proton-j/src/main/java/org/apache/qpid/proton/codec/AMQPDefinedTypes.java
index 2e2f9e0..79dea5d 100644
--- a/proton-j/src/main/java/org/apache/qpid/proton/codec/AMQPDefinedTypes.java
+++ b/proton-j/src/main/java/org/apache/qpid/proton/codec/AMQPDefinedTypes.java
@@ -23,15 +23,6 @@
 
 package org.apache.qpid.proton.codec;
 
-import org.apache.qpid.proton.amqp.transport.Attach;
-import org.apache.qpid.proton.amqp.transport.Begin;
-import org.apache.qpid.proton.amqp.transport.Close;
-import org.apache.qpid.proton.amqp.transport.Detach;
-import org.apache.qpid.proton.amqp.transport.Disposition;
-import org.apache.qpid.proton.amqp.transport.End;
-import org.apache.qpid.proton.amqp.transport.Flow;
-import org.apache.qpid.proton.amqp.transport.Open;
-import org.apache.qpid.proton.amqp.transport.Transfer;
 import org.apache.qpid.proton.codec.messaging.*;
 import org.apache.qpid.proton.codec.security.*;
 import org.apache.qpid.proton.codec.transaction.*;
@@ -47,7 +38,6 @@ public class AMQPDefinedTypes
         registerSecurityTypes(decoder, encoder);
     }
 
-
     public static void registerTransportTypes(Decoder decoder, EncoderImpl encoder)
     {
         OpenType.register(decoder, encoder);
@@ -60,21 +50,25 @@ public class AMQPDefinedTypes
         EndType.register(decoder, encoder);
         CloseType.register(decoder, encoder);
         ErrorConditionType.register(decoder, encoder);
+
+        FastPathFlowType.register(decoder, encoder);
+        FastPathTransferType.register(decoder, encoder);
+        FastPathDispositionType.register(decoder, encoder);
     }
 
     public static void registerMessagingTypes(Decoder decoder, EncoderImpl encoder)
     {
         HeaderType.register(decoder, encoder);
+        AcceptedType.register(decoder , encoder);
+        PropertiesType.register( decoder, encoder );
         DeliveryAnnotationsType.register(decoder, encoder);
         MessageAnnotationsType.register(decoder, encoder);
-        PropertiesType.register( decoder, encoder );
         ApplicationPropertiesType.register(decoder, encoder);
         DataType.register(decoder, encoder);
         AmqpSequenceType.register(decoder, encoder);
         AmqpValueType.register(decoder, encoder);
         FooterType.register(decoder, encoder);
         ReceivedType.register(decoder, encoder);
-        AcceptedType.register(decoder , encoder);
         RejectedType.register(decoder, encoder);
         ReleasedType.register(decoder, encoder);
         ModifiedType.register(decoder, encoder);
@@ -84,6 +78,17 @@ public class AMQPDefinedTypes
         DeleteOnNoLinksType.register(decoder, encoder);
         DeleteOnNoMessagesType.register(decoder, encoder);
         DeleteOnNoLinksOrMessagesType.register(decoder, encoder);
+
+        FastPathHeaderType.register(decoder, encoder);
+        FastPathAcceptedType.register(decoder , encoder);
+        FastPathPropertiesType.register( decoder, encoder );
+        FastPathDeliveryAnnotationsType.register(decoder, encoder);
+        FastPathMessageAnnotationsType.register(decoder, encoder);
+        FastPathApplicationPropertiesType.register(decoder, encoder);
+        FastPathDataType.register(decoder, encoder);
+        FastPathAmqpSequenceType.register(decoder, encoder);
+        FastPathAmqpValueType.register(decoder, encoder);
+        FastPathFooterType.register(decoder, encoder);
     }
 
     public static void registerTransactionTypes(Decoder decoder, EncoderImpl encoder)
@@ -103,5 +108,4 @@ public class AMQPDefinedTypes
         SaslResponseType.register(decoder, encoder);
         SaslOutcomeType.register(decoder, encoder);
     }
-
 }

http://git-wip-us.apache.org/repos/asf/qpid-proton-j/blob/2a2d3ff2/proton-j/src/main/java/org/apache/qpid/proton/codec/AbstractDescribedType.java
----------------------------------------------------------------------
diff --git a/proton-j/src/main/java/org/apache/qpid/proton/codec/AbstractDescribedType.java b/proton-j/src/main/java/org/apache/qpid/proton/codec/AbstractDescribedType.java
index d31ac40..2f34fd5 100644
--- a/proton-j/src/main/java/org/apache/qpid/proton/codec/AbstractDescribedType.java
+++ b/proton-j/src/main/java/org/apache/qpid/proton/codec/AbstractDescribedType.java
@@ -28,17 +28,27 @@ import org.apache.qpid.proton.amqp.UnsignedLong;
 
 abstract public class AbstractDescribedType<T,M> implements AMQPType<T>
 {
-
+    private final DecoderImpl _decoder;
     private final EncoderImpl _encoder;
     private final Map<TypeEncoding<M>, TypeEncoding<T>> _encodings = new HashMap<TypeEncoding<M>, TypeEncoding<T>>();
 
     public AbstractDescribedType(EncoderImpl encoder)
     {
         _encoder = encoder;
+        _decoder = encoder.getDecoder();
     }
 
     abstract protected UnsignedLong getDescriptor();
 
+    public EncoderImpl getEncoder()
+    {
+        return _encoder;
+    }
+
+    public DecoderImpl getDecoder()
+    {
+        return _decoder;
+    }
 
     public TypeEncoding<T> getEncoding(final T val)
     {

http://git-wip-us.apache.org/repos/asf/qpid-proton-j/blob/2a2d3ff2/proton-j/src/main/java/org/apache/qpid/proton/codec/ArrayType.java
----------------------------------------------------------------------
diff --git a/proton-j/src/main/java/org/apache/qpid/proton/codec/ArrayType.java b/proton-j/src/main/java/org/apache/qpid/proton/codec/ArrayType.java
index 45b8dd5..32d6f85 100644
--- a/proton-j/src/main/java/org/apache/qpid/proton/codec/ArrayType.java
+++ b/proton-j/src/main/java/org/apache/qpid/proton/codec/ArrayType.java
@@ -21,6 +21,7 @@
 package org.apache.qpid.proton.codec;
 
 import java.lang.reflect.Array;
+import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.Collection;
 
@@ -631,8 +632,13 @@ public class ArrayType implements PrimitiveType<Object[]>
             return decodeArrayAsObject(decoder, count);
         }
 
-
-
+        public void skipValue()
+        {
+            DecoderImpl decoder = getDecoder();
+            ByteBuffer buffer = decoder.getByteBuffer();
+            int size = decoder.readRawInt();
+            buffer.position(buffer.position() + size);
+        }
     }
 
 
@@ -891,6 +897,13 @@ public class ArrayType implements PrimitiveType<Object[]>
             return decodeArrayAsObject(decoder, count);
         }
 
+        public void skipValue()
+        {
+            DecoderImpl decoder = getDecoder();
+            ByteBuffer buffer = decoder.getByteBuffer();
+            int size = ((int)decoder.readRawByte()) & 0xFF;
+            buffer.position(buffer.position() + size);
+        }
     }
 
     private BooleanType.BooleanEncoding getUnderlyingEncoding(final boolean[] a)
@@ -969,7 +982,7 @@ public class ArrayType implements PrimitiveType<Object[]>
 
     private static Object[] decodeArray(final DecoderImpl decoder, final int count)
     {
-        TypeConstructor constructor = decoder.readConstructor();
+        TypeConstructor constructor = decoder.readConstructor(true);
         return decodeNonPrimitive(decoder, constructor, count);
     }
 
@@ -1009,7 +1022,7 @@ public class ArrayType implements PrimitiveType<Object[]>
 
     private static Object decodeArrayAsObject(final DecoderImpl decoder, final int count)
     {
-        TypeConstructor constructor = decoder.readConstructor();
+        TypeConstructor constructor = decoder.readConstructor(true);
         if(constructor.encodesJavaPrimitive())
         {
             if (count > decoder.getByteBufferRemaining()) {

http://git-wip-us.apache.org/repos/asf/qpid-proton-j/blob/2a2d3ff2/proton-j/src/main/java/org/apache/qpid/proton/codec/BinaryType.java
----------------------------------------------------------------------
diff --git a/proton-j/src/main/java/org/apache/qpid/proton/codec/BinaryType.java b/proton-j/src/main/java/org/apache/qpid/proton/codec/BinaryType.java
index 88c204f..1d739b8 100644
--- a/proton-j/src/main/java/org/apache/qpid/proton/codec/BinaryType.java
+++ b/proton-j/src/main/java/org/apache/qpid/proton/codec/BinaryType.java
@@ -22,6 +22,7 @@ package org.apache.qpid.proton.codec;
 
 import org.apache.qpid.proton.amqp.Binary;
 
+import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.Collection;
 
@@ -53,7 +54,6 @@ public class BinaryType extends AbstractPrimitiveType<Binary>
         return val.getLength() <= 255 ? _shortBinaryEncoding : _binaryEncoding;
     }
 
-
     public BinaryEncoding getCanonicalEncoding()
     {
         return _binaryEncoding;
@@ -64,6 +64,22 @@ public class BinaryType extends AbstractPrimitiveType<Binary>
         return Arrays.asList(_shortBinaryEncoding, _binaryEncoding);
     }
 
+    public void fastWrite(EncoderImpl encoder, Binary binary)
+    {
+        if (binary.getLength() <= 255)
+        {
+            encoder.writeRaw(EncodingCodes.VBIN8);
+            encoder.writeRaw((byte) binary.getLength());
+            encoder.writeRaw(binary.getArray(), binary.getArrayOffset(), binary.getLength());
+        }
+        else
+        {
+            encoder.writeRaw(EncodingCodes.VBIN32);
+            encoder.writeRaw(binary.getLength());
+            encoder.writeRaw(binary.getArray(), binary.getArrayOffset(), binary.getLength());
+        }
+    }
+
     private class LongBinaryEncoding
             extends LargeFloatingSizePrimitiveTypeEncoding<Binary>
             implements BinaryEncoding
@@ -115,6 +131,14 @@ public class BinaryType extends AbstractPrimitiveType<Binary>
             decoder.readRaw(data, 0, size);
             return new Binary(data);
         }
+
+        public void skipValue()
+        {
+            DecoderImpl decoder = getDecoder();
+            ByteBuffer buffer = decoder.getByteBuffer();
+            int size = decoder.readRawInt();
+            buffer.position(buffer.position() + size);
+        }
     }
 
     private class ShortBinaryEncoding
@@ -163,5 +187,12 @@ public class BinaryType extends AbstractPrimitiveType<Binary>
             getDecoder().readRaw(data, 0, size);
             return new Binary(data);
         }
+
+        public void skipValue()
+        {
+            ByteBuffer buffer = getDecoder().getByteBuffer();
+            int size = ((int)getDecoder().readRawByte()) & 0xff;
+            buffer.position(buffer.position() + size);
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/qpid-proton-j/blob/2a2d3ff2/proton-j/src/main/java/org/apache/qpid/proton/codec/Decoder.java
----------------------------------------------------------------------
diff --git a/proton-j/src/main/java/org/apache/qpid/proton/codec/Decoder.java b/proton-j/src/main/java/org/apache/qpid/proton/codec/Decoder.java
index 6053479..b1f47e0 100644
--- a/proton-j/src/main/java/org/apache/qpid/proton/codec/Decoder.java
+++ b/proton-j/src/main/java/org/apache/qpid/proton/codec/Decoder.java
@@ -145,5 +145,6 @@ public interface Decoder
 
     void register(final Object descriptor, final DescribedTypeConstructor dtc);
 
+    void register(final Object descriptor, final FastPathDescribedTypeConstructor<?> dtc);
 
 }

http://git-wip-us.apache.org/repos/asf/qpid-proton-j/blob/2a2d3ff2/proton-j/src/main/java/org/apache/qpid/proton/codec/DecoderImpl.java
----------------------------------------------------------------------
diff --git a/proton-j/src/main/java/org/apache/qpid/proton/codec/DecoderImpl.java b/proton-j/src/main/java/org/apache/qpid/proton/codec/DecoderImpl.java
index dd68f6a..1763129 100644
--- a/proton-j/src/main/java/org/apache/qpid/proton/codec/DecoderImpl.java
+++ b/proton-j/src/main/java/org/apache/qpid/proton/codec/DecoderImpl.java
@@ -20,6 +20,7 @@
  */
 package org.apache.qpid.proton.codec;
 
+import org.apache.qpid.proton.ProtonException;
 import org.apache.qpid.proton.amqp.Binary;
 import org.apache.qpid.proton.amqp.Decimal128;
 import org.apache.qpid.proton.amqp.Decimal32;
@@ -31,48 +32,97 @@ import org.apache.qpid.proton.amqp.UnsignedInteger;
 import org.apache.qpid.proton.amqp.UnsignedLong;
 import org.apache.qpid.proton.amqp.UnsignedShort;
 
+import java.io.IOException;
 import java.lang.reflect.Array;
 import java.nio.ByteBuffer;
+import java.nio.charset.CharsetDecoder;
+import java.nio.charset.StandardCharsets;
 import java.util.*;
 
 public class DecoderImpl implements ByteBufferDecoder
 {
-
     private ByteBuffer _buffer;
-    private PrimitiveTypeEncoding[] _constructors = new PrimitiveTypeEncoding[256];
-    private Map<Object, DescribedTypeConstructor> _dynamicTypeConstructors =
+
+    private final CharsetDecoder _charsetDecoder = StandardCharsets.UTF_8.newDecoder();
+
+    private final PrimitiveTypeEncoding[] _constructors = new PrimitiveTypeEncoding[256];
+    private final Map<Object, DescribedTypeConstructor> _dynamicTypeConstructors =
             new HashMap<Object, DescribedTypeConstructor>();
 
+    private final Map<Object, FastPathDescribedTypeConstructor<?>> _fastPathTypeConstructors =
+        new HashMap<Object, FastPathDescribedTypeConstructor<?>>();
 
     public DecoderImpl()
     {
     }
 
-
     DecoderImpl(final ByteBuffer buffer)
     {
         _buffer = buffer;
     }
 
-    TypeConstructor readConstructor()
+    public TypeConstructor<?> peekConstructor()
+    {
+        _buffer.mark();
+        try
+        {
+            return readConstructor();
+        }
+        finally
+        {
+            _buffer.reset();
+        }
+    }
+
+    @SuppressWarnings("rawtypes")
+    public TypeConstructor readConstructor()
+    {
+        return readConstructor(false);
+    }
+
+    @SuppressWarnings("rawtypes")
+    public TypeConstructor readConstructor(boolean excludeFastPathConstructors)
     {
         int code = ((int)readRawByte()) & 0xff;
         if(code == EncodingCodes.DESCRIBED_TYPE_INDICATOR)
         {
-            final Object descriptor = readObject();
-            TypeConstructor nestedEncoding = readConstructor();
-            DescribedTypeConstructor dtc = _dynamicTypeConstructors.get(descriptor);
+            final byte encoding = _buffer.get(_buffer.position());
+            final Object descriptor;
+
+            if (EncodingCodes.SMALLULONG == encoding || EncodingCodes.ULONG == encoding)
+            {
+                descriptor = readUnsignedLong();
+            }
+            else if (EncodingCodes.SYM8 == encoding || EncodingCodes.SYM32 == encoding)
+            {
+                descriptor = readSymbol();
+            }
+            else
+            {
+                descriptor = readObject();
+            }
+
+            if (!excludeFastPathConstructors)
+            {
+                TypeConstructor<?> fastPathTypeConstructor = _fastPathTypeConstructors.get(descriptor);
+                if (fastPathTypeConstructor != null)
+                {
+                    return fastPathTypeConstructor;
+                }
+            }
+
+            TypeConstructor<?> nestedEncoding = readConstructor();
+            DescribedTypeConstructor<?> dtc = _dynamicTypeConstructors.get(descriptor);
             if(dtc == null)
             {
                 dtc = new DescribedTypeConstructor()
                 {
-
                     public DescribedType newInstance(final Object described)
                     {
                         return new UnknownDescribedType(descriptor, described);
                     }
 
-                    public Class getTypeClass()
+                    public Class<?> getTypeClass()
                     {
                         return UnknownDescribedType.class;
                     }
@@ -87,8 +137,15 @@ public class DecoderImpl implements ByteBufferDecoder
         }
     }
 
+    public void register(final Object descriptor, final FastPathDescribedTypeConstructor<?> btc)
+    {
+        _fastPathTypeConstructors.put(descriptor, btc);
+    }
+
     public void register(final Object descriptor, final DescribedTypeConstructor dtc)
     {
+        // Allow external type constructors to replace the built-in instances.
+        _fastPathTypeConstructors.remove(descriptor);
         _dynamicTypeConstructors.put(descriptor, dtc);
     }
 
@@ -108,37 +165,39 @@ public class DecoderImpl implements ByteBufferDecoder
 
     public Boolean readBoolean(final Boolean defaultVal)
     {
-        TypeConstructor constructor = readConstructor();
-        Object val = constructor.readValue();
-        if(val == null)
-        {
-            return defaultVal;
-        }
-        else if(val instanceof Boolean)
+        byte encodingCode = _buffer.get();
+
+        switch (encodingCode)
         {
-            return (Boolean) val;
+            case EncodingCodes.BOOLEAN_TRUE:
+                return (Boolean) _constructors[EncodingCodes.BOOLEAN_TRUE & 0xff].readValue();
+            case EncodingCodes.BOOLEAN_FALSE:
+                return (Boolean) _constructors[EncodingCodes.BOOLEAN_FALSE & 0xff].readValue();
+            case EncodingCodes.BOOLEAN:
+                return (Boolean) _constructors[EncodingCodes.BOOLEAN & 0xff].readValue();
+            case EncodingCodes.NULL:
+                return defaultVal;
+            default:
+                throw new DecodeException("Expected boolean type but found encoding: " + encodingCode);
         }
-        throw unexpectedType(val, Boolean.class);
     }
 
     public boolean readBoolean(final boolean defaultVal)
     {
-        TypeConstructor constructor = readConstructor();
-        if(constructor instanceof BooleanType.BooleanEncoding)
-        {
-            return ((BooleanType.BooleanEncoding)constructor).readPrimitiveValue();
-        }
-        else
+        byte encodingCode = _buffer.get();
+
+        switch (encodingCode)
         {
-            Object val = constructor.readValue();
-            if(val == null)
-            {
+            case EncodingCodes.BOOLEAN_TRUE:
+                return (Boolean) _constructors[EncodingCodes.BOOLEAN_TRUE & 0xff].readValue();
+            case EncodingCodes.BOOLEAN_FALSE:
+                return (Boolean) _constructors[EncodingCodes.BOOLEAN_FALSE & 0xff].readValue();
+            case EncodingCodes.BOOLEAN:
+                return (Boolean) _constructors[EncodingCodes.BOOLEAN & 0xff].readValue();
+            case EncodingCodes.NULL:
                 return defaultVal;
-            }
-            else
-            {
-                throw unexpectedType(val, Boolean.class);
-            }
+            default:
+                throw new DecodeException("Expected boolean type but found encoding: " + encodingCode);
         }
     }
 
@@ -149,17 +208,16 @@ public class DecoderImpl implements ByteBufferDecoder
 
     public Byte readByte(final Byte defaultVal)
     {
-        TypeConstructor constructor = readConstructor();
-        Object val = constructor.readValue();
-        if(val == null)
-        {
-            return defaultVal;
-        }
-        else if(val instanceof Byte)
-        {
-            return (Byte) val;
+        byte encodingCode = _buffer.get();
+
+        switch (encodingCode) {
+            case EncodingCodes.BYTE:
+                return (Byte) _constructors[EncodingCodes.BYTE & 0xff].readValue();
+            case EncodingCodes.NULL:
+                return defaultVal;
+            default:
+                throw new DecodeException("Expected byte type but found encoding: " + encodingCode);
         }
-        throw unexpectedType(val, Byte.class);
     }
 
     public byte readByte(final byte defaultVal)
@@ -190,18 +248,17 @@ public class DecoderImpl implements ByteBufferDecoder
 
     public Short readShort(final Short defaultVal)
     {
-        TypeConstructor constructor = readConstructor();
-        Object val = constructor.readValue();
-        if(val == null)
-        {
-            return defaultVal;
-        }
-        else if(val instanceof Short)
+        byte encodingCode = _buffer.get();
+
+        switch (encodingCode)
         {
-            return (Short) val;
+            case EncodingCodes.SHORT:
+                return (Short) _constructors[EncodingCodes.SHORT & 0xff].readValue();
+            case EncodingCodes.NULL:
+                return defaultVal;
+            default:
+                throw new DecodeException("Expected Short type but found encoding: " + encodingCode);
         }
-        throw unexpectedType(val, Short.class);
-
     }
 
     public short readShort(final short defaultVal)
@@ -233,18 +290,19 @@ public class DecoderImpl implements ByteBufferDecoder
 
     public Integer readInteger(final Integer defaultVal)
     {
-        TypeConstructor constructor = readConstructor();
-        Object val = constructor.readValue();
-        if(val == null)
-        {
-            return defaultVal;
-        }
-        else if(val instanceof Integer)
+        byte encodingCode = _buffer.get();
+
+        switch (encodingCode)
         {
-            return (Integer) val;
+            case EncodingCodes.SMALLINT:
+                return (Integer) _constructors[EncodingCodes.SMALLINT & 0xff].readValue();
+            case EncodingCodes.INT:
+                return (Integer) _constructors[EncodingCodes.INT & 0xff].readValue();
+            case EncodingCodes.NULL:
+                return defaultVal;
+            default:
+                throw new DecodeException("Expected Integer type but found encoding: " + encodingCode);
         }
-        throw unexpectedType(val, Integer.class);
-
     }
 
     public int readInteger(final int defaultVal)
@@ -276,19 +334,19 @@ public class DecoderImpl implements ByteBufferDecoder
 
     public Long readLong(final Long defaultVal)
     {
+        byte encodingCode = _buffer.get();
 
-        TypeConstructor constructor = readConstructor();
-        Object val = constructor.readValue();
-        if(val == null)
-        {
-            return defaultVal;
-        }
-        else if(val instanceof Long)
+        switch (encodingCode)
         {
-            return (Long) val;
+            case EncodingCodes.SMALLLONG:
+                return (Long) _constructors[EncodingCodes.SMALLLONG & 0xff].readValue();
+            case EncodingCodes.LONG:
+                return (Long) _constructors[EncodingCodes.LONG & 0xff].readValue();
+            case EncodingCodes.NULL:
+                return defaultVal;
+            default:
+                throw new DecodeException("Expected Long type but found encoding: " + encodingCode);
         }
-        throw unexpectedType(val, Long.class);
-
     }
 
     public long readLong(final long defaultVal)
@@ -320,19 +378,17 @@ public class DecoderImpl implements ByteBufferDecoder
 
     public UnsignedByte readUnsignedByte(final UnsignedByte defaultVal)
     {
+        byte encodingCode = _buffer.get();
 
-        TypeConstructor constructor = readConstructor();
-        Object val = constructor.readValue();
-        if(val == null)
-        {
-            return defaultVal;
-        }
-        else if(val instanceof UnsignedByte)
+        switch (encodingCode)
         {
-            return (UnsignedByte) val;
+            case EncodingCodes.UBYTE:
+                return (UnsignedByte) _constructors[EncodingCodes.UBYTE & 0xff].readValue();
+            case EncodingCodes.NULL:
+                return defaultVal;
+            default:
+                throw new DecodeException("Expected unsigned byte type but found encoding: " + encodingCode);
         }
-        throw unexpectedType(val, UnsignedByte.class);
-
     }
 
     public UnsignedShort readUnsignedShort()
@@ -342,19 +398,17 @@ public class DecoderImpl implements ByteBufferDecoder
 
     public UnsignedShort readUnsignedShort(final UnsignedShort defaultVal)
     {
+        byte encodingCode = _buffer.get();
 
-        TypeConstructor constructor = readConstructor();
-        Object val = constructor.readValue();
-        if(val == null)
-        {
-            return defaultVal;
-        }
-        else if(val instanceof UnsignedShort)
+        switch (encodingCode)
         {
-            return (UnsignedShort) val;
+            case EncodingCodes.USHORT:
+                return (UnsignedShort) _constructors[EncodingCodes.USHORT & 0xff].readValue();
+            case EncodingCodes.NULL:
+                return defaultVal;
+            default:
+                throw new DecodeException("Expected UnsignedShort type but found encoding: " + encodingCode);
         }
-        throw unexpectedType(val, UnsignedShort.class);
-
     }
 
     public UnsignedInteger readUnsignedInteger()
@@ -364,19 +418,21 @@ public class DecoderImpl implements ByteBufferDecoder
 
     public UnsignedInteger readUnsignedInteger(final UnsignedInteger defaultVal)
     {
+        byte encodingCode = _buffer.get();
 
-        TypeConstructor constructor = readConstructor();
-        Object val = constructor.readValue();
-        if(val == null)
+        switch (encodingCode)
         {
-            return defaultVal;
-        }
-        else if(val instanceof UnsignedInteger)
-        {
-            return (UnsignedInteger) val;
+            case EncodingCodes.UINT0:
+                return (UnsignedInteger) _constructors[EncodingCodes.UINT0 & 0xff].readValue();
+            case EncodingCodes.SMALLUINT:
+                return (UnsignedInteger) _constructors[EncodingCodes.SMALLUINT & 0xff].readValue();
+            case EncodingCodes.UINT:
+                return (UnsignedInteger) _constructors[EncodingCodes.UINT & 0xff].readValue();
+            case EncodingCodes.NULL:
+                return defaultVal;
+            default:
+                throw new DecodeException("Expected UnsignedInteger type but found encoding: " + encodingCode);
         }
-        throw unexpectedType(val, UnsignedInteger.class);
-
     }
 
     public UnsignedLong readUnsignedLong()
@@ -386,19 +442,21 @@ public class DecoderImpl implements ByteBufferDecoder
 
     public UnsignedLong readUnsignedLong(final UnsignedLong defaultVal)
     {
+        byte encodingCode = _buffer.get();
 
-        TypeConstructor constructor = readConstructor();
-        Object val = constructor.readValue();
-        if(val == null)
+        switch (encodingCode)
         {
-            return defaultVal;
-        }
-        else if(val instanceof UnsignedLong)
-        {
-            return (UnsignedLong) val;
+            case EncodingCodes.ULONG0:
+                return (UnsignedLong) _constructors[EncodingCodes.ULONG0 & 0xff].readValue();
+            case EncodingCodes.SMALLULONG:
+                return (UnsignedLong) _constructors[EncodingCodes.SMALLULONG & 0xff].readValue();
+            case EncodingCodes.ULONG:
+                return (UnsignedLong) _constructors[EncodingCodes.ULONG & 0xff].readValue();
+            case EncodingCodes.NULL:
+                return defaultVal;
+            default:
+                throw new DecodeException("Expected UnsignedLong type but found encoding: " + encodingCode);
         }
-        throw unexpectedType(val, UnsignedLong.class);
-
     }
 
     public Character readCharacter()
@@ -408,40 +466,31 @@ public class DecoderImpl implements ByteBufferDecoder
 
     public Character readCharacter(final Character defaultVal)
     {
+        byte encodingCode = _buffer.get();
 
-        TypeConstructor constructor = readConstructor();
-        Object val = constructor.readValue();
-        if(val == null)
+        switch (encodingCode)
         {
-            return defaultVal;
-        }
-        else if(val instanceof Character)
-        {
-            return (Character) val;
+            case EncodingCodes.CHAR:
+                return (Character) _constructors[EncodingCodes.CHAR & 0xff].readValue();
+            case EncodingCodes.NULL:
+                return defaultVal;
+            default:
+                throw new DecodeException("Expected Character type but found encoding: " + encodingCode);
         }
-        throw unexpectedType(val, Character.class);
-
     }
 
     public char readCharacter(final char defaultVal)
     {
+        byte encodingCode = _buffer.get();
 
-        TypeConstructor constructor = readConstructor();
-        if(constructor instanceof CharacterType.CharacterEncoding)
-        {
-            return ((CharacterType.CharacterEncoding)constructor).readPrimitiveValue();
-        }
-        else
+        switch (encodingCode)
         {
-            Object val = constructor.readValue();
-            if(val == null)
-            {
+            case EncodingCodes.CHAR:
+                return (Character) _constructors[EncodingCodes.CHAR & 0xff].readValue();
+            case EncodingCodes.NULL:
                 return defaultVal;
-            }
-            else
-            {
-                throw unexpectedType(val, Character.class);
-            }
+            default:
+                throw new DecodeException("Expected Character type but found encoding: " + encodingCode);
         }
     }
 
@@ -452,19 +501,17 @@ public class DecoderImpl implements ByteBufferDecoder
 
     public Float readFloat(final Float defaultVal)
     {
+        byte encodingCode = _buffer.get();
 
-        TypeConstructor constructor = readConstructor();
-        Object val = constructor.readValue();
-        if(val == null)
+        switch (encodingCode)
         {
-            return defaultVal;
-        }
-        else if(val instanceof Float)
-        {
-            return (Float) val;
+            case EncodingCodes.FLOAT:
+                return (Float) _constructors[EncodingCodes.FLOAT & 0xff].readValue();
+            case EncodingCodes.NULL:
+                return defaultVal;
+            default:
+                throw new ProtonException("Expected Float type but found encoding: " + encodingCode);
         }
-        throw unexpectedType(val, Float.class);
-
     }
 
     public float readFloat(final float defaultVal)
@@ -496,19 +543,17 @@ public class DecoderImpl implements ByteBufferDecoder
 
     public Double readDouble(final Double defaultVal)
     {
+        byte encodingCode = _buffer.get();
 
-        TypeConstructor constructor = readConstructor();
-        Object val = constructor.readValue();
-        if(val == null)
+        switch (encodingCode)
         {
-            return defaultVal;
-        }
-        else if(val instanceof Double)
-        {
-            return (Double) val;
+            case EncodingCodes.DOUBLE:
+                return (Double) _constructors[EncodingCodes.DOUBLE & 0xff].readValue();
+            case EncodingCodes.NULL:
+                return defaultVal;
+            default:
+                throw new ProtonException("Expected Double type but found encoding: " + encodingCode);
         }
-        throw unexpectedType(val, Double.class);
-
     }
 
     public double readDouble(final double defaultVal)
@@ -540,19 +585,17 @@ public class DecoderImpl implements ByteBufferDecoder
 
     public UUID readUUID(final UUID defaultVal)
     {
+        byte encodingCode = _buffer.get();
 
-        TypeConstructor constructor = readConstructor();
-        Object val = constructor.readValue();
-        if(val == null)
+        switch (encodingCode)
         {
-            return defaultVal;
-        }
-        else if(val instanceof UUID)
-        {
-            return (UUID) val;
+            case EncodingCodes.UUID:
+                return (UUID) _constructors[EncodingCodes.UUID & 0xff].readValue();
+            case EncodingCodes.NULL:
+                return defaultVal;
+            default:
+                throw new ProtonException("Expected UUID type but found encoding: " + encodingCode);
         }
-        throw unexpectedType(val, UUID.class);
-
     }
 
     public Decimal32 readDecimal32()
@@ -562,19 +605,17 @@ public class DecoderImpl implements ByteBufferDecoder
 
     public Decimal32 readDecimal32(final Decimal32 defaultValue)
     {
+        byte encodingCode = _buffer.get();
 
-        TypeConstructor constructor = readConstructor();
-        Object val = constructor.readValue();
-        if(val == null)
-        {
-            return defaultValue;
-        }
-        else if(val instanceof Decimal32)
+        switch (encodingCode)
         {
-            return (Decimal32) val;
+            case EncodingCodes.DECIMAL32:
+                return (Decimal32) _constructors[EncodingCodes.DECIMAL32 & 0xff].readValue();
+            case EncodingCodes.NULL:
+                return defaultValue;
+            default:
+                throw new ProtonException("Expected Decimal32 type but found encoding: " + encodingCode);
         }
-        throw unexpectedType(val, Decimal32.class);
-
     }
 
     public Decimal64 readDecimal64()
@@ -584,18 +625,17 @@ public class DecoderImpl implements ByteBufferDecoder
 
     public Decimal64 readDecimal64(final Decimal64 defaultValue)
     {
+        byte encodingCode = _buffer.get();
 
-        TypeConstructor constructor = readConstructor();
-        Object val = constructor.readValue();
-        if(val == null)
+        switch (encodingCode)
         {
-            return defaultValue;
+            case EncodingCodes.DECIMAL64:
+                return (Decimal64) _constructors[EncodingCodes.DECIMAL64 & 0xff].readValue();
+            case EncodingCodes.NULL:
+                return defaultValue;
+            default:
+                throw new ProtonException("Expected Decimal64 type but found encoding: " + encodingCode);
         }
-        else if(val instanceof Decimal64)
-        {
-            return (Decimal64) val;
-        }
-        throw unexpectedType(val, Decimal64.class);
     }
 
     public Decimal128 readDecimal128()
@@ -605,18 +645,17 @@ public class DecoderImpl implements ByteBufferDecoder
 
     public Decimal128 readDecimal128(final Decimal128 defaultValue)
     {
+        byte encodingCode = _buffer.get();
 
-        TypeConstructor constructor = readConstructor();
-        Object val = constructor.readValue();
-        if(val == null)
-        {
-            return defaultValue;
-        }
-        else if(val instanceof Decimal128)
+        switch (encodingCode)
         {
-            return (Decimal128) val;
+            case EncodingCodes.DECIMAL128:
+                return (Decimal128) _constructors[EncodingCodes.DECIMAL128 & 0xff].readValue();
+            case EncodingCodes.NULL:
+                return defaultValue;
+            default:
+                throw new ProtonException("Expected Decimal128 type but found encoding: " + encodingCode);
         }
-        throw unexpectedType(val, Decimal128.class);
     }
 
     public Date readTimestamp()
@@ -626,18 +665,17 @@ public class DecoderImpl implements ByteBufferDecoder
 
     public Date readTimestamp(final Date defaultValue)
     {
+        byte encodingCode = _buffer.get();
 
-        TypeConstructor constructor = readConstructor();
-        Object val = constructor.readValue();
-        if(val == null)
+        switch (encodingCode)
         {
-            return defaultValue;
+            case EncodingCodes.TIMESTAMP:
+                return (Date) _constructors[EncodingCodes.TIMESTAMP & 0xff].readValue();
+            case EncodingCodes.NULL:
+                return defaultValue;
+            default:
+                throw new ProtonException("Expected Timestamp type but found encoding: " + encodingCode);
         }
-        else if(val instanceof Date)
-        {
-            return (Date) val;
-        }
-        throw unexpectedType(val, Date.class);
     }
 
     public Binary readBinary()
@@ -647,18 +685,19 @@ public class DecoderImpl implements ByteBufferDecoder
 
     public Binary readBinary(final Binary defaultValue)
     {
+        byte encodingCode = _buffer.get();
 
-        TypeConstructor constructor = readConstructor();
-        Object val = constructor.readValue();
-        if(val == null)
+        switch (encodingCode)
         {
-            return defaultValue;
+            case EncodingCodes.VBIN8:
+                return (Binary) _constructors[EncodingCodes.VBIN8 & 0xff].readValue();
+            case EncodingCodes.VBIN32:
+                return (Binary) _constructors[EncodingCodes.VBIN32 & 0xff].readValue();
+            case EncodingCodes.NULL:
+                return defaultValue;
+            default:
+                throw new ProtonException("Expected Binary type but found encoding: " + encodingCode);
         }
-        else if(val instanceof Binary)
-        {
-            return (Binary) val;
-        }
-        throw unexpectedType(val, Binary.class);
     }
 
     public Symbol readSymbol()
@@ -668,18 +707,19 @@ public class DecoderImpl implements ByteBufferDecoder
 
     public Symbol readSymbol(final Symbol defaultValue)
     {
+        byte encodingCode = _buffer.get();
 
-        TypeConstructor constructor = readConstructor();
-        Object val = constructor.readValue();
-        if(val == null)
-        {
-            return defaultValue;
-        }
-        else if(val instanceof Symbol)
+        switch (encodingCode)
         {
-            return (Symbol) val;
+            case EncodingCodes.SYM8:
+                return (Symbol) _constructors[EncodingCodes.SYM8 & 0xff].readValue();
+            case EncodingCodes.SYM32:
+                return (Symbol) _constructors[EncodingCodes.SYM32 & 0xff].readValue();
+            case EncodingCodes.NULL:
+                return defaultValue;
+            default:
+                throw new ProtonException("Expected Symbol type but found encoding: " + encodingCode);
         }
-        throw unexpectedType(val, Symbol.class);
     }
 
     public String readString()
@@ -689,34 +729,39 @@ public class DecoderImpl implements ByteBufferDecoder
 
     public String readString(final String defaultValue)
     {
+        byte encodingCode = _buffer.get();
 
-        TypeConstructor constructor = readConstructor();
-        Object val = constructor.readValue();
-        if(val == null)
+        switch (encodingCode)
         {
-            return defaultValue;
+            case EncodingCodes.STR8:
+                return (String) _constructors[EncodingCodes.STR8 & 0xff].readValue();
+            case EncodingCodes.STR32:
+                return (String) _constructors[EncodingCodes.STR32 & 0xff].readValue();
+            case EncodingCodes.NULL:
+                return defaultValue;
+            default:
+                throw new ProtonException("Expected String type but found encoding: " + encodingCode);
         }
-        else if(val instanceof String)
-        {
-            return (String) val;
-        }
-        throw unexpectedType(val, String.class);
     }
 
+    @SuppressWarnings("rawtypes")
     public List readList()
     {
+        byte encodingCode = _buffer.get();
 
-        TypeConstructor constructor = readConstructor();
-        Object val = constructor.readValue();
-        if(val == null)
-        {
-            return null;
-        }
-        else if(val instanceof List)
+        switch (encodingCode)
         {
-            return (List) val;
+            case EncodingCodes.LIST0:
+                return (List) _constructors[EncodingCodes.LIST0 & 0xff].readValue();
+            case EncodingCodes.LIST8:
+                return (List) _constructors[EncodingCodes.LIST8 & 0xff].readValue();
+            case EncodingCodes.LIST32:
+                return (List) _constructors[EncodingCodes.LIST32 & 0xff].readValue();
+            case EncodingCodes.NULL:
+                return null;
+            default:
+                throw new ProtonException("Expected List type but found encoding: " + encodingCode);
         }
-        throw unexpectedType(val, List.class);
     }
 
     public <T> void readList(final ListProcessor<T> processor)
@@ -724,20 +769,22 @@ public class DecoderImpl implements ByteBufferDecoder
         //TODO.
     }
 
+    @SuppressWarnings("rawtypes")
     public Map readMap()
     {
+        byte encodingCode = _buffer.get();
 
-        TypeConstructor constructor = readConstructor();
-        Object val = constructor.readValue();
-        if(val == null)
-        {
-            return null;
-        }
-        else if(val instanceof Map)
+        switch (encodingCode)
         {
-            return (Map) val;
+            case EncodingCodes.MAP8:
+                return (Map) _constructors[EncodingCodes.MAP8 & 0xff].readValue();
+            case EncodingCodes.MAP32:
+                return (Map) _constructors[EncodingCodes.MAP32 & 0xff].readValue();
+            case EncodingCodes.NULL:
+                return null;
+            default:
+                throw new ProtonException("Expected Map type but found encoding: " + encodingCode);
         }
-        throw unexpectedType(val, Map.class);
     }
 
     public <T> T[] readArray(final Class<T> clazz)
@@ -877,14 +924,26 @@ public class DecoderImpl implements ByteBufferDecoder
 
     public Object readObject()
     {
+        boolean arrayType = false;
+        byte code = _buffer.get(_buffer.position());
+        switch (code)
+        {
+            case EncodingCodes.ARRAY8:
+            case EncodingCodes.ARRAY32:
+                arrayType = true;
+        }
+
         TypeConstructor constructor = readConstructor();
         if(constructor== null)
         {
             throw new DecodeException("Unknown constructor");
         }
-        return constructor instanceof ArrayType.ArrayEncoding
-               ? ((ArrayType.ArrayEncoding)constructor).readValueArray()
-               : constructor.readValue();
+
+        if (arrayType) {
+            return ((ArrayType.ArrayEncoding)constructor).readValueArray();
+        } else {
+            return constructor.readValue();
+        }
     }
 
     public Object readObject(final Object defaultValue)
@@ -942,7 +1001,7 @@ public class DecoderImpl implements ByteBufferDecoder
 
     <V> V readRaw(TypeDecoder<V> decoder, int size)
     {
-        V decode = decoder.decode((ByteBuffer) _buffer.slice().limit(size));
+        V decode = decoder.decode(this, (ByteBuffer) _buffer.slice().limit(size));
         _buffer.position(_buffer.position()+size);
         return decode;
     }
@@ -952,9 +1011,19 @@ public class DecoderImpl implements ByteBufferDecoder
         _buffer = buffer;
     }
 
+    public ByteBuffer getByteBuffer()
+    {
+        return _buffer;
+    }
+
+    CharsetDecoder getCharsetDecoder()
+    {
+        return _charsetDecoder;
+    }
+
     interface TypeDecoder<V>
     {
-        V decode(ByteBuffer buf);
+        V decode(DecoderImpl decoder, ByteBuffer buf);
     }
 
     private static class UnknownDescribedType implements DescribedType

http://git-wip-us.apache.org/repos/asf/qpid-proton-j/blob/2a2d3ff2/proton-j/src/main/java/org/apache/qpid/proton/codec/DynamicTypeConstructor.java
----------------------------------------------------------------------
diff --git a/proton-j/src/main/java/org/apache/qpid/proton/codec/DynamicTypeConstructor.java b/proton-j/src/main/java/org/apache/qpid/proton/codec/DynamicTypeConstructor.java
index 0cee927..2f8b9f9 100644
--- a/proton-j/src/main/java/org/apache/qpid/proton/codec/DynamicTypeConstructor.java
+++ b/proton-j/src/main/java/org/apache/qpid/proton/codec/DynamicTypeConstructor.java
@@ -53,6 +53,11 @@ public class DynamicTypeConstructor implements TypeConstructor
         return false;
     }
 
+    public void skipValue()
+    {
+        _underlyingEncoding.skipValue();
+    }
+
     public Class getTypeClass()
     {
         return _describedTypeConstructor.getTypeClass();

http://git-wip-us.apache.org/repos/asf/qpid-proton-j/blob/2a2d3ff2/proton-j/src/main/java/org/apache/qpid/proton/codec/EncoderImpl.java
----------------------------------------------------------------------
diff --git a/proton-j/src/main/java/org/apache/qpid/proton/codec/EncoderImpl.java b/proton-j/src/main/java/org/apache/qpid/proton/codec/EncoderImpl.java
index 97e1005..7bef1ae 100644
--- a/proton-j/src/main/java/org/apache/qpid/proton/codec/EncoderImpl.java
+++ b/proton-j/src/main/java/org/apache/qpid/proton/codec/EncoderImpl.java
@@ -42,9 +42,9 @@ public final class EncoderImpl implements ByteBufferEncoder
 {
     private static final byte DESCRIBED_TYPE_OP = (byte)0;
 
-
     private WritableBuffer _buffer;
 
+    private final DecoderImpl _decoder;
     private final Map<Class, AMQPType> _typeRegistry = new HashMap<Class, AMQPType>();
     private Map<Object, AMQPType> _describedDescriptorRegistry = new HashMap<Object, AMQPType>();
     private Map<Class, AMQPType>  _describedTypesClassRegistry = new HashMap<Class, AMQPType>();
@@ -88,7 +88,7 @@ public final class EncoderImpl implements ByteBufferEncoder
 
     public EncoderImpl(DecoderImpl decoder)
     {
-
+        _decoder                = decoder;
         _nullType               = new NullType(this, decoder);
         _booleanType            = new BooleanType(this, decoder);
         _byteType               = new ByteType(this, decoder);
@@ -143,36 +143,32 @@ public final class EncoderImpl implements ByteBufferEncoder
         _buffer = buf;
     }
 
+    public WritableBuffer getBuffer()
+    {
+        return _buffer;
+    }
+
+    public DecoderImpl getDecoder()
+    {
+        return _decoder;
+    }
 
     @Override
     public AMQPType getType(final Object element)
     {
-        if(element instanceof DescribedType)
-        {
-            AMQPType amqpType;
-
-            Object descriptor = ((DescribedType)element).getDescriptor();
-            amqpType = _describedDescriptorRegistry.get(descriptor);
-            if(amqpType == null)
-            {
-                amqpType = new DynamicDescribedType(this, descriptor);
-                _describedDescriptorRegistry.put(descriptor, amqpType);
-            }
-            return amqpType;
-
-        }
-        else
-        {
-            return getTypeFromClass(element == null ? Void.class : element.getClass());
-        }
+        return getTypeFromClass(element == null ? Void.class : element.getClass(), element);
     }
 
     public AMQPType getTypeFromClass(final Class clazz)
     {
+        return getTypeFromClass(clazz, null);
+    }
+
+    private AMQPType getTypeFromClass(final Class clazz, Object instance)
+    {
         AMQPType amqpType = _typeRegistry.get(clazz);
         if(amqpType == null)
         {
-
             if(clazz.isArray())
             {
                 amqpType = _arrayType;
@@ -190,10 +186,21 @@ public final class EncoderImpl implements ByteBufferEncoder
                 else if(DescribedType.class.isAssignableFrom(clazz))
                 {
                     amqpType = _describedTypesClassRegistry.get(clazz);
+                    if(amqpType == null && instance != null)
+                    {
+                        Object descriptor = ((DescribedType) instance).getDescriptor();
+                        amqpType = _describedDescriptorRegistry.get(descriptor);
+                        if(amqpType == null)
+                        {
+                            amqpType = new DynamicDescribedType(this, descriptor);
+                            _describedDescriptorRegistry.put(descriptor, amqpType);
+                        }
+                    }
                 }
             }
             _typeRegistry.put(clazz, amqpType);
         }
+
         return amqpType;
     }
 
@@ -221,12 +228,19 @@ public final class EncoderImpl implements ByteBufferEncoder
 
     public void writeNull()
     {
-        _nullType.write();
+        _buffer.put(EncodingCodes.NULL);
     }
 
     public void writeBoolean(final boolean bool)
     {
-        _booleanType.writeValue(bool);
+        if (bool)
+        {
+            _buffer.put(EncodingCodes.BOOLEAN_TRUE);
+        }
+        else
+        {
+            _buffer.put(EncodingCodes.BOOLEAN_FALSE);
+        }
     }
 
     public void writeBoolean(final Boolean bool)
@@ -235,9 +249,13 @@ public final class EncoderImpl implements ByteBufferEncoder
         {
             writeNull();
         }
+        else if (Boolean.TRUE.equals(bool))
+        {
+            _buffer.put(EncodingCodes.BOOLEAN_TRUE);
+        }
         else
         {
-            _booleanType.write(bool);
+            _buffer.put(EncodingCodes.BOOLEAN_FALSE);
         }
     }
 
@@ -249,7 +267,7 @@ public final class EncoderImpl implements ByteBufferEncoder
         }
         else
         {
-            _unsignedByteType.write(ubyte);
+            _unsignedByteType.fastWrite(this, ubyte);
         }
     }
 
@@ -261,7 +279,7 @@ public final class EncoderImpl implements ByteBufferEncoder
         }
         else
         {
-            _unsignedShortType.write(ushort);
+            _unsignedShortType.fastWrite(this, ushort);
         }
     }
 
@@ -273,7 +291,7 @@ public final class EncoderImpl implements ByteBufferEncoder
         }
         else
         {
-            _unsignedIntegerType.write(uint);
+            _unsignedIntegerType.fastWrite(this, uint);
         }
     }
 
@@ -285,7 +303,7 @@ public final class EncoderImpl implements ByteBufferEncoder
         }
         else
         {
-            _unsignedLongType.write(ulong);
+            _unsignedLongType.fastWrite(this, ulong);
         }
     }
 
@@ -446,9 +464,9 @@ public final class EncoderImpl implements ByteBufferEncoder
         }
     }
 
-    public void writeTimestamp(final long d)
+    public void writeTimestamp(final long timestamp)
     {
-        _timestampType.write(d);
+        _timestampType.fastWrite(this, timestamp);
     }
 
     public void writeTimestamp(final Date d)
@@ -459,7 +477,7 @@ public final class EncoderImpl implements ByteBufferEncoder
         }
         else
         {
-            writeTimestamp(d.getTime());
+            _timestampType.fastWrite(this, d.getTime());
         }
     }
 
@@ -471,7 +489,7 @@ public final class EncoderImpl implements ByteBufferEncoder
         }
         else
         {
-            _uuidType.write(uuid);
+            _uuidType.fastWrite(this, uuid);
         }
 
     }
@@ -484,7 +502,7 @@ public final class EncoderImpl implements ByteBufferEncoder
         }
         else
         {
-            _binaryType.write(b);
+            _binaryType.fastWrite(this, b);
         }
     }
 
@@ -508,7 +526,7 @@ public final class EncoderImpl implements ByteBufferEncoder
         }
         else
         {
-            _symbolType.write(s);
+            _symbolType.fastWrite(this, s);
         }
 
     }
@@ -662,56 +680,19 @@ public final class EncoderImpl implements ByteBufferEncoder
 
     public void writeObject(final Object o)
     {
-        AMQPType type = _typeRegistry.get(o == null ? Void.class : o.getClass());
+        if (o == null)
+        {
+            getBuffer().put(EncodingCodes.NULL);
+            return;
+        }
+
+        AMQPType type = _typeRegistry.get(o.getClass());
 
         if(type == null)
         {
             if(o.getClass().isArray())
             {
-                Class<?> componentType = o.getClass().getComponentType();
-                if(componentType.isPrimitive())
-                {
-                    if(componentType == Boolean.TYPE)
-                    {
-                        writeArray((boolean[])o);
-                    }
-                    else if(componentType == Byte.TYPE)
-                    {
-                        writeArray((byte[])o);
-                    }
-                    else if(componentType == Short.TYPE)
-                    {
-                        writeArray((short[])o);
-                    }
-                    else if(componentType == Integer.TYPE)
-                    {
-                        writeArray((int[])o);
-                    }
-                    else if(componentType == Long.TYPE)
-                    {
-                        writeArray((long[])o);
-                    }
-                    else if(componentType == Float.TYPE)
-                    {
-                        writeArray((float[])o);
-                    }
-                    else if(componentType == Double.TYPE)
-                    {
-                        writeArray((double[])o);
-                    }
-                    else if(componentType == Character.TYPE)
-                    {
-                        writeArray((char[])o);
-                    }
-                    else
-                    {
-                        throw new IllegalArgumentException("Cannot write arrays of type " + componentType.getName());
-                    }
-                }
-                else
-                {
-                    writeArray((Object[]) o);
-                }
+                writeArrayType(o);
             }
             else if(o instanceof List)
             {
@@ -727,9 +708,8 @@ public final class EncoderImpl implements ByteBufferEncoder
             }
             else
             {
-                throw new IllegalArgumentException("Do not know how to write Objects of class " + o.getClass()
-                                                                                                       .getName());
-
+                throw new IllegalArgumentException(
+                    "Do not know how to write Objects of class " + o.getClass().getName());
             }
         }
         else
@@ -738,6 +718,53 @@ public final class EncoderImpl implements ByteBufferEncoder
         }
     }
 
+    private void writeArrayType(Object array) {
+        Class<?> componentType = array.getClass().getComponentType();
+        if(componentType.isPrimitive())
+        {
+            if(componentType == Boolean.TYPE)
+            {
+                writeArray((boolean[])array);
+            }
+            else if(componentType == Byte.TYPE)
+            {
+                writeArray((byte[])array);
+            }
+            else if(componentType == Short.TYPE)
+            {
+                writeArray((short[])array);
+            }
+            else if(componentType == Integer.TYPE)
+            {
+                writeArray((int[])array);
+            }
+            else if(componentType == Long.TYPE)
+            {
+                writeArray((long[])array);
+            }
+            else if(componentType == Float.TYPE)
+            {
+                writeArray((float[])array);
+            }
+            else if(componentType == Double.TYPE)
+            {
+                writeArray((double[])array);
+            }
+            else if(componentType == Character.TYPE)
+            {
+                writeArray((char[])array);
+            }
+            else
+            {
+                throw new IllegalArgumentException("Cannot write arrays of type " + componentType.getName());
+            }
+        }
+        else
+        {
+            writeArray((Object[]) array);
+        }
+    }
+
     public void writeRaw(final byte b)
     {
         _buffer.put(b);
@@ -814,4 +841,9 @@ public final class EncoderImpl implements ByteBufferEncoder
             }
         }
     }
+
+    AMQPType getNullTypeEncoder()
+    {
+        return _nullType;
+    }
 }

http://git-wip-us.apache.org/repos/asf/qpid-proton-j/blob/2a2d3ff2/proton-j/src/main/java/org/apache/qpid/proton/codec/FastPathDescribedTypeConstructor.java
----------------------------------------------------------------------
diff --git a/proton-j/src/main/java/org/apache/qpid/proton/codec/FastPathDescribedTypeConstructor.java b/proton-j/src/main/java/org/apache/qpid/proton/codec/FastPathDescribedTypeConstructor.java
new file mode 100644
index 0000000..13b3c80
--- /dev/null
+++ b/proton-j/src/main/java/org/apache/qpid/proton/codec/FastPathDescribedTypeConstructor.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      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.proton.codec;
+
+/**
+ * Marker interface that indicates the TypeConstructor can decode known Proton-J types
+ * using a fast path read / write operation.  These types may result in an encode that
+ * does not always write the smallest form of the given type to save time.
+ *
+ * @param <V> The type that this constructor handles
+ */
+public interface FastPathDescribedTypeConstructor<V> extends TypeConstructor<V> {
+
+}

http://git-wip-us.apache.org/repos/asf/qpid-proton-j/blob/2a2d3ff2/proton-j/src/main/java/org/apache/qpid/proton/codec/FixedSizePrimitiveTypeEncoding.java
----------------------------------------------------------------------
diff --git a/proton-j/src/main/java/org/apache/qpid/proton/codec/FixedSizePrimitiveTypeEncoding.java b/proton-j/src/main/java/org/apache/qpid/proton/codec/FixedSizePrimitiveTypeEncoding.java
index 2762064..7c055ae 100644
--- a/proton-j/src/main/java/org/apache/qpid/proton/codec/FixedSizePrimitiveTypeEncoding.java
+++ b/proton-j/src/main/java/org/apache/qpid/proton/codec/FixedSizePrimitiveTypeEncoding.java
@@ -38,5 +38,10 @@ abstract class FixedSizePrimitiveTypeEncoding<T> extends AbstractPrimitiveTypeEn
         return getFixedSize();
     }
 
+    public final void skipValue()
+    {
+        getDecoder().getByteBuffer().position(getDecoder().getByteBuffer().position() + getFixedSize());
+    }
+
     protected abstract int getFixedSize();
 }

http://git-wip-us.apache.org/repos/asf/qpid-proton-j/blob/2a2d3ff2/proton-j/src/main/java/org/apache/qpid/proton/codec/ListType.java
----------------------------------------------------------------------
diff --git a/proton-j/src/main/java/org/apache/qpid/proton/codec/ListType.java b/proton-j/src/main/java/org/apache/qpid/proton/codec/ListType.java
index 185373f..67e9d6e 100644
--- a/proton-j/src/main/java/org/apache/qpid/proton/codec/ListType.java
+++ b/proton-j/src/main/java/org/apache/qpid/proton/codec/ListType.java
@@ -20,6 +20,7 @@
  */
 package org.apache.qpid.proton.codec;
 
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -55,10 +56,9 @@ public class ListType extends AbstractPrimitiveType<List>
 
     public ListEncoding getEncoding(final List val)
     {
-
         int calculatedSize = calculateSize(val, _encoder);
-        ListEncoding encoding = val.isEmpty() 
-                                    ? _zeroListEncoding 
+        ListEncoding encoding = val.isEmpty()
+                                    ? _zeroListEncoding
                                     : (val.size() > 255 || calculatedSize >= 254)
                                         ? _listEncoding
                                         : _shortListEncoding;
@@ -86,7 +86,6 @@ public class ListType extends AbstractPrimitiveType<List>
         return len;
     }
 
-
     public ListEncoding getCanonicalEncoding()
     {
         return _listEncoding;
@@ -152,6 +151,8 @@ public class ListType extends AbstractPrimitiveType<List>
         public List readValue()
         {
             DecoderImpl decoder = getDecoder();
+            ByteBuffer buffer = decoder.getByteBuffer();
+
             int size = decoder.readRawInt();
             // todo - limit the decoder with size
             int count = decoder.readRawInt();
@@ -160,14 +161,78 @@ public class ListType extends AbstractPrimitiveType<List>
                 throw new IllegalArgumentException("List element count "+count+" is specified to be greater than the amount of data available ("+
                                                    decoder.getByteBufferRemaining()+")");
             }
-            List list = new ArrayList(count);
-            for(int i = 0; i < count; i++)
+
+            TypeConstructor<?> typeConstructor = null;
+
+            List<Object> list = new ArrayList<>(count);
+            for (int i = 0; i < count; i++)
             {
-                list.add(decoder.readObject());
+                boolean arrayType = false;
+                byte code = buffer.get(buffer.position());
+                switch (code)
+                {
+                    case EncodingCodes.ARRAY8:
+                    case EncodingCodes.ARRAY32:
+                        arrayType = true;
+                }
+
+                // Whenever we can just reuse the previously used TypeDecoder instead
+                // of spending time looking up the same one again.
+                if (typeConstructor == null)
+                {
+                    typeConstructor = getDecoder().readConstructor();
+                }
+                else
+                {
+                    buffer.mark();
+
+                    byte encodingCode = buffer.get();
+                    if (encodingCode == EncodingCodes.DESCRIBED_TYPE_INDICATOR || !(typeConstructor instanceof PrimitiveTypeEncoding<?>))
+                    {
+                        buffer.reset();
+                        typeConstructor = getDecoder().readConstructor();
+                    }
+                    else
+                    {
+                        PrimitiveTypeEncoding<?> primitiveConstructor = (PrimitiveTypeEncoding<?>) typeConstructor;
+                        if (encodingCode != primitiveConstructor.getEncodingCode())
+                        {
+                            buffer.reset();
+                            typeConstructor = getDecoder().readConstructor();
+                        }
+                    }
+                }
+
+                if(typeConstructor == null)
+                {
+                    throw new DecodeException("Unknown constructor");
+                }
+
+                final Object value;
+
+                if (arrayType)
+                {
+                    value = ((ArrayType.ArrayEncoding) typeConstructor).readValueArray();
+                }
+                else
+                {
+                    value = typeConstructor.readValue();
+                }
+
+                list.add(value);
             }
+
             return list;
         }
 
+        public void skipValue()
+        {
+            DecoderImpl decoder = getDecoder();
+            ByteBuffer buffer = decoder.getByteBuffer();
+            int size = decoder.readRawInt();
+            buffer.position(buffer.position() + size);
+        }
+
         public void setValue(final List value, final int length)
         {
             _value = value;
@@ -229,19 +294,84 @@ public class ListType extends AbstractPrimitiveType<List>
 
         public List readValue()
         {
-
             DecoderImpl decoder = getDecoder();
+            ByteBuffer buffer = decoder.getByteBuffer();
+
             int size = ((int)decoder.readRawByte()) & 0xff;
             // todo - limit the decoder with size
             int count = ((int)decoder.readRawByte()) & 0xff;
-            List list = new ArrayList(count);
-            for(int i = 0; i < count; i++)
+
+            TypeConstructor<?> typeConstructor = null;
+
+            List<Object> list = new ArrayList<>(count);
+            for (int i = 0; i < count; i++)
             {
-                list.add(decoder.readObject());
+                boolean arrayType = false;
+                byte code = buffer.get(buffer.position());
+                switch (code)
+                {
+                    case EncodingCodes.ARRAY8:
+                    case EncodingCodes.ARRAY32:
+                        arrayType = true;
+                }
+
+                // Whenever we can just reuse the previously used TypeDecoder instead
+                // of spending time looking up the same one again.
+                if (typeConstructor == null)
+                {
+                    typeConstructor = getDecoder().readConstructor();
+                }
+                else
+                {
+                    buffer.mark();
+
+                    byte encodingCode = buffer.get();
+                    if (encodingCode == EncodingCodes.DESCRIBED_TYPE_INDICATOR || !(typeConstructor instanceof PrimitiveTypeEncoding<?>))
+                    {
+                        buffer.reset();
+                        typeConstructor = getDecoder().readConstructor();
+                    }
+                    else
+                    {
+                        PrimitiveTypeEncoding<?> primitiveConstructor = (PrimitiveTypeEncoding<?>) typeConstructor;
+                        if (encodingCode != primitiveConstructor.getEncodingCode())
+                        {
+                            buffer.reset();
+                            typeConstructor = getDecoder().readConstructor();
+                        }
+                    }
+                }
+
+                if (typeConstructor == null)
+                {
+                    throw new DecodeException("Unknown constructor");
+                }
+
+                final Object value;
+
+                if (arrayType)
+                {
+                    value = ((ArrayType.ArrayEncoding) typeConstructor).readValueArray();
+                }
+                else
+                {
+                    value = typeConstructor.readValue();
+                }
+
+                list.add(value);
             }
+
             return list;
         }
 
+        public void skipValue()
+        {
+            DecoderImpl decoder = getDecoder();
+            ByteBuffer buffer = decoder.getByteBuffer();
+            int size = ((int)decoder.readRawByte()) & 0xff;
+            buffer.position(buffer.position() + size);
+        }
+
         public void setValue(final List value, final int length)
         {
             _value = value;
@@ -249,7 +379,7 @@ public class ListType extends AbstractPrimitiveType<List>
         }
     }
 
-    
+
     private class ZeroListEncoding
             extends FixedSizePrimitiveTypeEncoding<List>
             implements ListEncoding
@@ -271,7 +401,6 @@ public class ListType extends AbstractPrimitiveType<List>
             return 0;
         }
 
-
         public ListType getType()
         {
            return ListType.this;
@@ -294,7 +423,5 @@ public class ListType extends AbstractPrimitiveType<List>
         {
             return Collections.EMPTY_LIST;
         }
-
-
     }
 }

http://git-wip-us.apache.org/repos/asf/qpid-proton-j/blob/2a2d3ff2/proton-j/src/main/java/org/apache/qpid/proton/codec/MapType.java
----------------------------------------------------------------------
diff --git a/proton-j/src/main/java/org/apache/qpid/proton/codec/MapType.java b/proton-j/src/main/java/org/apache/qpid/proton/codec/MapType.java
index 5c8a7c7..eba075a 100644
--- a/proton-j/src/main/java/org/apache/qpid/proton/codec/MapType.java
+++ b/proton-j/src/main/java/org/apache/qpid/proton/codec/MapType.java
@@ -20,7 +20,13 @@
  */
 package org.apache.qpid.proton.codec;
 
-import java.util.*;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Map.Entry;
 
 public class MapType extends AbstractPrimitiveType<Map>
 {
@@ -28,6 +34,8 @@ public class MapType extends AbstractPrimitiveType<Map>
     private final MapEncoding _shortMapEncoding;
     private EncoderImpl _encoder;
 
+    private AMQPType fixedKeyType;
+
     private static interface MapEncoding extends PrimitiveTypeEncoding<Map>
     {
         void setValue(Map value, int length);
@@ -42,15 +50,21 @@ public class MapType extends AbstractPrimitiveType<Map>
         decoder.register(this);
     }
 
+    @Override
     public Class<Map> getTypeClass()
     {
         return Map.class;
     }
 
-    public MapEncoding getEncoding(final Map val)
+    public void setKeyEncoding(AMQPType<?> keyType)
     {
+        this.fixedKeyType = keyType;
+    }
 
-        int calculatedSize = calculateSize(val, _encoder);
+    @Override
+    public MapEncoding getEncoding(final Map val)
+    {
+        int calculatedSize = calculateSize(val, _encoder, fixedKeyType);
         MapEncoding encoding = (val.size() > 127 || calculatedSize >= 254)
                                     ? _mapEncoding
                                     : _shortMapEncoding;
@@ -59,29 +73,78 @@ public class MapType extends AbstractPrimitiveType<Map>
         return encoding;
     }
 
-    private static int calculateSize(final Map val, EncoderImpl encoder)
+    private static int calculateSize(final Map map, EncoderImpl encoder, AMQPType<?> fixedKeyType)
     {
         int len = 0;
-        Iterator<Map.Entry> iter = val.entrySet().iterator();
+        Iterator<Map.Entry> iter = map.entrySet().iterator();
 
-        while(iter.hasNext())
+        while (iter.hasNext())
         {
             Map.Entry element = iter.next();
-            TypeEncoding elementEncoding = encoder.getType(element.getKey()).getEncoding(element.getKey());
+
+            AMQPType keyType = fixedKeyType;
+            if (fixedKeyType == null)
+            {
+                keyType = encoder.getType(element.getKey());
+            }
+
+            TypeEncoding elementEncoding = keyType.getEncoding(element.getKey());
             len += elementEncoding.getConstructorSize()+elementEncoding.getValueSize(element.getKey());
             elementEncoding = encoder.getType(element.getValue()).getEncoding(element.getValue());
             len += elementEncoding.getConstructorSize()+elementEncoding.getValueSize(element.getValue());
-
         }
         return len;
     }
 
+    private AMQPType<?> getKeyEncoding(EncoderImpl encoder, Object key)
+    {
+        if (fixedKeyType != null)
+        {
+            return fixedKeyType;
+        }
+        else
+        {
+            return encoder.getType(key);
+        }
+    }
 
+    private static TypeConstructor<?> findNextDecoder(DecoderImpl decoder, ByteBuffer buffer, TypeConstructor<?> previousConstructor)
+    {
+        if (previousConstructor == null)
+        {
+            return decoder.readConstructor();
+        }
+        else
+        {
+            buffer.mark();
+
+            byte encodingCode = buffer.get();
+            if (encodingCode == EncodingCodes.DESCRIBED_TYPE_INDICATOR || !(previousConstructor instanceof PrimitiveTypeEncoding<?>))
+            {
+                buffer.reset();
+                return decoder.readConstructor();
+            }
+            else
+            {
+                PrimitiveTypeEncoding<?> primitiveConstructor = (PrimitiveTypeEncoding<?>) previousConstructor;
+                if (encodingCode != primitiveConstructor.getEncodingCode())
+                {
+                    buffer.reset();
+                    return decoder.readConstructor();
+                }
+            }
+        }
+
+        return previousConstructor;
+    }
+
+    @Override
     public MapEncoding getCanonicalEncoding()
     {
         return _mapEncoding;
     }
 
+    @Override
     public Collection<MapEncoding> getAllEncodings()
     {
         return Arrays.asList(_shortMapEncoding, _mapEncoding);
@@ -101,17 +164,23 @@ public class MapType extends AbstractPrimitiveType<Map>
         }
 
         @Override
-        protected void writeEncodedValue(final Map val)
+        protected void writeEncodedValue(final Map map)
         {
-            getEncoder().writeRaw(2 * val.size());
-            
+            getEncoder().writeRaw(2 * map.size());
 
-            Iterator<Map.Entry> iter = val.entrySet().iterator();
+            Iterator<Map.Entry> iter = map.entrySet().iterator();
 
-            while(iter.hasNext())
+            while (iter.hasNext())
             {
                 Map.Entry element = iter.next();
-                TypeEncoding elementEncoding = getEncoder().getType(element.getKey()).getEncoding(element.getKey());
+
+                AMQPType keyType = fixedKeyType;
+                if (keyType == null)
+                {
+                    keyType = getEncoder().getType(element.getKey());
+                }
+
+                TypeEncoding elementEncoding = keyType.getEncoding(element.getKey());
                 elementEncoding.writeConstructor();
                 elementEncoding.writeValue(element.getKey());
                 elementEncoding = getEncoder().getType(element.getValue()).getEncoding(element.getValue());
@@ -123,30 +192,33 @@ public class MapType extends AbstractPrimitiveType<Map>
         @Override
         protected int getEncodedValueSize(final Map val)
         {
-            return 4 + ((val == _value) ? _length : calculateSize(val, getEncoder()));
+            return 4 + ((val == _value) ? _length : calculateSize(val, getEncoder(), fixedKeyType));
         }
 
-
         @Override
         public byte getEncodingCode()
         {
             return EncodingCodes.MAP32;
         }
 
+        @Override
         public MapType getType()
         {
             return MapType.this;
         }
 
+        @Override
         public boolean encodesSuperset(final TypeEncoding<Map> encoding)
         {
             return (getType() == encoding.getType());
         }
 
+        @Override
         public Map readValue()
         {
-
             DecoderImpl decoder = getDecoder();
+            ByteBuffer buffer = decoder.getByteBuffer();
+
             int size = decoder.readRawInt();
             // todo - limit the decoder with size
             int count = decoder.readRawInt();
@@ -154,17 +226,62 @@ public class MapType extends AbstractPrimitiveType<Map>
                 throw new IllegalArgumentException("Map element count "+count+" is specified to be greater than the amount of data available ("+
                                                    decoder.getByteBufferRemaining()+")");
             }
-            Map map = new LinkedHashMap(count);
-            for(int i = 0; i < count; i++)
+
+            TypeConstructor<?> keyConstructor = null;
+            TypeConstructor<?> valueConstructor = null;
+
+            Map<Object, Object> map = new LinkedHashMap<>(count);
+            for(int i = 0; i < count / 2; i++)
             {
-                Object key = decoder.readObject();
-                i++;
-                Object value = decoder.readObject();
+                keyConstructor = findNextDecoder(decoder, buffer, keyConstructor);
+                if(keyConstructor == null)
+                {
+                    throw new DecodeException("Unknown constructor");
+                }
+
+                Object key = keyConstructor.readValue();
+
+                boolean arrayType = false;
+                byte code = buffer.get(buffer.position());
+                switch (code)
+                {
+                    case EncodingCodes.ARRAY8:
+                    case EncodingCodes.ARRAY32:
+                        arrayType = true;
+                }
+
+                valueConstructor = findNextDecoder(decoder, buffer, valueConstructor);
+                if (valueConstructor == null)
+                {
+                    throw new DecodeException("Unknown constructor");
+                }
+
+                final Object value;
+
+                if (arrayType)
+                {
+                    value = ((ArrayType.ArrayEncoding) valueConstructor).readValueArray();
+                }
+                else
+                {
+                    value = valueConstructor.readValue();
+                }
+
                 map.put(key, value);
             }
+
             return map;
         }
 
+        public void skipValue()
+        {
+            DecoderImpl decoder = getDecoder();
+            ByteBuffer buffer = decoder.getByteBuffer();
+            int size = decoder.readRawInt();
+            buffer.position(buffer.position() + size);
+        }
+
+        @Override
         public void setValue(final Map value, final int length)
         {
             _value = value;
@@ -186,17 +303,22 @@ public class MapType extends AbstractPrimitiveType<Map>
         }
 
         @Override
-        protected void writeEncodedValue(final Map val)
+        protected void writeEncodedValue(final Map map)
         {
-            getEncoder().writeRaw((byte)(2*val.size()));
-                
+            getEncoder().writeRaw((byte)(2 * map.size()));
 
-            Iterator<Map.Entry> iter = val.entrySet().iterator();
-
-            while(iter.hasNext())
+            Iterator<Map.Entry> iter = map.entrySet().iterator();
+            while (iter.hasNext())
             {
                 Map.Entry element = iter.next();
-                TypeEncoding elementEncoding = getEncoder().getType(element.getKey()).getEncoding(element.getKey());
+
+                AMQPType keyType = fixedKeyType;
+                if (keyType == null)
+                {
+                    keyType = getEncoder().getType(element.getKey());
+                }
+
+                TypeEncoding elementEncoding = keyType.getEncoding(element.getKey());
                 elementEncoding.writeConstructor();
                 elementEncoding.writeValue(element.getKey());
                 elementEncoding = getEncoder().getType(element.getValue()).getEncoding(element.getValue());
@@ -208,44 +330,92 @@ public class MapType extends AbstractPrimitiveType<Map>
         @Override
         protected int getEncodedValueSize(final Map val)
         {
-            return 1 + ((val == _value) ? _length : calculateSize(val, getEncoder()));
+            return 1 + ((val == _value) ? _length : calculateSize(val, getEncoder(), fixedKeyType));
         }
 
-
         @Override
         public byte getEncodingCode()
         {
             return EncodingCodes.MAP8;
         }
 
+        @Override
         public MapType getType()
         {
             return MapType.this;
         }
 
+        @Override
         public boolean encodesSuperset(final TypeEncoding<Map> encoder)
         {
             return encoder == this;
         }
 
+        @Override
         public Map readValue()
         {
             DecoderImpl decoder = getDecoder();
-            int size = ((int)decoder.readRawByte()) & 0xff;
+            ByteBuffer buffer = decoder.getByteBuffer();
+
+            int size = (decoder.readRawByte()) & 0xff;
             // todo - limit the decoder with size
-            int count = ((int)decoder.readRawByte()) & 0xff;
+            int count = (decoder.readRawByte()) & 0xff;
+
+            TypeConstructor<?> keyConstructor = null;
+            TypeConstructor<?> valueConstructor = null;
 
-            Map map = new LinkedHashMap(count);
-            for(int i = 0; i < count; i++)
+            Map<Object, Object> map = new LinkedHashMap<>(count);
+            for(int i = 0; i < count / 2; i++)
             {
-                Object key = decoder.readObject();
-                i++;
-                Object value = decoder.readObject();
+                keyConstructor = findNextDecoder(decoder, buffer, keyConstructor);
+                if(keyConstructor == null)
+                {
+                    throw new DecodeException("Unknown constructor");
+                }
+
+                Object key = keyConstructor.readValue();
+
+                boolean arrayType = false;
+                byte code = buffer.get(buffer.position());
+                switch (code)
+                {
+                    case EncodingCodes.ARRAY8:
+                    case EncodingCodes.ARRAY32:
+                        arrayType = true;
+                }
+
+                valueConstructor = findNextDecoder(decoder, buffer, valueConstructor);
+                if(valueConstructor== null)
+                {
+                    throw new DecodeException("Unknown constructor");
+                }
+
+                final Object value;
+
+                if (arrayType)
+                {
+                    value = ((ArrayType.ArrayEncoding) valueConstructor).readValueArray();
+                }
+                else
+                {
+                    value = valueConstructor.readValue();
+                }
+
                 map.put(key, value);
             }
+
             return map;
         }
 
+        public void skipValue()
+        {
+            DecoderImpl decoder = getDecoder();
+            ByteBuffer buffer = decoder.getByteBuffer();
+            int size = ((int)decoder.readRawByte()) & 0xff;
+            buffer.position(buffer.position() + size);
+        }
+
+        @Override
         public void setValue(final Map value, final int length)
         {
             _value = value;


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