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:29:00 UTC

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

IGNITE-1917: Binary protocol performance optimizations.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/4a1af37e
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/4a1af37e
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/4a1af37e

Branch: refs/heads/ignite-1282
Commit: 4a1af37e39783d57b924138fb1e4aeddc137fc8a
Parents: 5ea0625
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Fri Nov 20 09:29:19 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Nov 20 09:29:19 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/IgniteKernal.java    |    5 +-
 .../internal/portable/BinaryEnumCache.java      |   69 +
 .../internal/portable/BinaryFieldAccessor.java  |  805 ++++++++
 .../portable/BinaryMetadataCollector.java       |   66 +-
 .../internal/portable/BinaryObjectEx.java       |    8 +-
 .../internal/portable/BinaryObjectImpl.java     |   47 +-
 .../portable/BinaryObjectOffheapImpl.java       |  105 +-
 .../internal/portable/BinaryReaderExImpl.java   | 1823 +++++++++---------
 .../internal/portable/BinaryReaderHandles.java  |  108 ++
 .../portable/BinaryThreadLocalContext.java      |   69 +
 .../internal/portable/BinaryTypeImpl.java       |    1 +
 .../internal/portable/BinaryWriteMode.java      |  178 ++
 .../internal/portable/BinaryWriterExImpl.java   |  866 ++++-----
 .../internal/portable/BinaryWriterHandles.java  |  101 +
 .../portable/BinaryWriterSchemaHolder.java      |  148 ++
 .../portable/GridPortableMarshaller.java        |   11 +-
 .../portable/PortableClassDescriptor.java       |  747 +------
 .../internal/portable/PortableContext.java      |    8 +-
 .../portable/PortableReaderContext.java         |  105 -
 .../internal/portable/PortableSchema.java       |  405 ++--
 .../PortableThreadLocalMemoryAllocator.java     |  162 --
 .../ignite/internal/portable/PortableUtils.java |  120 +-
 .../builder/BinaryObjectBuilderImpl.java        |    8 +-
 .../portable/builder/PortableBuilderReader.java |   54 +-
 .../streams/PortableAbstractOutputStream.java   |   40 +-
 .../streams/PortableHeapInputStream.java        |   17 +
 .../streams/PortableHeapOutputStream.java       |  100 +-
 .../streams/PortableMemoryAllocator.java        |   67 +-
 .../streams/PortableMemoryAllocatorChunk.java   |  117 ++
 .../streams/PortableOffheapOutputStream.java    |   53 +-
 .../portable/streams/PortableOutputStream.java  |   79 +
 .../streams/PortableSimpleMemoryAllocator.java  |   66 -
 .../PlatformBigEndianOutputStreamImpl.java      |   30 +
 .../memory/PlatformOutputStreamImpl.java        |   63 +
 .../ignite/internal/util/GridEnumCache.java     |   56 -
 .../portable/BinaryMarshallerSelfTest.java      |   30 +-
 .../testframework/junits/GridAbstractTest.java  |    4 +-
 .../Cache/Query/CacheQueriesTest.cs             |    2 -
 38 files changed, 3832 insertions(+), 2911 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4a1af37e/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 2b6eaad..2e8520e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -102,7 +102,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
 import org.apache.ignite.internal.processors.cache.GridCacheUtilityKey;
 import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
-import org.apache.ignite.internal.processors.cache.portable.CacheObjectBinaryProcessor;
 import org.apache.ignite.internal.processors.cache.portable.CacheObjectBinaryProcessorImpl;
 import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
 import org.apache.ignite.internal.processors.clock.GridClockSyncProcessor;
@@ -132,7 +131,7 @@ import org.apache.ignite.internal.processors.service.GridServiceProcessor;
 import org.apache.ignite.internal.processors.session.GridTaskSessionProcessor;
 import org.apache.ignite.internal.processors.task.GridTaskProcessor;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
-import org.apache.ignite.internal.util.GridEnumCache;
+import org.apache.ignite.internal.portable.BinaryEnumCache;
 import org.apache.ignite.internal.util.GridTimerTask;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
@@ -1941,7 +1940,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
             // Clean internal class/classloader caches to avoid stopped contexts held in memory.
             U.clearClassCache();
             MarshallerExclusions.clearCache();
-            GridEnumCache.clear();
+            BinaryEnumCache.clear();
 
             gw.writeLock();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4a1af37e/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryEnumCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryEnumCache.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryEnumCache.java
