You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2015/11/26 11:51:21 UTC

[24/38] ignite git commit: IGNITE-2006: Moved "typeId()" method from BinaryObject to BinaryType.

IGNITE-2006: Moved "typeId()" method from BinaryObject to BinaryType.


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

Branch: refs/heads/ignite-1537
Commit: 58b2944b975adb398ce5f468227767dbaaf29dd7
Parents: 041cd38
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Nov 26 12:00:28 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Nov 26 12:00:28 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/binary/BinaryObject.java  |   7 -
 .../org/apache/ignite/binary/BinaryType.java    |   7 +
 .../cache/store/jdbc/CacheJdbcPojoStore.java    |   3 +-
 .../internal/portable/BinaryFieldImpl.java      |   6 +-
 .../internal/portable/BinaryObjectEx.java       | 225 +----------------
 .../internal/portable/BinaryObjectExImpl.java   | 245 +++++++++++++++++++
 .../internal/portable/BinaryObjectImpl.java     |   2 +-
 .../portable/BinaryObjectOffheapImpl.java       |   2 +-
 .../internal/portable/BinaryTypeImpl.java       |   5 +
 .../ignite/internal/portable/PortableUtils.java |   2 +-
 .../builder/PortableBuilderSerializer.java      |   4 +-
 .../CacheObjectBinaryProcessorImpl.java         |   3 +-
 .../portable/BinaryFieldsAbstractSelfTest.java  |  12 +-
 .../portable/BinaryFieldsHeapSelfTest.java      |   2 +-
 .../portable/BinaryFieldsOffheapSelfTest.java   |   2 +-
 .../BinaryFooterOffsetsAbstractSelfTest.java    |   4 +-
 .../BinaryFooterOffsetsHeapSelfTest.java        |   2 +-
 .../BinaryFooterOffsetsOffheapSelfTest.java     |   2 +-
 .../portable/BinaryMarshallerSelfTest.java      |  16 +-
 .../portable/BinaryObjectBuilderSelfTest.java   |  71 +++---
 .../PlatformComputeBinarizableArgTask.java      |   6 +-
 21 files changed, 335 insertions(+), 293 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/main/java/org/apache/ignite/binary/BinaryObject.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryObject.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryObject.java
