You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2015/11/20 07:28:58 UTC

[3/5] ignite git commit: IGNITE-1917: Binary protocol performance optimizations.

http://git-wip-us.apache.org/repos/asf/ignite/blob/4a1af37e/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderHandles.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderHandles.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderHandles.java
new file mode 100644
index 0000000..0024db0
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderHandles.java
@@ -0,0 +1,108 @@
+/*
+ * 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.ignite.internal.portable;
+
+import org.jetbrains.annotations.Nullable;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Reader handles.
+ */
+public class BinaryReaderHandles {
+    /** Mode: empty. */
+    private static final int MODE_EMPTY = 0;
+
+    /** Mode: single object. */
+    private static final int MODE_SINGLE = 1;
+
+    /** Mode: multiple objects. */
+    private static final int MODE_MULTIPLE = 2;
+
+    /** Position.  */
+    private int singlePos;
+
+    /** Data. This is either an object or a map. */
+    private Object data;
+
+    /** Mode. */
+    private int mode = MODE_EMPTY;
+
+    /**
+     * Get object by position.
+     *
+     * @param pos Position.
+     * @return Object.
+     */
+    @SuppressWarnings("unchecked")
+    public @Nullable <T> T get(int pos) {
+        switch (mode) {
+            case MODE_EMPTY:
+                return null;
+
+            case MODE_SINGLE:
+                return (T)data;
+
+            default:
+                assert mode == MODE_MULTIPLE;
+
+                return (T)((Map<Integer, Object>)data).get(pos);
+        }
+    }
+
+    /**
+     * Put object to registry and return previous position (if any).
+     *
+     * @param pos Position.
+     * @param obj Object.
+     */
+    @SuppressWarnings("unchecked")
+    public void put(int pos, Object obj) {
+        assert pos >= 0;
+        assert obj != null;
+
+        switch (mode) {
+            case MODE_EMPTY:
+                this.singlePos = pos;
+                this.data = obj;
+                this.mode = MODE_SINGLE;
+
+                break;
+
+            case MODE_SINGLE:
+                Map<Integer, Object> newData = new HashMap(3, 1.0f);
+
+                newData.put(singlePos, data);
+                newData.put(pos, obj);
+
+                this.singlePos = -1;
+                this.data = newData;
+                this.mode = MODE_MULTIPLE;
+
+                break;
+
+            default:
+                assert mode == MODE_MULTIPLE;
+
+                Map<Integer, Object> data0 = (Map<Integer, Object>)data;
+
+                data0.put(pos, obj);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4a1af37e/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryThreadLocalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryThreadLocalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryThreadLocalContext.java
new file mode 100644
index 0000000..c6a7fc3
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryThreadLocalContext.java
@@ -0,0 +1,69 @@
+/*
+ * 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.ignite.internal.portable;
+
+import org.apache.ignite.internal.portable.streams.PortableMemoryAllocator;
+import org.apache.ignite.internal.portable.streams.PortableMemoryAllocatorChunk;
+
+/**
+ * Contains thread-local data for binary marshalling.
+ */
+public class BinaryThreadLocalContext {
+    /** Thread-local instance. */
+    private static final ThreadLocal<BinaryThreadLocalContext> CTX = new ThreadLocal<BinaryThreadLocalContext>() {
+        @Override protected BinaryThreadLocalContext initialValue() {
+            return new BinaryThreadLocalContext();
+        }
+    };
+
+    /** Memory chunk. */
+    private final PortableMemoryAllocatorChunk chunk = PortableMemoryAllocator.INSTANCE.chunk();
+
+    /** Schema holder. */
+    private final BinaryWriterSchemaHolder schema = new BinaryWriterSchemaHolder();
+
+    /**
+     * Get current context.
+     *
+     * @return Context.
+     */
+    public static BinaryThreadLocalContext get() {
+        return CTX.get();
+    }
+
+    /**
+     * Private constructor.
+     */
+    private BinaryThreadLocalContext() {
+        // No-op.
+    }
+
+    /**
+     * @return Memory chunk.
+     */
+    public PortableMemoryAllocatorChunk chunk() {
+        return chunk;
+    }
+
+    /**
+     * @return Schema holder.
+     */
+    public BinaryWriterSchemaHolder schemaHolder() {
+        return schema;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4a1af37e/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryTypeImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryTypeImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryTypeImpl.java
index 60c135d..2630a40 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryTypeImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryTypeImpl.java
@@ -71,6 +71,7 @@ public class BinaryTypeImpl implements BinaryType {
     public PortableContext context() {
         return ctx;
     }
+
     /**
      * @return Metadata.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/4a1af37e/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriteMode.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriteMode.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriteMode.java
new file mode 100644
index 0000000..a26b741
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriteMode.java
@@ -0,0 +1,178 @@
+/*
+ * 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.ignite.internal.portable;
+
+/**
+ * Various write modes for binary objects.
+ */
+public enum BinaryWriteMode {
+    /** Primitive byte. */
+    P_BYTE(GridPortableMarshaller.BYTE),
+
+    /** Primitive boolean. */
+    P_BOOLEAN(GridPortableMarshaller.BOOLEAN),
+
+    /** Primitive short. */
+    P_SHORT(GridPortableMarshaller.SHORT),
+
+    /** Primitive char. */
+    P_CHAR(GridPortableMarshaller.CHAR),
+
+    /** Primitive int. */
+    P_INT(GridPortableMarshaller.INT),
+
+    /** Primitive long. */
+    P_LONG(GridPortableMarshaller.LONG),
+
+    /** Primitive float. */
+    P_FLOAT(GridPortableMarshaller.FLOAT),
+
+    /** Primitive int. */
+    P_DOUBLE(GridPortableMarshaller.DOUBLE),
+
+    /** */
+    BYTE(GridPortableMarshaller.BYTE),
+
+    /** */
+    SHORT(GridPortableMarshaller.SHORT),
+
+    /** */
+    INT(GridPortableMarshaller.INT),
+
+    /** */
+    LONG(GridPortableMarshaller.LONG),
+
+    /** */
+    FLOAT(GridPortableMarshaller.FLOAT),
+
+    /** */
+    DOUBLE(GridPortableMarshaller.DOUBLE),
+
+    /** */
+    CHAR(GridPortableMarshaller.CHAR),
+
+    /** */
+    BOOLEAN(GridPortableMarshaller.BOOLEAN),
+
+    /** */
+    DECIMAL(GridPortableMarshaller.DECIMAL),
+
+    /** */
+    STRING(GridPortableMarshaller.STRING),
+
+    /** */
+    UUID(GridPortableMarshaller.UUID),
+
+    /** */
+    DATE(GridPortableMarshaller.DATE),
+
+    /** */
+    TIMESTAMP(GridPortableMarshaller.TIMESTAMP),
+
+    /** */
+    BYTE_ARR(GridPortableMarshaller.BYTE_ARR),
+
+    /** */
+    SHORT_ARR(GridPortableMarshaller.SHORT_ARR),
+
+    /** */
+    INT_ARR(GridPortableMarshaller.INT_ARR),
+
+    /** */
+    LONG_ARR(GridPortableMarshaller.LONG_ARR),
+
+    /** */
+    FLOAT_ARR(GridPortableMarshaller.FLOAT_ARR),
+
+    /** */
+    DOUBLE_ARR(GridPortableMarshaller.DOUBLE_ARR),
+
+    /** */
+    CHAR_ARR(GridPortableMarshaller.CHAR_ARR),
+
+    /** */
+    BOOLEAN_ARR(GridPortableMarshaller.BOOLEAN_ARR),
+
+    /** */
+    DECIMAL_ARR(GridPortableMarshaller.DECIMAL_ARR),
+
+    /** */
+    STRING_ARR(GridPortableMarshaller.STRING_ARR),
+
+    /** */
+    UUID_ARR(GridPortableMarshaller.UUID_ARR),
+
+    /** */
+    DATE_ARR(GridPortableMarshaller.DATE_ARR),
+
+    /** */
+    TIMESTAMP_ARR(GridPortableMarshaller.TIMESTAMP_ARR),
+
+    /** */
+    OBJECT_ARR(GridPortableMarshaller.OBJ_ARR),
+
+    /** */
+    COL(GridPortableMarshaller.COL),
+
+    /** */
+    MAP(GridPortableMarshaller.MAP),
+
+    /** */
+    MAP_ENTRY(GridPortableMarshaller.MAP_ENTRY),
+
+    /** */
+    PORTABLE_OBJ(GridPortableMarshaller.OBJ),
+
+    /** */
+    ENUM(GridPortableMarshaller.ENUM),
+
+    /** */
+    ENUM_ARR(GridPortableMarshaller.ENUM_ARR),
+
+    /** */
+    CLASS(GridPortableMarshaller.CLASS),
+
+    /** */
+    PORTABLE(GridPortableMarshaller.PORTABLE_OBJ),
+
+    /** */
+    EXTERNALIZABLE(GridPortableMarshaller.OBJ),
+
+    /** */
+    OBJECT(GridPortableMarshaller.OBJ),
+
+    /** */
+    EXCLUSION(GridPortableMarshaller.OBJ);
+
+    /** Type ID. */
+    private final int typeId;
+
+    /**
+     * @param typeId Type ID.
+     */
+    private BinaryWriteMode(int typeId) {
+        this.typeId = typeId;
+    }
+
+    /**
+     * @return Type ID.
+     */
+    public int typeId() {
+        return typeId;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4a1af37e/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriterExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriterExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriterExImpl.java
index 6cb18fb..7bb4c49 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriterExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriterExImpl.java
@@ -19,12 +19,12 @@ package org.apache.ignite.internal.portable;
 
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.binary.BinaryIdMapper;
-import org.apache.ignite.internal.portable.streams.PortableHeapOutputStream;
-import org.apache.ignite.internal.portable.streams.PortableOutputStream;
-import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.binary.BinaryRawWriter;
 import org.apache.ignite.binary.BinaryWriter;
+import org.apache.ignite.internal.portable.streams.PortableHeapOutputStream;
+import org.apache.ignite.internal.portable.streams.PortableOutputStream;
+import org.apache.ignite.internal.util.typedef.internal.A;
 import org.jetbrains.annotations.Nullable;
 
 import java.io.IOException;
@@ -35,7 +35,6 @@ import java.math.BigInteger;
 import java.sql.Timestamp;
 import java.util.Collection;
 import java.util.Date;
-import java.util.IdentityHashMap;
 import java.util.Map;
 import java.util.UUID;
 
@@ -66,7 +65,6 @@ import static org.apache.ignite.internal.portable.GridPortableMarshaller.LONG_AR
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.MAP;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.MAP_ENTRY;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.NULL;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.OBJ;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.OBJ_ARR;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.OPTM_MARSH;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.PORTABLE_OBJ;
@@ -93,44 +91,26 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
     /** Initial capacity. */
     private static final int INIT_CAP = 1024;
 
-    /** Maximum offset which fits in 1 byte. */
-    private static final int MAX_OFFSET_1 = 1 << 8;
-
-    /** Maximum offset which fits in 2 bytes. */
-    private static final int MAX_OFFSET_2 = 1 << 16;
-
-    /** Thread-local schema. */
-    private static final ThreadLocal<SchemaHolder> SCHEMA = new ThreadLocal<>();
-
     /** */
     private final PortableContext ctx;
 
-    /** */
-    private final int start;
+    /** Output stream. */
+    private final PortableOutputStream out;
 
-    /** */
-    private Class<?> cls;
+    /** Schema. */
+    private final BinaryWriterSchemaHolder schema;
 
     /** */
     private int typeId;
 
-    /** Raw offset position. */
-    private int rawOffPos;
-
     /** */
-    private boolean metaEnabled;
+    private final int start;
 
-    /** */
-    private int metaHashSum;
+    /** Raw offset position. */
+    private int rawOffPos;
 
     /** Handles. */
-    private Map<Object, Integer> handles;
-
-    /** Output stream. */
-    private PortableOutputStream out;
-
-    /** Schema. */
-    private SchemaHolder schema;
+    private BinaryWriterHandles handles;
 
     /** Schema ID. */
     private int schemaId = PortableUtils.schemaInitialId();
@@ -144,40 +124,38 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
     /**
      * @param ctx Context.
      */
-    BinaryWriterExImpl(PortableContext ctx) {
-        this(ctx, new PortableHeapOutputStream(INIT_CAP));
+    public BinaryWriterExImpl(PortableContext ctx) {
+        this(ctx, BinaryThreadLocalContext.get());
     }
 
     /**
      * @param ctx Context.
-     * @param out Output stream.
+     * @param tlsCtx TLS context.
      */
-    BinaryWriterExImpl(PortableContext ctx, PortableOutputStream out) {
-        this(ctx, out, new IdentityHashMap<Object, Integer>());
+    public BinaryWriterExImpl(PortableContext ctx, BinaryThreadLocalContext tlsCtx) {
+        this(ctx, new PortableHeapOutputStream(INIT_CAP, tlsCtx.chunk()), tlsCtx.schemaHolder(), null);
     }
 
-     /**
-      * @param ctx Context.
-      * @param out Output stream.
-      * @param handles Handles.
-      */
-     private BinaryWriterExImpl(PortableContext ctx, PortableOutputStream out, Map<Object, Integer> handles) {
-         this.ctx = ctx;
-         this.out = out;
-         this.handles = handles;
+    /**
+     * @param ctx Context.
+     * @param out Output stream.
+     * @param handles Handles.
+     */
+    public BinaryWriterExImpl(PortableContext ctx, PortableOutputStream out, BinaryWriterSchemaHolder schema,
+        BinaryWriterHandles handles) {
+        this.ctx = ctx;
+        this.out = out;
+        this.schema = schema;
+        this.handles = handles;
 
-         start = out.position();
-     }
+        start = out.position();
+    }
 
     /**
-     * @param ctx Context.
      * @param typeId Type ID.
      */
-    public BinaryWriterExImpl(PortableContext ctx, int typeId, boolean metaEnabled) {
-        this(ctx);
-
+    public void typeId(int typeId) {
         this.typeId = typeId;
-        this.metaEnabled = metaEnabled;
     }
 
     /**
@@ -188,13 +166,6 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
     }
 
     /**
-     * @return Meta data hash sum or {@code null} if meta data is disabled.
-     */
-    @Nullable Integer metaDataHashSum() {
-        return metaEnabled ? metaHashSum : null;
-    }
-
-    /**
      * @param obj Object.
      * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
      */
@@ -210,7 +181,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
     void marshal(Object obj, boolean enableReplace) throws BinaryObjectException {
         assert obj != null;
 
-        cls = obj.getClass();
+        Class<?> cls = obj.getClass();
 
         PortableClassDescriptor desc = ctx.descriptorForClass(cls);
 
@@ -218,12 +189,13 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
             throw new BinaryObjectException("Object is not portable: [class=" + cls + ']');
 
         if (desc.excluded()) {
-            doWriteByte(NULL);
+            out.writeByte(NULL);
+
             return;
         }
 
         if (desc.useOptimizedMarshaller()) {
-            writeByte(OPTM_MARSH);
+            out.writeByte(OPTM_MARSH);
 
             try {
                 byte[] arr = ctx.optimizedMarsh().marshal(obj);
@@ -256,7 +228,8 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
             }
 
             if (replacedObj == null) {
-                doWriteByte(NULL);
+                out.writeByte(NULL);
+
                 return;
             }
 
@@ -265,31 +238,10 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
             return;
         }
 
-        typeId = desc.typeId();
-        metaEnabled = desc.userType();
-
         desc.write(obj, this);
     }
 
     /**
-     * @param obj Object.
-     * @return Handle.
-     */
-    int handle(Object obj) {
-        assert obj != null;
-
-        Integer h = handles.get(obj);
-
-        if (h != null)
-            return out.position() - h;
-        else {
-            handles.put(obj, out.position());
-
-            return -1;
-        }
-    }
-
-    /**
      * @return Array.
      */
     public byte[] array() {
@@ -340,18 +292,18 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
 
         if (useCompactFooter)
             flags |= PortableUtils.FLAG_COMPACT_FOOTER;
-        
-        if (schema != null) {
+
+        if (fieldCnt != 0) {
             flags |= PortableUtils.FLAG_HAS_SCHEMA;
 
             // Write schema ID.
-            out.writeInt(start + SCHEMA_ID_POS, schemaId);
+            out.unsafeWriteInt(start + SCHEMA_ID_POS, schemaId);
 
             // Write schema offset.
-            out.writeInt(start + SCHEMA_OR_RAW_OFF_POS, out.position() - start);
+            out.unsafeWriteInt(start + SCHEMA_OR_RAW_OFF_POS, out.position() - start);
 
             // Write the schema.
-            int offsetByteCnt = schema.write(this, fieldCnt, useCompactFooter);
+            int offsetByteCnt = schema.write(out, fieldCnt, useCompactFooter);
 
             if (offsetByteCnt == PortableUtils.OFFSET_1)
                 flags |= PortableUtils.FLAG_OFFSET_ONE_BYTE;
@@ -370,28 +322,25 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
                 // If there are no schema, we are free to write raw offset to schema offset.
                 flags |= PortableUtils.FLAG_HAS_RAW;
 
-                out.writeInt(start + SCHEMA_OR_RAW_OFF_POS, rawOffPos - start);
+                out.unsafeWriteInt(start + SCHEMA_OR_RAW_OFF_POS, rawOffPos - start);
             }
             else
-                out.writeInt(start + SCHEMA_OR_RAW_OFF_POS, 0);
+                out.unsafeWriteInt(start + SCHEMA_OR_RAW_OFF_POS, 0);
         }
 
         // Write flags.
-        out.writeShort(start + FLAGS_POS, flags);
+        out.unsafeWriteShort(start + FLAGS_POS, flags);
 
         // Write length.
-        out.writeInt(start + TOTAL_LEN_POS, out.position() - start);
+        out.unsafeWriteInt(start + TOTAL_LEN_POS, out.position() - start);
     }
 
     /**
      * Pop schema.
      */
     public void popSchema() {
-        if (schema != null) {
-            assert fieldCnt > 0;
-
+        if (fieldCnt > 0)
             schema.pop(fieldCnt);
-        }
     }
 
     /**
@@ -415,83 +364,29 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
     }
 
     /**
-     * @param val Value.
-     */
-    public void doWriteByte(byte val) {
-        out.writeByte(val);
-    }
-
-    /**
-     * @param val Value.
-     */
-    public void doWriteShort(short val) {
-        out.writeShort(val);
-    }
-
-    /**
-     * @param val Value.
-     */
-    public void doWriteInt(int val) {
-        out.writeInt(val);
-    }
-
-    /**
-     * @param val Value.
-     */
-    public void doWriteLong(long val) {
-        out.writeLong(val);
-    }
-
-    /**
-     * @param val Value.
-     */
-    public void doWriteFloat(float val) {
-        out.writeFloat(val);
-    }
-
-    /**
-     * @param val Value.
-     */
-    public void doWriteDouble(double val) {
-        out.writeDouble(val);
-    }
-
-    /**
-     * @param val Value.
-     */
-    public void doWriteChar(char val) {
-        out.writeChar(val);
-    }
-
-    /**
-     * @param val Value.
-     */
-    public void doWriteBoolean(boolean val) {
-        out.writeBoolean(val);
-    }
-
-    /**
      * @param val String value.
      */
     public void doWriteDecimal(@Nullable BigDecimal val) {
         if (val == null)
-            doWriteByte(NULL);
+            out.writeByte(NULL);
         else {
-            doWriteByte(DECIMAL);
+            out.unsafeEnsure(1 + 4 + 4);
+
+            out.unsafeWriteByte(DECIMAL);
 
             BigInteger intVal = val.unscaledValue();
 
             if (intVal.signum() == -1) {
                 intVal = intVal.negate();
 
-                out.writeInt(val.scale() | 0x80000000);
+                out.unsafeWriteInt(val.scale() | 0x80000000);
             }
             else
-                out.writeInt(val.scale());
+                out.unsafeWriteInt(val.scale());
 
             byte[] vals = intVal.toByteArray();
 
-            out.writeInt(vals.length);
+            out.unsafeWriteInt(vals.length);
             out.writeByteArray(vals);
         }
     }
@@ -501,13 +396,13 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      */
     public void doWriteString(@Nullable String val) {
         if (val == null)
-            doWriteByte(NULL);
+            out.writeByte(NULL);
         else {
-            doWriteByte(STRING);
-
             byte[] strArr = val.getBytes(UTF_8);
 
-            doWriteInt(strArr.length);
+            out.unsafeEnsure(1 + 4);
+            out.unsafeWriteByte(STRING);
+            out.unsafeWriteInt(strArr.length);
 
             out.writeByteArray(strArr);
         }
@@ -518,11 +413,12 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      */
     public void doWriteUuid(@Nullable UUID uuid) {
         if (uuid == null)
-            doWriteByte(NULL);
+            out.writeByte(NULL);
         else {
-            doWriteByte(UUID);
-            doWriteLong(uuid.getMostSignificantBits());
-            doWriteLong(uuid.getLeastSignificantBits());
+            out.unsafeEnsure(1 + 8 + 8);
+            out.unsafeWriteByte(UUID);
+            out.unsafeWriteLong(uuid.getMostSignificantBits());
+            out.unsafeWriteLong(uuid.getLeastSignificantBits());
         }
     }
 
@@ -531,10 +427,11 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      */
     public void doWriteDate(@Nullable Date date) {
         if (date == null)
-            doWriteByte(NULL);
+            out.writeByte(NULL);
         else {
-            doWriteByte(DATE);
-            doWriteLong(date.getTime());
+            out.unsafeEnsure(1 + 8);
+            out.unsafeWriteByte(DATE);
+            out.unsafeWriteLong(date.getTime());
         }
     }
 
@@ -543,11 +440,12 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      */
     public void doWriteTimestamp(@Nullable Timestamp ts) {
         if (ts== null)
-            doWriteByte(NULL);
+            out.writeByte(NULL);
         else {
-            doWriteByte(TIMESTAMP);
-            doWriteLong(ts.getTime());
-            doWriteInt(ts.getNanos() % 1000000);
+            out.unsafeEnsure(1 + 8 + 4);
+            out.unsafeWriteByte(TIMESTAMP);
+            out.unsafeWriteLong(ts.getTime());
+            out.unsafeWriteInt(ts.getNanos() % 1000000);
         }
     }
 
@@ -559,9 +457,9 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      */
     public void doWriteObject(@Nullable Object obj) throws BinaryObjectException {
         if (obj == null)
-            doWriteByte(NULL);
+            out.writeByte(NULL);
         else {
-            BinaryWriterExImpl writer = new BinaryWriterExImpl(ctx, out, handles);
+            BinaryWriterExImpl writer = new BinaryWriterExImpl(ctx, out, schema, handles());
 
             writer.marshal(obj);
         }
@@ -572,13 +470,14 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      */
     void doWriteByteArray(@Nullable byte[] val) {
         if (val == null)
-            doWriteByte(NULL);
+            out.writeByte(NULL);
         else {
             if (tryWriteAsHandle(val))
                 return;
 
-            doWriteByte(BYTE_ARR);
-            doWriteInt(val.length);
+            out.unsafeEnsure(1 + 4);
+            out.unsafeWriteByte(BYTE_ARR);
+            out.unsafeWriteInt(val.length);
 
             out.writeByteArray(val);
         }
@@ -589,13 +488,14 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      */
     void doWriteShortArray(@Nullable short[] val) {
         if (val == null)
-            doWriteByte(NULL);
+            out.writeByte(NULL);
         else {
             if (tryWriteAsHandle(val))
                 return;
 
-            doWriteByte(SHORT_ARR);
-            doWriteInt(val.length);
+            out.unsafeEnsure(1 + 4);
+            out.unsafeWriteByte(SHORT_ARR);
+            out.unsafeWriteInt(val.length);
 
             out.writeShortArray(val);
         }
@@ -606,13 +506,14 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      */
     void doWriteIntArray(@Nullable int[] val) {
         if (val == null)
-            doWriteByte(NULL);
+            out.writeByte(NULL);
         else {
             if (tryWriteAsHandle(val))
                 return;
 
-            doWriteByte(INT_ARR);
-            doWriteInt(val.length);
+            out.unsafeEnsure(1 + 4);
+            out.unsafeWriteByte(INT_ARR);
+            out.unsafeWriteInt(val.length);
 
             out.writeIntArray(val);
         }
@@ -623,13 +524,14 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      */
     void doWriteLongArray(@Nullable long[] val) {
         if (val == null)
-            doWriteByte(NULL);
+            out.writeByte(NULL);
         else {
             if (tryWriteAsHandle(val))
                 return;
 
-            doWriteByte(LONG_ARR);
-            doWriteInt(val.length);
+            out.unsafeEnsure(1 + 4);
+            out.unsafeWriteByte(LONG_ARR);
+            out.unsafeWriteInt(val.length);
 
             out.writeLongArray(val);
         }
@@ -640,13 +542,14 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      */
     void doWriteFloatArray(@Nullable float[] val) {
         if (val == null)
-            doWriteByte(NULL);
+            out.writeByte(NULL);
         else {
             if (tryWriteAsHandle(val))
                 return;
 
-            doWriteByte(FLOAT_ARR);
-            doWriteInt(val.length);
+            out.unsafeEnsure(1 + 4);
+            out.unsafeWriteByte(FLOAT_ARR);
+            out.unsafeWriteInt(val.length);
 
             out.writeFloatArray(val);
         }
@@ -657,13 +560,14 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      */
     void doWriteDoubleArray(@Nullable double[] val) {
         if (val == null)
-            doWriteByte(NULL);
+            out.writeByte(NULL);
         else {
             if (tryWriteAsHandle(val))
                 return;
 
-            doWriteByte(DOUBLE_ARR);
-            doWriteInt(val.length);
+            out.unsafeEnsure(1 + 4);
+            out.unsafeWriteByte(DOUBLE_ARR);
+            out.unsafeWriteInt(val.length);
 
             out.writeDoubleArray(val);
         }
@@ -674,13 +578,14 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      */
     void doWriteCharArray(@Nullable char[] val) {
         if (val == null)
-            doWriteByte(NULL);
+            out.writeByte(NULL);
         else {
             if (tryWriteAsHandle(val))
                 return;
 
-            doWriteByte(CHAR_ARR);
-            doWriteInt(val.length);
+            out.unsafeEnsure(1 + 4);
+            out.unsafeWriteByte(CHAR_ARR);
+            out.unsafeWriteInt(val.length);
 
             out.writeCharArray(val);
         }
@@ -691,13 +596,14 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      */
     void doWriteBooleanArray(@Nullable boolean[] val) {
         if (val == null)
-            doWriteByte(NULL);
+            out.writeByte(NULL);
         else {
             if (tryWriteAsHandle(val))
                 return;
 
-            doWriteByte(BOOLEAN_ARR);
-            doWriteInt(val.length);
+            out.unsafeEnsure(1 + 4);
+            out.unsafeWriteByte(BOOLEAN_ARR);
+            out.unsafeWriteInt(val.length);
 
             out.writeBooleanArray(val);
         }
@@ -708,13 +614,14 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      */
     void doWriteDecimalArray(@Nullable BigDecimal[] val) {
         if (val == null)
-            doWriteByte(NULL);
+            out.writeByte(NULL);
         else {
             if (tryWriteAsHandle(val))
                 return;
 
-            doWriteByte(DECIMAL_ARR);
-            doWriteInt(val.length);
+            out.unsafeEnsure(1 + 4);
+            out.unsafeWriteByte(DECIMAL_ARR);
+            out.unsafeWriteInt(val.length);
 
             for (BigDecimal str : val)
                 doWriteDecimal(str);
@@ -726,13 +633,14 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      */
     void doWriteStringArray(@Nullable String[] val) {
         if (val == null)
-            doWriteByte(NULL);
+            out.writeByte(NULL);
         else {
             if (tryWriteAsHandle(val))
                 return;
 
-            doWriteByte(STRING_ARR);
-            doWriteInt(val.length);
+            out.unsafeEnsure(1 + 4);
+            out.unsafeWriteByte(STRING_ARR);
+            out.unsafeWriteInt(val.length);
 
             for (String str : val)
                 doWriteString(str);
@@ -744,13 +652,14 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      */
     void doWriteUuidArray(@Nullable UUID[] val) {
         if (val == null)
-            doWriteByte(NULL);
+            out.writeByte(NULL);
         else {
             if (tryWriteAsHandle(val))
                 return;
 
-            doWriteByte(UUID_ARR);
-            doWriteInt(val.length);
+            out.unsafeEnsure(1 + 4);
+            out.unsafeWriteByte(UUID_ARR);
+            out.unsafeWriteInt(val.length);
 
             for (UUID uuid : val)
                 doWriteUuid(uuid);
@@ -762,13 +671,14 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      */
     void doWriteDateArray(@Nullable Date[] val) {
         if (val == null)
-            doWriteByte(NULL);
+            out.writeByte(NULL);
         else {
             if (tryWriteAsHandle(val))
                 return;
 
-            doWriteByte(DATE_ARR);
-            doWriteInt(val.length);
+            out.unsafeEnsure(1 + 4);
+            out.unsafeWriteByte(DATE_ARR);
+            out.unsafeWriteInt(val.length);
 
             for (Date date : val)
                 doWriteDate(date);
@@ -780,13 +690,14 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
       */
      void doWriteTimestampArray(@Nullable Timestamp[] val) {
          if (val == null)
-             doWriteByte(NULL);
+             out.writeByte(NULL);
          else {
              if (tryWriteAsHandle(val))
                  return;
 
-             doWriteByte(TIMESTAMP_ARR);
-             doWriteInt(val.length);
+             out.unsafeEnsure(1 + 4);
+             out.unsafeWriteByte(TIMESTAMP_ARR);
+             out.unsafeWriteInt(val.length);
 
              for (Timestamp ts : val)
                  doWriteTimestamp(ts);
@@ -799,23 +710,25 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      */
     void doWriteObjectArray(@Nullable Object[] val) throws BinaryObjectException {
         if (val == null)
-            doWriteByte(NULL);
+            out.writeByte(NULL);
         else {
             if (tryWriteAsHandle(val))
                 return;
 
             PortableClassDescriptor desc = ctx.descriptorForClass(val.getClass().getComponentType());
 
-            doWriteByte(OBJ_ARR);
+            out.unsafeEnsure(1 + 4);
+            out.unsafeWriteByte(OBJ_ARR);
 
             if (desc.registered())
-                doWriteInt(desc.typeId());
+                out.unsafeWriteInt(desc.typeId());
             else {
-                doWriteInt(UNREGISTERED_TYPE_ID);
+                out.unsafeWriteInt(UNREGISTERED_TYPE_ID);
+
                 doWriteString(val.getClass().getComponentType().getName());
             }
 
-            doWriteInt(val.length);
+            out.writeInt(val.length);
 
             for (Object obj : val)
                 doWriteObject(obj);
@@ -828,14 +741,15 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      */
     void doWriteCollection(@Nullable Collection<?> col) throws BinaryObjectException {
         if (col == null)
-            doWriteByte(NULL);
+            out.writeByte(NULL);
         else {
             if (tryWriteAsHandle(col))
                 return;
 
-            doWriteByte(COL);
-            doWriteInt(col.size());
-            doWriteByte(ctx.collectionType(col.getClass()));
+            out.unsafeEnsure(1 + 4 + 1);
+            out.unsafeWriteByte(COL);
+            out.unsafeWriteInt(col.size());
+            out.unsafeWriteByte(ctx.collectionType(col.getClass()));
 
             for (Object obj : col)
                 doWriteObject(obj);
@@ -848,14 +762,15 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      */
     void doWriteMap(@Nullable Map<?, ?> map) throws BinaryObjectException {
         if (map == null)
-            doWriteByte(NULL);
+            out.writeByte(NULL);
         else {
             if (tryWriteAsHandle(map))
                 return;
 
-            doWriteByte(MAP);
-            doWriteInt(map.size());
-            doWriteByte(ctx.mapType(map.getClass()));
+            out.unsafeEnsure(1 + 4 + 1);
+            out.unsafeWriteByte(MAP);
+            out.unsafeWriteInt(map.size());
+            out.unsafeWriteByte(ctx.mapType(map.getClass()));
 
             for (Map.Entry<?, ?> e : map.entrySet()) {
                 doWriteObject(e.getKey());
@@ -870,12 +785,12 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      */
     void doWriteMapEntry(@Nullable Map.Entry<?, ?> e) throws BinaryObjectException {
         if (e == null)
-            doWriteByte(NULL);
+            out.writeByte(NULL);
         else {
             if (tryWriteAsHandle(e))
                 return;
 
-            doWriteByte(MAP_ENTRY);
+            out.writeByte(MAP_ENTRY);
             doWriteObject(e.getKey());
             doWriteObject(e.getValue());
         }
@@ -886,20 +801,22 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      */
     void doWriteEnum(@Nullable Enum<?> val) {
         if (val == null)
-            doWriteByte(NULL);
+            out.writeByte(NULL);
         else {
             PortableClassDescriptor desc = ctx.descriptorForClass(val.getClass());
 
-            doWriteByte(ENUM);
+            out.unsafeEnsure(1 + 4);
+
+            out.unsafeWriteByte(ENUM);
 
             if (desc.registered())
-                doWriteInt(desc.typeId());
+                out.unsafeWriteInt(desc.typeId());
             else {
-                doWriteInt(UNREGISTERED_TYPE_ID);
+                out.unsafeWriteInt(UNREGISTERED_TYPE_ID);
                 doWriteString(val.getClass().getName());
             }
 
-            doWriteInt(val.ordinal());
+            out.writeInt(val.ordinal());
         }
     }
 
@@ -910,19 +827,23 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
         assert val == null || val.getClass().getComponentType().isEnum();
 
         if (val == null)
-            doWriteByte(NULL);
+            out.writeByte(NULL);
         else {
             PortableClassDescriptor desc = ctx.descriptorForClass(val.getClass().getComponentType());
-            doWriteByte(ENUM_ARR);
+
+            out.unsafeEnsure(1 + 4);
+
+            out.unsafeWriteByte(ENUM_ARR);
 
             if (desc.registered())
-                doWriteInt(desc.typeId());
+                out.unsafeWriteInt(desc.typeId());
             else {
-                doWriteInt(UNREGISTERED_TYPE_ID);
+                out.unsafeWriteInt(UNREGISTERED_TYPE_ID);
+
                 doWriteString(val.getClass().getComponentType().getName());
             }
 
-            doWriteInt(val.length);
+            out.writeInt(val.length);
 
             // TODO: Denis: Redundant data for each element of the array.
             for (Object o : val)
@@ -935,16 +856,19 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      */
     void doWriteClass(@Nullable Class val) {
         if (val == null)
-            doWriteByte(NULL);
+            out.writeByte(NULL);
         else {
             PortableClassDescriptor desc = ctx.descriptorForClass(val);
 
-            doWriteByte(CLASS);
+            out.unsafeEnsure(1 + 4);
+
+            out.unsafeWriteByte(CLASS);
 
             if (desc.registered())
-                doWriteInt(desc.typeId());
+                out.unsafeWriteInt(desc.typeId());
             else {
-                doWriteInt(UNREGISTERED_TYPE_ID);
+                out.unsafeWriteInt(UNREGISTERED_TYPE_ID);
+
                 doWriteString(val.getClass().getName());
             }
         }
@@ -955,30 +879,37 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      */
     public void doWritePortableObject(@Nullable BinaryObjectImpl po) {
         if (po == null)
-            doWriteByte(NULL);
+            out.writeByte(NULL);
         else {
-            doWriteByte(PORTABLE_OBJ);
-
             byte[] poArr = po.array();
 
-            doWriteInt(poArr.length);
+            out.unsafeEnsure(1 + 4 + poArr.length + 4);
 
+            out.unsafeWriteByte(PORTABLE_OBJ);
+            out.unsafeWriteInt(poArr.length);
             out.writeByteArray(poArr);
-
-            doWriteInt(po.start());
+            out.unsafeWriteInt(po.start());
         }
     }
 
     /**
      * @param val Value.
      */
+    void writeByteFieldPrimitive(byte val) {
+        out.unsafeEnsure(1 + 1);
+
+        out.unsafeWriteByte(BYTE);
+        out.unsafeWriteByte(val);
+    }
+
+    /**
+     * @param val Value.
+     */
     void writeByteField(@Nullable Byte val) {
         if (val == null)
-            doWriteByte(NULL);
-        else {
-            doWriteByte(BYTE);
-            doWriteByte(val);
-        }
+            out.writeByte(NULL);
+        else
+            writeByteFieldPrimitive(val);
     }
 
     /**
@@ -991,13 +922,31 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
     /**
      * @param val Value.
      */
+    void writeShortFieldPrimitive(short val) {
+        out.unsafeEnsure(1 + 2);
+
+        out.unsafeWriteByte(SHORT);
+        out.unsafeWriteShort(val);
+    }
+
+    /**
+     * @param val Value.
+     */
     void writeShortField(@Nullable Short val) {
         if (val == null)
-            doWriteByte(NULL);
-        else {
-            doWriteByte(SHORT);
-            doWriteShort(val);
-        }
+            out.writeByte(NULL);
+        else
+            writeShortFieldPrimitive(val);
+    }
+
+    /**
+     * @param val Value.
+     */
+    void writeIntFieldPrimitive(int val) {
+        out.unsafeEnsure(1 + 4);
+
+        out.unsafeWriteByte(INT);
+        out.unsafeWriteInt(val);
     }
 
     /**
@@ -1005,11 +954,19 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      */
     void writeIntField(@Nullable Integer val) {
         if (val == null)
-            doWriteByte(NULL);
-        else {
-            doWriteByte(INT);
-            doWriteInt(val);
-        }
+            out.writeByte(NULL);
+        else
+            writeIntFieldPrimitive(val);
+    }
+
+    /**
+     * @param val Value.
+     */
+    void writeLongFieldPrimitive(long val) {
+        out.unsafeEnsure(1 + 8);
+
+        out.unsafeWriteByte(LONG);
+        out.unsafeWriteLong(val);
     }
 
     /**
@@ -1017,11 +974,19 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      */
     void writeLongField(@Nullable Long val) {
         if (val == null)
-            doWriteByte(NULL);
-        else {
-            doWriteByte(LONG);
-            doWriteLong(val);
-        }
+            out.writeByte(NULL);
+        else
+            writeLongFieldPrimitive(val);
+    }
+
+    /**
+     * @param val Value.
+     */
+    void writeFloatFieldPrimitive(float val) {
+        out.unsafeEnsure(1 + 4);
+
+        out.unsafeWriteByte(FLOAT);
+        out.unsafeWriteFloat(val);
     }
 
     /**
@@ -1029,11 +994,19 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      */
     void writeFloatField(@Nullable Float val) {
         if (val == null)
-            doWriteByte(NULL);
-        else {
-            doWriteByte(FLOAT);
-            doWriteFloat(val);
-        }
+            out.writeByte(NULL);
+        else
+            writeFloatFieldPrimitive(val);
+    }
+
+    /**
+     * @param val Value.
+     */
+    void writeDoubleFieldPrimitive(double val) {
+        out.unsafeEnsure(1 + 8);
+
+        out.unsafeWriteByte(DOUBLE);
+        out.unsafeWriteDouble(val);
     }
 
     /**
@@ -1041,11 +1014,19 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      */
     void writeDoubleField(@Nullable Double val) {
         if (val == null)
-            doWriteByte(NULL);
-        else {
-            doWriteByte(DOUBLE);
-            doWriteDouble(val);
-        }
+            out.writeByte(NULL);
+        else
+            writeDoubleFieldPrimitive(val);
+    }
+
+    /**
+     * @param val Value.
+     */
+    void writeCharFieldPrimitive(char val) {
+        out.unsafeEnsure(1 + 2);
+
+        out.unsafeWriteByte(CHAR);
+        out.unsafeWriteChar(val);
     }
 
     /**
@@ -1053,11 +1034,19 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      */
     void writeCharField(@Nullable Character val) {
         if (val == null)
-            doWriteByte(NULL);
-        else {
-            doWriteByte(CHAR);
-            doWriteChar(val);
-        }
+            out.writeByte(NULL);
+        else
+            writeCharFieldPrimitive(val);
+    }
+
+    /**
+     * @param val Value.
+     */
+    void writeBooleanFieldPrimitive(boolean val) {
+        out.unsafeEnsure(1 + 1);
+
+        out.unsafeWriteByte(BOOLEAN);
+        out.unsafeWriteBoolean(val);
     }
 
     /**
@@ -1065,11 +1054,9 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      */
     void writeBooleanField(@Nullable Boolean val) {
         if (val == null)
-            doWriteByte(NULL);
-        else {
-            doWriteByte(BOOLEAN);
-            doWriteBoolean(val);
-        }
+            out.writeByte(NULL);
+        else
+            writeBooleanFieldPrimitive(val);
     }
 
     /**
@@ -1262,95 +1249,95 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
 
     /** {@inheritDoc} */
     @Override public void writeByte(String fieldName, byte val) throws BinaryObjectException {
-        writeFieldId(fieldName, BYTE);
+        writeFieldId(fieldName);
         writeByteField(val);
     }
 
     /** {@inheritDoc} */
     @Override public void writeByte(byte val) throws BinaryObjectException {
-        doWriteByte(val);
+        out.writeByte(val);
     }
 
     /** {@inheritDoc} */
     @Override public void writeShort(String fieldName, short val) throws BinaryObjectException {
-        writeFieldId(fieldName, SHORT);
+        writeFieldId(fieldName);
         writeShortField(val);
     }
 
     /** {@inheritDoc} */
     @Override public void writeShort(short val) throws BinaryObjectException {
-        doWriteShort(val);
+        out.writeShort(val);
     }
 
     /** {@inheritDoc} */
     @Override public void writeInt(String fieldName, int val) throws BinaryObjectException {
-        writeFieldId(fieldName, INT);
+        writeFieldId(fieldName);
         writeIntField(val);
     }
 
     /** {@inheritDoc} */
     @Override public void writeInt(int val) throws BinaryObjectException {
-        doWriteInt(val);
+        out.writeInt(val);
     }
 
     /** {@inheritDoc} */
     @Override public void writeLong(String fieldName, long val) throws BinaryObjectException {
-        writeFieldId(fieldName, LONG);
+        writeFieldId(fieldName);
         writeLongField(val);
     }
 
     /** {@inheritDoc} */
     @Override public void writeLong(long val) throws BinaryObjectException {
-        doWriteLong(val);
+        out.writeLong(val);
     }
 
     /** {@inheritDoc} */
     @Override public void writeFloat(String fieldName, float val) throws BinaryObjectException {
-        writeFieldId(fieldName, FLOAT);
+        writeFieldId(fieldName);
         writeFloatField(val);
     }
 
     /** {@inheritDoc} */
     @Override public void writeFloat(float val) throws BinaryObjectException {
-        doWriteFloat(val);
+        out.writeFloat(val);
     }
 
     /** {@inheritDoc} */
     @Override public void writeDouble(String fieldName, double val) throws BinaryObjectException {
-        writeFieldId(fieldName, DOUBLE);
+        writeFieldId(fieldName);
         writeDoubleField(val);
     }
 
     /** {@inheritDoc} */
     @Override public void writeDouble(double val) throws BinaryObjectException {
-        doWriteDouble(val);
+        out.writeDouble(val);
     }
 
     /** {@inheritDoc} */
     @Override public void writeChar(String fieldName, char val) throws BinaryObjectException {
-        writeFieldId(fieldName, CHAR);
+        writeFieldId(fieldName);
         writeCharField(val);
     }
 
     /** {@inheritDoc} */
     @Override public void writeChar(char val) throws BinaryObjectException {
-        doWriteChar(val);
+        out.writeChar(val);
     }
 
     /** {@inheritDoc} */
     @Override public void writeBoolean(String fieldName, boolean val) throws BinaryObjectException {
-        writeFieldId(fieldName, BOOLEAN);
+        writeFieldId(fieldName);
         writeBooleanField(val);
     }
 
     /** {@inheritDoc} */
     @Override public void writeBoolean(boolean val) throws BinaryObjectException {
-        doWriteBoolean(val);
+        out.writeBoolean(val);
     }
 
     /** {@inheritDoc} */
     @Override public void writeDecimal(String fieldName, @Nullable BigDecimal val) throws BinaryObjectException {
-        writeFieldId(fieldName, DECIMAL);
+        writeFieldId(fieldName);
         writeDecimalField(val);
     }
 
@@ -1361,7 +1348,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
 
     /** {@inheritDoc} */
     @Override public void writeString(String fieldName, @Nullable String val) throws BinaryObjectException {
-        writeFieldId(fieldName, STRING);
+        writeFieldId(fieldName);
         writeStringField(val);
     }
 
@@ -1372,7 +1359,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
 
     /** {@inheritDoc} */
     @Override public void writeUuid(String fieldName, @Nullable UUID val) throws BinaryObjectException {
-        writeFieldId(fieldName, UUID);
+        writeFieldId(fieldName);
         writeUuidField(val);
     }
 
@@ -1383,7 +1370,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
 
     /** {@inheritDoc} */
     @Override public void writeDate(String fieldName, @Nullable Date val) throws BinaryObjectException {
-        writeFieldId(fieldName, DATE);
+        writeFieldId(fieldName);
         writeDateField(val);
     }
 
@@ -1394,7 +1381,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
 
     /** {@inheritDoc} */
     @Override public void writeTimestamp(String fieldName, @Nullable Timestamp val) throws BinaryObjectException {
-        writeFieldId(fieldName, TIMESTAMP);
+        writeFieldId(fieldName);
         writeTimestampField(val);
     }
 
@@ -1405,7 +1392,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
 
     /** {@inheritDoc} */
     @Override public void writeObject(String fieldName, @Nullable Object obj) throws BinaryObjectException {
-        writeFieldId(fieldName, OBJ);
+        writeFieldId(fieldName);
         writeObjectField(obj);
     }
 
@@ -1417,9 +1404,9 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
     /** {@inheritDoc} */
     @Override public void writeObjectDetached(@Nullable Object obj) throws BinaryObjectException {
         if (obj == null)
-            doWriteByte(NULL);
+            out.writeByte(NULL);
         else {
-            BinaryWriterExImpl writer = new BinaryWriterExImpl(ctx, out, new IdentityHashMap<Object, Integer>());
+            BinaryWriterExImpl writer = new BinaryWriterExImpl(ctx, out, schema, null);
 
             writer.marshal(obj);
         }
@@ -1427,7 +1414,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
 
     /** {@inheritDoc} */
     @Override public void writeByteArray(String fieldName, @Nullable byte[] val) throws BinaryObjectException {
-        writeFieldId(fieldName, BYTE_ARR);
+        writeFieldId(fieldName);
         writeByteArrayField(val);
     }
 
@@ -1438,7 +1425,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
 
     /** {@inheritDoc} */
     @Override public void writeShortArray(String fieldName, @Nullable short[] val) throws BinaryObjectException {
-        writeFieldId(fieldName, SHORT_ARR);
+        writeFieldId(fieldName);
         writeShortArrayField(val);
     }
 
@@ -1449,7 +1436,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
 
     /** {@inheritDoc} */
     @Override public void writeIntArray(String fieldName, @Nullable int[] val) throws BinaryObjectException {
-        writeFieldId(fieldName, INT_ARR);
+        writeFieldId(fieldName);
         writeIntArrayField(val);
     }
 
@@ -1460,7 +1447,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
 
     /** {@inheritDoc} */
     @Override public void writeLongArray(String fieldName, @Nullable long[] val) throws BinaryObjectException {
-        writeFieldId(fieldName, LONG_ARR);
+        writeFieldId(fieldName);
         writeLongArrayField(val);
     }
 
@@ -1471,7 +1458,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
 
     /** {@inheritDoc} */
     @Override public void writeFloatArray(String fieldName, @Nullable float[] val) throws BinaryObjectException {
-        writeFieldId(fieldName, FLOAT_ARR);
+        writeFieldId(fieldName);
         writeFloatArrayField(val);
     }
 
@@ -1483,7 +1470,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
     /** {@inheritDoc} */
     @Override public void writeDoubleArray(String fieldName, @Nullable double[] val)
         throws BinaryObjectException {
-        writeFieldId(fieldName, DOUBLE_ARR);
+        writeFieldId(fieldName);
         writeDoubleArrayField(val);
     }
 
@@ -1494,7 +1481,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
 
     /** {@inheritDoc} */
     @Override public void writeCharArray(String fieldName, @Nullable char[] val) throws BinaryObjectException {
-        writeFieldId(fieldName, CHAR_ARR);
+        writeFieldId(fieldName);
         writeCharArrayField(val);
     }
 
@@ -1506,7 +1493,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
     /** {@inheritDoc} */
     @Override public void writeBooleanArray(String fieldName, @Nullable boolean[] val)
         throws BinaryObjectException {
-        writeFieldId(fieldName, BOOLEAN_ARR);
+        writeFieldId(fieldName);
         writeBooleanArrayField(val);
     }
 
@@ -1518,7 +1505,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
     /** {@inheritDoc} */
     @Override public void writeDecimalArray(String fieldName, @Nullable BigDecimal[] val)
         throws BinaryObjectException {
-        writeFieldId(fieldName, DECIMAL_ARR);
+        writeFieldId(fieldName);
         writeDecimalArrayField(val);
     }
 
@@ -1530,7 +1517,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
     /** {@inheritDoc} */
     @Override public void writeStringArray(String fieldName, @Nullable String[] val)
         throws BinaryObjectException {
-        writeFieldId(fieldName, STRING_ARR);
+        writeFieldId(fieldName);
         writeStringArrayField(val);
     }
 
@@ -1541,7 +1528,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
 
     /** {@inheritDoc} */
     @Override public void writeUuidArray(String fieldName, @Nullable UUID[] val) throws BinaryObjectException {
-        writeFieldId(fieldName, UUID_ARR);
+        writeFieldId(fieldName);
         writeUuidArrayField(val);
     }
 
@@ -1552,7 +1539,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
 
     /** {@inheritDoc} */
     @Override public void writeDateArray(String fieldName, @Nullable Date[] val) throws BinaryObjectException {
-        writeFieldId(fieldName, DATE_ARR);
+        writeFieldId(fieldName);
         writeDateArrayField(val);
     }
 
@@ -1563,7 +1550,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
 
     /** {@inheritDoc} */
     @Override public void writeTimestampArray(String fieldName, @Nullable Timestamp[] val) throws BinaryObjectException {
-        writeFieldId(fieldName, TIMESTAMP_ARR);
+        writeFieldId(fieldName);
         writeTimestampArrayField(val);
     }
 
@@ -1574,7 +1561,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
 
      /** {@inheritDoc} */
     @Override public void writeObjectArray(String fieldName, @Nullable Object[] val) throws BinaryObjectException {
-        writeFieldId(fieldName, OBJ_ARR);
+        writeFieldId(fieldName);
         writeObjectArrayField(val);
     }
 
@@ -1586,7 +1573,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
     /** {@inheritDoc} */
     @Override public <T> void writeCollection(String fieldName, @Nullable Collection<T> col)
         throws BinaryObjectException {
-        writeFieldId(fieldName, COL);
+        writeFieldId(fieldName);
         writeCollectionField(col);
     }
 
@@ -1598,7 +1585,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
     /** {@inheritDoc} */
     @Override public <K, V> void writeMap(String fieldName, @Nullable Map<K, V> map)
         throws BinaryObjectException {
-        writeFieldId(fieldName, MAP);
+        writeFieldId(fieldName);
         writeMapField(map);
     }
 
@@ -1609,7 +1596,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
 
     /** {@inheritDoc} */
     @Override public <T extends Enum<?>> void writeEnum(String fieldName, T val) throws BinaryObjectException {
-        writeFieldId(fieldName, ENUM);
+        writeFieldId(fieldName);
         writeEnumField(val);
     }
 
@@ -1620,7 +1607,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
 
     /** {@inheritDoc} */
     @Override public <T extends Enum<?>> void writeEnumArray(String fieldName, T[] val) throws BinaryObjectException {
-        writeFieldId(fieldName, ENUM_ARR);
+        writeFieldId(fieldName);
         writeEnumArrayField(val);
     }
 
@@ -1672,22 +1659,22 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
 
     /** {@inheritDoc} */
     @Override public void writeByte(int v) throws IOException {
-        doWriteByte((byte) v);
+        out.writeByte((byte) v);
     }
 
     /** {@inheritDoc} */
     @Override public void writeShort(int v) throws IOException {
-        doWriteShort((short) v);
+        out.writeShort((short) v);
     }
 
     /** {@inheritDoc} */
     @Override public void writeChar(int v) throws IOException {
-        doWriteChar((char) v);
+        out.writeChar((char) v);
     }
 
     /** {@inheritDoc} */
     @Override public void write(int b) throws IOException {
-        doWriteByte((byte) b);
+        out.writeByte((byte) b);
     }
 
     /** {@inheritDoc} */
@@ -1709,12 +1696,11 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      * @param fieldName Field name.
      * @throws org.apache.ignite.binary.BinaryObjectException If fields are not allowed.
      */
-    private void writeFieldId(String fieldName, byte fieldType) throws BinaryObjectException {
+    private void writeFieldId(String fieldName) throws BinaryObjectException {
         A.notNull(fieldName, "fieldName");
 
         if (rawOffPos != 0)
-            throw new BinaryObjectException("Individual field can't be written after raw writer is acquired " +
-                "via rawWriter() method. Consider fixing serialization logic for class: " + cls.getName());
+            throw new BinaryObjectException("Individual field can't be written after raw writer is acquired.");
 
         if (idMapper == null)
             idMapper = ctx.userTypeIdMapper(typeId);
@@ -1722,9 +1708,6 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
         int id = idMapper.fieldId(typeId, fieldName);
 
         writeFieldId(id);
-
-        if (metaEnabled)
-            metaHashSum = 31 * metaHashSum + (id + fieldType);
     }
 
     /**
@@ -1734,17 +1717,22 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
     public void writeFieldId(int fieldId) {
         int fieldOff = out.position() - start;
 
-        if (schema == null) {
-            schema = SCHEMA.get();
+        // Advance schema hash.
+        schemaId = PortableUtils.updateSchemaId(schemaId, fieldId);
 
-            if (schema == null) {
-                schema = new SchemaHolder();
+        schema.push(fieldId, fieldOff);
 
-                SCHEMA.set(schema);
-            }
-        }
+        fieldCnt++;
+    }
 
-        schemaId = PortableUtils.updateSchemaId(schemaId, fieldId);
+    /**
+     * Write field ID without schema ID update. This method should be used when schema ID is stable because class
+     * is seializable.
+     *
+     * @param fieldId Field ID.
+     */
+    public void writeFieldIdNoSchemaUpdate(int fieldId) {
+        int fieldOff = out.position() - start;
 
         schema.push(fieldId, fieldOff);
 
@@ -1752,7 +1740,14 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
     }
 
     /**
-     * @return Current schema ID.
+     * @param schemaId Schema ID.
+     */
+    public void schemaId(int schemaId) {
+        this.schemaId = schemaId;
+    }
+
+    /**
+     * @return Schema ID.
      */
     public int schemaId() {
         return schemaId;
@@ -1771,22 +1766,43 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
     }
 
     /**
+     * Get current handles. If they are {@code null}, then we should create them. Otherwise we will not see updates
+     * performed by child writers.
+     *
+     * @return Handles.
+     */
+    private BinaryWriterHandles handles() {
+        if (handles == null)
+            handles = new BinaryWriterHandles();
+
+        return handles;
+    }
+
+    /**
      * Attempts to write the object as a handle.
      *
      * @param obj Object to write.
      * @return {@code true} if the object has been written as a handle.
      */
     boolean tryWriteAsHandle(Object obj) {
-        int handle = handle(obj);
+        assert obj != null;
 
-        if (handle >= 0) {
-            doWriteByte(GridPortableMarshaller.HANDLE);
-            doWriteInt(handle);
+        int pos = out.position();
+
+        BinaryWriterHandles handles0 = handles();
+
+        int old = handles0.put(obj, pos);
+
+        if (old == BinaryWriterHandles.POS_NULL)
+            return false;
+        else {
+            out.unsafeEnsure(1 + 4);
+
+            out.unsafeWriteByte(GridPortableMarshaller.HANDLE);
+            out.unsafeWriteInt(pos - old);
 
             return true;
         }
-
-        return false;
     }
 
     /**
@@ -1796,9 +1812,9 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      * @return New writer.
      */
     public BinaryWriterExImpl newWriter(int typeId) {
-        BinaryWriterExImpl res = new BinaryWriterExImpl(ctx, out, handles);
+        BinaryWriterExImpl res = new BinaryWriterExImpl(ctx, out, schema, handles());
 
-        res.typeId = typeId;
+        res.typeId(typeId);
 
         return res;
     }
@@ -1809,138 +1825,4 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
     public PortableContext context() {
         return ctx;
     }
-
-    /**
-     * Schema holder.
-     */
-    private static class SchemaHolder {
-        /** Grow step. */
-        private static final int GROW_STEP = 64;
-
-        /** Maximum stable size. */
-        private static final int MAX_SIZE = 1024;
-
-        /** Data. */
-        private int[] data;
-
-        /** Index. */
-        private int idx;
-
-        /**
-         * Constructor.
-         */
-        public SchemaHolder() {
-            data = new int[GROW_STEP];
-        }
-
-        /**
-         * Push another frame.
-         *
-         * @param id Field ID.
-         * @param off Field offset.
-         */
-        public void push(int id, int off) {
-            if (idx == data.length) {
-                int[] data0 = new int[data.length + GROW_STEP];
-
-                System.arraycopy(data, 0, data0, 0, data.length);
-
-                data = data0;
-            }
-
-            data[idx] = id;
-            data[idx + 1] = off;
-
-            idx += 2;
-        }
-
-        /**
-         * Build the schema.
-         *
-         * @param builder Builder.
-         * @param fieldCnt Fields count.
-         */
-        public void build(PortableSchema.Builder builder, int fieldCnt) {
-            for (int curIdx = idx - fieldCnt * 2; curIdx < idx; curIdx += 2)
-                builder.addField(data[curIdx]);
-        }
-
-        /**
-         * Write collected frames and pop them.
-         *
-         * @param writer Writer.
-         * @param fieldCnt Count.
-         * @param compactFooter Whether footer should be written in compact form.
-         * @return Amount of bytes dedicated to each field offset. Could be 1, 2 or 4.
-         */
-        public int write(BinaryWriterExImpl writer, int fieldCnt, boolean compactFooter) {
-            int startIdx = idx - fieldCnt * 2;
-
-            assert startIdx >= 0;
-
-            int lastOffset = data[idx - 1];
-
-            int res;
-
-            if (compactFooter) {
-                if (lastOffset < MAX_OFFSET_1) {
-                    for (int curIdx = startIdx + 1; curIdx < idx; curIdx += 2)
-                        writer.writeByte((byte)data[curIdx]);
-
-                    res = PortableUtils.OFFSET_1;
-                }
-                else if (lastOffset < MAX_OFFSET_2) {
-                    for (int curIdx = startIdx + 1; curIdx < idx; curIdx += 2)
-                        writer.writeShort((short)data[curIdx]);
-
-                    res = PortableUtils.OFFSET_2;
-                }
-                else {
-                    for (int curIdx = startIdx + 1; curIdx < idx; curIdx += 2)
-                        writer.writeInt(data[curIdx]);
-
-                    res = PortableUtils.OFFSET_4;
-                }
-            }
-            else {
-                if (lastOffset < MAX_OFFSET_1) {
-                    for (int curIdx = startIdx; curIdx < idx;) {
-                        writer.writeInt(data[curIdx++]);
-                        writer.writeByte((byte) data[curIdx++]);
-                    }
-
-                    res = PortableUtils.OFFSET_1;
-                }
-                else if (lastOffset < MAX_OFFSET_2) {
-                    for (int curIdx = startIdx; curIdx < idx;) {
-                        writer.writeInt(data[curIdx++]);
-                        writer.writeShort((short)data[curIdx++]);
-                    }
-
-                    res = PortableUtils.OFFSET_2;
-                }
-                else {
-                    for (int curIdx = startIdx; curIdx < idx;) {
-                        writer.writeInt(data[curIdx++]);
-                        writer.writeInt(data[curIdx++]);
-                    }
-
-                    res = PortableUtils.OFFSET_4;
-                }
-            }
-
-            return res;
-        }
-
-        /**
-         * Pop current object's frame.
-         */
-        public void pop(int fieldCnt) {
-            idx = idx - fieldCnt * 2;
-
-            // Shrink data array if needed.
-            if (idx == 0 && data.length > MAX_SIZE)
-                data = new int[MAX_SIZE];
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4a1af37e/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriterHandles.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriterHandles.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriterHandles.java
new file mode 100644
index 0000000..2a47a2b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriterHandles.java
@@ -0,0 +1,101 @@
+/*
+ * 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.ignite.internal.portable;
+
+import java.util.IdentityHashMap;
+
+/**
+ * Writer handles. Aimed to delay hash map allocation for some time until it is clearly evident that it is needed.
+ */
+public class BinaryWriterHandles {
+    /** Value denoting null position. */
+    public static final int POS_NULL = -1;
+
+    /** Mode: empty. */
+    private static final int MODE_EMPTY = 0;
+
+    /** Mode: single object. */
+    private static final int MODE_SINGLE = 1;
+
+    /** Mode: multiple objects. */
+    private static final int MODE_MULTIPLE = 2;
+
+    /** Data. This is either an object or a map. */
+    private Object data;
+
+    /** Position.  */
+    private int singlePos;
+
+    /** Mode. */
+    private int mode = MODE_EMPTY;
+
+    /**
+     * Put object to registry and return previous position (if any).
+     *
+     * @param obj Object.
+     * @param pos Position.
+     * @return Old position.
+     */
+    @SuppressWarnings("unchecked")
+    public int put(Object obj, int pos) {
+        assert obj != null;
+        assert pos >= 0;
+
+        switch (mode) {
+            case MODE_EMPTY:
+                this.data = obj;
+                this.singlePos = pos;
+                this.mode = MODE_SINGLE;
+
+                return POS_NULL;
+
+            case MODE_SINGLE:
+                if (this.data == obj)
+                    return singlePos;
+                else {
+                    IdentityHashMap<Object, Integer> newData = new IdentityHashMap<>(2);
+
+                    newData.put(data, singlePos);
+                    newData.put(obj, pos);
+
+                    this.data = newData;
+                    this.singlePos = -1;
+                    this.mode = MODE_MULTIPLE;
+
+                    return POS_NULL;
+                }
+
+            default:
+                assert mode == MODE_MULTIPLE;
+
+                IdentityHashMap<Object, Integer> data0 = (IdentityHashMap<Object, Integer>)data;
+
+                Integer oldPos = data0.put(obj, pos);
+
+                if (oldPos != null) {
+                    // Restore initial position and return it.
+                    data0.put(obj, oldPos);
+
+                    return oldPos;
+                }
+                else
+                    return POS_NULL;
+
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4a1af37e/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriterSchemaHolder.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriterSchemaHolder.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriterSchemaHolder.java
new file mode 100644
index 0000000..c7400d0
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriterSchemaHolder.java
@@ -0,0 +1,148 @@
+/*
+ * 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.ignite.internal.portable;
+
+import org.apache.ignite.internal.portable.streams.PortableOutputStream;
+
+/**
+ * Binary writer schema holder.
+ */
+public class BinaryWriterSchemaHolder {
+    /** Maximum offset which fits in 1 byte. */
+    private static final int MAX_OFFSET_1 = 1 << 8;
+
+    /** Maximum offset which fits in 2 bytes. */
+    private static final int MAX_OFFSET_2 = 1 << 16;
+
+    /** Grow step. */
+    private static final int GROW_STEP = 64;
+
+    /** Data. */
+    private int[] data = new int[GROW_STEP];
+
+    /** Index. */
+    private int idx;
+
+    /**
+     * Push another frame.
+     *
+     * @param id Field ID.
+     * @param off Field offset.
+     */
+    public void push(int id, int off) {
+        if (idx == data.length) {
+            int[] data0 = new int[data.length + GROW_STEP];
+
+            System.arraycopy(data, 0, data0, 0, data.length);
+
+            data = data0;
+        }
+
+        data[idx] = id;
+        data[idx + 1] = off;
+
+        idx += 2;
+    }
+
+    /**
+     * Build the schema.
+     *
+     * @param builder Builder.
+     * @param fieldCnt Fields count.
+     */
+    public void build(PortableSchema.Builder builder, int fieldCnt) {
+        for (int curIdx = idx - fieldCnt * 2; curIdx < idx; curIdx += 2)
+            builder.addField(data[curIdx]);
+    }
+
+    /**
+     * Write collected frames and pop them.
+     *
+     * @param out Output stream.
+     * @param fieldCnt Count.
+     * @param compactFooter Whether footer should be written in compact form.
+     * @return Amount of bytes dedicated to each field offset. Could be 1, 2 or 4.
+     */
+    public int write(PortableOutputStream out, int fieldCnt, boolean compactFooter) {
+        int startIdx = idx - fieldCnt * 2;
+        assert startIdx >= 0;
+
+        // Ensure there are at least 8 bytes for each field to allow for unsafe writes.
+        out.unsafeEnsure(fieldCnt << 3);
+
+        int lastOffset = data[idx - 1];
+
+        int res;
+
+        if (compactFooter) {
+            if (lastOffset < MAX_OFFSET_1) {
+                for (int curIdx = startIdx + 1; curIdx < idx; curIdx += 2)
+                    out.unsafeWriteByte((byte)data[curIdx]);
+
+                res = PortableUtils.OFFSET_1;
+            }
+            else if (lastOffset < MAX_OFFSET_2) {
+                for (int curIdx = startIdx + 1; curIdx < idx; curIdx += 2)
+                    out.unsafeWriteShort((short) data[curIdx]);
+
+                res = PortableUtils.OFFSET_2;
+            }
+            else {
+                for (int curIdx = startIdx + 1; curIdx < idx; curIdx += 2)
+                    out.unsafeWriteInt(data[curIdx]);
+
+                res = PortableUtils.OFFSET_4;
+            }
+        }
+        else {
+            if (lastOffset < MAX_OFFSET_1) {
+                for (int curIdx = startIdx; curIdx < idx;) {
+                    out.unsafeWriteInt(data[curIdx++]);
+                    out.unsafeWriteByte((byte) data[curIdx++]);
+                }
+
+                res = PortableUtils.OFFSET_1;
+            }
+            else if (lastOffset < MAX_OFFSET_2) {
+                for (int curIdx = startIdx; curIdx < idx;) {
+                    out.unsafeWriteInt(data[curIdx++]);
+                    out.unsafeWriteShort((short) data[curIdx++]);
+                }
+
+                res = PortableUtils.OFFSET_2;
+            }
+            else {
+                for (int curIdx = startIdx; curIdx < idx;) {
+                    out.unsafeWriteInt(data[curIdx++]);
+                    out.unsafeWriteInt(data[curIdx++]);
+                }
+
+                res = PortableUtils.OFFSET_4;
+            }
+        }
+
+        return res;
+    }
+
+    /**
+     * Pop current object's frame.
+     */
+    public void pop(int fieldCnt) {
+        idx = idx - fieldCnt * 2;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4a1af37e/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java
index 056a7c7..9c61ef2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.portable;
 
+import org.apache.ignite.internal.portable.streams.PortableHeapInputStream;
 import org.apache.ignite.internal.portable.streams.PortableInputStream;
 import org.apache.ignite.internal.portable.streams.PortableOutputStream;
 import org.apache.ignite.binary.BinaryObjectException;
@@ -254,7 +255,8 @@ public class GridPortableMarshaller {
     @Nullable public <T> T unmarshal(byte[] bytes, @Nullable ClassLoader clsLdr) throws BinaryObjectException {
         assert bytes != null;
 
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx, bytes, 0, clsLdr);
+        BinaryReaderExImpl reader =
+            new BinaryReaderExImpl(ctx, PortableHeapInputStream.create(bytes, 0), clsLdr, new BinaryReaderHandles());
 
         return (T)reader.unmarshal();
     }
@@ -283,7 +285,8 @@ public class GridPortableMarshaller {
         if (arr[0] == NULL)
             return null;
 
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx, arr, 0, ldr);
+        BinaryReaderExImpl reader =
+            new BinaryReaderExImpl(ctx, PortableHeapInputStream.create(arr, 0), ldr, new BinaryReaderHandles());
 
         return (T)reader.deserialize();
     }
@@ -295,7 +298,7 @@ public class GridPortableMarshaller {
      * @return Writer.
      */
     public BinaryWriterExImpl writer(PortableOutputStream out) {
-        return new BinaryWriterExImpl(ctx, out);
+        return new BinaryWriterExImpl(ctx, out, BinaryThreadLocalContext.get().schemaHolder(), null);
     }
 
     /**
@@ -306,7 +309,7 @@ public class GridPortableMarshaller {
      */
     public BinaryReaderExImpl reader(PortableInputStream in) {
         // TODO: IGNITE-1272 - Is class loader needed here?
-        return new BinaryReaderExImpl(ctx, in, in.position(), null);
+        return new BinaryReaderExImpl(ctx, in, null, new BinaryReaderHandles());
     }
 
     /**