new file mode 100644
index 0000000..fc042e5
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryEnumCache.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.binary.BinaryObjectException;
+import org.jsr166.ConcurrentHashMap8;
+
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * Cache for enum constants.
+ */
+public class BinaryEnumCache {
+    /** Cache for enum constants. */
+    private static final ConcurrentMap<Class<?>, Object[]> ENUM_CACHE = new ConcurrentHashMap8<>();
+
+    /**
+     * Get value for the given class and ordinal.
+     *
+     * @param cls Class.
+     * @param ord Ordinal.
+     * @return Value.
+     * @throws BinaryObjectException In case of invalid ordinal.
+     */
+    @SuppressWarnings("unchecked")
+    public static <T> T get(Class<?> cls, int ord) throws BinaryObjectException {
+        assert cls != null;
+
+        if (ord >= 0) {
+            Object[] vals = ENUM_CACHE.get(cls);
+
+            if (vals == null) {
+                vals = cls.getEnumConstants();
+
+                ENUM_CACHE.putIfAbsent(cls, vals);
+            }
+
+            if (ord < vals.length)
+                return (T) vals[ord];
+            else
+                throw new BinaryObjectException("Failed to get enum value for ordinal (do you have correct class " +
+                    "version?) [cls=" + cls.getName() + ", ordinal=" + ord + ", totalValues=" + vals.length + ']');
+        }
+        else
+            return null;
+    }
+
+    /**
+     * Clears cache.
+     */
+    public static void clear() {
+        ENUM_CACHE.clear();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/4a1af37e/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryFieldAccessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryFieldAccessor.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryFieldAccessor.java
new file mode 100644
index 0000000..0eda375
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryFieldAccessor.java
@@ -0,0 +1,805 @@
+/*
+ * 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.binary.BinaryObjectException;
+import org.apache.ignite.internal.util.GridUnsafe;
+import sun.misc.Unsafe;
+
+import java.lang.reflect.Field;
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.util.Collection;
+import java.util.Date;
+import java.util.Map;
+import java.util.UUID;
+
+/**
+ * Field accessor to speedup access.
+ */
+public abstract class BinaryFieldAccessor {
+    /** Field ID. */
+    protected final int id;
+
+    /** Mode. */
+    protected final BinaryWriteMode mode;
+
+    /**
+     * Create accessor for the field.
+     *
+     * @param field Field.
+     * @param id FIeld ID.
+     * @return Accessor.
+     */
+    public static BinaryFieldAccessor create(Field field, int id) {
+        BinaryWriteMode mode = PortableUtils.mode(field.getType());
+
+        switch (mode) {
+            case P_BYTE:
+                return new BytePrimitiveAccessor(field, id);
+
+            case P_BOOLEAN:
+                return new BooleanPrimitiveAccessor(field, id);
+
+            case P_SHORT:
+                return new ShortPrimitiveAccessor(field, id);
+
+            case P_CHAR:
+                return new CharPrimitiveAccessor(field, id);
+
+            case P_INT:
+                return new IntPrimitiveAccessor(field, id);
+
+            case P_LONG:
+                return new LongPrimitiveAccessor(field, id);
+
+            case P_FLOAT:
+                return new FloatPrimitiveAccessor(field, id);
+
+            case P_DOUBLE:
+                return new DoublePrimitiveAccessor(field, id);
+
+            default:
+                return new DefaultAccessor(field, id, mode);
+        }
+    }
+
+    /**
+     * Protected constructor.
+     *
+     * @param id Field ID.
+     * @param mode Mode;
+     */
+    protected BinaryFieldAccessor(int id, BinaryWriteMode mode) {
+        assert id != 0;
+        assert mode != null;
+
+        this.id = id;
+        this.mode = mode;
+    }
+
+    /**
+     * Get mode.
+     *
+     * @return Mode.
+     */
+    public BinaryWriteMode mode() {
+        return mode;
+    }
+
+    /**
+     * Write field.
+     *
+     * @param obj Object.
+     * @param writer Writer.
+     * @throws BinaryObjectException If failed.
+     */
+    public abstract void write(Object obj, BinaryWriterExImpl writer) throws BinaryObjectException;
+
+    /**
+     * Read field.
+     *
+     * @param obj Object.
+     * @param reader Reader.
+     * @throws BinaryObjectException If failed.
+     */
+    public abstract void read(Object obj, BinaryReaderExImpl reader) throws BinaryObjectException;
+
+    /**
+     * Base primitive field accessor.
+     */
+    private static abstract class AbstractPrimitiveAccessor extends BinaryFieldAccessor {
+        /** Unsafe instance. */
+        protected static final Unsafe UNSAFE = GridUnsafe.unsafe();
+
+        /** Offset. */
+        protected final long offset;
+
+        /**
+         * Constructor.
+         *
+         * @param field Field.
+         * @param id Field ID.
+         * @param mode Mode.
+         */
+        protected AbstractPrimitiveAccessor(Field field, int id, BinaryWriteMode mode) {
+            super(id, mode);
+
+            assert field != null;
+
+            offset = UNSAFE.objectFieldOffset(field);
+        }
+    }
+
+    /**
+     * Byte field accessor.
+     */
+    private static class BytePrimitiveAccessor extends AbstractPrimitiveAccessor {
+        /**
+         * Constructor.
+         *
+         * @param field Field.
+         */
+        public BytePrimitiveAccessor(Field field, int id) {
+            super(field, id, BinaryWriteMode.P_BYTE);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Object obj, BinaryWriterExImpl writer) throws BinaryObjectException {
+            writer.writeFieldIdNoSchemaUpdate(id);
+
+            byte val = UNSAFE.getByte(obj, offset);
+
+            writer.writeByteFieldPrimitive(val);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void read(Object obj, BinaryReaderExImpl reader) throws BinaryObjectException {
+            byte val = reader.readByte(id);
+
+            UNSAFE.putByte(obj, offset, val);
+        }
+    }
+
+    /**
+     * Boolean field accessor.
+     */
+    private static class BooleanPrimitiveAccessor extends AbstractPrimitiveAccessor {
+        /**
+         * Constructor.
+         *
+         * @param field Field.
+         */
+        public BooleanPrimitiveAccessor(Field field, int id) {
+            super(field, id, BinaryWriteMode.P_BOOLEAN);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Object obj, BinaryWriterExImpl writer) throws BinaryObjectException {
+            writer.writeFieldIdNoSchemaUpdate(id);
+
+            boolean val = UNSAFE.getBoolean(obj, offset);
+
+            writer.writeBooleanFieldPrimitive(val);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void read(Object obj, BinaryReaderExImpl reader) throws BinaryObjectException {
+            boolean val = reader.readBoolean(id);
+
+            UNSAFE.putBoolean(obj, offset, val);
+        }
+    }
+
+    /**
+     * Short field accessor.
+     */
+    private static class ShortPrimitiveAccessor extends AbstractPrimitiveAccessor {
+        /**
+         * Constructor.
+         *
+         * @param field Field.
+         */
+        public ShortPrimitiveAccessor(Field field, int id) {
+            super(field, id, BinaryWriteMode.P_SHORT);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Object obj, BinaryWriterExImpl writer) throws BinaryObjectException {
+            writer.writeFieldIdNoSchemaUpdate(id);
+
+            short val = UNSAFE.getShort(obj, offset);
+
+            writer.writeShortFieldPrimitive(val);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void read(Object obj, BinaryReaderExImpl reader) throws BinaryObjectException {
+            short val = reader.readShort(id);
+
+            UNSAFE.putShort(obj, offset, val);
+        }
+    }
+
+    /**
+     * Char field accessor.
+     */
+    private static class CharPrimitiveAccessor extends AbstractPrimitiveAccessor {
+        /**
+         * Constructor.
+         *
+         * @param field Field.
+         */
+        public CharPrimitiveAccessor(Field field, int id) {
+            super(field, id, BinaryWriteMode.P_CHAR);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Object obj, BinaryWriterExImpl writer) throws BinaryObjectException {
+            writer.writeFieldIdNoSchemaUpdate(id);
+
+            char val = UNSAFE.getChar(obj, offset);
+
+            writer.writeCharFieldPrimitive(val);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void read(Object obj, BinaryReaderExImpl reader) throws BinaryObjectException {
+            char val = reader.readChar(id);
+
+            UNSAFE.putChar(obj, offset, val);
+        }
+    }
+
+    /**
+     * Int field accessor.
+     */
+    private static class IntPrimitiveAccessor extends AbstractPrimitiveAccessor {
+        /**
+         * Constructor.
+         *
+         * @param field Field.
+         */
+        public IntPrimitiveAccessor(Field field, int id) {
+            super(field, id, BinaryWriteMode.P_INT);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Object obj, BinaryWriterExImpl writer) throws BinaryObjectException {
+            writer.writeFieldIdNoSchemaUpdate(id);
+
+            int val = UNSAFE.getInt(obj, offset);
+
+            writer.writeIntFieldPrimitive(val);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void read(Object obj, BinaryReaderExImpl reader) throws BinaryObjectException {
+            int val = reader.readInt(id);
+
+            UNSAFE.putInt(obj, offset, val);
+        }
+    }
+
+    /**
+     * Long field accessor.
+     */
+    private static class LongPrimitiveAccessor extends AbstractPrimitiveAccessor {
+        /**
+         * Constructor.
+         *
+         * @param field Field.
+         */
+        public LongPrimitiveAccessor(Field field, int id) {
+            super(field, id, BinaryWriteMode.P_LONG);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Object obj, BinaryWriterExImpl writer) throws BinaryObjectException {
+            writer.writeFieldIdNoSchemaUpdate(id);
+
+            long val = UNSAFE.getLong(obj, offset);
+
+            writer.writeLongFieldPrimitive(val);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void read(Object obj, BinaryReaderExImpl reader) throws BinaryObjectException {
+            long val = reader.readLong(id);
+
+            UNSAFE.putLong(obj, offset, val);
+        }
+    }
+
+    /**
+     * Float field accessor.
+     */
+    private static class FloatPrimitiveAccessor extends AbstractPrimitiveAccessor {
+        /**
+         * Constructor.
+         *
+         * @param field Field.
+         */
+        public FloatPrimitiveAccessor(Field field, int id) {
+            super(field, id, BinaryWriteMode.P_FLOAT);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Object obj, BinaryWriterExImpl writer) throws BinaryObjectException {
+            writer.writeFieldIdNoSchemaUpdate(id);
+
+            float val = UNSAFE.getFloat(obj, offset);
+
+            writer.writeFloatFieldPrimitive(val);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void read(Object obj, BinaryReaderExImpl reader) throws BinaryObjectException {
+            float val = reader.readFloat(id);
+
+            UNSAFE.putFloat(obj, offset, val);
+        }
+    }
+
+    /**
+     * Double field accessor.
+     */
+    private static class DoublePrimitiveAccessor extends AbstractPrimitiveAccessor {
+        /**
+         * Constructor.
+         *
+         * @param field Field.
+         */
+        public DoublePrimitiveAccessor(Field field, int id) {
+            super(field, id, BinaryWriteMode.P_DOUBLE);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Object obj, BinaryWriterExImpl writer) throws BinaryObjectException {
+            writer.writeFieldIdNoSchemaUpdate(id);
+
+            double val = UNSAFE.getDouble(obj, offset);
+
+            writer.writeDoubleFieldPrimitive(val);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void read(Object obj, BinaryReaderExImpl reader) throws BinaryObjectException {
+            double val = reader.readDouble(id);
+
+            UNSAFE.putDouble(obj, offset, val);
+        }
+    }
+
+    /**
+     * Default accessor.
+     */
+    private static class DefaultAccessor extends BinaryFieldAccessor {
+        /** Target field. */
+        private final Field field;
+
+        /**
+         * Constructor.
+         *
+         * @param field Field.
+         * @param id Field ID.
+         * @param mode Mode.
+         */
+        public DefaultAccessor(Field field, int id, BinaryWriteMode mode) {
+            super(id, mode);
+
+            assert field != null;
+
+            this.field = field;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Object obj, BinaryWriterExImpl writer) throws BinaryObjectException {
+            assert obj != null;
+            assert writer != null;
+
+            writer.writeFieldIdNoSchemaUpdate(id);
+
+            Object val;
+
+            try {
+                val = field.get(obj);
+            }
+            catch (IllegalAccessException e) {
+                throw new BinaryObjectException("Failed to get value for field: " + field, e);
+            }
+
+            switch (mode) {
+                case BYTE:
+                    writer.writeByteField((Byte) val);
+
+                    break;
+
+                case SHORT:
+                    writer.writeShortField((Short) val);
+
+                    break;
+
+                case INT:
+                    writer.writeIntField((Integer) val);
+
+                    break;
+
+                case LONG:
+                    writer.writeLongField((Long)val);
+
+                    break;
+
+                case FLOAT:
+                    writer.writeFloatField((Float)val);
+
+                    break;
+
+                case DOUBLE:
+                    writer.writeDoubleField((Double)val);
+
+                    break;
+
+                case CHAR:
+                    writer.writeCharField((Character)val);
+
+                    break;
+
+                case BOOLEAN:
+                    writer.writeBooleanField((Boolean)val);
+
+                    break;
+
+                case DECIMAL:
+                    writer.writeDecimalField((BigDecimal)val);
+
+                    break;
+
+                case STRING:
+                    writer.writeStringField((String)val);
+
+                    break;
+
+                case UUID:
+                    writer.writeUuidField((UUID)val);
+
+                    break;
+
+                case DATE:
+                    writer.writeDateField((Date)val);
+
+                    break;
+
+                case TIMESTAMP:
+                    writer.writeTimestampField((Timestamp)val);
+
+                    break;
+
+                case BYTE_ARR:
+                    writer.writeByteArrayField((byte[])val);
+
+                    break;
+
+                case SHORT_ARR:
+                    writer.writeShortArrayField((short[])val);
+
+                    break;
+
+                case INT_ARR:
+                    writer.writeIntArrayField((int[])val);
+
+                    break;
+
+                case LONG_ARR:
+                    writer.writeLongArrayField((long[])val);
+
+                    break;
+
+                case FLOAT_ARR:
+                    writer.writeFloatArrayField((float[])val);
+
+                    break;
+
+                case DOUBLE_ARR:
+                    writer.writeDoubleArrayField((double[])val);
+
+                    break;
+
+                case CHAR_ARR:
+                    writer.writeCharArrayField((char[])val);
+
+                    break;
+
+                case BOOLEAN_ARR:
+                    writer.writeBooleanArrayField((boolean[])val);
+
+                    break;
+
+                case DECIMAL_ARR:
+                    writer.writeDecimalArrayField((BigDecimal[])val);
+
+                    break;
+
+                case STRING_ARR:
+                    writer.writeStringArrayField((String[])val);
+
+                    break;
+
+                case UUID_ARR:
+                    writer.writeUuidArrayField((UUID[])val);
+
+                    break;
+
+                case DATE_ARR:
+                    writer.writeDateArrayField((Date[])val);
+
+                    break;
+
+                case TIMESTAMP_ARR:
+                    writer.writeTimestampArrayField((Timestamp[])val);
+
+                    break;
+
+                case OBJECT_ARR:
+                    writer.writeObjectArrayField((Object[])val);
+
+                    break;
+
+                case COL:
+                    writer.writeCollectionField((Collection<?>)val);
+
+                    break;
+
+                case MAP:
+                    writer.writeMapField((Map<?, ?>)val);
+
+                    break;
+
+                case MAP_ENTRY:
+                    writer.writeMapEntryField((Map.Entry<?, ?>)val);
+
+                    break;
+
+                case PORTABLE_OBJ:
+                    writer.writePortableObjectField((BinaryObjectImpl)val);
+
+                    break;
+
+                case ENUM:
+                    writer.writeEnumField((Enum<?>)val);
+
+                    break;
+
+                case ENUM_ARR:
+                    writer.writeEnumArrayField((Object[])val);
+
+                    break;
+
+                case PORTABLE:
+                case EXTERNALIZABLE:
+                case OBJECT:
+                    writer.writeObjectField(val);
+
+                    break;
+
+                case CLASS:
+                    writer.writeClassField((Class)val);
+
+                    break;
+
+                default:
+                    assert false : "Invalid mode: " + mode;
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void read(Object obj, BinaryReaderExImpl reader) throws BinaryObjectException {
+            Object val = null;
+
+            switch (mode) {
+                case BYTE:
+                    val = reader.readByteNullable(id);
+
+                    break;
+
+                case SHORT:
+                    val = reader.readShortNullable(id);
+
+                    break;
+
+                case INT:
+                    val = reader.readIntNullable(id);
+
+                    break;
+
+                case LONG:
+                    val = reader.readLongNullable(id);
+
+                    break;
+
+                case FLOAT:
+                    val = reader.readFloatNullable(id);
+
+                    break;
+
+                case DOUBLE:
+                    val = reader.readDoubleNullable(id);
+
+                    break;
+
+                case CHAR:
+                    val = reader.readCharNullable(id);
+
+                    break;
+
+                case BOOLEAN:
+                    val = reader.readBooleanNullable(id);
+
+                    break;
+
+                case DECIMAL:
+                    val = reader.readDecimal(id);
+
+                    break;
+
+                case STRING:
+                    val = reader.readString(id);
+
+                    break;
+
+                case UUID:
+                    val = reader.readUuid(id);
+
+                    break;
+
+                case DATE:
+                    val = reader.readDate(id);
+
+                    break;
+
+                case TIMESTAMP:
+                    val = reader.readTimestamp(id);
+
+                    break;
+
+                case BYTE_ARR:
+                    val = reader.readByteArray(id);
+
+                    break;
+
+                case SHORT_ARR:
+                    val = reader.readShortArray(id);
+
+                    break;
+
+                case INT_ARR:
+                    val = reader.readIntArray(id);
+
+                    break;
+
+                case LONG_ARR:
+                    val = reader.readLongArray(id);
+
+                    break;
+
+                case FLOAT_ARR:
+                    val = reader.readFloatArray(id);
+
+                    break;
+
+                case DOUBLE_ARR:
+                    val = reader.readDoubleArray(id);
+
+                    break;
+
+                case CHAR_ARR:
+                    val = reader.readCharArray(id);
+
+                    break;
+
+                case BOOLEAN_ARR:
+                    val = reader.readBooleanArray(id);
+
+                    break;
+
+                case DECIMAL_ARR:
+                    val = reader.readDecimalArray(id);
+
+                    break;
+
+                case STRING_ARR:
+                    val = reader.readStringArray(id);
+
+                    break;
+
+                case UUID_ARR:
+                    val = reader.readUuidArray(id);
+
+                    break;
+
+                case DATE_ARR:
+                    val = reader.readDateArray(id);
+
+                    break;
+
+                case TIMESTAMP_ARR:
+                    val = reader.readTimestampArray(id);
+
+                    break;
+
+                case OBJECT_ARR:
+                    val = reader.readObjectArray(id);
+
+                    break;
+
+                case COL:
+                    val = reader.readCollection(id, null);
+
+                    break;
+
+                case MAP:
+                    val = reader.readMap(id, null);
+
+                    break;
+
+                case MAP_ENTRY:
+                    val = reader.readMapEntry(id);
+
+                    break;
+
+                case PORTABLE_OBJ:
+                    val = reader.readPortableObject(id);
+
+                    break;
+
+                case ENUM:
+                    val = reader.readEnum(id, field.getType());
+
+                    break;
+
+                case ENUM_ARR:
+                    val = reader.readEnumArray(id, field.getType().getComponentType());
+
+                    break;
+
+                case PORTABLE:
+                case EXTERNALIZABLE:
+                case OBJECT:
+                    val = reader.readObject(id);
+
+                    break;
+
+                case CLASS:
+                    val = reader.readClass(id);
+
+                    break;
+
+                default:
+                    assert false : "Invalid mode: " + mode;
+            }
+
+            try {
+                if (val != null || !field.getType().isPrimitive())
+                    field.set(obj, val);
+            }
+            catch (IllegalAccessException e) {
+                throw new BinaryObjectException("Failed to set value for field: " + field, e);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4a1af37e/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadataCollector.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadataCollector.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadataCollector.java
index 28eb1d0..6ad0ad1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadataCollector.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadataCollector.java
@@ -82,163 +82,163 @@ class BinaryMetadataCollector implements BinaryWriter {
 
     /** {@inheritDoc} */
     @Override public void writeByte(String fieldName, byte val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.BYTE);
+        add(fieldName, BinaryWriteMode.BYTE);
     }
 
     /** {@inheritDoc} */
     @Override public void writeShort(String fieldName, short val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.SHORT);
+        add(fieldName, BinaryWriteMode.SHORT);
     }
 
     /** {@inheritDoc} */
     @Override public void writeInt(String fieldName, int val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.INT);
+        add(fieldName, BinaryWriteMode.INT);
     }
 
     /** {@inheritDoc} */
     @Override public void writeLong(String fieldName, long val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.LONG);
+        add(fieldName, BinaryWriteMode.LONG);
     }
 
     /** {@inheritDoc} */
     @Override public void writeFloat(String fieldName, float val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.FLOAT);
+        add(fieldName, BinaryWriteMode.FLOAT);
     }
 
     /** {@inheritDoc} */
     @Override public void writeDouble(String fieldName, double val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.DOUBLE);
+        add(fieldName, BinaryWriteMode.DOUBLE);
     }
 
     /** {@inheritDoc} */
     @Override public void writeChar(String fieldName, char val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.CHAR);
+        add(fieldName, BinaryWriteMode.CHAR);
     }
 
     /** {@inheritDoc} */
     @Override public void writeBoolean(String fieldName, boolean val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.BOOLEAN);
+        add(fieldName, BinaryWriteMode.BOOLEAN);
     }
 
     /** {@inheritDoc} */
     @Override public void writeDecimal(String fieldName, @Nullable BigDecimal val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.DECIMAL);
+        add(fieldName, BinaryWriteMode.DECIMAL);
     }
 
     /** {@inheritDoc} */
     @Override public void writeString(String fieldName, @Nullable String val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.STRING);
+        add(fieldName, BinaryWriteMode.STRING);
     }
 
     /** {@inheritDoc} */
     @Override public void writeUuid(String fieldName, @Nullable UUID val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.UUID);
+        add(fieldName, BinaryWriteMode.UUID);
     }
 
     /** {@inheritDoc} */
     @Override public void writeDate(String fieldName, @Nullable Date val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.DATE);
+        add(fieldName, BinaryWriteMode.DATE);
     }
 
     /** {@inheritDoc} */
     @Override public void writeTimestamp(String fieldName, @Nullable Timestamp val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.TIMESTAMP);
+        add(fieldName, BinaryWriteMode.TIMESTAMP);
     }
 
     /** {@inheritDoc} */
     @Override public <T extends Enum<?>> void writeEnum(String fieldName, T val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.ENUM);
+        add(fieldName, BinaryWriteMode.ENUM);
     }
 
     /** {@inheritDoc} */
     @Override public <T extends Enum<?>> void writeEnumArray(String fieldName, T[] val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.ENUM_ARR);
+        add(fieldName, BinaryWriteMode.ENUM_ARR);
     }
 
     /** {@inheritDoc} */
     @Override public void writeObject(String fieldName, @Nullable Object obj) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.OBJECT);
+        add(fieldName, BinaryWriteMode.OBJECT);
     }
 
     /** {@inheritDoc} */
     @Override public void writeByteArray(String fieldName, @Nullable byte[] val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.BYTE_ARR);
