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/02 13:12:13 UTC

ignite git commit: IGNITE-1282: Improved portable primitives handling.

Repository: ignite
Updated Branches:
  refs/heads/ignite-1282 ce402cdce -> 2cdbb2ff5


IGNITE-1282: Improved portable primitives handling.


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

Branch: refs/heads/ignite-1282
Commit: 2cdbb2ff516ee9269f0813428be2ce5cebbe2a0a
Parents: ce402cd
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Nov 2 15:12:57 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Nov 2 15:12:57 2015 +0300

----------------------------------------------------------------------
 .../internal/portable/PortableObjectImpl.java   |  33 +-
 .../portable/PortablePositionReadable.java      |   8 +
 .../internal/portable/PortablePrimitives.java   | 755 ++++---------------
 .../ignite/internal/portable/PortableUtils.java |  15 +
 .../portable/builder/PortableBuilderImpl.java   |   2 +-
 .../portable/builder/PortableBuilderReader.java |  41 +-
 .../streams/PortableAbstractInputStream.java    |  18 +
 .../streams/PortableHeapInputStream.java        |   5 +
 .../streams/PortableOffheapInputStream.java     |   5 +
 .../PlatformBigEndianInputStreamImpl.java       |   5 +
 .../memory/PlatformInputStreamImpl.java         |  10 +
 11 files changed, 261 insertions(+), 636 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2cdbb2ff/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectImpl.java
index dfe6d88..a1272d0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectImpl.java
@@ -63,9 +63,6 @@ public final class PortableObjectImpl extends PortableObjectEx implements Extern
     private static final long serialVersionUID = 0L;
 
     /** */
-    private static final PortablePrimitives PRIM = PortablePrimitives.get();
-
-    /** */
     @GridDirectTransient
     private PortableContext ctx;
 
@@ -154,7 +151,7 @@ public final class PortableObjectImpl extends PortableObjectEx implements Extern
 
     /** {@inheritDoc} */
     @Override public int length() {
-        return PRIM.readInt(arr, start + GridPortableMarshaller.TOTAL_LEN_POS);
+        return PortablePrimitives.readInt(arr, start + GridPortableMarshaller.TOTAL_LEN_POS);
     }
 
     /**
@@ -230,7 +227,7 @@ public final class PortableObjectImpl extends PortableObjectEx implements Extern
 
     /** {@inheritDoc} */
     @Override public int typeId() {
-        return PRIM.readInt(arr, start + GridPortableMarshaller.TYPE_ID_POS);
+        return PortablePrimitives.readInt(arr, start + GridPortableMarshaller.TYPE_ID_POS);
     }
 
     /** {@inheritDoc} */
