You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ak...@apache.org on 2015/12/11 16:56:22 UTC

[54/59] [abbrv] ignite git commit: ignite-2065: rename "portable" classes to "binary"

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldImpl.java
index 36fde02..7f51631 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldImpl.java
@@ -32,7 +32,7 @@ public class BinaryFieldImpl implements BinaryField {
 
     /** Well-known object schemas. */
     @GridToStringExclude
-    private final PortableSchemaRegistry schemas;
+    private final BinarySchemaRegistry schemas;
 
     /** Field name. */
     private final String fieldName;
@@ -47,7 +47,7 @@ public class BinaryFieldImpl implements BinaryField {
      * @param fieldName Field name.
      * @param fieldId Field ID.
      */
-    public BinaryFieldImpl(int typeId, PortableSchemaRegistry schemas, String fieldName, int fieldId) {
+    public BinaryFieldImpl(int typeId, BinarySchemaRegistry schemas, String fieldName, int fieldId) {
         assert typeId != 0;
         assert schemas != null;
         assert fieldName != null;
@@ -68,7 +68,7 @@ public class BinaryFieldImpl implements BinaryField {
     @Override public boolean exists(BinaryObject obj) {
         BinaryObjectExImpl obj0 = (BinaryObjectExImpl)obj;
 
-        return fieldOrder(obj0) != PortableSchema.ORDER_NOT_FOUND;
+        return fieldOrder(obj0) != BinarySchema.ORDER_NOT_FOUND;
     }
 
     /** {@inheritDoc} */
@@ -78,7 +78,7 @@ public class BinaryFieldImpl implements BinaryField {
 
         int order = fieldOrder(obj0);
 
-        return order != PortableSchema.ORDER_NOT_FOUND ? (T)obj0.fieldByOrder(order) : null;
+        return order != BinarySchema.ORDER_NOT_FOUND ? (T)obj0.fieldByOrder(order) : null;
     }
 
     /**
@@ -96,7 +96,7 @@ public class BinaryFieldImpl implements BinaryField {
 
         int schemaId = obj.schemaId();
 
-        PortableSchema schema = schemas.schema(schemaId);
+        BinarySchema schema = schemas.schema(schemaId);
 
         if (schema == null) {
             schema = obj.createSchema();

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMarshaller.java
index 13435e6..03bf9f9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMarshaller.java
@@ -36,7 +36,7 @@ import sun.misc.Unsafe;
  */
 public class BinaryMarshaller extends AbstractMarshaller {
     /** */
-    private GridPortableMarshaller impl;
+    private GridBinaryMarshaller impl;
 
     /**
      * Checks whether {@code BinaryMarshaller} is able to work on the current JVM.
@@ -79,15 +79,15 @@ public class BinaryMarshaller extends AbstractMarshaller {
     }
 
     /**
-     * Sets {@link PortableContext}.
+     * Sets {@link BinaryContext}.
      * <p/>
      * @param ctx Portable context.
      */
     @SuppressWarnings("UnusedDeclaration")
-    private void setPortableContext(PortableContext ctx, IgniteConfiguration cfg) {
+    private void setPortableContext(BinaryContext ctx, IgniteConfiguration cfg) {
         ctx.configure(this, cfg);
 
-        impl = new GridPortableMarshaller(ctx);
+        impl = new GridBinaryMarshaller(ctx);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadata.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadata.java
index cefad9e..a8ff140 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadata.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadata.java
@@ -50,7 +50,7 @@ public class BinaryMetadata implements Externalizable {
     private String affKeyFieldName;
 
     /** Schemas associated with type. */
-    private Collection<PortableSchema> schemas;
+    private Collection<BinarySchema> schemas;
 
     /** Whether this is enum type. */
     private boolean isEnum;
@@ -73,7 +73,7 @@ public class BinaryMetadata implements Externalizable {
      * @param isEnum Enum flag.
      */
     public BinaryMetadata(int typeId, String typeName, @Nullable Map<String, Integer> fields,
-        @Nullable String affKeyFieldName, @Nullable Collection<PortableSchema> schemas, boolean isEnum) {
+        @Nullable String affKeyFieldName, @Nullable Collection<BinarySchema> schemas, boolean isEnum) {
         assert typeName != null;
 
         this.typeId = typeId;
@@ -119,7 +119,7 @@ public class BinaryMetadata implements Externalizable {
     @Nullable public String fieldTypeName(String fieldName) {
         Integer typeId = fields != null ? fields.get(fieldName) : null;
 
-        return typeId != null ? PortableUtils.fieldTypeName(typeId) : null;
+        return typeId != null ? BinaryUtils.fieldTypeName(typeId) : null;
     }
 
     /**
@@ -132,8 +132,8 @@ public class BinaryMetadata implements Externalizable {
     /**
      * @return Schemas.
      */
-    public Collection<PortableSchema> schemas() {
-        return schemas != null ? schemas : Collections.<PortableSchema>emptyList();
+    public Collection<BinarySchema> schemas() {
+        return schemas != null ? schemas : Collections.<BinarySchema>emptyList();
     }
 
     /**
@@ -149,7 +149,7 @@ public class BinaryMetadata implements Externalizable {
      * @param ctx Portable context.
      * @return Binary type.
      */
-    public BinaryTypeImpl wrap(PortableContext ctx) {
+    public BinaryTypeImpl wrap(BinaryContext ctx) {
         return new BinaryTypeImpl(ctx, this);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadataCollector.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadataCollector.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadataCollector.java
index af99cce..54f2b13 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadataCollector.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadataCollector.java
@@ -51,7 +51,7 @@ class BinaryMetadataCollector implements BinaryWriter {
     private final Map<String, Integer> meta = new HashMap<>();
 
     /** Schema builder. */
-    private PortableSchema.Builder schemaBuilder = PortableSchema.Builder.newBuilder();
+    private BinarySchema.Builder schemaBuilder = BinarySchema.Builder.newBuilder();
 
     /**
      * Constructor.
@@ -76,7 +76,7 @@ class BinaryMetadataCollector implements BinaryWriter {
     /**
      * @return Schemas.
      */
-    PortableSchema schema() {
+    BinarySchema schema() {
         return schemaBuilder.build();
     }
 
@@ -267,8 +267,8 @@ class BinaryMetadataCollector implements BinaryWriter {
         if (oldFieldTypeId != null && !oldFieldTypeId.equals(fieldTypeId)) {
             throw new BinaryObjectException(
                 "Field is written twice with different types [" + "typeName=" + typeName + ", fieldName=" + name +
-                ", fieldTypeName1=" + PortableUtils.fieldTypeName(oldFieldTypeId) +
-                ", fieldTypeName2=" + PortableUtils.fieldTypeName(fieldTypeId) + ']'
+                ", fieldTypeName1=" + BinaryUtils.fieldTypeName(oldFieldTypeId) +
+                ", fieldTypeName2=" + BinaryUtils.fieldTypeName(fieldTypeId) + ']'
             );
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java
index d09bc28..252c495 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java
@@ -98,7 +98,7 @@ public abstract class BinaryObjectExImpl implements BinaryObjectEx {
      *
      * @return Schema.
      */
-    protected abstract PortableSchema createSchema();
+    protected abstract BinarySchema createSchema();
 
     /** {@inheritDoc} */
     @Override public BinaryObject clone() throws CloneNotSupportedException {

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java
index 18adbc1..9fd5901 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java
@@ -33,7 +33,7 @@ import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.binary.BinaryType;
 import org.apache.ignite.internal.GridDirectTransient;
 import org.apache.ignite.internal.IgniteCodeGeneratingFail;
-import org.apache.ignite.internal.binary.streams.PortableHeapInputStream;
+import org.apache.ignite.internal.binary.streams.BinaryHeapInputStream;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
@@ -55,7 +55,7 @@ public final class BinaryObjectImpl extends BinaryObjectExImpl implements Extern
 
     /** */
     @GridDirectTransient
-    private PortableContext ctx;
+    private BinaryContext ctx;
 
     /** */
     private byte[] arr;
@@ -83,7 +83,7 @@ public final class BinaryObjectImpl extends BinaryObjectExImpl implements Extern
      * @param arr Array.
      * @param start Start.
      */
-    public BinaryObjectImpl(PortableContext ctx, byte[] arr, int start) {
+    public BinaryObjectImpl(BinaryContext ctx, byte[] arr, int start) {
         assert ctx != null;
         assert arr != null;
 
@@ -152,7 +152,7 @@ public final class BinaryObjectImpl extends BinaryObjectExImpl implements Extern
 
     /** {@inheritDoc} */
     @Override public int length() {
-        return PortablePrimitives.readInt(arr, start + GridPortableMarshaller.TOTAL_LEN_POS);
+        return BinaryPrimitives.readInt(arr, start + GridBinaryMarshaller.TOTAL_LEN_POS);
     }
 
     /**
@@ -188,14 +188,14 @@ public final class BinaryObjectImpl extends BinaryObjectExImpl implements Extern
     /**
      * @return Context.
      */
-    public PortableContext context() {
+    public BinaryContext context() {
         return ctx;
     }
 
     /**
      * @param ctx Context.
      */
-    public void context(PortableContext ctx) {
+    public void context(BinaryContext ctx) {
         this.ctx = ctx;
     }
 
@@ -221,7 +221,7 @@ public final class BinaryObjectImpl extends BinaryObjectExImpl implements Extern
 
     /** {@inheritDoc} */
     @Override public int typeId() {
-        return PortablePrimitives.readInt(arr, start + GridPortableMarshaller.TYPE_ID_POS);
+        return BinaryPrimitives.readInt(arr, start + GridBinaryMarshaller.TYPE_ID_POS);
     }
 
     /** {@inheritDoc} */
@@ -250,87 +250,87 @@ public final class BinaryObjectImpl extends BinaryObjectExImpl implements Extern
         Object val;
 
         // Calculate field position.
-        int schemaOffset = PortablePrimitives.readInt(arr, start + GridPortableMarshaller.SCHEMA_OR_RAW_OFF_POS);
+        int schemaOffset = BinaryPrimitives.readInt(arr, start + GridBinaryMarshaller.SCHEMA_OR_RAW_OFF_POS);
 
-        short flags = PortablePrimitives.readShort(arr, start + GridPortableMarshaller.FLAGS_POS);
+        short flags = BinaryPrimitives.readShort(arr, start + GridBinaryMarshaller.FLAGS_POS);
 
-        int fieldIdLen = PortableUtils.isCompactFooter(flags) ? 0 : PortableUtils.FIELD_ID_LEN;
-        int fieldOffsetLen = PortableUtils.fieldOffsetLength(flags);
+        int fieldIdLen = BinaryUtils.isCompactFooter(flags) ? 0 : BinaryUtils.FIELD_ID_LEN;
+        int fieldOffsetLen = BinaryUtils.fieldOffsetLength(flags);
 
         int fieldOffsetPos = start + schemaOffset + order * (fieldIdLen + fieldOffsetLen) + fieldIdLen;
 
         int fieldPos;
 
-        if (fieldOffsetLen == PortableUtils.OFFSET_1)
-            fieldPos = start + ((int)PortablePrimitives.readByte(arr, fieldOffsetPos) & 0xFF);
-        else if (fieldOffsetLen == PortableUtils.OFFSET_2)
-            fieldPos = start + ((int)PortablePrimitives.readShort(arr, fieldOffsetPos) & 0xFFFF);
+        if (fieldOffsetLen == BinaryUtils.OFFSET_1)
+            fieldPos = start + ((int)BinaryPrimitives.readByte(arr, fieldOffsetPos) & 0xFF);
+        else if (fieldOffsetLen == BinaryUtils.OFFSET_2)
+            fieldPos = start + ((int)BinaryPrimitives.readShort(arr, fieldOffsetPos) & 0xFFFF);
         else
-            fieldPos = start + PortablePrimitives.readInt(arr, fieldOffsetPos);
+            fieldPos = start + BinaryPrimitives.readInt(arr, fieldOffsetPos);
 
         // Read header and try performing fast lookup for well-known types (the most common types go first).
-        byte hdr = PortablePrimitives.readByte(arr, fieldPos);
+        byte hdr = BinaryPrimitives.readByte(arr, fieldPos);
 
         switch (hdr) {
-            case GridPortableMarshaller.INT:
-                val = PortablePrimitives.readInt(arr, fieldPos + 1);
+            case GridBinaryMarshaller.INT:
+                val = BinaryPrimitives.readInt(arr, fieldPos + 1);
 
                 break;
 
-            case GridPortableMarshaller.LONG:
-                val = PortablePrimitives.readLong(arr, fieldPos + 1);
+            case GridBinaryMarshaller.LONG:
+                val = BinaryPrimitives.readLong(arr, fieldPos + 1);
 
                 break;
 
-            case GridPortableMarshaller.BOOLEAN:
-                val = PortablePrimitives.readBoolean(arr, fieldPos + 1);
+            case GridBinaryMarshaller.BOOLEAN:
+                val = BinaryPrimitives.readBoolean(arr, fieldPos + 1);
 
                 break;
 
-            case GridPortableMarshaller.SHORT:
-                val = PortablePrimitives.readShort(arr, fieldPos + 1);
+            case GridBinaryMarshaller.SHORT:
+                val = BinaryPrimitives.readShort(arr, fieldPos + 1);
 
                 break;
 
-            case GridPortableMarshaller.BYTE:
-                val = PortablePrimitives.readByte(arr, fieldPos + 1);
+            case GridBinaryMarshaller.BYTE:
+                val = BinaryPrimitives.readByte(arr, fieldPos + 1);
 
                 break;
 
-            case GridPortableMarshaller.CHAR:
-                val = PortablePrimitives.readChar(arr, fieldPos + 1);
+            case GridBinaryMarshaller.CHAR:
+                val = BinaryPrimitives.readChar(arr, fieldPos + 1);
 
                 break;
 
-            case GridPortableMarshaller.FLOAT:
-                val = PortablePrimitives.readFloat(arr, fieldPos + 1);
+            case GridBinaryMarshaller.FLOAT:
+                val = BinaryPrimitives.readFloat(arr, fieldPos + 1);
 
                 break;
 
-            case GridPortableMarshaller.DOUBLE:
-                val = PortablePrimitives.readDouble(arr, fieldPos + 1);
+            case GridBinaryMarshaller.DOUBLE:
+                val = BinaryPrimitives.readDouble(arr, fieldPos + 1);
 
                 break;
 
-            case GridPortableMarshaller.STRING: {
-                int dataLen = PortablePrimitives.readInt(arr, fieldPos + 1);
+            case GridBinaryMarshaller.STRING: {
+                int dataLen = BinaryPrimitives.readInt(arr, fieldPos + 1);
 
                 val = new String(arr, fieldPos + 5, dataLen, UTF_8);
 
                 break;
             }
 
-            case GridPortableMarshaller.DATE: {
-                long time = PortablePrimitives.readLong(arr, fieldPos + 1);
+            case GridBinaryMarshaller.DATE: {
+                long time = BinaryPrimitives.readLong(arr, fieldPos + 1);
 
                 val = new Date(time);
 
                 break;
             }
 
-            case GridPortableMarshaller.TIMESTAMP: {
-                long time = PortablePrimitives.readLong(arr, fieldPos + 1);
-                int nanos = PortablePrimitives.readInt(arr, fieldPos + 1 + 8);
+            case GridBinaryMarshaller.TIMESTAMP: {
+                long time = BinaryPrimitives.readLong(arr, fieldPos + 1);
+                int nanos = BinaryPrimitives.readInt(arr, fieldPos + 1 + 8);
 
                 Timestamp ts = new Timestamp(time);
 
@@ -341,20 +341,20 @@ public final class BinaryObjectImpl extends BinaryObjectExImpl implements Extern
                 break;
             }
 
-            case GridPortableMarshaller.UUID: {
-                long most = PortablePrimitives.readLong(arr, fieldPos + 1);
-                long least = PortablePrimitives.readLong(arr, fieldPos + 1 + 8);
+            case GridBinaryMarshaller.UUID: {
+                long most = BinaryPrimitives.readLong(arr, fieldPos + 1);
+                long least = BinaryPrimitives.readLong(arr, fieldPos + 1 + 8);
 
                 val = new UUID(most, least);
 
                 break;
             }
 
-            case GridPortableMarshaller.DECIMAL: {
-                int scale = PortablePrimitives.readInt(arr, fieldPos + 1);
+            case GridBinaryMarshaller.DECIMAL: {
+                int scale = BinaryPrimitives.readInt(arr, fieldPos + 1);
 
-                int dataLen = PortablePrimitives.readInt(arr, fieldPos + 5);
-                byte[] data = PortablePrimitives.readByteArray(arr, fieldPos + 9, dataLen);
+                int dataLen = BinaryPrimitives.readInt(arr, fieldPos + 5);
+                byte[] data = BinaryPrimitives.readByteArray(arr, fieldPos + 9, dataLen);
 
                 BigInteger intVal = new BigInteger(data);
 
@@ -369,13 +369,13 @@ public final class BinaryObjectImpl extends BinaryObjectExImpl implements Extern
                 break;
             }
 
-            case GridPortableMarshaller.NULL:
+            case GridBinaryMarshaller.NULL:
                 val = null;
 
                 break;
 
             default:
-                val = PortableUtils.unmarshal(PortableHeapInputStream.create(arr, fieldPos), ctx, null);
+                val = BinaryUtils.unmarshal(BinaryHeapInputStream.create(arr, fieldPos), ctx, null);
 
                 break;
         }
@@ -412,16 +412,16 @@ public final class BinaryObjectImpl extends BinaryObjectExImpl implements Extern
 
     /** {@inheritDoc} */
     @Override public int hashCode() {
-        return PortablePrimitives.readInt(arr, start + GridPortableMarshaller.HASH_CODE_POS);
+        return BinaryPrimitives.readInt(arr, start + GridBinaryMarshaller.HASH_CODE_POS);
     }
 
     /** {@inheritDoc} */
     @Override protected int schemaId() {
-        return PortablePrimitives.readInt(arr, start + GridPortableMarshaller.SCHEMA_ID_POS);
+        return BinaryPrimitives.readInt(arr, start + GridBinaryMarshaller.SCHEMA_ID_POS);
     }
 
     /** {@inheritDoc} */
-    @Override protected PortableSchema createSchema() {
+    @Override protected BinarySchema createSchema() {
         return reader(null).getOrCreateSchema();
     }
 
@@ -445,7 +445,7 @@ public final class BinaryObjectImpl extends BinaryObjectExImpl implements Extern
 
     /** {@inheritDoc} */
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        ctx = (PortableContext)in.readObject();
+        ctx = (BinaryContext)in.readObject();
 
         arr = new byte[in.readInt()];
 
@@ -536,7 +536,7 @@ public final class BinaryObjectImpl extends BinaryObjectExImpl implements Extern
 
         Object obj0 = reader.deserialize();
 
-        PortableClassDescriptor desc = reader.descriptor();
+        BinaryClassDescriptor desc = reader.descriptor();
 
         assert desc != null;
 
@@ -562,7 +562,7 @@ public final class BinaryObjectImpl extends BinaryObjectExImpl implements Extern
      */
     private BinaryReaderExImpl reader(@Nullable BinaryReaderHandles rCtx) {
         return new BinaryReaderExImpl(ctx,
-            PortableHeapInputStream.create(arr, start),
+            BinaryHeapInputStream.create(arr, start),
             ctx.configuration().getClassLoader(),
             rCtx);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java
index 2944099..1206db7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java
@@ -21,8 +21,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.binary.BinaryType;
-import org.apache.ignite.internal.binary.streams.PortableOffheapInputStream;
-import org.apache.ignite.internal.binary.streams.PortableOffheapInputStream;
+import org.apache.ignite.internal.binary.streams.BinaryOffheapInputStream;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
 import org.apache.ignite.internal.util.GridUnsafe;
@@ -44,20 +43,6 @@ import java.util.Date;
 import java.util.UUID;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.BOOLEAN;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.BYTE;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.CHAR;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.DATE;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.DECIMAL;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.DOUBLE;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.FLOAT;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.INT;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.LONG;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.NULL;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.SHORT;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.STRING;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.TIMESTAMP;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.UUID;
 
 /**
  *  Portable object implementation over offheap memory
@@ -70,7 +55,7 @@ public class BinaryObjectOffheapImpl extends BinaryObjectExImpl implements Exter
     private static final Unsafe UNSAFE = GridUnsafe.unsafe();
 
     /** */
-    private final PortableContext ctx;
+    private final BinaryContext ctx;
 
     /** */
     private final long ptr;
@@ -94,7 +79,7 @@ public class BinaryObjectOffheapImpl extends BinaryObjectExImpl implements Exter
      * @param start Object start.
      * @param size Memory size.
      */
-    public BinaryObjectOffheapImpl(PortableContext ctx, long ptr, int start, int size) {
+    public BinaryObjectOffheapImpl(BinaryContext ctx, long ptr, int start, int size) {
         this.ctx = ctx;
         this.ptr = ptr;
         this.start = start;
@@ -110,26 +95,26 @@ public class BinaryObjectOffheapImpl extends BinaryObjectExImpl implements Exter
 
     /** {@inheritDoc} */
     @Override public int typeId() {
-        return UNSAFE.getInt(ptr + start + GridPortableMarshaller.TYPE_ID_POS);
+        return UNSAFE.getInt(ptr + start + GridBinaryMarshaller.TYPE_ID_POS);
     }
 
     /** {@inheritDoc} */
     @Override public int length() {
-        return UNSAFE.getInt(ptr + start + GridPortableMarshaller.TOTAL_LEN_POS);
+        return UNSAFE.getInt(ptr + start + GridBinaryMarshaller.TOTAL_LEN_POS);
     }
 
     /** {@inheritDoc} */
     @Override public int hashCode() {
-        return UNSAFE.getInt(ptr + start + GridPortableMarshaller.HASH_CODE_POS);
+        return UNSAFE.getInt(ptr + start + GridBinaryMarshaller.HASH_CODE_POS);
     }
 
     /** {@inheritDoc} */
     @Override protected int schemaId() {
-        return UNSAFE.getInt(ptr + start + GridPortableMarshaller.SCHEMA_ID_POS);
+        return UNSAFE.getInt(ptr + start + GridBinaryMarshaller.SCHEMA_ID_POS);
     }
 
     /** {@inheritDoc} */
-    @Override protected PortableSchema createSchema() {
+    @Override protected BinarySchema createSchema() {
         return reader(null).getOrCreateSchema();
     }
 
@@ -179,88 +164,88 @@ public class BinaryObjectOffheapImpl extends BinaryObjectExImpl implements Exter
         Object val;
 
         // Calculate field position.
-        int schemaOffset = PortablePrimitives.readInt(ptr, start + GridPortableMarshaller.SCHEMA_OR_RAW_OFF_POS);
+        int schemaOffset = BinaryPrimitives.readInt(ptr, start + GridBinaryMarshaller.SCHEMA_OR_RAW_OFF_POS);
 
-        short flags = PortablePrimitives.readShort(ptr, start + GridPortableMarshaller.FLAGS_POS);
+        short flags = BinaryPrimitives.readShort(ptr, start + GridBinaryMarshaller.FLAGS_POS);
 
-        int fieldIdLen = PortableUtils.isCompactFooter(flags) ? 0 : PortableUtils.FIELD_ID_LEN;
-        int fieldOffsetLen = PortableUtils.fieldOffsetLength(flags);
+        int fieldIdLen = BinaryUtils.isCompactFooter(flags) ? 0 : BinaryUtils.FIELD_ID_LEN;
+        int fieldOffsetLen = BinaryUtils.fieldOffsetLength(flags);
 
         int fieldOffsetPos = start + schemaOffset + order * (fieldIdLen + fieldOffsetLen) + fieldIdLen;
 
         int fieldPos;
 
-        if (fieldOffsetLen == PortableUtils.OFFSET_1)
-            fieldPos = start + ((int)PortablePrimitives.readByte(ptr, fieldOffsetPos) & 0xFF);
-        else if (fieldOffsetLen == PortableUtils.OFFSET_2)
-            fieldPos = start + ((int)PortablePrimitives.readShort(ptr, fieldOffsetPos) & 0xFFFF);
+        if (fieldOffsetLen == BinaryUtils.OFFSET_1)
+            fieldPos = start + ((int)BinaryPrimitives.readByte(ptr, fieldOffsetPos) & 0xFF);
+        else if (fieldOffsetLen == BinaryUtils.OFFSET_2)
+            fieldPos = start + ((int)BinaryPrimitives.readShort(ptr, fieldOffsetPos) & 0xFFFF);
         else
-            fieldPos = start + PortablePrimitives.readInt(ptr, fieldOffsetPos);
+            fieldPos = start + BinaryPrimitives.readInt(ptr, fieldOffsetPos);
 
         // Read header and try performing fast lookup for well-known types (the most common types go first).
-        byte hdr = PortablePrimitives.readByte(ptr, fieldPos);
+        byte hdr = BinaryPrimitives.readByte(ptr, fieldPos);
 
         switch (hdr) {
-            case GridPortableMarshaller.INT:
-                val = PortablePrimitives.readInt(ptr, fieldPos + 1);
+            case GridBinaryMarshaller.INT:
+                val = BinaryPrimitives.readInt(ptr, fieldPos + 1);
 
                 break;
 
-            case GridPortableMarshaller.LONG:
-                val = PortablePrimitives.readLong(ptr, fieldPos + 1);
+            case GridBinaryMarshaller.LONG:
+                val = BinaryPrimitives.readLong(ptr, fieldPos + 1);
 
                 break;
 
-            case GridPortableMarshaller.BOOLEAN:
-                val = PortablePrimitives.readBoolean(ptr, fieldPos + 1);
+            case GridBinaryMarshaller.BOOLEAN:
+                val = BinaryPrimitives.readBoolean(ptr, fieldPos + 1);
 
                 break;
 
-            case GridPortableMarshaller.SHORT:
-                val = PortablePrimitives.readShort(ptr, fieldPos + 1);
+            case GridBinaryMarshaller.SHORT:
+                val = BinaryPrimitives.readShort(ptr, fieldPos + 1);
 
                 break;
 
-            case GridPortableMarshaller.BYTE:
-                val = PortablePrimitives.readByte(ptr, fieldPos + 1);
+            case GridBinaryMarshaller.BYTE:
+                val = BinaryPrimitives.readByte(ptr, fieldPos + 1);
 
                 break;
 
-            case GridPortableMarshaller.CHAR:
-                val = PortablePrimitives.readChar(ptr, fieldPos + 1);
+            case GridBinaryMarshaller.CHAR:
+                val = BinaryPrimitives.readChar(ptr, fieldPos + 1);
 
                 break;
 
-            case GridPortableMarshaller.FLOAT:
-                val = PortablePrimitives.readFloat(ptr, fieldPos + 1);
+            case GridBinaryMarshaller.FLOAT:
+                val = BinaryPrimitives.readFloat(ptr, fieldPos + 1);
 
                 break;
 
-            case GridPortableMarshaller.DOUBLE:
-                val = PortablePrimitives.readDouble(ptr, fieldPos + 1);
+            case GridBinaryMarshaller.DOUBLE:
+                val = BinaryPrimitives.readDouble(ptr, fieldPos + 1);
 
                 break;
 
-            case GridPortableMarshaller.STRING: {
-                int dataLen = PortablePrimitives.readInt(ptr, fieldPos + 1);
-                byte[] data = PortablePrimitives.readByteArray(ptr, fieldPos + 5, dataLen);
+            case GridBinaryMarshaller.STRING: {
+                int dataLen = BinaryPrimitives.readInt(ptr, fieldPos + 1);
+                byte[] data = BinaryPrimitives.readByteArray(ptr, fieldPos + 5, dataLen);
 
                 val = new String(data, UTF_8);
 
                 break;
             }
 
-            case GridPortableMarshaller.DATE: {
-                long time = PortablePrimitives.readLong(ptr, fieldPos + 1);
+            case GridBinaryMarshaller.DATE: {
+                long time = BinaryPrimitives.readLong(ptr, fieldPos + 1);
 
                 val = new Date(time);
 
                 break;
             }
 
-            case GridPortableMarshaller.TIMESTAMP: {
-                long time = PortablePrimitives.readLong(ptr, fieldPos + 1);
-                int nanos = PortablePrimitives.readInt(ptr, fieldPos + 1 + 8);
+            case GridBinaryMarshaller.TIMESTAMP: {
+                long time = BinaryPrimitives.readLong(ptr, fieldPos + 1);
+                int nanos = BinaryPrimitives.readInt(ptr, fieldPos + 1 + 8);
 
                 Timestamp ts = new Timestamp(time);
 
@@ -271,20 +256,20 @@ public class BinaryObjectOffheapImpl extends BinaryObjectExImpl implements Exter
                 break;
             }
 
-            case GridPortableMarshaller.UUID: {
-                long most = PortablePrimitives.readLong(ptr, fieldPos + 1);
-                long least = PortablePrimitives.readLong(ptr, fieldPos + 1 + 8);
+            case GridBinaryMarshaller.UUID: {
+                long most = BinaryPrimitives.readLong(ptr, fieldPos + 1);
+                long least = BinaryPrimitives.readLong(ptr, fieldPos + 1 + 8);
 
                 val = new UUID(most, least);
 
                 break;
             }
 
-            case GridPortableMarshaller.DECIMAL: {
-                int scale = PortablePrimitives.readInt(ptr, fieldPos + 1);
+            case GridBinaryMarshaller.DECIMAL: {
+                int scale = BinaryPrimitives.readInt(ptr, fieldPos + 1);
 
-                int dataLen = PortablePrimitives.readInt(ptr, fieldPos + 5);
-                byte[] data = PortablePrimitives.readByteArray(ptr, fieldPos + 9, dataLen);
+                int dataLen = BinaryPrimitives.readInt(ptr, fieldPos + 5);
+                byte[] data = BinaryPrimitives.readByteArray(ptr, fieldPos + 9, dataLen);
 
                 BigInteger intVal = new BigInteger(data);
 
@@ -299,17 +284,17 @@ public class BinaryObjectOffheapImpl extends BinaryObjectExImpl implements Exter
                 break;
             }
 
-            case GridPortableMarshaller.NULL:
+            case GridBinaryMarshaller.NULL:
                 val = null;
 
                 break;
 
             default:
-                PortableOffheapInputStream stream = new PortableOffheapInputStream(ptr, size, false);
+                BinaryOffheapInputStream stream = new BinaryOffheapInputStream(ptr, size, false);
 
                 stream.position(fieldPos);
 
-                val = PortableUtils.unmarshal(stream, ctx, null);
+                val = BinaryUtils.unmarshal(stream, ctx, null);
 
                 break;
         }
@@ -420,7 +405,7 @@ public class BinaryObjectOffheapImpl extends BinaryObjectExImpl implements Exter
      * @return Reader.
      */
     private BinaryReaderExImpl reader(@Nullable BinaryReaderHandles rCtx) {
-        PortableOffheapInputStream stream = new PortableOffheapInputStream(ptr, size, false);
+        BinaryOffheapInputStream stream = new BinaryOffheapInputStream(ptr, size, false);
 
         stream.position(start);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryPositionReadable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryPositionReadable.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryPositionReadable.java
new file mode 100644
index 0000000..c85bb19
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryPositionReadable.java
@@ -0,0 +1,47 @@
+/*
+ * 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.binary;
+
+/**
+ * Interface allowing for positioned read.
+ */
+public interface BinaryPositionReadable {
+    /**
+     * 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.
+     * @return Value.
+     */
+    public short readShortPositioned(int pos);
+
+    /**
+     * Read integer at the given position.
+     *
+     * @param pos Position.
+     * @return Value.
+     */
+    public int readIntPositioned(int pos);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryPrimitives.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryPrimitives.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryPrimitives.java
new file mode 100644
index 0000000..a6a867c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryPrimitives.java
@@ -0,0 +1,382 @@
+/*
+ * 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.binary;
+
+import org.apache.ignite.internal.util.GridUnsafe;
+import sun.misc.Unsafe;
+
+import java.nio.ByteOrder;
+
+/**
+ * Primitives writer.
+ */
+public abstract class BinaryPrimitives {
+    /** */
+    private static final Unsafe UNSAFE = GridUnsafe.unsafe();
+
+    /** */
+    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 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 static byte readByte(byte[] arr, int off) {
+        return UNSAFE.getByte(arr, BYTE_ARR_OFF + off);
+    }
+
+    /**
+     * @param ptr Pointer.
+     * @param off Offset.
+     * @return Value.
+     */
+    public static byte readByte(long ptr, int off) {
+        return UNSAFE.getByte(ptr + off);
+    }
+
+    /**
+     * @param arr Array.
+     * @param off Offset.
+     * @return Value.
+     */
+    public static 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);
+
+        return arr0;
+    }
+
+    /**
+     * @param ptr Pointer.
+     * @param off Offset.
+     * @return Value.
+     */
+    public static byte[] readByteArray(long ptr, int off, int len) {
+        byte[] arr0 = new byte[len];
+
+        UNSAFE.copyMemory(null, ptr + off, arr0, BYTE_ARR_OFF, len);
+
+        return arr0;
+    }
+
+    /**
+     * @param arr Array.
+     * @param off Offset.
+     * @param val Value.
+     */
+    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 static boolean readBoolean(byte[] arr, int off) {
+        return readByte(arr, off) == 1;
+    }
+
+    /**
+     * @param ptr Pointer.
+     * @param off Offset.
+     * @return Value.
+     */
+    public static boolean readBoolean(long ptr, int off) {
+        return readByte(ptr, off) == 1;
+    }
+
+    /**
+     * @param arr Array.
+     * @param off Offset.
+     * @param val Value.
+     */
+    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 static short readShort(byte[] arr, int off) {
+        short val = UNSAFE.getShort(arr, BYTE_ARR_OFF + off);
+
+        if (BIG_ENDIAN)
+            val = Short.reverseBytes(val);
+
+        return val;
+    }
+
+    /**
+     * @param ptr Pointer.
+     * @param off Offset.
+     * @return Value.
+     */
+    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 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 static char readChar(byte[] arr, int off) {
+        char val = UNSAFE.getChar(arr, BYTE_ARR_OFF + off);
+
+        if (BIG_ENDIAN)
+            val = Character.reverseBytes(val);
+
+        return val;
+    }
+
+    /**
+     * @param ptr Pointer.
+     * @param off Offset.
+     * @return Value.
+     */
+    public static char readChar(long ptr, int off) {
+        char val = UNSAFE.getChar(ptr + off);
+
+        if (BIG_ENDIAN)
+            val = Character.reverseBytes(val);
+
+        return val;
+    }
+
+    /**
+     * @param arr Array.
+     * @param off Offset.
+     * @return Value.
+     */
+    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 ptr Pointer.
+     * @param off Offset.
+     * @return Value.
+     */
+    public static char[] readCharArray(long ptr, int off, int len) {
+        char[] arr0 = new char[len];
+
+        UNSAFE.copyMemory(null, ptr + 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 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 static int readInt(byte[] arr, int off) {
+        int val = UNSAFE.getInt(arr, BYTE_ARR_OFF + off);
+
+        if (BIG_ENDIAN)
+            val = Integer.reverseBytes(val);
+
+        return val;
+    }
+
+    /**
+     * @param ptr Pointer.
+     * @param off Offset.
+     * @return Value.
+     */
+    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 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 static long readLong(byte[] arr, int off) {
+        long val = UNSAFE.getLong(arr, BYTE_ARR_OFF + off);
+
+        if (BIG_ENDIAN)
+            val = Long.reverseBytes(val);
+
+        return val;
+    }
+
+    /**
+     * @param ptr Pointer.
+     * @param off Offset.
+     * @return Value.
+     */
+    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 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 static float readFloat(byte[] arr, int off) {
+        int val = readInt(arr, off);
+
+        return Float.intBitsToFloat(val);
+    }
+
+    /**
+     * @param ptr Pointer.
+     * @param off Offset.
+     * @return Value.
+     */
+    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 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 static double readDouble(byte[] arr, int off) {
+        long val = readLong(arr, off);
+
+        return Double.longBitsToDouble(val);
+    }
+
+    /**
+     * @param ptr Pointer.
+     * @param off Offset.
+     * @return Value.
+     */
+    public static double readDouble(long ptr, int off) {
+        long val = readLong(ptr, off);
+
+        return Double.longBitsToDouble(val);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryRawWriterEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryRawWriterEx.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryRawWriterEx.java
index 6e7e0cb..0ba0d90 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryRawWriterEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryRawWriterEx.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.binary;
 
-import org.apache.ignite.internal.binary.streams.PortableOutputStream;
+import org.apache.ignite.internal.binary.streams.BinaryOutputStream;
 import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.binary.BinaryRawWriter;
 import org.jetbrains.annotations.Nullable;
@@ -35,7 +35,7 @@ public interface BinaryRawWriterEx extends BinaryRawWriter, AutoCloseable {
     /**
      * @return Output stream.
      */
-    public PortableOutputStream out();
+    public BinaryOutputStream out();
 
     /**
      * Cleans resources.

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java
index bf47d53..b673e27 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java
@@ -35,50 +35,50 @@ import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.binary.BinaryRawReader;
 import org.apache.ignite.binary.BinaryReader;
-import org.apache.ignite.internal.binary.streams.PortableInputStream;
+import org.apache.ignite.internal.binary.streams.BinaryInputStream;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.BOOLEAN;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.BOOLEAN_ARR;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.BYTE;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.BYTE_ARR;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.CHAR;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.CHAR_ARR;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.CLASS;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.COL;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.DATE;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.DATE_ARR;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.DECIMAL;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.DECIMAL_ARR;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.DFLT_HDR_LEN;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.DOUBLE;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.DOUBLE_ARR;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.ENUM;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.ENUM_ARR;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.FLOAT;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.FLOAT_ARR;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.HANDLE;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.INT;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.INT_ARR;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.LONG;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.LONG_ARR;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.MAP;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.NULL;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.OBJ;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.OBJ_ARR;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.OPTM_MARSH;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.PORTABLE_OBJ;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.SHORT;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.SHORT_ARR;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.STRING;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.STRING_ARR;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.TIMESTAMP;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.TIMESTAMP_ARR;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.UNREGISTERED_TYPE_ID;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.UUID;
-import static org.apache.ignite.internal.binary.GridPortableMarshaller.UUID_ARR;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.BOOLEAN;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.BOOLEAN_ARR;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.BYTE;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.BYTE_ARR;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.CHAR;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.CHAR_ARR;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.CLASS;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.COL;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.DATE;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.DATE_ARR;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.DECIMAL;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.DECIMAL_ARR;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.DFLT_HDR_LEN;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.DOUBLE;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.DOUBLE_ARR;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.ENUM;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.ENUM_ARR;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.FLOAT;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.FLOAT_ARR;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.HANDLE;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.INT;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.INT_ARR;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.LONG;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.LONG_ARR;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.MAP;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.NULL;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.OBJ;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.OBJ_ARR;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.OPTM_MARSH;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.PORTABLE_OBJ;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.SHORT;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.SHORT_ARR;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.STRING;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.STRING_ARR;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.TIMESTAMP;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.TIMESTAMP_ARR;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.UNREGISTERED_TYPE_ID;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.UUID;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.UUID_ARR;
 
 /**
  * Portable reader implementation.
@@ -86,10 +86,10 @@ import static org.apache.ignite.internal.binary.GridPortableMarshaller.UUID_ARR;
 @SuppressWarnings("unchecked")
 public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, BinaryReaderHandlesHolder, ObjectInput {
     /** Portable context. */
-    private final PortableContext ctx;
+    private final BinaryContext ctx;
 
     /** Input stream. */
-    private final PortableInputStream in;
+    private final BinaryInputStream in;
 
     /** Class loaded. */
     private final ClassLoader ldr;
@@ -131,7 +131,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
     private final int fieldOffsetLen;
 
     /** Object schema. */
-    private final PortableSchema schema;
+    private final BinarySchema schema;
 
     /** Whether passed IDs matches schema order. Reset to false as soon as a single mismatch detected. */
     private boolean matching = true;
@@ -149,7 +149,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
      * @param in Input stream.
      * @param ldr Class loader.
      */
-    public BinaryReaderExImpl(PortableContext ctx, PortableInputStream in, ClassLoader ldr) {
+    public BinaryReaderExImpl(BinaryContext ctx, BinaryInputStream in, ClassLoader ldr) {
         this(ctx, in, ldr, null);
     }
 
@@ -161,7 +161,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
      * @param ldr Class loader.
      * @param hnds Context.
      */
-    public BinaryReaderExImpl(PortableContext ctx, PortableInputStream in, ClassLoader ldr,
+    public BinaryReaderExImpl(BinaryContext ctx, BinaryInputStream in, ClassLoader ldr,
         @Nullable BinaryReaderHandles hnds) {
         this(ctx, in, ldr, hnds, false);
     }
@@ -175,7 +175,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
      * @param hnds Context.
      * @param skipHdrCheck Whether to skip header check.
      */
-    public BinaryReaderExImpl(PortableContext ctx, PortableInputStream in, ClassLoader ldr,
+    public BinaryReaderExImpl(BinaryContext ctx, BinaryInputStream in, ClassLoader ldr,
         @Nullable BinaryReaderHandles hnds, boolean skipHdrCheck) {
         // Initialize base members.
         this.ctx = ctx;
@@ -186,9 +186,9 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
         start = in.position();
 
         // Perform full header parsing in case of portable object.
-        if (!skipHdrCheck && (in.readByte() == GridPortableMarshaller.OBJ)) {
+        if (!skipHdrCheck && (in.readByte() == GridBinaryMarshaller.OBJ)) {
             // Ensure protocol is fine.
-            PortableUtils.checkProtocolVersion(in.readByte());
+            BinaryUtils.checkProtocolVersion(in.readByte());
 
             // Read header content.
             short flags = in.readShort();
@@ -201,16 +201,16 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
             int offset = in.readInt();
 
             // Get trivial flag values.
-            userType = PortableUtils.isUserType(flags);
-            fieldIdLen = PortableUtils.fieldIdLength(flags);
-            fieldOffsetLen = PortableUtils.fieldOffsetLength(flags);
+            userType = BinaryUtils.isUserType(flags);
+            fieldIdLen = BinaryUtils.fieldIdLength(flags);
+            fieldOffsetLen = BinaryUtils.fieldOffsetLength(flags);
 
             // Calculate footer borders and raw offset.
-            if (PortableUtils.hasSchema(flags)) {
+            if (BinaryUtils.hasSchema(flags)) {
                 // Schema exists.
                 footerStart = start + offset;
 
-                if (PortableUtils.hasRaw(flags)) {
+                if (BinaryUtils.hasRaw(flags)) {
                     footerLen = len - offset - 4;
                     rawOff = start + in.readIntPositioned(start + len - 4);
                 }
@@ -224,7 +224,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
                 footerStart = start + len;
                 footerLen = 0;
 
-                if (PortableUtils.hasRaw(flags))
+                if (BinaryUtils.hasRaw(flags))
                     rawOff = start + offset;
                 else
                     rawOff = start + len;
@@ -235,7 +235,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
                 int off = in.position();
 
                 // Registers class by type ID, at least locally if the cache is not ready yet.
-                typeId = ctx.descriptorForClass(PortableUtils.doReadClass(in, ctx, ldr, typeId0), false).typeId();
+                typeId = ctx.descriptorForClass(BinaryUtils.doReadClass(in, ctx, ldr, typeId0), false).typeId();
 
                 int clsNameLen = in.position() - off;
 
@@ -248,7 +248,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
             }
 
             idMapper = userType ? ctx.userTypeIdMapper(typeId) : BinaryInternalIdMapper.defaultInstance();
-            schema = PortableUtils.hasSchema(flags) ? getOrCreateSchema() : null;
+            schema = BinaryUtils.hasSchema(flags) ? getOrCreateSchema() : null;
         }
         else {
             dataStart = 0;
@@ -270,14 +270,14 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
     /**
      * @return Input stream.
      */
-    public PortableInputStream in() {
+    public BinaryInputStream in() {
         return in;
     }
 
     /**
      * @return Descriptor.
      */
-    PortableClassDescriptor descriptor() {
+    BinaryClassDescriptor descriptor() {
         return ctx.descriptorForTypeId(userType, typeId, ldr, true);
     }
 
@@ -289,7 +289,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
     public Object unmarshal(int offset) throws BinaryObjectException {
         streamPosition(offset);
 
-        return in.position() >= 0 ? PortableUtils.unmarshal(in, ctx, ldr, this) : null;
+        return in.position() >= 0 ? BinaryUtils.unmarshal(in, ctx, ldr, this) : null;
     }
 
     /**
@@ -298,7 +298,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
      * @throws BinaryObjectException In case of error.
      */
     @Nullable Object unmarshalField(String fieldName) throws BinaryObjectException {
-        return findFieldByName(fieldName) ? PortableUtils.unmarshal(in, ctx, ldr, this) : null;
+        return findFieldByName(fieldName) ? BinaryUtils.unmarshal(in, ctx, ldr, this) : null;
     }
 
     /**
@@ -307,7 +307,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
      * @throws BinaryObjectException In case of error.
      */
     @Nullable Object unmarshalField(int fieldId) throws BinaryObjectException {
-        return findFieldById(fieldId) ? PortableUtils.unmarshal(in, ctx, ldr, this) : null;
+        return findFieldById(fieldId) ? BinaryUtils.unmarshal(in, ctx, ldr, this) : null;
     }
 
     /**
@@ -320,7 +320,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
             if (checkFlag(PORTABLE_OBJ) == Flag.NULL)
                 return null;
 
-            return new BinaryObjectImpl(ctx, PortableUtils.doReadByteArray(in), in.readInt());
+            return new BinaryObjectImpl(ctx, BinaryUtils.doReadByteArray(in), in.readInt());
         }
         else
             return null;
@@ -336,7 +336,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
             if (checkFlag(CLASS) == Flag.NULL)
                 return null;
 
-            return PortableUtils.doReadClass(in, ctx, ldr);
+            return BinaryUtils.doReadClass(in, ctx, ldr);
         }
 
         return null;
@@ -374,7 +374,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
      * @return Field.
      */
     private <T> T readHandleField() {
-        int handlePos = PortableUtils.positionForHandle(in) - in.readInt();
+        int handlePos = BinaryUtils.positionForHandle(in) - in.readInt();
 
         Object obj = getHandle(handlePos);
 
@@ -383,7 +383,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
 
             streamPosition(handlePos);
 
-            obj = PortableUtils.doReadObject(in, ctx, ldr, this);
+            obj = BinaryUtils.doReadObject(in, ctx, ldr, this);
 
             streamPosition(retPos);
         }
@@ -436,7 +436,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
     @Nullable @Override public byte[] readByteArray() throws BinaryObjectException {
         switch (checkFlag(BYTE_ARR)) {
             case NORMAL:
-                return PortableUtils.doReadByteArray(in);
+                return BinaryUtils.doReadByteArray(in);
 
             case HANDLE:
                 return readHandleField();
@@ -492,7 +492,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
     @Nullable @Override public boolean[] readBooleanArray() throws BinaryObjectException {
         switch (checkFlag(BOOLEAN_ARR)) {
             case NORMAL:
-                return PortableUtils.doReadBooleanArray(in);
+                return BinaryUtils.doReadBooleanArray(in);
 
             case HANDLE:
                 return readHandleField();
@@ -548,7 +548,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
     @Nullable @Override public short[] readShortArray() throws BinaryObjectException {
         switch (checkFlag(SHORT_ARR)) {
             case NORMAL:
-                return PortableUtils.doReadShortArray(in);
+                return BinaryUtils.doReadShortArray(in);
 
             case HANDLE:
                 return readHandleField();
@@ -604,7 +604,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
     @Nullable @Override public char[] readCharArray() throws BinaryObjectException {
         switch (checkFlag(CHAR_ARR)) {
             case NORMAL:
-                return PortableUtils.doReadCharArray(in);
+                return BinaryUtils.doReadCharArray(in);
 
             case HANDLE:
                 return readHandleField();
@@ -660,7 +660,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
     @Nullable @Override public int[] readIntArray() throws BinaryObjectException {
         switch (checkFlag(INT_ARR)) {
             case NORMAL:
-                return PortableUtils.doReadIntArray(in);
+                return BinaryUtils.doReadIntArray(in);
 
             case HANDLE:
                 return readHandleField();
@@ -716,7 +716,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
     @Nullable @Override public long[] readLongArray() throws BinaryObjectException {
         switch (checkFlag(LONG_ARR)) {
             case NORMAL:
-                return PortableUtils.doReadLongArray(in);
+                return BinaryUtils.doReadLongArray(in);
 
             case HANDLE:
                 return readHandleField();
@@ -772,7 +772,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
     @Nullable @Override public float[] readFloatArray() throws BinaryObjectException {
         switch (checkFlag(FLOAT_ARR)) {
             case NORMAL:
-                return PortableUtils.doReadFloatArray(in);
+                return BinaryUtils.doReadFloatArray(in);
 
             case HANDLE:
                 return readHandleField();
@@ -828,7 +828,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
     @Nullable @Override public double[] readDoubleArray() throws BinaryObjectException {
         switch (checkFlag(DOUBLE_ARR)) {
             case NORMAL:
-                return PortableUtils.doReadDoubleArray(in);
+                return BinaryUtils.doReadDoubleArray(in);
 
             case HANDLE:
                 return readHandleField();
@@ -854,7 +854,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
 
     /** {@inheritDoc} */
     @Override @Nullable public BigDecimal readDecimal() throws BinaryObjectException {
-        return checkFlagNoHandles(DECIMAL) == Flag.NORMAL ? PortableUtils.doReadDecimal(in) : null;
+        return checkFlagNoHandles(DECIMAL) == Flag.NORMAL ? BinaryUtils.doReadDecimal(in) : null;
     }
 
     /** {@inheritDoc} */
@@ -875,7 +875,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
     @Override @Nullable public BigDecimal[] readDecimalArray() throws BinaryObjectException {
         switch (checkFlag(DECIMAL_ARR)) {
             case NORMAL:
-                return PortableUtils.doReadDecimalArray(in);
+                return BinaryUtils.doReadDecimalArray(in);
 
             case HANDLE:
                 return readHandleField();
@@ -901,7 +901,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
 
     /** {@inheritDoc} */
     @Override @Nullable public String readString() throws BinaryObjectException {
-        return checkFlagNoHandles(STRING) == Flag.NORMAL ? PortableUtils.doReadString(in) : null;
+        return checkFlagNoHandles(STRING) == Flag.NORMAL ? BinaryUtils.doReadString(in) : null;
     }
 
     /** {@inheritDoc} */
@@ -922,7 +922,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
     @Override @Nullable public String[] readStringArray() throws BinaryObjectException {
         switch (checkFlag(STRING_ARR)) {
             case NORMAL:
-                return PortableUtils.doReadStringArray(in);
+                return BinaryUtils.doReadStringArray(in);
 
             case HANDLE:
                 return readHandleField();
@@ -948,7 +948,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
 
     /** {@inheritDoc} */
     @Override @Nullable public UUID readUuid() throws BinaryObjectException {
-        return checkFlagNoHandles(UUID) == Flag.NORMAL ? PortableUtils.doReadUuid(in) : null;
+        return checkFlagNoHandles(UUID) == Flag.NORMAL ? BinaryUtils.doReadUuid(in) : null;
     }
 
     /** {@inheritDoc} */
@@ -969,7 +969,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
     @Override @Nullable public UUID[] readUuidArray() throws BinaryObjectException {
         switch (checkFlag(UUID_ARR)) {
             case NORMAL:
-                return PortableUtils.doReadUuidArray(in);
+                return BinaryUtils.doReadUuidArray(in);
 
             case HANDLE:
                 return readHandleField();
@@ -995,7 +995,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
 
     /** {@inheritDoc} */
     @Override @Nullable public Date readDate() throws BinaryObjectException {
-        return checkFlagNoHandles(DATE) == Flag.NORMAL ? PortableUtils.doReadDate(in) : null;
+        return checkFlagNoHandles(DATE) == Flag.NORMAL ? BinaryUtils.doReadDate(in) : null;
     }
 
     /** {@inheritDoc} */
@@ -1016,7 +1016,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
     @Override @Nullable public Date[] readDateArray() throws BinaryObjectException {
         switch (checkFlag(DATE_ARR)) {
             case NORMAL:
-                return PortableUtils.doReadDateArray(in);
+                return BinaryUtils.doReadDateArray(in);
 
             case HANDLE:
                 return readHandleField();
@@ -1042,7 +1042,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
 
     /** {@inheritDoc} */
     @Override @Nullable public Timestamp readTimestamp() throws BinaryObjectException {
-        return checkFlagNoHandles(TIMESTAMP) == Flag.NORMAL ? PortableUtils.doReadTimestamp(in) : null;
+        return checkFlagNoHandles(TIMESTAMP) == Flag.NORMAL ? BinaryUtils.doReadTimestamp(in) : null;
     }
 
     /** {@inheritDoc} */
@@ -1063,7 +1063,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
     @Override @Nullable public Timestamp[] readTimestampArray() throws BinaryObjectException {
         switch (checkFlag(TIMESTAMP_ARR)) {
             case NORMAL:
-                return PortableUtils.doReadTimestampArray(in);
+                return BinaryUtils.doReadTimestampArray(in);
 
             case HANDLE:
                 return readHandleField();
@@ -1076,7 +1076,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Nullable @Override public <T> T readObject(String fieldName) throws BinaryObjectException {
-        return findFieldByName(fieldName) ? (T)PortableUtils.doReadObject(in, ctx, ldr, this) : null;
+        return findFieldByName(fieldName) ? (T)BinaryUtils.doReadObject(in, ctx, ldr, this) : null;
     }
 
     /**
@@ -1085,17 +1085,17 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
      * @throws BinaryObjectException In case of error.
      */
     @Nullable Object readObject(int fieldId) throws BinaryObjectException {
-        return findFieldById(fieldId) ? PortableUtils.doReadObject(in, ctx, ldr, this) : null;
+        return findFieldById(fieldId) ? BinaryUtils.doReadObject(in, ctx, ldr, this) : null;
     }
 
     /** {@inheritDoc} */
     @Override public Object readObject() throws BinaryObjectException {
-        return PortableUtils.doReadObject(in, ctx, ldr, this);
+        return BinaryUtils.doReadObject(in, ctx, ldr, this);
     }
 
     /** {@inheritDoc} */
     @Nullable @Override public Object readObjectDetached() throws BinaryObjectException {
-        return PortableUtils.unmarshal(in, ctx, ldr, this, true);
+        return BinaryUtils.unmarshal(in, ctx, ldr, this, true);
     }
 
     /** {@inheritDoc} */
@@ -1116,7 +1116,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
     @Nullable @Override public Object[] readObjectArray() throws BinaryObjectException {
         switch (checkFlag(OBJ_ARR)) {
             case NORMAL:
-                return PortableUtils.doReadObjectArray(in, ctx, ldr, this, true);
+                return BinaryUtils.doReadObjectArray(in, ctx, ldr, this, true);
 
             case HANDLE:
                 return readHandleField();
@@ -1156,12 +1156,12 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
     private Enum<?> readEnum0(@Nullable Class<?> cls) throws BinaryObjectException {
         if (checkFlagNoHandles(ENUM) == Flag.NORMAL) {
             // Read class even if we know it in advance to set correct stream position.
-            Class<?> cls0 = PortableUtils.doReadClass(in, ctx, ldr);
+            Class<?> cls0 = BinaryUtils.doReadClass(in, ctx, ldr);
 
             if (cls == null)
                 cls = cls0;
 
-            return PortableUtils.doReadEnum(in, cls);
+            return BinaryUtils.doReadEnum(in, cls);
         }
         else
             return null;
@@ -1199,12 +1199,12 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
         switch (checkFlag(ENUM_ARR)) {
             case NORMAL:
                 // Read class even if we know it in advance to set correct stream position.
-                Class<?> cls0 = PortableUtils.doReadClass(in, ctx, ldr);
+                Class<?> cls0 = BinaryUtils.doReadClass(in, ctx, ldr);
 
                 if (cls == null)
                     cls = cls0;
 
-                return PortableUtils.doReadEnumArray(in, ctx, ldr, cls);
+                return BinaryUtils.doReadEnumArray(in, ctx, ldr, cls);
 
             case HANDLE:
                 return readHandleField();
@@ -1258,10 +1258,10 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
         throws BinaryObjectException {
         switch (checkFlag(COL)) {
             case NORMAL:
-                return (Collection)PortableUtils.doReadCollection(in, ctx, ldr, this, true, factory);
+                return (Collection)BinaryUtils.doReadCollection(in, ctx, ldr, this, true, factory);
 
             case HANDLE: {
-                int handlePos = PortableUtils.positionForHandle(in) - in.readInt();
+                int handlePos = BinaryUtils.positionForHandle(in) - in.readInt();
 
                 Object obj = getHandle(handlePos);
 
@@ -1325,10 +1325,10 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
     private Map readMap0(@Nullable BinaryMapFactory factory) throws BinaryObjectException {
         switch (checkFlag(MAP)) {
             case NORMAL:
-                return (Map)PortableUtils.doReadMap(in, ctx, ldr, this, true, factory);
+                return (Map)BinaryUtils.doReadMap(in, ctx, ldr, this, true, factory);
 
             case HANDLE: {
-                int handlePos = PortableUtils.positionForHandle(in) - in.readInt();
+                int handlePos = BinaryUtils.positionForHandle(in) - in.readInt();
 
                 Object obj = getHandle(handlePos);
 
@@ -1367,7 +1367,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
         else if (flag == HANDLE)
             return Flag.HANDLE;
 
-        int pos = PortableUtils.positionForHandle(in);
+        int pos = BinaryUtils.positionForHandle(in);
 
         throw new BinaryObjectException("Unexpected flag value [pos=" + pos + ", expected=" + expFlag +
             ", actual=" + flag + ']');
@@ -1388,7 +1388,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
         else if (flag == NULL)
             return Flag.NULL;
 
-        int pos = PortableUtils.positionForHandle(in);
+        int pos = BinaryUtils.positionForHandle(in);
 
         throw new BinaryObjectException("Unexpected flag value [pos=" + pos + ", expected=" + expFlag +
             ", actual=" + flag + ']');
@@ -1432,7 +1432,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
 
                     streamPosition(handlePos);
 
-                    obj = PortableUtils.doReadObject(in, ctx, ldr, this);
+                    obj = BinaryUtils.doReadObject(in, ctx, ldr, this);
 
                     streamPosition(retPos);
                 }
@@ -1440,7 +1440,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
                 break;
 
             case OBJ:
-                PortableClassDescriptor desc = ctx.descriptorForTypeId(userType, typeId, ldr, true);
+                BinaryClassDescriptor desc = ctx.descriptorForTypeId(userType, typeId, ldr, true);
 
                 streamPosition(dataStart);
 
@@ -1494,137 +1494,137 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
                 break;
 
             case DECIMAL:
-                obj = PortableUtils.doReadDecimal(in);
+                obj = BinaryUtils.doReadDecimal(in);
 
                 break;
 
             case STRING:
-                obj = PortableUtils.doReadString(in);
+                obj = BinaryUtils.doReadString(in);
 
                 break;
 
             case UUID:
-                obj = PortableUtils.doReadUuid(in);
+                obj = BinaryUtils.doReadUuid(in);
 
                 break;
 
             case DATE:
-                obj = PortableUtils.doReadDate(in);
+                obj = BinaryUtils.doReadDate(in);
 
                 break;
 
             case TIMESTAMP:
-                obj = PortableUtils.doReadTimestamp(in);
+                obj = BinaryUtils.doReadTimestamp(in);
 
                 break;
 
             case BYTE_ARR:
-                obj = PortableUtils.doReadByteArray(in);
+                obj = BinaryUtils.doReadByteArray(in);
 
                 break;
 
             case SHORT_ARR:
-                obj = PortableUtils.doReadShortArray(in);
+                obj = BinaryUtils.doReadShortArray(in);
 
                 break;
 
             case INT_ARR:
-                obj = PortableUtils.doReadIntArray(in);
+                obj = BinaryUtils.doReadIntArray(in);
 
                 break;
 
             case LONG_ARR:
-                obj = PortableUtils.doReadLongArray(in);
+                obj = BinaryUtils.doReadLongArray(in);
 
                 break;
 
             case FLOAT_ARR:
-                obj = PortableUtils.doReadFloatArray(in);
+                obj = BinaryUtils.doReadFloatArray(in);
 
                 break;
 
             case DOUBLE_ARR:
-                obj = PortableUtils.doReadDoubleArray(in);
+                obj = BinaryUtils.doReadDoubleArray(in);
 
                 break;
 
             case CHAR_ARR:
-                obj = PortableUtils.doReadCharArray(in);
+                obj = BinaryUtils.doReadCharArray(in);
 
                 break;
 
             case BOOLEAN_ARR:
-                obj = PortableUtils.doReadBooleanArray(in);
+                obj = BinaryUtils.doReadBooleanArray(in);
 
                 break;
 
             case DECIMAL_ARR:
-                obj = PortableUtils.doReadDecimalArray(in);
+                obj = BinaryUtils.doReadDecimalArray(in);
 
                 break;
 
             case STRING_ARR:
-                obj = PortableUtils.doReadStringArray(in);
+                obj = BinaryUtils.doReadStringArray(in);
 
                 break;
 
             case UUID_ARR:
-                obj = PortableUtils.doReadUuidArray(in);
+                obj = BinaryUtils.doReadUuidArray(in);
 
                 break;
 
             case DATE_ARR:
-                obj = PortableUtils.doReadDateArray(in);
+                obj = BinaryUtils.doReadDateArray(in);
 
                 break;
 
             case TIMESTAMP_ARR:
-                obj = PortableUtils.doReadTimestampArray(in);
+                obj = BinaryUtils.doReadTimestampArray(in);
 
                 break;
 
             case OBJ_ARR:
-                obj = PortableUtils.doReadObjectArray(in, ctx, ldr, this, true);
+                obj = BinaryUtils.doReadObjectArray(in, ctx, ldr, this, true);
 
                 break;
 
             case COL:
-                obj = PortableUtils.doReadCollection(in, ctx, ldr, this, true, null);
+                obj = BinaryUtils.doReadCollection(in, ctx, ldr, this, true, null);
 
                 break;
 
             case MAP:
-                obj = PortableUtils.doReadMap(in, ctx, ldr, this, true, null);
+                obj = BinaryUtils.doReadMap(in, ctx, ldr, this, true, null);
 
                 break;
 
             case PORTABLE_OBJ:
-                obj = PortableUtils.doReadPortableObject(in, ctx);
+                obj = BinaryUtils.doReadPortableObject(in, ctx);
 
                 ((BinaryObjectImpl)obj).context(ctx);
 
-                if (!GridPortableMarshaller.KEEP_PORTABLES.get())
+                if (!GridBinaryMarshaller.KEEP_PORTABLES.get())
                     obj = ((BinaryObject)obj).deserialize();
 
                 break;
 
             case ENUM:
-                obj = PortableUtils.doReadEnum(in, PortableUtils.doReadClass(in, ctx, ldr));
+                obj = BinaryUtils.doReadEnum(in, BinaryUtils.doReadClass(in, ctx, ldr));
 
                 break;
 
             case ENUM_ARR:
-                obj = PortableUtils.doReadEnumArray(in, ctx, ldr, PortableUtils.doReadClass(in, ctx, ldr));
+                obj = BinaryUtils.doReadEnumArray(in, ctx, ldr, BinaryUtils.doReadClass(in, ctx, ldr));
 
                 break;
 
             case CLASS:
-                obj = PortableUtils.doReadClass(in, ctx, ldr);
+                obj = BinaryUtils.doReadClass(in, ctx, ldr);
 
                 break;
 
             case OPTM_MARSH:
-                obj = PortableUtils.doReadOptimized(in, ctx, ldr);
+                obj = BinaryUtils.doReadOptimized(in, ctx, ldr);
 
                 break;
 
@@ -1661,18 +1661,18 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
      *
      * @return Schema.
      */
-    public PortableSchema getOrCreateSchema() {
-        PortableSchema schema = ctx.schemaRegistry(typeId).schema(schemaId);
+    public BinarySchema getOrCreateSchema() {
+        BinarySchema schema = ctx.schemaRegistry(typeId).schema(schemaId);
 
         if (schema == null) {
-            if (fieldIdLen != PortableUtils.FIELD_ID_LEN) {
+            if (fieldIdLen != BinaryUtils.FIELD_ID_LEN) {
                 BinaryTypeImpl type = (BinaryTypeImpl)ctx.metadata(typeId);
 
                 if (type == null || type.metadata() == null)
                     throw new BinaryObjectException("Cannot find metadata for object with compact footer: " +
                         typeId);
 
-                for (PortableSchema typeSchema : type.metadata().schemas()) {
+                for (BinarySchema typeSchema : type.metadata().schemas()) {
                     if (schemaId == typeSchema.schemaId()) {
                         schema = typeSchema;
 
@@ -1700,10 +1700,10 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
      *
      * @return Schema.
      */
-    private PortableSchema createSchema() {
-        assert fieldIdLen == PortableUtils.FIELD_ID_LEN;
+    private BinarySchema createSchema() {
+        assert fieldIdLen == BinaryUtils.FIELD_ID_LEN;
 
-        PortableSchema.Builder builder = PortableSchema.Builder.newBuilder();
+        BinarySchema.Builder builder = BinarySchema.Builder.newBuilder();
 
         int searchPos = footerStart;
         int searchEnd = searchPos + footerLen;
@@ -1713,7 +1713,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
 
             builder.addField(fieldId);
 
-            searchPos += PortableUtils.FIELD_ID_LEN + fieldOffsetLen;
+            searchPos += BinaryUtils.FIELD_ID_LEN + fieldOffsetLen;
         }
 
         return builder.build();
@@ -1740,7 +1740,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
             if (matching) {
                 int expOrder = matchingOrder++;
 
-                PortableSchema.Confirmation confirm = schema.confirmOrder(expOrder, name);
+                BinarySchema.Confirmation confirm = schema.confirmOrder(expOrder, name);
 
                 switch (confirm) {
                     case CONFIRMED:
@@ -1761,7 +1761,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
 
                     default:
                         // Field name is not know for this order. Need to calculate ID and repeat speculation.
-                        assert confirm == PortableSchema.Confirmation.CLARIFY;
+                        assert confirm == BinarySchema.Confirmation.CLARIFY;
 
                         int id = fieldId(name);
                         int realId = schema.fieldId(expOrder);
@@ -1846,10 +1846,10 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
      * @return {@code True} if field was found and stream was positioned accordingly.
      */
     private boolean trySetUserFieldPosition(int order) {
-        if (order != PortableSchema.ORDER_NOT_FOUND) {
+        if (order != BinarySchema.ORDER_NOT_FOUND) {
             int offsetPos = footerStart + order * (fieldIdLen + fieldOffsetLen) + fieldIdLen;
 
-            int pos = start + PortableUtils.fieldOffsetRelative(in, offsetPos, fieldOffsetLen);
+            int pos = start + BinaryUtils.fieldOffsetRelative(in, offsetPos, fieldOffsetLen);
 
             streamPosition(pos);
 
@@ -1867,7 +1867,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
      */
     private boolean trySetSystemFieldPosition(int id) {
         // System types are never written with compact footers because they do not have metadata.
-        assert fieldIdLen == PortableUtils.FIELD_ID_LEN;
+        assert fieldIdLen == BinaryUtils.FIELD_ID_LEN;
 
         int searchPos = footerStart;
         int searchTail = searchPos + footerLen;
@@ -1879,7 +1879,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
             int id0 = in.readIntPositioned(searchPos);
 
             if (id0 == id) {
-                int pos = start + PortableUtils.fieldOffsetRelative(in, searchPos + PortableUtils.FIELD_ID_LEN,
+                int pos = start + BinaryUtils.fieldOffsetRelative(in, searchPos + BinaryUtils.FIELD_ID_LEN,
                     fieldOffsetLen);
 
                 streamPosition(pos);
@@ -1887,7 +1887,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
                 return true;
             }
 
-            searchPos += PortableUtils.FIELD_ID_LEN + fieldOffsetLen;
+            searchPos += BinaryUtils.FIELD_ID_LEN + fieldOffsetLen;
         }
     }