+        add(fieldName, BinaryWriteMode.BYTE_ARR);
     }
 
     /** {@inheritDoc} */
     @Override public void writeShortArray(String fieldName, @Nullable short[] val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.SHORT_ARR);
+        add(fieldName, BinaryWriteMode.SHORT_ARR);
     }
 
     /** {@inheritDoc} */
     @Override public void writeIntArray(String fieldName, @Nullable int[] val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.INT_ARR);
+        add(fieldName, BinaryWriteMode.INT_ARR);
     }
 
     /** {@inheritDoc} */
     @Override public void writeLongArray(String fieldName, @Nullable long[] val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.LONG_ARR);
+        add(fieldName, BinaryWriteMode.LONG_ARR);
     }
 
     /** {@inheritDoc} */
     @Override public void writeFloatArray(String fieldName, @Nullable float[] val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.FLOAT_ARR);
+        add(fieldName, BinaryWriteMode.FLOAT_ARR);
     }
 
     /** {@inheritDoc} */
     @Override public void writeDoubleArray(String fieldName, @Nullable double[] val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.DOUBLE_ARR);
+        add(fieldName, BinaryWriteMode.DOUBLE_ARR);
     }
 
     /** {@inheritDoc} */
     @Override public void writeCharArray(String fieldName, @Nullable char[] val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.CHAR_ARR);