@@ -262,50 +259,50 @@ public final class PortableObjectImpl extends PortableObjectEx implements Extern
     @Nullable @Override protected <F> F fieldByOffset(int fieldOffset) {
         Object val;
 
-        int schemaOffset = PRIM.readInt(arr, start + GridPortableMarshaller.SCHEMA_OR_RAW_OFF_POS);
-        int fieldPos = PRIM.readInt(arr, start + schemaOffset + fieldOffset);
+        int schemaOffset = PortablePrimitives.readInt(arr, start + GridPortableMarshaller.SCHEMA_OR_RAW_OFF_POS);
+        int fieldPos = PortablePrimitives.readInt(arr, start + schemaOffset + fieldOffset);
 
         // Read header and try performing fast lookup for well-known types (the most common types go first).
-        byte hdr = PRIM.readByte(arr, fieldPos);
+        byte hdr = PortablePrimitives.readByte(arr, fieldPos);
 
         switch (hdr) {
             case INT:
-                val = PRIM.readInt(arr, fieldPos + 1);
+                val = PortablePrimitives.readInt(arr, fieldPos + 1);
 
                 break;
 
             case LONG:
-                val = PRIM.readLong(arr, fieldPos + 1);
+                val = PortablePrimitives.readLong(arr, fieldPos + 1);
 
                 break;
 
             case BOOLEAN:
-                val = PRIM.readBoolean(arr, fieldPos + 1);
+                val = PortablePrimitives.readBoolean(arr, fieldPos + 1);
 
                 break;
 
             case SHORT:
-                val = PRIM.readShort(arr, fieldPos + 1);
+                val = PortablePrimitives.readShort(arr, fieldPos + 1);
 
                 break;
 
             case BYTE:
-                val = PRIM.readByte(arr, fieldPos + 1);
+                val = PortablePrimitives.readByte(arr, fieldPos + 1);
 
                 break;
 
             case CHAR:
-                val = PRIM.readChar(arr, fieldPos + 1);
+                val = PortablePrimitives.readChar(arr, fieldPos + 1);
 
                 break;
 
             case FLOAT:
-                val = PRIM.readFloat(arr, fieldPos + 1);
+                val = PortablePrimitives.readFloat(arr, fieldPos + 1);
 
                 break;
 
             case DOUBLE:
-                val = PRIM.readDouble(arr, fieldPos + 1);
+                val = PortablePrimitives.readDouble(arr, fieldPos + 1);
 
                 break;
 
@@ -393,12 +390,12 @@ public final class PortableObjectImpl extends PortableObjectEx implements Extern
 
     /** {@inheritDoc} */
     @Override public int hashCode() {
-        return PRIM.readInt(arr, start + GridPortableMarshaller.HASH_CODE_POS);
+        return PortablePrimitives.readInt(arr, start + GridPortableMarshaller.HASH_CODE_POS);
     }
 
     /** {@inheritDoc} */
     @Override protected int schemaId() {
-        return PRIM.readInt(arr, start + GridPortableMarshaller.SCHEMA_ID_POS);
+        return PortablePrimitives.readInt(arr, start + GridPortableMarshaller.SCHEMA_ID_POS);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/2cdbb2ff/modules/core/src/main/java/org/apache/ignite/internal/portable/PortablePositionReadable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortablePositionReadable.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortablePositionReadable.java
index 7e8d9d3..c49c898 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortablePositionReadable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortablePositionReadable.java
@@ -22,6 +22,14 @@ package org.apache.ignite.internal.portable;
  */
 public interface PortablePositionReadable {
     /**
+     * Read byte at the given position.
+     *
+     * @param pos Position.
+     * @return Value.
+     */
+    public byte readBytePositioned(int pos);
+
+    /**
      * Read short at the given position.
      *
      * @param pos Position.

http://git-wip-us.apache.org/repos/asf/ignite/blob/2cdbb2ff/modules/core/src/main/java/org/apache/ignite/internal/portable/PortablePrimitives.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortablePrimitives.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortablePrimitives.java
index 4c6b5fe..d5a78bc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortablePrimitives.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortablePrimitives.java
@@ -20,754 +20,319 @@ package org.apache.ignite.internal.portable;
 import org.apache.ignite.internal.util.GridUnsafe;
 import sun.misc.Unsafe;
 
-import static java.nio.ByteOrder.LITTLE_ENDIAN;
-import static java.nio.ByteOrder.nativeOrder;
+import java.nio.ByteOrder;
 
 /**
  * Primitives writer.
  */
 public abstract class PortablePrimitives {
     /** */
-    private static final PortablePrimitives INSTANCE =
-        nativeOrder() == LITTLE_ENDIAN ? new UnsafePrimitives() : new BytePrimitives();
+    private static final Unsafe UNSAFE = GridUnsafe.unsafe();
 
-    /**
-     * @return Primitives writer.
-     */
-    public static PortablePrimitives get() {
-        return INSTANCE;
-    }
+    /** */
+    private static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** */
+    private static final long CHAR_ARR_OFF = UNSAFE.arrayBaseOffset(char[].class);
+
+    /** Whether little endian is set. */
+    private static final boolean BIG_ENDIAN = ByteOrder.nativeOrder() == ByteOrder.BIG_ENDIAN;
 
     /**
      * @param arr Array.
      * @param off Offset.
      * @param val Value.
      */
-    public abstract void writeByte(byte[] arr, int off, byte val);
+    public static void writeByte(byte[] arr, int off, byte val) {
+        UNSAFE.putByte(arr, BYTE_ARR_OFF + off, val);
+    }
 
     /**
      * @param arr Array.
      * @param off Offset.
      * @return Value.
      */
-    public abstract byte readByte(byte[] arr, int off);
-
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @param val Value.
-     */
-    public abstract void writeShort(byte[] arr, int off, short val);
+    public static byte readByte(byte[] arr, int off) {
+        return UNSAFE.getByte(arr, BYTE_ARR_OFF + off);
+    }
 
     /**
-     * @param arr Array.
+     * @param ptr Pointer.
      * @param off Offset.
      * @return Value.
      */
-    public abstract short readShort(byte[] arr, int off);
+    public static byte readByte(long ptr, int off) {
+        return UNSAFE.getByte(ptr + off);
+    }
 
     /**
      * @param arr Array.
      * @param off Offset.
      * @param val Value.
      */
-    public abstract void writeInt(byte[] arr, int off, int val);
+    public static void writeBoolean(byte[] arr, int off, boolean val) {
+        writeByte(arr, off, val ? (byte)1 : (byte)0);
+    }
 
     /**
      * @param arr Array.
      * @param off Offset.
      * @return Value.
      */
-    public abstract int readInt(byte[] arr, int off);
-
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @param val Value.
-     */
-    public abstract void writeLong(byte[] arr, int off, long val);
+    public static boolean readBoolean(byte[] arr, int off) {
+        return readByte(arr, off) == 1;
+    }
 
     /**
-     * @param arr Array.
+     * @param ptr Pointer.
      * @param off Offset.
      * @return Value.
      */
-    public abstract long readLong(byte[] arr, int off);
+    public static boolean readBoolean(long ptr, int off) {
+        return readByte(ptr, off) == 1;
+    }
 
     /**
      * @param arr Array.
      * @param off Offset.
      * @param val Value.
      */
-    public abstract void writeFloat(byte[] arr, int off, float val);
+    public static void writeShort(byte[] arr, int off, short val) {
+        if (BIG_ENDIAN)
+            val = Short.reverseBytes(val);
+
+        UNSAFE.putShort(arr, BYTE_ARR_OFF + off, val);
+    }
 
     /**
      * @param arr Array.
      * @param off Offset.
      * @return Value.
      */
-    public abstract float readFloat(byte[] arr, int off);
+    public static short readShort(byte[] arr, int off) {
+        short val = UNSAFE.getShort(arr, BYTE_ARR_OFF + off);
 
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @param val Value.
-     */
-    public abstract void writeDouble(byte[] arr, int off, double val);
+        if (BIG_ENDIAN)
+            val = Short.reverseBytes(val);
+
+        return val;
+    }
 
     /**
-     * @param arr Array.
+     * @param ptr Pointer.
      * @param off Offset.
      * @return Value.
      */
-    public abstract double readDouble(byte[] arr, int off);
+    public static short readShort(long ptr, int off) {
+        short val = UNSAFE.getShort(ptr + off);
+
+        if (BIG_ENDIAN)
+            val = Short.reverseBytes(val);
+
+        return val;
+    }
 
     /**
      * @param arr Array.
      * @param off Offset.
      * @param val Value.
      */
-    public abstract void writeChar(byte[] arr, int off, char val);
+    public static void writeChar(byte[] arr, int off, char val) {
+        if (BIG_ENDIAN)
+            val = Character.reverseBytes(val);
+
+        UNSAFE.putChar(arr, BYTE_ARR_OFF + off, val);
+    }
 
     /**
      * @param arr Array.
      * @param off Offset.
      * @return Value.
      */
-    public abstract char readChar(byte[] arr, int off);
+    public static char readChar(byte[] arr, int off) {
+        char val = UNSAFE.getChar(arr, BYTE_ARR_OFF + off);
 
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @param val Value.
-     */
-    public abstract void writeBoolean(byte[] arr, int off, boolean val);
+        if (BIG_ENDIAN)
+            val = Character.reverseBytes(val);
+
+        return val;
+    }
 
     /**
-     * @param arr Array.
+     * @param ptr Pointer.
      * @param off Offset.
      * @return Value.
      */
-    public abstract boolean readBoolean(byte[] arr, int off);
+    public static char readChar(long ptr, int off) {
+        char val = UNSAFE.getChar(ptr + off);
 
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @param val Value.
-     */
-    public abstract void writeByteArray(byte[] arr, int off, byte[] val);
+        if (BIG_ENDIAN)
+            val = Character.reverseBytes(val);
+
+        return val;
+    }
 
     /**
      * @param arr Array.
      * @param off Offset.
      * @return Value.
      */
-    public abstract byte[] readByteArray(byte[] arr, int off, int len);
+    public static char[] readCharArray(byte[] arr, int off, int len) {
+        char[] arr0 = new char[len];
+
+        UNSAFE.copyMemory(arr, BYTE_ARR_OFF + off, arr0, CHAR_ARR_OFF, len << 1);
+
+        if (BIG_ENDIAN) {
+            for (int i = 0; i < len; i++)
+                arr0[i] = Character.reverseBytes(arr0[i]);
+        }
+
+        return arr0;
+    }
 
     /**
      * @param arr Array.
      * @param off Offset.
      * @param val Value.
      */
-    public abstract void writeShortArray(byte[] arr, int off, short[] val);
+    public static void writeInt(byte[] arr, int off, int val) {
+        if (BIG_ENDIAN)
+            val = Integer.reverseBytes(val);
+
+        UNSAFE.putInt(arr, BYTE_ARR_OFF + off, val);
+    }
 
     /**
      * @param arr Array.
      * @param off Offset.
      * @return Value.
      */
-    public abstract short[] readShortArray(byte[] arr, int off, int len);
+    public static int readInt(byte[] arr, int off) {
+        int val = UNSAFE.getInt(arr, BYTE_ARR_OFF + off);
 
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @param val Value.
-     */
-    public abstract void writeIntArray(byte[] arr, int off, int[] val);
+        if (BIG_ENDIAN)
+            val = Integer.reverseBytes(val);
+
+        return val;
+    }
 
     /**
-     * @param arr Array.
+     * @param ptr Pointer.
      * @param off Offset.
      * @return Value.
      */
-    public abstract int[] readIntArray(byte[] arr, int off, int len);
+    public static int readInt(long ptr, int off) {
+        int val = UNSAFE.getInt(ptr + off);
+
+        if (BIG_ENDIAN)
+            val = Integer.reverseBytes(val);
+
+        return val;
+    }
 
     /**
      * @param arr Array.
      * @param off Offset.
      * @param val Value.
      */
-    public abstract void writeLongArray(byte[] arr, int off, long[] val);
+    public static void writeLong(byte[] arr, int off, long val) {
+        if (BIG_ENDIAN)
+            val = Long.reverseBytes(val);
+
+        UNSAFE.putLong(arr, BYTE_ARR_OFF + off, val);
+    }
 
     /**
      * @param arr Array.
      * @param off Offset.
      * @return Value.
      */
-    public abstract long[] readLongArray(byte[] arr, int off, int len);
+    public static long readLong(byte[] arr, int off) {
+        long val = UNSAFE.getLong(arr, BYTE_ARR_OFF + off);
 
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @param val Value.
-     */
-    public abstract void writeFloatArray(byte[] arr, int off, float[] val);
+        if (BIG_ENDIAN)
+            val = Long.reverseBytes(val);
+
+        return val;
+    }
 
     /**
-     * @param arr Array.
+     * @param ptr Pointer.
      * @param off Offset.
      * @return Value.
      */
-    public abstract float[] readFloatArray(byte[] arr, int off, int len);
+    public static long readLong(long ptr, int off) {
+        long val = UNSAFE.getLong(ptr + off);
+
+        if (BIG_ENDIAN)
+            val = Long.reverseBytes(val);
+
+        return val;
+    }
 
     /**
      * @param arr Array.
      * @param off Offset.
      * @param val Value.
      */
-    public abstract void writeDoubleArray(byte[] arr, int off, double[] val);
+    public static void writeFloat(byte[] arr, int off, float val) {
+        int val0 = Float.floatToIntBits(val);
+
+        writeInt(arr, off, val0);
+    }
 
     /**
      * @param arr Array.
      * @param off Offset.
      * @return Value.
      */
-    public abstract double[] readDoubleArray(byte[] arr, int off, int len);
+    public static float readFloat(byte[] arr, int off) {
+        int val = readInt(arr, off);
 
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @param val Value.
-     */
-    public abstract void writeCharArray(byte[] arr, int off, char[] val);
+        return Float.intBitsToFloat(val);
+    }
 
     /**
-     * @param arr Array.
+     * @param ptr Pointer.
      * @param off Offset.
      * @return Value.
      */
-    public abstract char[] readCharArray(byte[] arr, int off, int len);
+    public static float readFloat(long ptr, int off) {
+        int val = readInt(ptr, off);
+
+        return Float.intBitsToFloat(val);
+    }
 
     /**
      * @param arr Array.
      * @param off Offset.
      * @param val Value.
      */
-    public abstract void writeBooleanArray(byte[] arr, int off, boolean[] val);
+    public static void writeDouble(byte[] arr, int off, double val) {
+        long val0 = Double.doubleToLongBits(val);
+
+        writeLong(arr, off, val0);
+    }
 
     /**
      * @param arr Array.
      * @param off Offset.
      * @return Value.
      */
-    public abstract boolean[] readBooleanArray(byte[] arr, int off, int len);
-
-    /** */
-    private static class UnsafePrimitives extends PortablePrimitives {
-        /** */
-        private static final Unsafe UNSAFE = GridUnsafe.unsafe();
-
-        /** */
-        private static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
-
-        /** */
-        private static final long SHORT_ARR_OFF = UNSAFE.arrayBaseOffset(short[].class);
-
-        /** */
-        private static final long INT_ARR_OFF = UNSAFE.arrayBaseOffset(int[].class);
-
-        /** */
-        private static final long LONG_ARR_OFF = UNSAFE.arrayBaseOffset(long[].class);
-
-        /** */
-        private static final long FLOAT_ARR_OFF = UNSAFE.arrayBaseOffset(float[].class);
-
-        /** */
-        private static final long DOUBLE_ARR_OFF = UNSAFE.arrayBaseOffset(double[].class);
-
-        /** */
-        private static final long CHAR_ARR_OFF = UNSAFE.arrayBaseOffset(char[].class);
-
-        /** */
-        private static final long BOOLEAN_ARR_OFF = UNSAFE.arrayBaseOffset(boolean[].class);
-
-        /** {@inheritDoc} */
-        @Override public void writeByte(byte[] arr, int off, byte val) {
-            UNSAFE.putByte(arr, BYTE_ARR_OFF + off, val);
-        }
-
-        /** {@inheritDoc} */
-        @Override public byte readByte(byte[] arr, int off) {
-            return UNSAFE.getByte(arr, BYTE_ARR_OFF + off);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeShort(byte[] arr, int off, short val) {
-            UNSAFE.putShort(arr, BYTE_ARR_OFF + off, val);
-        }
-
-        /** {@inheritDoc} */
-        @Override public short readShort(byte[] arr, int off) {
-            return UNSAFE.getShort(arr, BYTE_ARR_OFF + off);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeInt(byte[] arr, int off, int val) {
-            UNSAFE.putInt(arr, BYTE_ARR_OFF + off, val);
-        }
-
-        /** {@inheritDoc} */
-        @Override public int readInt(byte[] arr, int off) {
-            return UNSAFE.getInt(arr, BYTE_ARR_OFF + off);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeLong(byte[] arr, int off, long val) {
-            UNSAFE.putLong(arr, BYTE_ARR_OFF + off, val);
-        }
-
-        /** {@inheritDoc} */
-        @Override public long readLong(byte[] arr, int off) {
-            return UNSAFE.getLong(arr, BYTE_ARR_OFF + off);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeFloat(byte[] arr, int off, float val) {
-            UNSAFE.putFloat(arr, BYTE_ARR_OFF + off, val);
-        }
-
-        /** {@inheritDoc} */
-        @Override public float readFloat(byte[] arr, int off) {
-            return UNSAFE.getFloat(arr, BYTE_ARR_OFF + off);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeDouble(byte[] arr, int off, double val) {
-            UNSAFE.putDouble(arr, BYTE_ARR_OFF + off, val);
-        }
-
-        /** {@inheritDoc} */
-        @Override public double readDouble(byte[] arr, int off) {
-            return UNSAFE.getDouble(arr, BYTE_ARR_OFF + off);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeChar(byte[] arr, int off, char val) {
-            UNSAFE.putChar(arr, BYTE_ARR_OFF + off, val);
-        }
-
-        /** {@inheritDoc} */
-        @Override public char readChar(byte[] arr, int off) {
-            return UNSAFE.getChar(arr, BYTE_ARR_OFF + off);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeBoolean(byte[] arr, int off, boolean val) {
-            UNSAFE.putBoolean(arr, BYTE_ARR_OFF + off, val);
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean readBoolean(byte[] arr, int off) {
-            return UNSAFE.getBoolean(arr, BYTE_ARR_OFF + off);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeByteArray(byte[] arr, int off, byte[] val) {
-            UNSAFE.copyMemory(val, BYTE_ARR_OFF, arr, BYTE_ARR_OFF + off, val.length);
-        }
-
-        /** {@inheritDoc} */
-        @Override public byte[] readByteArray(byte[] arr, int off, int len) {
-            byte[] arr0 = new byte[len];
-
-            UNSAFE.copyMemory(arr, BYTE_ARR_OFF + off, arr0, BYTE_ARR_OFF, len);
+    public static double readDouble(byte[] arr, int off) {
+        long val = readLong(arr, off);
 
-            return arr0;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeShortArray(byte[] arr, int off, short[] val) {
-            UNSAFE.copyMemory(val, SHORT_ARR_OFF, arr, BYTE_ARR_OFF + off, val.length << 1);
-        }
-
-        /** {@inheritDoc} */
-        @Override public short[] readShortArray(byte[] arr, int off, int len) {
-            short[] arr0 = new short[len];
-
-            UNSAFE.copyMemory(arr, BYTE_ARR_OFF + off, arr0, SHORT_ARR_OFF, len << 1);
-
-            return arr0;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeIntArray(byte[] arr, int off, int[] val) {
-            UNSAFE.copyMemory(val, INT_ARR_OFF, arr, BYTE_ARR_OFF + off, val.length << 2);
-        }
-
-        /** {@inheritDoc} */
-        @Override public int[] readIntArray(byte[] arr, int off, int len) {
-            int[] arr0 = new int[len];
-
-            UNSAFE.copyMemory(arr, BYTE_ARR_OFF + off, arr0, INT_ARR_OFF, len << 2);
-
-            return arr0;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeLongArray(byte[] arr, int off, long[] val) {
-            UNSAFE.copyMemory(val, LONG_ARR_OFF, arr, BYTE_ARR_OFF + off, val.length << 3);
-        }
-
-        /** {@inheritDoc} */
-        @Override public long[] readLongArray(byte[] arr, int off, int len) {
-            long[] arr0 = new long[len];
-
-            UNSAFE.copyMemory(arr, BYTE_ARR_OFF + off, arr0, LONG_ARR_OFF, len << 3);
-
-            return arr0;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeFloatArray(byte[] arr, int off, float[] val) {
-            UNSAFE.copyMemory(val, FLOAT_ARR_OFF, arr, BYTE_ARR_OFF + off, val.length << 2);
-        }
-
-        /** {@inheritDoc} */
-        @Override public float[] readFloatArray(byte[] arr, int off, int len) {
-            float[] arr0 = new float[len];
-
-            UNSAFE.copyMemory(arr, BYTE_ARR_OFF + off, arr0, FLOAT_ARR_OFF, len << 2);
-
-            return arr0;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeDoubleArray(byte[] arr, int off, double[] val) {
-            UNSAFE.copyMemory(val, DOUBLE_ARR_OFF, arr, BYTE_ARR_OFF + off, val.length << 3);
-        }
-
-        /** {@inheritDoc} */
-        @Override public double[] readDoubleArray(byte[] arr, int off, int len) {
-            double[] arr0 = new double[len];
-
-            UNSAFE.copyMemory(arr, BYTE_ARR_OFF + off, arr0, DOUBLE_ARR_OFF, len << 3);
-
-            return arr0;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeCharArray(byte[] arr, int off, char[] val) {
-            UNSAFE.copyMemory(val, CHAR_ARR_OFF, arr, BYTE_ARR_OFF + off, val.length << 1);
-        }
-
-        /** {@inheritDoc} */
-        @Override public char[] readCharArray(byte[] arr, int off, int len) {
-            char[] arr0 = new char[len];
-
-            UNSAFE.copyMemory(arr, BYTE_ARR_OFF + off, arr0, CHAR_ARR_OFF, len << 1);
-
-            return arr0;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeBooleanArray(byte[] arr, int off, boolean[] val) {
-            UNSAFE.copyMemory(val, BOOLEAN_ARR_OFF, arr, BYTE_ARR_OFF + off, val.length);
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean[] readBooleanArray(byte[] arr, int off, int len) {
-            boolean[] arr0 = new boolean[len];
-
-            UNSAFE.copyMemory(arr, BYTE_ARR_OFF + off, arr0, BOOLEAN_ARR_OFF, len);
-
-            return arr0;
-        }
+        return Double.longBitsToDouble(val);
     }
 
-    /** */
-    private static class BytePrimitives extends PortablePrimitives {
-        /** {@inheritDoc} */
-        @Override public void writeByte(byte[] arr, int off, byte val) {
-            arr[off] = val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public byte readByte(byte[] arr, int off) {
-            return arr[off];
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeShort(byte[] arr, int off, short val) {
-            arr[off++] = (byte)(val & 0xff);
-            arr[off] = (byte)((val >>> 8) & 0xff);
-        }
-
-        /** {@inheritDoc} */
-        @Override public short readShort(byte[] arr, int off) {
-            short val = 0;
-
-            val |= (arr[off++] & 0xff);
-            val |= (arr[off] & 0xff) << 8;
-
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeInt(byte[] arr, int off, int val) {
-            arr[off++] = (byte)(val & 0xff);
-            arr[off++] = (byte)((val >>> 8) & 0xff);
-            arr[off++] = (byte)((val >>> 16) & 0xff);
-            arr[off] = (byte)((val >>> 24) & 0xff);
-        }
-
-        /** {@inheritDoc} */
-        @Override public int readInt(byte[] arr, int off) {
-            int val = 0;
-
-            val |= (arr[off++] & 0xff);
-            val |= (arr[off++] & 0xff) << 8;
-            val |= (arr[off++] & 0xff) << 16;
-            val |= (arr[off] & 0xff) << 24;
-
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeLong(byte[] arr, int off, long val) {
-            arr[off++] = (byte)(val & 0xffL);
-            arr[off++] = (byte)((val >>> 8) & 0xffL);
-            arr[off++] = (byte)((val >>> 16) & 0xffL);
-            arr[off++] = (byte)((val >>> 24) & 0xffL);
-            arr[off++] = (byte)((val >>> 32) & 0xffL);
-            arr[off++] = (byte)((val >>> 40) & 0xffL);
-            arr[off++] = (byte)((val >>> 48) & 0xffL);
-            arr[off] = (byte)((val >>> 56) & 0xffL);
-        }
-
-        /** {@inheritDoc} */
-        @Override public long readLong(byte[] arr, int off) {
-            long val = 0;
-
-            val |= (arr[off++] & 0xffL);
-            val |= (arr[off++] & 0xffL) << 8;
-            val |= (arr[off++] & 0xffL) << 16;
-            val |= (arr[off++] & 0xffL) << 24;
-            val |= (arr[off++] & 0xffL) << 32;
-            val |= (arr[off++] & 0xffL) << 40;
-            val |= (arr[off++] & 0xffL) << 48;
-            val |= (arr[off] & 0xffL) << 56;
-
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeFloat(byte[] arr, int off, float val) {
-            writeInt(arr, off, Float.floatToIntBits(val));
-        }
-
-        /** {@inheritDoc} */
-        @Override public float readFloat(byte[] arr, int off) {
-            return Float.intBitsToFloat(readInt(arr, off));
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeDouble(byte[] arr, int off, double val) {
-            writeLong(arr, off, Double.doubleToLongBits(val));
-        }
-
-        /** {@inheritDoc} */
-        @Override public double readDouble(byte[] arr, int off) {
-            return Double.longBitsToDouble(readLong(arr, off));
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeChar(byte[] arr, int off, char val) {
-            arr[off++] = (byte)(val & 0xff);
-            arr[off] = (byte)((val >>> 8) & 0xff);
-        }
-
-        /** {@inheritDoc} */
-        @Override public char readChar(byte[] arr, int off) {
-            char val = 0;
-
-            val |= (arr[off++] & 0xff);
-            val |= (arr[off] & 0xff) << 8;
-
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeBoolean(byte[] arr, int off, boolean val) {
-            arr[off] = (byte)(val ? 1 : 0);
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean readBoolean(byte[] arr, int off) {
-            return arr[off] != 0;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeByteArray(byte[] arr, int off, byte[] val) {
-            for (byte b : val)
-                arr[off++] = b;
-        }
-
-        /** {@inheritDoc} */
-        @Override public byte[] readByteArray(byte[] arr, int off, int len) {
-            byte[] val = new byte[len];
-
-            for (int i = 0; i < len; i++)
-                val[i] = arr[off++];
-
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeShortArray(byte[] arr, int off, short[] val) {
-            for (short s : val) {
-                writeShort(arr, off, s);
-
-                off += 2;
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override public short[] readShortArray(byte[] arr, int off, int len) {
-            short[] val = new short[len];
-
-            for (int i = 0; i < len; i++) {
-                val[i] = readShort(arr, off);
-
-                off += 2;
-            }
-
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeIntArray(byte[] arr, int off, int[] val) {
-            for (int i : val) {
-                writeInt(arr, off, i);
-
-                off += 4;
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override public int[] readIntArray(byte[] arr, int off, int len) {
-            int[] val = new int[len];
-
-            for (int i = 0; i < len; i++) {
-                val[i] = readInt(arr, off);
-
-                off += 4;
-            }
-
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeLongArray(byte[] arr, int off, long[] val) {
-            for (long l : val) {
-                writeLong(arr, off, l);
-
-                off += 8;
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override public long[] readLongArray(byte[] arr, int off, int len) {
-            long[] val = new long[len];
-
-            for (int i = 0; i < len; i++) {
-                val[i] = readLong(arr, off);
-
-                off += 8;
-            }
-
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeFloatArray(byte[] arr, int off, float[] val) {
-            for (float f : val) {
-                writeFloat(arr, off, f);
-
-                off += 4;
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override public float[] readFloatArray(byte[] arr, int off, int len) {
-            float[] val = new float[len];
-
-            for (int i = 0; i < len; i++) {
-                val[i] = readFloat(arr, off);
-
-                off += 4;
-            }
-
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeDoubleArray(byte[] arr, int off, double[] val) {
-            for (double d : val) {
-                writeDouble(arr, off, d);
-
-                off += 8;
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override public double[] readDoubleArray(byte[] arr, int off, int len) {
-            double[] val = new double[len];
-
-            for (int i = 0; i < len; i++) {
-                val[i] = readDouble(arr, off);
-
-                off += 8;
-            }
-
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeCharArray(byte[] arr, int off, char[] val) {
-            for (char c : val) {
-                writeChar(arr, off, c);
-
-                off += 2;
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override public char[] readCharArray(byte[] arr, int off, int len) {
-            char[] val = new char[len];
-
-            for (int i = 0; i < len; i++) {
-                val[i] = readChar(arr, off);
-
-                off += 2;
-            }
-
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeBooleanArray(byte[] arr, int off, boolean[] val) {
-            for (boolean b : val)
-                writeBoolean(arr, off++, b);
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean[] readBooleanArray(byte[] arr, int off, int len) {
-            boolean[] val = new boolean[len];
-
-            for (int i = 0; i < len; i++)
-                val[i] = readBoolean(arr, off++);
+    /**
+     * @param ptr Pointer.
+     * @param off Offset.
+     * @return Value.
+     */
+    public static double readDouble(long ptr, int off) {
+        long val = readLong(ptr, off);
 
-            return val;
-        }
+        return Double.longBitsToDouble(val);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/2cdbb2ff/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java
index eafcbd1..f57a29b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java
@@ -100,6 +100,21 @@ public class PortableUtils {
     /** Flag: only raw data exists. */
     public static final short FLAG_RAW_ONLY = 0x2;
 
+    /** Flag: offsets take 1 byte. */
+    public static final short FLAG_OFFSET_ONE_BYTE = 0x4;
+
+    /** Flag: offsets take 2 bytes. */
+    public static final short FLAG_OFFSET_TWO_BYTES = 0x8;
+
+    /** Offset which fits into 1 byte. */
+    public static final int OFFSET_1 = 1;
+
+    /** Offset which fits into 2 bytes. */
+    public static final int OFFSET_2 = 2;
+
+    /** Offset which fits into 4 bytes. */
+    public static final int OFFSET_4 = 4;
+
     /**
      * Write flags.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/2cdbb2ff/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderImpl.java
index 442fc35..34e1d6d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderImpl.java
@@ -138,7 +138,7 @@ public class PortableBuilderImpl implements PortableBuilder {
         this.reader = reader;
         this.start = start;
 
-        byte ver = reader.readByteAbsolute(start + PROTO_VER_POS);
+        byte ver = reader.readBytePositioned(start + PROTO_VER_POS);
 
         PortableUtils.checkProtocolVersion(ver);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/2cdbb2ff/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
index b999cde..9645ced 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
@@ -40,9 +40,6 @@ import static org.apache.ignite.internal.portable.GridPortableMarshaller.STRING;
  */
 public class PortableBuilderReader implements PortablePositionReadable {
     /** */
-    private static final PortablePrimitives PRIM = PortablePrimitives.get();
-
-    /** */
     private final Map<Integer, PortableBuilderImpl> objMap = new HashMap<>();
 
     /** */
@@ -120,32 +117,32 @@ public class PortableBuilderReader implements PortablePositionReadable {
      * @return Read int value.
      */
     public int readInt(int off) {
-        return PRIM.readInt(arr, pos + off);
+        return PortablePrimitives.readInt(arr, pos + off);
     }
 
     /**
      * @param pos Position in the source array.
      * @return Read byte value.
      */
-    public byte readByteAbsolute(int pos) {
-        return PRIM.readByte(arr, pos);
+    public byte readBytePositioned(int pos) {
+        return PortablePrimitives.readByte(arr, pos);
     }
 
     /** {@inheritDoc} */
     @Override public short readShortPositioned(int pos) {
-        return PRIM.readShort(arr, pos);
+        return PortablePrimitives.readShort(arr, pos);
     }
 
     /** {@inheritDoc} */
     @Override public int readIntPositioned(int pos) {
-        return PRIM.readInt(arr, pos);
+        return PortablePrimitives.readInt(arr, pos);
     }
 
     /**
      * @return Read length of array.
      */
     public int readLength() {
-        return PRIM.readInt(arr, pos);
+        return PortablePrimitives.readInt(arr, pos);
     }
 
     /**
@@ -154,9 +151,9 @@ public class PortableBuilderReader implements PortablePositionReadable {
      * @return String length.
      */
     public int readStringLength() {
-        boolean utf = PRIM.readBoolean(arr, pos);
+        boolean utf = PortablePrimitives.readBoolean(arr, pos);
 
-        int arrLen = PRIM.readInt(arr, pos + 1);
+        int arrLen = PortablePrimitives.readInt(arr, pos + 1);
 
         return 1 + (utf ? arrLen : arrLen << 1);
     }
@@ -186,7 +183,7 @@ public class PortableBuilderReader implements PortablePositionReadable {
             pos += len;
         }
         else {
-            str = String.valueOf(PRIM.readCharArray(arr, pos, len));
+            str = String.valueOf(PortablePrimitives.readCharArray(arr, pos, len));
 
             pos += len << 1;
         }
@@ -389,22 +386,22 @@ public class PortableBuilderReader implements PortablePositionReadable {
                 return arr[pos + 1];
 
             case GridPortableMarshaller.SHORT:
-                return PRIM.readShort(arr, pos + 1);
+                return PortablePrimitives.readShort(arr, pos + 1);
 
             case GridPortableMarshaller.INT:
-                return PRIM.readInt(arr, pos + 1);
+                return PortablePrimitives.readInt(arr, pos + 1);
 
             case GridPortableMarshaller.LONG:
-                return PRIM.readLong(arr, pos + 1);
+                return PortablePrimitives.readLong(arr, pos + 1);
 
             case GridPortableMarshaller.FLOAT:
-                return PRIM.readFloat(arr, pos + 1);
+                return PortablePrimitives.readFloat(arr, pos + 1);
 
             case GridPortableMarshaller.DOUBLE:
-                return PRIM.readDouble(arr, pos + 1);
+                return PortablePrimitives.readDouble(arr, pos + 1);
 
             case GridPortableMarshaller.CHAR:
-                return PRIM.readChar(arr, pos + 1);
+                return PortablePrimitives.readChar(arr, pos + 1);
 
             case GridPortableMarshaller.BOOLEAN:
                 return arr[pos + 1] != 0;
@@ -518,7 +515,7 @@ public class PortableBuilderReader implements PortablePositionReadable {
                 return arr[pos++];
 
             case GridPortableMarshaller.SHORT: {
-                Object res = PRIM.readShort(arr, pos);
+                Object res = PortablePrimitives.readShort(arr, pos);
                 pos += 2;
                 return res;
             }
@@ -638,7 +635,7 @@ public class PortableBuilderReader implements PortablePositionReadable {
                     if (flag != GridPortableMarshaller.DATE)
                         throw new PortableException("Invalid flag value: " + flag);
 
-                    long time = PRIM.readLong(arr, pos);
+                    long time = PortablePrimitives.readLong(arr, pos);
 
                     pos += 8;
 
@@ -662,11 +659,11 @@ public class PortableBuilderReader implements PortablePositionReadable {
                     if (flag != GridPortableMarshaller.TIMESTAMP)
                         throw new PortableException("Invalid flag value: " + flag);
 
-                    long time = PRIM.readLong(arr, pos);
+                    long time = PortablePrimitives.readLong(arr, pos);
 
                     pos += 8;
 
-                    int nano = PRIM.readInt(arr, pos);
+                    int nano = PortablePrimitives.readInt(arr, pos);
 
                     pos += 4;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/2cdbb2ff/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableAbstractInputStream.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableAbstractInputStream.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableAbstractInputStream.java
index 7dbee92..d7f66c9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableAbstractInputStream.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableAbstractInputStream.java
@@ -158,6 +158,16 @@ public abstract class PortableAbstractInputStream extends PortableAbstractStream
     }
 
     /** {@inheritDoc} */
+    @Override public byte readBytePositioned(int pos) {
+        int delta = pos + 1 - this.pos;
+
+        if (delta > 0)
+            ensureEnoughData(delta);
+
+        return readBytePositioned0(pos);
+    }
+
+    /** {@inheritDoc} */
     @Override public short readShortPositioned(int pos) {
         int delta = pos + 2 - this.pos;
 
@@ -344,6 +354,14 @@ public abstract class PortableAbstractInputStream extends PortableAbstractStream
     protected abstract long readLongFast();
 
     /**
+     * Internal routine for positioned byte value read.
+     *
+     * @param pos Position.
+     * @return Int value.
+     */
+    protected abstract byte readBytePositioned0(int pos);
+
+    /**
      * Internal routine for positioned short value read.
      *
      * @param pos Position.

http://git-wip-us.apache.org/repos/asf/ignite/blob/2cdbb2ff/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableHeapInputStream.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableHeapInputStream.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableHeapInputStream.java
index adfeaad..e027d70 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableHeapInputStream.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableHeapInputStream.java
@@ -123,6 +123,11 @@ public final class PortableHeapInputStream extends PortableAbstractInputStream {
     }
 
     /** {@inheritDoc} */
+    @Override protected byte readBytePositioned0(int pos) {
+        return UNSAFE.getByte(data, BYTE_ARR_OFF + pos);
+    }
+
+    /** {@inheritDoc} */
     @Override protected short readShortPositioned0(int pos) {
         short res = UNSAFE.getShort(data, BYTE_ARR_OFF + pos);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/2cdbb2ff/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableOffheapInputStream.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableOffheapInputStream.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableOffheapInputStream.java
index 75bffb9..1dc9b5b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableOffheapInputStream.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableOffheapInputStream.java
@@ -113,6 +113,11 @@ public class PortableOffheapInputStream extends PortableAbstractInputStream {
     }
 
     /** {@inheritDoc} */
+    @Override protected byte readBytePositioned0(int pos) {
+        return UNSAFE.getByte(ptr + pos);
+    }
+
+    /** {@inheritDoc} */
     @Override protected short readShortPositioned0(int pos) {
         short res = UNSAFE.getShort(ptr + pos);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/2cdbb2ff/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformBigEndianInputStreamImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformBigEndianInputStreamImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformBigEndianInputStreamImpl.java
index a4d711e..41bc43f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformBigEndianInputStreamImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformBigEndianInputStreamImpl.java
@@ -65,6 +65,11 @@ public class PlatformBigEndianInputStreamImpl extends PlatformInputStreamImpl {
     }
 
     /** {@inheritDoc} */
+    @Override public byte readBytePositioned(int pos) {
+        return super.readBytePositioned(pos);
+    }
+
+    /** {@inheritDoc} */
     @Override public short readShortPositioned(int pos) {
         return Short.reverseBytes(super.readShortPositioned(pos));
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/2cdbb2ff/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformInputStreamImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformInputStreamImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformInputStreamImpl.java
index 68b4141..13da8c5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformInputStreamImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformInputStreamImpl.java
@@ -146,6 +146,16 @@ public class PlatformInputStreamImpl implements PlatformInputStream {
     }
 
     /** {@inheritDoc} */
+    @Override public byte readBytePositioned(int pos) {
+        int delta = pos + 1 - this.pos;
+
+        if (delta > 0)
+            ensureEnoughData(delta);
+
+        return UNSAFE.getByte(data + pos);
+    }
+
+    /** {@inheritDoc} */
     @Override public short readShortPositioned(int pos) {
         int delta = pos + 2 - this.pos;