index 2691c7b..432ace2 100644
--- a/modules/core/src/main/java/org/apache/ignite/binary/BinaryObject.java
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryObject.java
@@ -100,13 +100,6 @@ import java.util.TreeMap;
  */
 public interface BinaryObject extends Serializable, Cloneable {
     /**
-     * Gets binary object type ID.
-     *
-     * @return Type ID.
-     */
-    public int typeId();
-
-    /**
      * Gets type information for this binary object.
      *
      * @return Binary object type information.

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/main/java/org/apache/ignite/binary/BinaryType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryType.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryType.java
index 52bb212..710bf55 100644
--- a/modules/core/src/main/java/org/apache/ignite/binary/BinaryType.java
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryType.java
@@ -34,6 +34,13 @@ public interface BinaryType {
     public String typeName();
 
     /**
+     * Gets binary type ID.
+     *
+     * @return Binary type ID.
+     */
+    public int typeId();
+
+    /**
      * Gets collection of all field names for this binary type.
      *
      * @return Collection of all field names for this binary type.

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java
index aa013b9..b837e5c 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java
@@ -32,6 +32,7 @@ import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryObjectBuilder;
 import org.apache.ignite.cache.store.CacheStore;
 import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.portable.BinaryObjectEx;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.jetbrains.annotations.Nullable;
 
@@ -270,7 +271,7 @@ public class CacheJdbcPojoStore<K, V> extends CacheAbstractJdbcStore<K, V> {
      */
     @Override protected Object typeIdForObject(Object obj) throws CacheException {
         if (obj instanceof BinaryObject)
-            return ((BinaryObject)obj).typeId();
+            return ((BinaryObjectEx)obj).typeId();
 
         return obj.getClass();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryFieldImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryFieldImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryFieldImpl.java
index 810c820..b471fbe 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryFieldImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryFieldImpl.java
@@ -66,7 +66,7 @@ public class BinaryFieldImpl implements BinaryField {
 
     /** {@inheritDoc} */
     @Override public boolean exists(BinaryObject obj) {
-        BinaryObjectEx obj0 = (BinaryObjectEx)obj;
+        BinaryObjectExImpl obj0 = (BinaryObjectExImpl)obj;
 
         return fieldOrder(obj0) != PortableSchema.ORDER_NOT_FOUND;
     }
@@ -74,7 +74,7 @@ public class BinaryFieldImpl implements BinaryField {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public <T> T value(BinaryObject obj) {
-        BinaryObjectEx obj0 = (BinaryObjectEx)obj;
+        BinaryObjectExImpl obj0 = (BinaryObjectExImpl)obj;
 
         int order = fieldOrder(obj0);
 
@@ -87,7 +87,7 @@ public class BinaryFieldImpl implements BinaryField {
      * @param obj Object.
      * @return Field offset.
      */
-    private int fieldOrder(BinaryObjectEx obj) {
+    private int fieldOrder(BinaryObjectExImpl obj) {
         if (typeId != obj.typeId()) {
             throw new BinaryObjectException("Failed to get field because type ID of passed object differs" +
                 " from type ID this " + BinaryField.class.getSimpleName() + " belongs to [expected=" + typeId +

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectEx.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectEx.java
index 597fad5..acc8e4b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectEx.java
@@ -17,229 +17,14 @@
 
 package org.apache.ignite.internal.portable;
 
-import java.math.BigDecimal;
-import java.util.Arrays;
-import java.util.IdentityHashMap;
-import org.apache.ignite.IgniteException;
-import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory;
-import org.apache.ignite.internal.util.typedef.internal.SB;
-import org.apache.ignite.binary.BinaryObjectException;
-import org.apache.ignite.binary.BinaryType;
 import org.apache.ignite.binary.BinaryObject;
-import org.jetbrains.annotations.Nullable;
 
 /**
- * Internal portable object interface.
+ * Extended binary object interface.
  */
-public abstract class BinaryObjectEx implements BinaryObject {
+public interface BinaryObjectEx extends BinaryObject {
     /**
-     * @return Length.
+     * @return Type ID.
      */
-    public abstract int length();
-
-    /**
-     * @return Object start.
-     */
-    public abstract int start();
-
-    /**
-     * @return {@code True} if object is array based.
-     */
-    protected abstract boolean hasArray();
-
-    /**
-     * @return Object array if object is array based, otherwise {@code null}.
-     */
-    public abstract byte[] array();
-
-    /**
-     * @return Object offheap address is object is offheap based, otherwise 0.
-     */
-    public abstract long offheapAddress();
-
-    /**
-     * Gets field value.
-     *
-     * @param fieldId Field ID.
-     * @return Field value.
-     * @throws org.apache.ignite.binary.BinaryObjectException In case of any other error.
-     */
-    @Nullable public abstract <F> F field(int fieldId) throws BinaryObjectException;
-
-    /**
-     * Get field by offset.
-     *
-     * @param fieldOffset Field offset.
-     * @return Field value.
-     */
-    @Nullable protected abstract <F> F fieldByOrder(int fieldOffset);
-
-    /**
-     * @param ctx Reader context.
-     * @param fieldName Field name.
-     * @return Field value.
-     */
-    @Nullable protected abstract <F> F field(BinaryReaderHandles ctx, String fieldName);
-
-    /**
-     * Get schema ID.
-     *
-     * @return Schema ID.
-     */
-    protected abstract int schemaId();
-
-    /**
-     * Create schema for object.
-     *
-     * @return Schema.
-     */
-    protected abstract PortableSchema createSchema();
-
-    /** {@inheritDoc} */
-    @Override public BinaryObject clone() throws CloneNotSupportedException {
-        return (BinaryObject)super.clone();
-    }
-
-    /** {@inheritDoc} */
-    public boolean equals(Object other) {
-        if (other == this)
-            return true;
-
-        if (other == null)
-            return false;
-
-        if (!(other instanceof BinaryObjectEx))
-            return false;
-
-        BinaryObjectEx otherPo = (BinaryObjectEx)other;
-
-        if (length() != otherPo.length() || typeId() != otherPo.typeId())
-            return false;
-
-        if (hasArray()) {
-            if (otherPo.hasArray()) {
-                int len = length();
-                int end = start() + len;
-
-                byte[] arr = array();
-                byte[] otherArr = otherPo.array();
-
-                for (int i = start(), j = otherPo.start(); i < end; i++, j++) {
-                    if (arr[i] != otherArr[j])
-                        return false;
-                }
-
-                return true;
-            }
-            else {
-                assert otherPo.offheapAddress() > 0;
-
-                return GridUnsafeMemory.compare(otherPo.offheapAddress() + otherPo.start(), array());
-            }
-        }
-        else {
-            assert offheapAddress() > 0;
-
-            if (otherPo.hasArray())
-                return GridUnsafeMemory.compare(offheapAddress() + start(), otherPo.array());
-            else {
-                assert otherPo.offheapAddress() > 0;
-
-                return GridUnsafeMemory.compare(offheapAddress() + start(),
-                    otherPo.offheapAddress() + otherPo.start(),
-                    length());
-            }
-        }
-    }
-
-    /**
-     * @param ctx Reader context.
-     * @param handles Handles for already traversed objects.
-     * @return String representation.
-     */
-    private String toString(BinaryReaderHandles ctx, IdentityHashMap<BinaryObject, Integer> handles) {
-        int idHash = System.identityHashCode(this);
-
-        BinaryType meta;
-
-        try {
-            meta = type();
-        }
-        catch (BinaryObjectException ignore) {
-            meta = null;
-        }
-
-        if (meta == null)
-            return BinaryObject.class.getSimpleName() +  " [hash=" + idHash + ", typeId=" + typeId() + ']';
-
-        handles.put(this, idHash);
-
-        SB buf = new SB(meta.typeName());
-
-        if (meta.fieldNames() != null) {
-            buf.a(" [hash=").a(idHash);
-
-            for (String name : meta.fieldNames()) {
-                Object val = field(ctx, name);
-
-                buf.a(", ").a(name).a('=');
-
-                if (val instanceof byte[])
-                    buf.a(Arrays.toString((byte[]) val));
-                else if (val instanceof short[])
-                    buf.a(Arrays.toString((short[])val));
-                else if (val instanceof int[])
-                    buf.a(Arrays.toString((int[])val));
-                else if (val instanceof long[])
-                    buf.a(Arrays.toString((long[])val));
-                else if (val instanceof float[])
-                    buf.a(Arrays.toString((float[])val));
-                else if (val instanceof double[])
-                    buf.a(Arrays.toString((double[])val));
-                else if (val instanceof char[])
-                    buf.a(Arrays.toString((char[])val));
-                else if (val instanceof boolean[])
-                    buf.a(Arrays.toString((boolean[]) val));
-                else if (val instanceof BigDecimal[])
-                    buf.a(Arrays.toString((BigDecimal[])val));
-                else {
-                    if (val instanceof BinaryObjectEx) {
-                        BinaryObjectEx po = (BinaryObjectEx)val;
-
-                        Integer idHash0 = handles.get(val);
-
-                        if (idHash0 != null) {  // Circular reference.
-                            BinaryType meta0 = po.type();
-
-                            assert meta0 != null;
-
-                            buf.a(meta0.typeName()).a(" [hash=").a(idHash0).a(", ...]");
-                        }
-                        else
-                            buf.a(po.toString(ctx, handles));
-                    }
-                    else
-                        buf.a(val);
-                }
-            }
-
-            buf.a(']');
-        }
-
-        return buf.toString();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        try {
-            BinaryReaderHandles ctx = new BinaryReaderHandles();
-
-            ctx.put(start(), this);
-
-            return toString(ctx, new IdentityHashMap<BinaryObject, Integer>());
-        }
-        catch (BinaryObjectException e) {
-            throw new IgniteException("Failed to create string representation of portable object.", e);
-        }
-    }
-}
\ No newline at end of file
+    public int typeId();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectExImpl.java
new file mode 100644
index 0000000..7497bd3
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectExImpl.java
@@ -0,0 +1,245 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.portable;
+
+import java.math.BigDecimal;
+import java.util.Arrays;
+import java.util.IdentityHashMap;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory;
+import org.apache.ignite.internal.util.typedef.internal.SB;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryType;
+import org.apache.ignite.binary.BinaryObject;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Internal portable object interface.
+ */
+public abstract class BinaryObjectExImpl implements BinaryObjectEx {
+    /**
+     * @return Length.
+     */
+    public abstract int length();
+
+    /**
+     * @return Object start.
+     */
+    public abstract int start();
+
+    /**
+     * @return {@code True} if object is array based.
+     */
+    protected abstract boolean hasArray();
+
+    /**
+     * @return Object array if object is array based, otherwise {@code null}.
+     */
+    public abstract byte[] array();
+
+    /**
+     * @return Object offheap address is object is offheap based, otherwise 0.
+     */
+    public abstract long offheapAddress();
+
+    /**
+     * Gets field value.
+     *
+     * @param fieldId Field ID.
+     * @return Field value.
+     * @throws org.apache.ignite.binary.BinaryObjectException In case of any other error.
+     */
+    @Nullable public abstract <F> F field(int fieldId) throws BinaryObjectException;
+
+    /**
+     * Get field by offset.
+     *
+     * @param fieldOffset Field offset.
+     * @return Field value.
+     */
+    @Nullable protected abstract <F> F fieldByOrder(int fieldOffset);
+
+    /**
+     * @param ctx Reader context.
+     * @param fieldName Field name.
+     * @return Field value.
+     */
+    @Nullable protected abstract <F> F field(BinaryReaderHandles ctx, String fieldName);
+
+    /**
+     * Get schema ID.
+     *
+     * @return Schema ID.
+     */
+    protected abstract int schemaId();
+
+    /**
+     * Create schema for object.
+     *
+     * @return Schema.
+     */
+    protected abstract PortableSchema createSchema();
+
+    /** {@inheritDoc} */
+    @Override public BinaryObject clone() throws CloneNotSupportedException {
+        return (BinaryObject)super.clone();
+    }
+
+    /** {@inheritDoc} */
+    public boolean equals(Object other) {
+        if (other == this)
+            return true;
+
+        if (other == null)
+            return false;
+
+        if (!(other instanceof BinaryObjectExImpl))
+            return false;
+
+        BinaryObjectExImpl otherPo = (BinaryObjectExImpl)other;
+
+        if (length() != otherPo.length() || typeId() != otherPo.typeId())
+            return false;
+
+        if (hasArray()) {
+            if (otherPo.hasArray()) {
+                int len = length();
+                int end = start() + len;
+
+                byte[] arr = array();
+                byte[] otherArr = otherPo.array();
+
+                for (int i = start(), j = otherPo.start(); i < end; i++, j++) {
+                    if (arr[i] != otherArr[j])
+                        return false;
+                }
+
+                return true;
+            }
+            else {
+                assert otherPo.offheapAddress() > 0;
+
+                return GridUnsafeMemory.compare(otherPo.offheapAddress() + otherPo.start(), array());
+            }
+        }
+        else {
+            assert offheapAddress() > 0;
+
+            if (otherPo.hasArray())
+                return GridUnsafeMemory.compare(offheapAddress() + start(), otherPo.array());
+            else {
+                assert otherPo.offheapAddress() > 0;
+
+                return GridUnsafeMemory.compare(offheapAddress() + start(),
+                    otherPo.offheapAddress() + otherPo.start(),
+                    length());
+            }
+        }
+    }
+
+    /**
+     * @param ctx Reader context.
+     * @param handles Handles for already traversed objects.
+     * @return String representation.
+     */
+    private String toString(BinaryReaderHandles ctx, IdentityHashMap<BinaryObject, Integer> handles) {
+        int idHash = System.identityHashCode(this);
+
+        BinaryType meta;
+
+        try {
+            meta = type();
+        }
+        catch (BinaryObjectException ignore) {
+            meta = null;
+        }
+
+        if (meta == null)
+            return BinaryObject.class.getSimpleName() +  " [hash=" + idHash + ", typeId=" + typeId() + ']';
+
+        handles.put(this, idHash);
+
+        SB buf = new SB(meta.typeName());
+
+        if (meta.fieldNames() != null) {
+            buf.a(" [hash=").a(idHash);
+
+            for (String name : meta.fieldNames()) {
+                Object val = field(ctx, name);
+
+                buf.a(", ").a(name).a('=');
+
+                if (val instanceof byte[])
+                    buf.a(Arrays.toString((byte[]) val));
+                else if (val instanceof short[])
+                    buf.a(Arrays.toString((short[])val));
+                else if (val instanceof int[])
+                    buf.a(Arrays.toString((int[])val));
+                else if (val instanceof long[])
+                    buf.a(Arrays.toString((long[])val));
+                else if (val instanceof float[])
+                    buf.a(Arrays.toString((float[])val));
+                else if (val instanceof double[])
+                    buf.a(Arrays.toString((double[])val));
+                else if (val instanceof char[])
+                    buf.a(Arrays.toString((char[])val));
+                else if (val instanceof boolean[])
+                    buf.a(Arrays.toString((boolean[]) val));
+                else if (val instanceof BigDecimal[])
+                    buf.a(Arrays.toString((BigDecimal[])val));
+                else {
+                    if (val instanceof BinaryObjectExImpl) {
+                        BinaryObjectExImpl po = (BinaryObjectExImpl)val;
+
+                        Integer idHash0 = handles.get(val);
+
+                        if (idHash0 != null) {  // Circular reference.
+                            BinaryType meta0 = po.type();
+
+                            assert meta0 != null;
+
+                            buf.a(meta0.typeName()).a(" [hash=").a(idHash0).a(", ...]");
+                        }
+                        else
+                            buf.a(po.toString(ctx, handles));
+                    }
+                    else
+                        buf.a(val);
+                }
+            }
+
+            buf.a(']');
+        }
+
+        return buf.toString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        try {
+            BinaryReaderHandles ctx = new BinaryReaderHandles();
+
+            ctx.put(start(), this);
+
+            return toString(ctx, new IdentityHashMap<BinaryObject, Integer>());
+        }
+        catch (BinaryObjectException e) {
+            throw new IgniteException("Failed to create string representation of portable object.", e);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
index 7db4b4a..f683ed9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
@@ -64,7 +64,7 @@ import static org.apache.ignite.internal.portable.GridPortableMarshaller.UUID;
  * Portable object implementation.
  */
 @IgniteCodeGeneratingFail // Fields arr and start should not be generated by MessageCodeGenerator.
-public final class BinaryObjectImpl extends BinaryObjectEx implements Externalizable, KeyCacheObject {
+public final class BinaryObjectImpl extends BinaryObjectExImpl implements Externalizable, KeyCacheObject {
     /** */
     public static final byte TYPE_BINARY = 100;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java
index 8b8e0e8..6de0432 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java
@@ -61,7 +61,7 @@ import static org.apache.ignite.internal.portable.GridPortableMarshaller.UUID;
 /**
  *  Portable object implementation over offheap memory
  */
-public class BinaryObjectOffheapImpl extends BinaryObjectEx implements Externalizable, CacheObject {
+public class BinaryObjectOffheapImpl extends BinaryObjectExImpl implements Externalizable, CacheObject {
     /** */
     private static final long serialVersionUID = 0L;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryTypeImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryTypeImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryTypeImpl.java
index 5f2a91e..c3fee7e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryTypeImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryTypeImpl.java
@@ -48,6 +48,11 @@ public class BinaryTypeImpl implements BinaryType {
     }
 
     /** {@inheritDoc} */
+    @Override public int typeId() {
+        return meta.typeId();
+    }
+
+    /** {@inheritDoc} */
     @Override public Collection<String> fieldNames() {
         return meta.fields();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/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 5c798b8..0c569cf 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
@@ -1470,7 +1470,7 @@ public class PortableUtils {
 
                 int len = length(in, start);
 
-                BinaryObjectEx po;
+                BinaryObjectExImpl po;
 
                 if (detach) {
                     // In detach mode we simply copy object's content.

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializer.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializer.java
index 0e8eaa4..ee7bd65 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializer.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.portable.builder;
 
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.internal.portable.GridPortableMarshaller;
-import org.apache.ignite.internal.portable.BinaryObjectEx;
+import org.apache.ignite.internal.portable.BinaryObjectExImpl;
 import org.apache.ignite.internal.portable.PortableUtils;
 import org.apache.ignite.internal.portable.BinaryWriterExImpl;
 import org.apache.ignite.internal.util.*;
@@ -61,7 +61,7 @@ class PortableBuilderSerializer {
             return;
         }
 
-        if (val instanceof BinaryObjectEx) {
+        if (val instanceof BinaryObjectExImpl) {
             if (portableObjToWrapper == null)
                 portableObjToWrapper = new IdentityHashMap<>();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
index 05e9263..d461041 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
@@ -32,6 +32,7 @@ import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.portable.BinaryMetadata;
 import org.apache.ignite.internal.portable.BinaryMetadataHandler;
+import org.apache.ignite.internal.portable.BinaryObjectEx;
 import org.apache.ignite.internal.portable.BinaryObjectImpl;
 import org.apache.ignite.internal.portable.BinaryObjectOffheapImpl;
 import org.apache.ignite.internal.portable.BinaryTypeImpl;
@@ -575,7 +576,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
         if (obj == null)
             return 0;
 
-        return isPortableObject(obj) ? ((BinaryObject)obj).typeId() : typeId(obj.getClass().getSimpleName());
+        return isPortableObject(obj) ? ((BinaryObjectEx)obj).typeId() : typeId(obj.getClass().getSimpleName());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java
index dd08390..ca41358 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java
@@ -487,7 +487,7 @@ public abstract class BinaryFieldsAbstractSelfTest extends GridCommonAbstractTes
     private TestContext context(BinaryMarshaller marsh, String fieldName) throws Exception {
         TestObject obj = createObject();
 
-        BinaryObjectEx portObj = toPortable(marsh, obj);
+        BinaryObjectExImpl portObj = toPortable(marsh, obj);
 
         BinaryField field = portObj.type().field(fieldName);
 
@@ -507,8 +507,8 @@ public abstract class BinaryFieldsAbstractSelfTest extends GridCommonAbstractTes
         TestObject obj = createObject();
         TestOuterObject outObj = new TestOuterObject(obj);
 
-        BinaryObjectEx portOutObj = toPortable(marsh, outObj);
-        BinaryObjectEx portObj = portOutObj.field("fInner");
+        BinaryObjectExImpl portOutObj = toPortable(marsh, outObj);
+        BinaryObjectExImpl portObj = portOutObj.field("fInner");
 
         assert portObj != null;
 
@@ -534,7 +534,7 @@ public abstract class BinaryFieldsAbstractSelfTest extends GridCommonAbstractTes
      * @return Portable object.
      * @throws Exception If failed.
      */
-    protected abstract BinaryObjectEx toPortable(BinaryMarshaller marsh, Object obj) throws Exception;
+    protected abstract BinaryObjectExImpl toPortable(BinaryMarshaller marsh, Object obj) throws Exception;
 
     /**
      * Outer test object.
@@ -698,7 +698,7 @@ public abstract class BinaryFieldsAbstractSelfTest extends GridCommonAbstractTes
         public final TestObject obj;
 
         /** Portable object. */
-        public final BinaryObjectEx portObj;
+        public final BinaryObjectExImpl portObj;
 
         /** Field. */
         public final BinaryField field;
@@ -710,7 +710,7 @@ public abstract class BinaryFieldsAbstractSelfTest extends GridCommonAbstractTes
          * @param portObj Portable object.
          * @param field Field.
          */
-        public TestContext(TestObject obj, BinaryObjectEx portObj, BinaryField field) {
+        public TestContext(TestObject obj, BinaryObjectExImpl portObj, BinaryField field) {
             this.obj = obj;
             this.portObj = portObj;
             this.field = field;

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsHeapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsHeapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsHeapSelfTest.java
index 53ba212..dbabba5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsHeapSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsHeapSelfTest.java
@@ -24,7 +24,7 @@ import org.apache.ignite.marshaller.portable.BinaryMarshaller;
  */
 public class BinaryFieldsHeapSelfTest extends BinaryFieldsAbstractSelfTest {
     /** {@inheritDoc} */
-    @Override protected BinaryObjectEx toPortable(BinaryMarshaller marsh, Object obj) throws Exception {
+    @Override protected BinaryObjectExImpl toPortable(BinaryMarshaller marsh, Object obj) throws Exception {
         byte[] bytes = marsh.marshal(obj);
 
         return new BinaryObjectImpl(portableContext(marsh), bytes, 0);

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsOffheapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsOffheapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsOffheapSelfTest.java
index 68bfb30..fe9f745 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsOffheapSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsOffheapSelfTest.java
@@ -47,7 +47,7 @@ public class BinaryFieldsOffheapSelfTest extends BinaryFieldsAbstractSelfTest {
     }
 
     /** {@inheritDoc} */
-    @Override protected BinaryObjectEx toPortable(BinaryMarshaller marsh, Object obj) throws Exception {
+    @Override protected BinaryObjectExImpl toPortable(BinaryMarshaller marsh, Object obj) throws Exception {
         byte[] arr = marsh.marshal(obj);
 
         long ptr = UNSAFE.allocateMemory(arr.length);

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsAbstractSelfTest.java
index 7267b16..d900064 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsAbstractSelfTest.java
@@ -128,7 +128,7 @@ public abstract class BinaryFooterOffsetsAbstractSelfTest extends GridCommonAbst
     private void check(int len) throws Exception {
         TestObject obj = new TestObject(len);
 
-        BinaryObjectEx portObj = toPortable(marsh, obj);
+        BinaryObjectExImpl portObj = toPortable(marsh, obj);
 
         // 1. Test portable object content.
         assert portObj.hasField("field1");
@@ -170,7 +170,7 @@ public abstract class BinaryFooterOffsetsAbstractSelfTest extends GridCommonAbst
      * @return Portable object.
      * @throws Exception If failed.
      */
-    protected abstract BinaryObjectEx toPortable(BinaryMarshaller marsh, Object obj) throws Exception;
+    protected abstract BinaryObjectExImpl toPortable(BinaryMarshaller marsh, Object obj) throws Exception;
 
     /**
      * Test object.

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsHeapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsHeapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsHeapSelfTest.java
index 471bd44..939f01d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsHeapSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsHeapSelfTest.java
@@ -24,7 +24,7 @@ import org.apache.ignite.marshaller.portable.BinaryMarshaller;
  */
 public class BinaryFooterOffsetsHeapSelfTest extends BinaryFooterOffsetsAbstractSelfTest {
     /** {@inheritDoc} */
-    @Override protected BinaryObjectEx toPortable(BinaryMarshaller marsh, Object obj) throws Exception {
+    @Override protected BinaryObjectExImpl toPortable(BinaryMarshaller marsh, Object obj) throws Exception {
         byte[] bytes = marsh.marshal(obj);
 
         return new BinaryObjectImpl(ctx, bytes, 0);

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsOffheapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsOffheapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsOffheapSelfTest.java
index 7b44b80..f7e743f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsOffheapSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsOffheapSelfTest.java
@@ -47,7 +47,7 @@ public class BinaryFooterOffsetsOffheapSelfTest extends BinaryFooterOffsetsAbstr
     }
 
     /** {@inheritDoc} */
-    @Override protected BinaryObjectEx toPortable(BinaryMarshaller marsh, Object obj) throws Exception {
+    @Override protected BinaryObjectExImpl toPortable(BinaryMarshaller marsh, Object obj) throws Exception {
         byte[] arr = marsh.marshal(obj);
 
         long ptr = UNSAFE.allocateMemory(arr.length);

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryMarshallerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryMarshallerSelfTest.java
index 19c40b3..0fbb4c2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryMarshallerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryMarshallerSelfTest.java
@@ -792,9 +792,9 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
 
         CustomMappedObject1 obj1 = new CustomMappedObject1(10, "str");
 
-        BinaryObjectEx po1 = marshal(obj1, marsh);
+        BinaryObjectExImpl po1 = marshal(obj1, marsh);
 
-        assertEquals(11111, po1.typeId());
+        assertEquals(11111, po1.type().typeId());
         assertEquals((Integer)10, po1.field(22222));
         assertEquals("str", po1.field(33333));
 
@@ -849,9 +849,9 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
 
         CustomMappedObject1 obj1 = new CustomMappedObject1(10, "str1");
 
-        BinaryObjectEx po1 = marshal(obj1, marsh);
+        BinaryObjectExImpl po1 = marshal(obj1, marsh);
 
-        assertEquals(11111, po1.typeId());
+        assertEquals(11111, po1.type().typeId());
         assertEquals((Integer)10, po1.field(22222));
         assertEquals("str1", po1.field(33333));
 
@@ -860,9 +860,9 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
 
         CustomMappedObject2 obj2 = new CustomMappedObject2(20, "str2");
 
-        BinaryObjectEx po2 = marshal(obj2, marsh);
+        BinaryObjectExImpl po2 = marshal(obj2, marsh);
 
-        assertEquals(44444, po2.typeId());
+        assertEquals(44444, po2.type().typeId());
         assertEquals((Integer)20, po2.field(55555));
         assertEquals("str2", po2.field(66666));
 
@@ -1774,7 +1774,7 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
             assertTrue(offheapObj.equals(offheapObj1));
             assertTrue(offheapObj1.equals(offheapObj));
 
-            assertEquals(obj.typeId(), offheapObj.typeId());
+            assertEquals(obj.type().typeId(), offheapObj.type().typeId());
             assertEquals(obj.hashCode(), offheapObj.hashCode());
 
             checkSimpleObjectData(simpleObj, offheapObj);
@@ -1787,7 +1787,7 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
 
             obj = (BinaryObjectImpl)offheapObj.heapCopy();
 
-            assertEquals(obj.typeId(), offheapObj.typeId());
+            assertEquals(obj.type().typeId(), offheapObj.type().typeId());
             assertEquals(obj.hashCode(), offheapObj.hashCode());
 
             checkSimpleObjectData(simpleObj, obj);

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderSelfTest.java
index 7f023f3..4864847 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderSelfTest.java
@@ -52,6 +52,7 @@ import sun.misc.Unsafe;
 /**
  * Portable builder test.
  */
+@SuppressWarnings("ResultOfMethodCallIgnored")
 public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
     /** */
     private static final Unsafe UNSAFE = GridUnsafe.unsafe();
@@ -135,7 +136,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertEquals((byte) 1, po.<Byte>field("byteField").byteValue());
@@ -153,7 +154,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertEquals((short)1, po.<Short>field("shortField").shortValue());
@@ -171,7 +172,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertEquals(1, po.<Integer>field("intField").intValue());
@@ -189,7 +190,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertEquals(1L, po.<Long>field("longField").longValue());
@@ -207,7 +208,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertEquals(1.0f, po.<Float>field("floatField").floatValue(), 0);
@@ -225,7 +226,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertEquals(1.0d, po.<Double>field("doubleField").doubleValue(), 0);
@@ -243,7 +244,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertEquals((char)1, po.<Character>field("charField").charValue());
@@ -261,7 +262,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertTrue(po.<Boolean>field("booleanField"));
@@ -279,10 +280,10 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
-        assertEquals(BigDecimal.TEN, po.<String>field("decimalField"));
+        assertEquals(BigDecimal.TEN, po.<BigDecimal>field("decimalField"));
     }
 
     /**
@@ -297,7 +298,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertEquals("str", po.<String>field("stringField"));
@@ -336,7 +337,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertEquals(uuid, po.<UUID>field("uuidField"));
@@ -354,7 +355,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertTrue(Arrays.equals(new byte[] {1, 2, 3}, po.<byte[]>field("byteArrayField")));
@@ -372,7 +373,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertTrue(Arrays.equals(new short[] {1, 2, 3}, po.<short[]>field("shortArrayField")));
@@ -390,7 +391,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertTrue(Arrays.equals(new int[] {1, 2, 3}, po.<int[]>field("intArrayField")));
@@ -408,7 +409,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertTrue(Arrays.equals(new long[] {1, 2, 3}, po.<long[]>field("longArrayField")));
@@ -426,7 +427,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertTrue(Arrays.equals(new float[] {1, 2, 3}, po.<float[]>field("floatArrayField")));
@@ -444,7 +445,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertTrue(Arrays.equals(new double[] {1, 2, 3}, po.<double[]>field("doubleArrayField")));
@@ -462,7 +463,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertTrue(Arrays.equals(new char[] {1, 2, 3}, po.<char[]>field("charArrayField")));
@@ -480,7 +481,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         boolean[] arr = po.field("booleanArrayField");
@@ -503,7 +504,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertTrue(Arrays.equals(new BigDecimal[] {BigDecimal.ONE, BigDecimal.TEN}, po.<String[]>field("decimalArrayField")));
@@ -521,7 +522,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertTrue(Arrays.equals(new String[] {"str1", "str2", "str3"}, po.<String[]>field("stringArrayField")));
@@ -568,7 +569,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertTrue(Arrays.equals(arr, po.<UUID[]>field("uuidArrayField")));
@@ -586,7 +587,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertEquals(1, po.<BinaryObject>field("objectField").<Value>deserialize().i);
@@ -604,7 +605,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         Object[] arr = po.field("objectArrayField");
@@ -627,7 +628,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         List<BinaryObject> list = po.field("collectionField");
@@ -650,7 +651,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         Map<BinaryObject, BinaryObject> map = po.field("mapField");
@@ -677,7 +678,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertEquals(111, po.<Integer>field("i").intValue());
@@ -727,7 +728,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
             assertEquals(BinaryObjectOffheapImpl.class, offheapObj.getClass());
 
-            assertEquals("class".hashCode(), offheapObj.typeId());
+            assertEquals("class".hashCode(), offheapObj.type().typeId());
             assertEquals(100, offheapObj.hashCode());
 
             assertEquals(111, offheapObj.<Integer>field("i").intValue());
@@ -762,7 +763,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("value".hashCode(), po.typeId());
+        assertEquals("value".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertEquals(1, po.<Value>deserialize().i);
@@ -856,6 +857,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
     /**
      *
      */
+    @SuppressWarnings("unchecked")
     public void testCopyFromInnerObjects() {
         ArrayList<Object> list = new ArrayList<>();
         list.add(new TestObjectAllTypes());
@@ -898,7 +900,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
         assertTrue(builder.getField("plainPortable") instanceof BinaryObject);
 
         TestObjectPlainPortable deserialized = builder.build().deserialize();
-        assertTrue(deserialized.plainPortable instanceof BinaryObject);
+        assertTrue(deserialized.plainPortable != null);
     }
 
     /**
@@ -987,20 +989,21 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
     /**
      * @return Builder.
      */
-    private <T> BinaryObjectBuilder builder(String clsName) {
+    private BinaryObjectBuilder builder(String clsName) {
         return portables().builder(clsName);
     }
 
     /**
      * @return Builder.
      */
-    private <T> BinaryObjectBuilderImpl builder(BinaryObject obj) {
+    private BinaryObjectBuilderImpl builder(BinaryObject obj) {
         return (BinaryObjectBuilderImpl)portables().builder(obj);
     }
 
     /**
      *
      */
+    @SuppressWarnings("UnusedDeclaration")
     private static class CustomIdMapper {
         /** */
         private String str = "a";
@@ -1011,6 +1014,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
     /**
      */
+    @SuppressWarnings("UnusedDeclaration")
     private static class Key {
         /** */
         private int i;
@@ -1049,6 +1053,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
     /**
      */
+    @SuppressWarnings("UnusedDeclaration")
     private static class Value {
         /** */
         private int i;

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputeBinarizableArgTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputeBinarizableArgTask.java b/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputeBinarizableArgTask.java
index 8eba80b..ffae2d1 100644
--- a/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputeBinarizableArgTask.java
+++ b/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputeBinarizableArgTask.java
@@ -19,14 +19,14 @@ package org.apache.ignite.platform;
 
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.binary.BinaryType;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.compute.ComputeJob;
 import org.apache.ignite.compute.ComputeJobAdapter;
 import org.apache.ignite.compute.ComputeJobResult;
 import org.apache.ignite.compute.ComputeTaskAdapter;
+import org.apache.ignite.internal.portable.BinaryObjectEx;
 import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.binary.BinaryType;
-import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.resources.IgniteInstanceResource;
 import org.jetbrains.annotations.Nullable;
 
@@ -87,7 +87,7 @@ public class PlatformComputeBinarizableArgTask extends ComputeTaskAdapter<Object
 
         /** {@inheritDoc} */
         @Nullable @Override public Object execute() {
-            BinaryObject arg0 = ((BinaryObject)arg);
+            BinaryObjectEx arg0 = ((BinaryObjectEx)arg);
 
             BinaryType meta = ignite.binary().metadata(arg0.typeId());