+        add(fieldName, BinaryWriteMode.CHAR_ARR);
     }
 
     /** {@inheritDoc} */
     @Override public void writeBooleanArray(String fieldName, @Nullable boolean[] val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.BOOLEAN_ARR);
+        add(fieldName, BinaryWriteMode.BOOLEAN_ARR);
     }
 
     /** {@inheritDoc} */
     @Override public void writeDecimalArray(String fieldName, @Nullable BigDecimal[] val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.DECIMAL_ARR);
+        add(fieldName, BinaryWriteMode.DECIMAL_ARR);
     }
 
     /** {@inheritDoc} */
     @Override public void writeStringArray(String fieldName, @Nullable String[] val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.STRING_ARR);
+        add(fieldName, BinaryWriteMode.STRING_ARR);
     }
 
     /** {@inheritDoc} */
     @Override public void writeUuidArray(String fieldName, @Nullable UUID[] val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.UUID_ARR);
+        add(fieldName, BinaryWriteMode.UUID_ARR);
     }
 
     /** {@inheritDoc} */
     @Override public void writeDateArray(String fieldName, @Nullable Date[] val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.DATE_ARR);
+        add(fieldName, BinaryWriteMode.DATE_ARR);
     }
 
     /** {@inheritDoc} */
     @Override public void writeTimestampArray(String fieldName, @Nullable Timestamp[] val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.TIMESTAMP_ARR);
+        add(fieldName, BinaryWriteMode.TIMESTAMP_ARR);
     }
 
     /** {@inheritDoc} */
     @Override public void writeObjectArray(String fieldName, @Nullable Object[] val) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.OBJECT_ARR);
+        add(fieldName, BinaryWriteMode.OBJECT_ARR);
     }
 
     /** {@inheritDoc} */
     @Override public <T> void writeCollection(String fieldName, @Nullable Collection<T> col)
         throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.COL);
+        add(fieldName, BinaryWriteMode.COL);
     }
 
     /** {@inheritDoc} */
     @Override public <K, V> void writeMap(String fieldName, @Nullable Map<K, V> map) throws BinaryObjectException {
-        add(fieldName, PortableClassDescriptor.Mode.MAP);
+        add(fieldName, BinaryWriteMode.MAP);
     }
 
     /** {@inheritDoc} */
@@ -257,7 +257,7 @@ class BinaryMetadataCollector implements BinaryWriter {
      * @param mode Field mode.
      * @throws BinaryObjectException In case of error.
      */
-    private void add(String name, PortableClassDescriptor.Mode mode) throws BinaryObjectException {
+    private void add(String name, BinaryWriteMode mode) throws BinaryObjectException {
         assert name != null;
 
         int fieldTypeId = mode.typeId();

http://git-wip-us.apache.org/repos/asf/ignite/blob/4a1af37e/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectEx.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectEx.java
index 6902675..597fad5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectEx.java
@@ -79,7 +79,7 @@ public abstract class BinaryObjectEx implements BinaryObject {
      * @param fieldName Field name.
      * @return Field value.
      */
-    @Nullable protected abstract <F> F field(PortableReaderContext ctx, String fieldName);
+    @Nullable protected abstract <F> F field(BinaryReaderHandles ctx, String fieldName);
 
     /**
      * Get schema ID.
@@ -157,7 +157,7 @@ public abstract class BinaryObjectEx implements BinaryObject {
      * @param handles Handles for already traversed objects.
      * @return String representation.
      */
-    private String toString(PortableReaderContext ctx, IdentityHashMap<BinaryObject, Integer> handles) {
+    private String toString(BinaryReaderHandles ctx, IdentityHashMap<BinaryObject, Integer> handles) {
         int idHash = System.identityHashCode(this);
 
         BinaryType meta;
@@ -232,9 +232,9 @@ public abstract class BinaryObjectEx implements BinaryObject {
     /** {@inheritDoc} */
     @Override public String toString() {
         try {
-            PortableReaderContext ctx = new PortableReaderContext();
+            BinaryReaderHandles ctx = new BinaryReaderHandles();
 
-            ctx.setPortableHandler(start(), this);
+            ctx.put(start(), this);
 
             return toString(ctx, new IdentityHashMap<BinaryObject, Integer>());
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4a1af37e/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
index d9339f8..7ab9497 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
@@ -25,7 +25,6 @@ import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.portable.CacheObjectBinaryProcessorImpl;
-import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
@@ -33,7 +32,6 @@ import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.binary.BinaryType;
 import org.apache.ignite.binary.BinaryObject;
-import org.apache.ignite.binary.BinaryField;
 import org.jetbrains.annotations.Nullable;
 
 import java.io.Externalizable;
@@ -257,17 +255,13 @@ public final class BinaryObjectImpl extends BinaryObjectEx implements Externaliz
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Nullable @Override public <F> F field(String fieldName) throws BinaryObjectException {
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx, arr, start, null);
-
-        return (F)reader.unmarshalField(fieldName);
+        return (F)newReader().unmarshalField(fieldName);
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Nullable @Override public <F> F field(int fieldId) throws BinaryObjectException {
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx, arr, start, null);
-
-        return (F)reader.unmarshalField(fieldId);
+        return (F)newReader().unmarshalField(fieldId);
     }
 
     /** {@inheritDoc} */
@@ -400,11 +394,13 @@ public final class BinaryObjectImpl extends BinaryObjectEx implements Externaliz
 
                 break;
 
-            default: {
-                BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx, arr, start, null);
+            default:
+                BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx, PortableHeapInputStream.create(arr, fieldPos),
+                    null, new BinaryReaderHandles());
 
-                val = reader.unmarshalFieldByAbsolutePosition(fieldPos);
-            }
+                val = reader.unmarshal();
+
+                break;
         }
 
         return (F)val;
@@ -412,21 +408,15 @@ public final class BinaryObjectImpl extends BinaryObjectEx implements Externaliz
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Nullable @Override protected <F> F field(PortableReaderContext rCtx, String fieldName) {
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx,
-            new PortableHeapInputStream(arr),
-            start,
-            null,
-            rCtx);
+    @Nullable @Override protected <F> F field(BinaryReaderHandles rCtx, String fieldName) {
+        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx, PortableHeapInputStream.create(arr, start), null, rCtx);
 
         return (F)reader.unmarshalField(fieldName);
     }
 
     /** {@inheritDoc} */
     @Override public boolean hasField(String fieldName) {
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx, arr, start, null);
-
-        return reader.hasField(fieldName);
+        return newReader().hasField(fieldName);
     }
 
     /** {@inheritDoc} */
@@ -458,9 +448,7 @@ public final class BinaryObjectImpl extends BinaryObjectEx implements Externaliz
 
     /** {@inheritDoc} */
     @Override protected PortableSchema createSchema() {
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx, arr, start, null);
-
-        return reader.getOrCreateSchema();
+        return newReader().getOrCreateSchema();
     }
 
     /** {@inheritDoc} */
@@ -569,7 +557,7 @@ public final class BinaryObjectImpl extends BinaryObjectEx implements Externaliz
      */
     private Object deserializeValue() {
         // TODO: IGNITE-1272 - Deserialize with proper class loader.
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx, arr, start, null);
+        BinaryReaderExImpl reader = newReader();
 
         Object obj0 = reader.deserialize();
 
@@ -582,4 +570,13 @@ public final class BinaryObjectImpl extends BinaryObjectEx implements Externaliz
 
         return obj0;
     }
+
+    /**
+     * Create new reader for this object.
+     *
+     * @return Reader.
+     */
+    private BinaryReaderExImpl newReader() {
+        return new BinaryReaderExImpl(ctx, PortableHeapInputStream.create(arr, start), null, new BinaryReaderHandles());
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/4a1af37e/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java
index a71c98a..6f9190a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java
@@ -17,18 +17,11 @@
 
 package org.apache.ignite.internal.portable;
 
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.nio.ByteBuffer;
-import java.sql.Timestamp;
-import java.util.Date;
-import java.util.UUID;
-
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.binary.BinaryField;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryType;
 import org.apache.ignite.internal.portable.streams.PortableOffheapInputStream;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
@@ -37,13 +30,20 @@ import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
 import org.apache.ignite.plugin.extensions.communication.MessageWriter;
-import org.apache.ignite.binary.BinaryObjectException;
-import org.apache.ignite.binary.BinaryType;
-import org.apache.ignite.binary.BinaryObject;
-import org.apache.ignite.binary.BinaryField;
 import org.jetbrains.annotations.Nullable;
 import sun.misc.Unsafe;
 
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.sql.Timestamp;
+import java.util.Date;
+import java.util.UUID;
+
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.BOOLEAN;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.BYTE;
@@ -131,12 +131,7 @@ public class BinaryObjectOffheapImpl extends BinaryObjectEx implements Externali
 
     /** {@inheritDoc} */
     @Override protected PortableSchema createSchema() {
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx,
-            new PortableOffheapInputStream(ptr, size, false),
-            start,
-            null);
-
-        return reader.getOrCreateSchema();
+        return newReader().getOrCreateSchema();
     }
 
     /** {@inheritDoc} */
@@ -170,23 +165,13 @@ public class BinaryObjectOffheapImpl extends BinaryObjectEx implements Externali
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Nullable @Override public <F> F field(String fieldName) throws BinaryObjectException {
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx,
-            new PortableOffheapInputStream(ptr, size, false),
-            start,
-            null);
-
-        return (F)reader.unmarshalField(fieldName);
+        return (F)newReader().unmarshalField(fieldName);
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Nullable @Override public <F> F field(int fieldId) throws BinaryObjectException {
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx,
-            new PortableOffheapInputStream(ptr, size, false),
-            start,
-            null);
-
-        return (F)reader.unmarshalField(fieldId);
+        return (F)newReader().unmarshalField(fieldId);
     }
 
     /** {@inheritDoc} */
@@ -320,14 +305,16 @@ public class BinaryObjectOffheapImpl extends BinaryObjectEx implements Externali
 
                 break;
 
-            default: {
-                BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx,
-                    new PortableOffheapInputStream(ptr, size, false),
-                    start,
-                    null);
+            default:
+                PortableOffheapInputStream stream = new PortableOffheapInputStream(ptr, size, false);
 
-                val = reader.unmarshalFieldByAbsolutePosition(fieldPos);
-            }
+                stream.position(fieldPos);
+
+                BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx, stream, null, new BinaryReaderHandles());
+
+                val = reader.unmarshal();
+
+                break;
         }
 
         return (F)val;
@@ -335,24 +322,19 @@ public class BinaryObjectOffheapImpl extends BinaryObjectEx implements Externali
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Nullable @Override protected <F> F field(PortableReaderContext rCtx, String fieldName) {
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx,
-            new PortableOffheapInputStream(ptr, size, false),
-            start,
-            null,
-            rCtx);
+    @Nullable @Override protected <F> F field(BinaryReaderHandles rCtx, String fieldName) {
+        PortableOffheapInputStream stream = new PortableOffheapInputStream(ptr, size, false);
+
+        stream.position(start);
+
+        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx, stream, null, rCtx);
 
         return (F)reader.unmarshalField(fieldName);
     }
 
     /** {@inheritDoc} */
     @Override public boolean hasField(String fieldName) {
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx,
-            new PortableOffheapInputStream(ptr, size, false),
-            start,
-            null);
-
-        return reader.hasField(fieldName);
+        return newReader().hasField(fieldName);
     }
 
     /** {@inheritDoc} */
@@ -438,12 +420,19 @@ public class BinaryObjectOffheapImpl extends BinaryObjectEx implements Externali
      */
     private Object deserializeValue() {
         // TODO: IGNITE-1272 - Deserialize with proper class loader.
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(
-            ctx,
-            new PortableOffheapInputStream(ptr, size, false),
-            start,
-            null);
+        return newReader().deserialize();
+    }
+
+    /**
+     * Create new reader for this object.
+     *
+     * @return Reader.
+     */
+    private BinaryReaderExImpl newReader() {
+        PortableOffheapInputStream stream = new PortableOffheapInputStream(ptr, size, false);
+
+        stream.position(start);
 
-        return reader.deserialize();
+        return new BinaryReaderExImpl(ctx, stream, null, new BinaryReaderHandles());
     }
 }
\ No newline at end of file