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

[01/13] ignite git commit: IGNITE-1816: Implemented compact footers.

Repository: ignite
Updated Branches:
  refs/heads/ignite-1945 29e586f47 -> a511fa178


http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsAbstractSelfTest.java
deleted file mode 100644
index 9225b97..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsAbstractSelfTest.java
+++ /dev/null
@@ -1,190 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.portable;
-
-import org.apache.ignite.binary.BinaryField;
-import org.apache.ignite.binary.BinaryTypeConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.util.IgniteUtils;
-import org.apache.ignite.marshaller.MarshallerContextTestImpl;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-
-import java.util.Arrays;
-
-/**
- * Contains tests for compact offsets.
- */
-public abstract class PortableCompactOffsetsAbstractSelfTest extends GridCommonAbstractTest {
-    /** 2 pow 8. */
-    private static int POW_8 = 1 << 8;
-
-    /** 2 pow 16. */
-    private static int POW_16 = 1 << 16;
-
-    /** Marshaller. */
-    protected PortableMarshaller marsh;
-
-    /** Portable context. */
-    protected PortableContext ctx;
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        super.beforeTest();
-
-        ctx = new PortableContext(new TestCachingMetadataHandler(), new IgniteConfiguration());
-
-        marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(new BinaryTypeConfiguration(TestObject.class.getName())));
-        marsh.setContext(new MarshallerContextTestImpl(null));
-
-        IgniteUtils.invoke(PortableMarshaller.class, marsh, "setPortableContext", ctx);
-    }
-
-    /**
-     * Test 1 byte.
-     *
-     * @throws Exception If failed.
-     */
-    public void test1Byte() throws Exception {
-        check(POW_8 >> 2);
-    }
-
-    /**
-     * Test 1 byte with sign altering.
-     *
-     * @throws Exception If failed.
-     */
-    public void test1ByteSign() throws Exception {
-        check(POW_8 >> 1);
-    }
-
-    /**
-     * Test 2 bytes.
-     *
-     * @throws Exception If failed.
-     */
-    public void test2Bytes() throws Exception {
-        check(POW_16 >> 2);
-    }
-
-    /**
-     * Test 2 bytes with sign altering.
-     *
-     * @throws Exception If failed.
-     */
-    public void test2BytesSign() throws Exception {
-        check(POW_16 >> 1);
-    }
-
-    /**
-     * Test 4 bytes.
-     *
-     * @throws Exception If failed.
-     */
-    public void test4Bytes() throws Exception {
-        check(POW_16 << 2);
-    }
-
-    /**
-     * Main check routine.
-     *
-     * @param len Length of the first field.
-     *
-     * @throws Exception If failed.
-     */
-    private void check(int len) throws Exception {
-        TestObject obj = new TestObject(len);
-
-        BinaryObjectEx portObj = toPortable(marsh, obj);
-
-        // 1. Test portable object content.
-        assert portObj.hasField("field1");
-        assert portObj.hasField("field2");
-
-        byte[] field1 = portObj.field("field1");
-        Integer field2 = portObj.field("field2");
-
-        assert field1 != null;
-        assert field2 != null;
-
-        assert Arrays.equals(obj.field1, field1);
-        assert obj.field2 == field2;
-
-        // 2. Test fields API.
-        BinaryField field1Desc = portObj.type().field("field1");
-        BinaryField field2Desc = portObj.type().field("field2");
-
-        assert field1Desc.exists(portObj);
-        assert field2Desc.exists(portObj);
-
-        assert Arrays.equals(obj.field1, (byte[])field1Desc.value(portObj));
-        assert obj.field2 == (Integer)field2Desc.value(portObj);
-
-        // 3. Test deserialize.
-        TestObject objRestored = portObj.deserialize();
-
-        assert objRestored != null;
-
-        assert Arrays.equals(obj.field1, objRestored.field1);
-        assert obj.field2 == objRestored.field2;
-    }
-
-    /**
-     * Convert object to portable object.
-     *
-     * @param marsh Marshaller.
-     * @param obj Object.
-     * @return Portable object.
-     * @throws Exception If failed.
-     */
-    protected abstract BinaryObjectEx toPortable(PortableMarshaller marsh, Object obj) throws Exception;
-
-    /**
-     * Test object.
-     */
-    public static class TestObject {
-        /** First field with variable length. */
-        public byte[] field1;
-
-        /** Second field. */
-        public int field2;
-
-        /**
-         * Default constructor.
-         */
-        public TestObject() {
-            // No-op.
-        }
-
-        /**
-         * Constructor.
-         *
-         * @param len Array length.
-         */
-        public TestObject(int len) {
-            field1 = new byte[len];
-
-            field1[0] = 1;
-            field1[len - 1] = 2;
-
-            field2 = len;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsHeapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsHeapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsHeapSelfTest.java
deleted file mode 100644
index ebdef38..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsHeapSelfTest.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.portable;
-
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-
-/**
- * Compact offsets tests for heap portable objects.
- */
-public class PortableCompactOffsetsHeapSelfTest extends PortableCompactOffsetsAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected BinaryObjectEx toPortable(PortableMarshaller 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/0b4a8f83/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsOffheapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsOffheapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsOffheapSelfTest.java
deleted file mode 100644
index e3b6bda..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsOffheapSelfTest.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.portable;
-
-import org.apache.ignite.internal.util.GridUnsafe;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.eclipse.jetty.util.ConcurrentHashSet;
-import sun.misc.Unsafe;
-
-/**
- * Compact offsets tests for offheap portable objects.
- */
-public class PortableCompactOffsetsOffheapSelfTest extends PortableCompactOffsetsAbstractSelfTest {
-    /** Unsafe instance. */
-    private static final Unsafe UNSAFE = GridUnsafe.unsafe();
-
-    /** Byte array offset for unsafe mechanics. */
-    protected static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
-
-    /** Allocated unsafe pointer. */
-    private final ConcurrentHashSet<Long> ptrs = new ConcurrentHashSet<>();
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        super.afterTest();
-
-        // Cleanup allocated objects.
-        for (Long ptr : ptrs)
-            UNSAFE.freeMemory(ptr);
-
-        ptrs.clear();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected BinaryObjectEx toPortable(PortableMarshaller marsh, Object obj) throws Exception {
-        byte[] arr = marsh.marshal(obj);
-
-        long ptr = UNSAFE.allocateMemory(arr.length);
-
-        ptrs.add(ptr);
-
-        UNSAFE.copyMemory(arr, BYTE_ARR_OFF, null, ptr, arr.length);
-
-        return new BinaryObjectOffheapImpl(ctx, ptr, 0, arr.length);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryFieldsHeapNonCompactSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryFieldsHeapNonCompactSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryFieldsHeapNonCompactSelfTest.java
new file mode 100644
index 0000000..9e7619f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryFieldsHeapNonCompactSelfTest.java
@@ -0,0 +1,34 @@
+/*
+ * 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.noncompact;
+
+import org.apache.ignite.internal.portable.BinaryFieldsAbstractSelfTest;
+import org.apache.ignite.internal.portable.BinaryFieldsHeapSelfTest;
+import org.apache.ignite.internal.portable.BinaryObjectEx;
+import org.apache.ignite.internal.portable.BinaryObjectImpl;
+import org.apache.ignite.marshaller.portable.PortableMarshaller;
+
+/**
+ * Field tests for heap-based portables with non-compact footer.
+ */
+public class BinaryFieldsHeapNonCompactSelfTest extends BinaryFieldsHeapSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean compactFooter() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryFieldsOffheapNonCompactSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryFieldsOffheapNonCompactSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryFieldsOffheapNonCompactSelfTest.java
new file mode 100644
index 0000000..0bca601
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryFieldsOffheapNonCompactSelfTest.java
@@ -0,0 +1,30 @@
+/*
+ * 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.noncompact;
+
+import org.apache.ignite.internal.portable.BinaryFieldsOffheapSelfTest;
+
+/**
+ * Field tests for offheap-based portables with non-compact footer.
+ */
+public class BinaryFieldsOffheapNonCompactSelfTest extends BinaryFieldsOffheapSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean compactFooter() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryFooterOffsetsHeapNonCompactSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryFooterOffsetsHeapNonCompactSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryFooterOffsetsHeapNonCompactSelfTest.java
new file mode 100644
index 0000000..8fba738
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryFooterOffsetsHeapNonCompactSelfTest.java
@@ -0,0 +1,30 @@
+/*
+ * 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.noncompact;
+
+import org.apache.ignite.internal.portable.BinaryFooterOffsetsHeapSelfTest;
+
+/**
+ * Compact offsets tests for heap portable objects with non-compact footer.
+ */
+public class BinaryFooterOffsetsHeapNonCompactSelfTest extends BinaryFooterOffsetsHeapSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean compactFooter() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryFooterOffsetsOffheapNonCompactSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryFooterOffsetsOffheapNonCompactSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryFooterOffsetsOffheapNonCompactSelfTest.java
new file mode 100644
index 0000000..b52bd83
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryFooterOffsetsOffheapNonCompactSelfTest.java
@@ -0,0 +1,30 @@
+/*
+ * 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.noncompact;
+
+import org.apache.ignite.internal.portable.BinaryFooterOffsetsOffheapSelfTest;
+
+/**
+ * Compact offsets tests for offheap portable objects with non-compact footer.
+ */
+public class BinaryFooterOffsetsOffheapNonCompactSelfTest extends BinaryFooterOffsetsOffheapSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean compactFooter() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryMarshallerNonCompactSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryMarshallerNonCompactSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryMarshallerNonCompactSelfTest.java
new file mode 100644
index 0000000..0484dea
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryMarshallerNonCompactSelfTest.java
@@ -0,0 +1,30 @@
+/*
+ * 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.noncompact;
+
+import org.apache.ignite.internal.portable.BinaryMarshallerSelfTest;
+
+/**
+ * Basic marshaller test with non-compact footer.
+ */
+public class BinaryMarshallerNonCompactSelfTest extends BinaryMarshallerSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean compactFooter() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryObjectBuilderAdditionalNonCompactSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryObjectBuilderAdditionalNonCompactSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryObjectBuilderAdditionalNonCompactSelfTest.java
new file mode 100644
index 0000000..8811029
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryObjectBuilderAdditionalNonCompactSelfTest.java
@@ -0,0 +1,30 @@
+/*
+ * 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.noncompact;
+
+import org.apache.ignite.internal.portable.BinaryObjectBuilderAdditionalSelfTest;
+
+/**
+ *
+ */
+public class BinaryObjectBuilderAdditionalNonCompactSelfTest extends BinaryObjectBuilderAdditionalSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean compactFooter() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryObjectBuilderNonCompactSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryObjectBuilderNonCompactSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryObjectBuilderNonCompactSelfTest.java
new file mode 100644
index 0000000..0b0916d
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryObjectBuilderNonCompactSelfTest.java
@@ -0,0 +1,30 @@
+/*
+ * 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.noncompact;
+
+import org.apache.ignite.internal.portable.BinaryObjectBuilderSelfTest;
+
+/**
+ * Portable builder test for objects with non-compact footer.
+ */
+public class BinaryObjectBuilderNonCompactSelfTest extends BinaryObjectBuilderSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean compactFooter() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java
index 4fe8633..1128d67 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java
@@ -19,16 +19,23 @@ package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
 import org.apache.ignite.internal.portable.GridPortableAffinityKeySelfTest;
-import org.apache.ignite.internal.portable.GridBinaryObjectBuilderAdditionalSelfTest;
-import org.apache.ignite.internal.portable.GridBinaryObjectBuilderSelfTest;
+import org.apache.ignite.internal.portable.BinaryObjectBuilderAdditionalSelfTest;
+import org.apache.ignite.internal.portable.BinaryObjectBuilderSelfTest;
 import org.apache.ignite.internal.portable.GridPortableMarshallerCtxDisabledSelfTest;
-import org.apache.ignite.internal.portable.GridPortableMarshallerSelfTest;
+import org.apache.ignite.internal.portable.BinaryMarshallerSelfTest;
 import org.apache.ignite.internal.portable.GridPortableMetaDataSelfTest;
 import org.apache.ignite.internal.portable.GridPortableWildcardsSelfTest;
-import org.apache.ignite.internal.portable.PortableCompactOffsetsHeapSelfTest;
-import org.apache.ignite.internal.portable.PortableCompactOffsetsOffheapSelfTest;
+import org.apache.ignite.internal.portable.BinaryFooterOffsetsHeapSelfTest;
+import org.apache.ignite.internal.portable.BinaryFooterOffsetsOffheapSelfTest;
 import org.apache.ignite.internal.portable.BinaryFieldsHeapSelfTest;
 import org.apache.ignite.internal.portable.BinaryFieldsOffheapSelfTest;
+import org.apache.ignite.internal.portable.noncompact.BinaryFieldsHeapNonCompactSelfTest;
+import org.apache.ignite.internal.portable.noncompact.BinaryFieldsOffheapNonCompactSelfTest;
+import org.apache.ignite.internal.portable.noncompact.BinaryFooterOffsetsHeapNonCompactSelfTest;
+import org.apache.ignite.internal.portable.noncompact.BinaryFooterOffsetsOffheapNonCompactSelfTest;
+import org.apache.ignite.internal.portable.noncompact.BinaryMarshallerNonCompactSelfTest;
+import org.apache.ignite.internal.portable.noncompact.BinaryObjectBuilderAdditionalNonCompactSelfTest;
+import org.apache.ignite.internal.portable.noncompact.BinaryObjectBuilderNonCompactSelfTest;
 import org.apache.ignite.internal.processors.cache.portable.GridCacheClientNodeBinaryObjectMetadataMultinodeTest;
 import org.apache.ignite.internal.processors.cache.portable.GridCacheClientNodeBinaryObjectMetadataTest;
 import org.apache.ignite.internal.processors.cache.portable.GridCachePortableStoreObjectsSelfTest;
@@ -57,18 +64,27 @@ public class IgnitePortableObjectsTestSuite extends TestSuite {
     public static TestSuite suite() throws Exception {
         TestSuite suite = new TestSuite("GridGain Portable Objects Test Suite");
 
-        suite.addTestSuite(GridPortableMarshallerSelfTest.class);
+        suite.addTestSuite(BinaryMarshallerSelfTest.class);
         suite.addTestSuite(GridPortableMarshallerCtxDisabledSelfTest.class);
-        suite.addTestSuite(GridBinaryObjectBuilderSelfTest.class);
-        suite.addTestSuite(GridBinaryObjectBuilderAdditionalSelfTest.class);
+        suite.addTestSuite(BinaryObjectBuilderSelfTest.class);
+        suite.addTestSuite(BinaryObjectBuilderAdditionalSelfTest.class);
         suite.addTestSuite(BinaryFieldsHeapSelfTest.class);
         suite.addTestSuite(BinaryFieldsOffheapSelfTest.class);
-        suite.addTestSuite(PortableCompactOffsetsHeapSelfTest.class);
-        suite.addTestSuite(PortableCompactOffsetsOffheapSelfTest.class);
+        suite.addTestSuite(BinaryFooterOffsetsHeapSelfTest.class);
+        suite.addTestSuite(BinaryFooterOffsetsOffheapSelfTest.class);
         suite.addTestSuite(GridPortableMetaDataSelfTest.class);
         suite.addTestSuite(GridPortableAffinityKeySelfTest.class);
         suite.addTestSuite(GridPortableWildcardsSelfTest.class);
 
+        // Tests for objects with non-compact footers.
+        suite.addTestSuite(BinaryMarshallerNonCompactSelfTest.class);
+        suite.addTestSuite(BinaryObjectBuilderNonCompactSelfTest.class);
+        suite.addTestSuite(BinaryObjectBuilderAdditionalNonCompactSelfTest.class);
+        suite.addTestSuite(BinaryFieldsHeapNonCompactSelfTest.class);
+        suite.addTestSuite(BinaryFieldsOffheapNonCompactSelfTest.class);
+        suite.addTestSuite(BinaryFooterOffsetsHeapNonCompactSelfTest.class);
+        suite.addTestSuite(BinaryFooterOffsetsOffheapNonCompactSelfTest.class);
+
         suite.addTestSuite(GridCacheBinaryObjectsLocalSelfTest.class);
         suite.addTestSuite(GridCacheBinaryObjectsAtomicLocalSelfTest.class);
         suite.addTestSuite(GridCacheBinaryObjectsReplicatedSelfTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/platforms/cpp/core-test/src/binary_reader_writer_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/src/binary_reader_writer_test.cpp b/modules/platforms/cpp/core-test/src/binary_reader_writer_test.cpp
index d273b11..b2551ec 100644
--- a/modules/platforms/cpp/core-test/src/binary_reader_writer_test.cpp
+++ b/modules/platforms/cpp/core-test/src/binary_reader_writer_test.cpp
@@ -70,7 +70,7 @@ void CheckPrimitive(T val)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
     try
@@ -150,7 +150,7 @@ void CheckPrimitiveArray(T dflt, T val1, T val2)
         int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
         int32_t footerEnd = footerBegin + 5;
 
-        BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+        BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
         BinaryReader reader(&readerImpl);
 
         in.Position(IGNITE_DFLT_HDR_LEN);
@@ -183,7 +183,7 @@ void CheckPrimitiveArray(T dflt, T val1, T val2)
         int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
         int32_t footerEnd = footerBegin + 5;
 
-        BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+        BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
         BinaryReader reader(&readerImpl);
 
         in.Position(IGNITE_DFLT_HDR_LEN);
@@ -220,7 +220,7 @@ void CheckPrimitiveArray(T dflt, T val1, T val2)
         int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
         int32_t footerEnd = footerBegin + 5;
 
-        BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+        BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
         BinaryReader reader(&readerImpl);
 
         in.Position(IGNITE_DFLT_HDR_LEN);
@@ -260,7 +260,7 @@ void CheckPrimitiveArray(T dflt, T val1, T val2)
         int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
         int32_t footerEnd = footerBegin + 5;
 
-        BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+        BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
         BinaryReader reader(&readerImpl);
 
         in.Position(IGNITE_DFLT_HDR_LEN);
@@ -512,7 +512,7 @@ void CheckCollectionEmpty(CollectionType* colType)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5 * 2;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -602,7 +602,7 @@ void CheckCollection(CollectionType* colType)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5 * 2;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -679,7 +679,7 @@ void CheckCollectionIterators(CollectionType* colType)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5 * 2;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -755,7 +755,7 @@ void CheckMapEmpty(MapType* mapType)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5 * 2;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -848,7 +848,7 @@ void CheckMap(MapType* mapType)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5 * 2;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -1030,7 +1030,7 @@ BOOST_AUTO_TEST_CASE(TestGuidNull)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
     
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -1115,7 +1115,7 @@ BOOST_AUTO_TEST_CASE(TestString) {
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5 * 5;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -1193,7 +1193,7 @@ BOOST_AUTO_TEST_CASE(TestStringArrayNull)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5 * 2;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -1310,7 +1310,7 @@ BOOST_AUTO_TEST_CASE(TestStringArrayEmpty)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5 * 2;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -1437,7 +1437,7 @@ BOOST_AUTO_TEST_CASE(TestStringArray)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5 * 2;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -1551,7 +1551,7 @@ BOOST_AUTO_TEST_CASE(TestObject)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5 * 3;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN); 
@@ -1595,7 +1595,7 @@ BOOST_AUTO_TEST_CASE(TestNestedObject)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5 * 3;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -1637,7 +1637,7 @@ BOOST_AUTO_TEST_CASE(TestArrayNull)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5 * 2;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -1713,7 +1713,7 @@ BOOST_AUTO_TEST_CASE(TestArrayEmpty)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5 * 2;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -1797,7 +1797,7 @@ BOOST_AUTO_TEST_CASE(TestArray)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5 * 2;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -1858,7 +1858,7 @@ BOOST_AUTO_TEST_CASE(TestCollectionNull)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5 * 2;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -1944,7 +1944,7 @@ BOOST_AUTO_TEST_CASE(TestMapNull)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5 * 2;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -2035,7 +2035,7 @@ BOOST_AUTO_TEST_CASE(TestRawMode)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 1000, footerBegin, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 1000, footerBegin, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
     
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -2091,10 +2091,10 @@ BOOST_AUTO_TEST_CASE(TestFieldSeek)
     int32_t rawOff;
     int32_t footerBegin;
 
-    if (flags & IGNITE_BINARY_FLAG_RAW_ONLY)
-        footerBegin = len;
-    else
+    if (flags & IGNITE_BINARY_FLAG_HAS_SCHEMA)
         footerBegin = schemaOrRawOff;
+    else
+        footerBegin = len;
 
     int32_t trailingBytes = (len - footerBegin) % 8;
 
@@ -2105,14 +2105,14 @@ BOOST_AUTO_TEST_CASE(TestFieldSeek)
     else
         rawOff = schemaOrRawOff;
 
-    bool usrType = flags & IGNITE_BINARY_FLAG_USER_OBJECT;
+    bool usrType = flags & IGNITE_BINARY_FLAG_USER_TYPE;
 
     footerBegin += pos;
     footerEnd += pos;
 
     BinaryReaderImpl readerImpl(&in, &idRslvr, pos, usrType, 
                                   typeId, hashCode, len, rawOff, 
-                                  footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+                                  footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
 
     BinaryReader reader(&readerImpl);
 
@@ -2243,7 +2243,7 @@ BOOST_AUTO_TEST_CASE(TestSchemaOffset2ByteFields)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 6 * fieldsNum;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_2_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_TWO_BYTES);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -2288,7 +2288,7 @@ BOOST_AUTO_TEST_CASE(TestSchemaOffset4ByteFields)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 8 * fieldsNum;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_4_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_FOUR_BYTES);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -2328,7 +2328,7 @@ BOOST_AUTO_TEST_CASE(TestSchemaOffset2ByteArray)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 6 * 2;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_2_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_TWO_BYTES);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -2362,7 +2362,7 @@ BOOST_AUTO_TEST_CASE(TestSchemaOffset4ByteArray)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 8 * 2;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_4_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_FOUR_BYTES);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/platforms/cpp/core/include/ignite/impl/binary/binary_common.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/binary/binary_common.h b/modules/platforms/cpp/core/include/ignite/impl/binary/binary_common.h
index d487941..979d2ed 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/binary/binary_common.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/binary/binary_common.h
@@ -21,7 +21,7 @@
 #include <stdint.h>
 
 namespace ignite
-{    
+{
     namespace impl
     {
         namespace binary
@@ -164,19 +164,25 @@ namespace ignite
             /** Read/write map. */
             const int32_t IGNITE_BINARY_MODE_MAP = 3;
 
-            /** User object flag. */
-            const int16_t IGNITE_BINARY_FLAG_USER_OBJECT = 0x0001;
+            /** User type flag. */
+            const int16_t IGNITE_BINARY_FLAG_USER_TYPE = 0x0001;
 
-            /** Raw only flag. */
-            const int16_t IGNITE_BINARY_FLAG_RAW_ONLY = 0x0002;
+            /** Flag: schema exists. */
+            const int16_t IGNITE_BINARY_FLAG_HAS_SCHEMA = 0x0002;
+
+            /** Flag indicating that object has raw data. */
+            const int16_t IGNITE_BINARY_FLAG_HAS_RAW = 0x0004;
 
             /** Flag indicating that schema field offset is one byte long. */
-            const int16_t IGNITE_BINARY_FLAG_OFFSET_1_BYTE = 0x0004;
+            const int16_t IGNITE_BINARY_FLAG_OFFSET_ONE_BYTE = 0x0008;
 
             /** Flag indicating that schema field offset is two byte long. */
-            const int16_t IGNITE_BINARY_FLAG_OFFSET_2_BYTE = 0x0008;
+            const int16_t IGNITE_BINARY_FLAG_OFFSET_TWO_BYTES = 0x0010;
+
+            /** Flag: compact footer, no field IDs. */
+            const int16_t IGNITE_BINARY_FLAG_COMPACT_FOOTER = 0x0020;
         }
-    }    
+    }
 }
 
 #endif
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/platforms/cpp/core/include/ignite/impl/binary/binary_reader_impl.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/binary/binary_reader_impl.h b/modules/platforms/cpp/core/include/ignite/impl/binary/binary_reader_impl.h
index 923553d..0b2b592 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/binary/binary_reader_impl.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/binary/binary_reader_impl.h
@@ -771,6 +771,13 @@ namespace ignite
                             }
 
                             int16_t flags = stream->ReadInt16();
+
+                            if (flags & IGNITE_BINARY_FLAG_COMPACT_FOOTER) {
+                                IGNITE_ERROR_2(ignite::IgniteError::IGNITE_ERR_BINARY,
+                                    "Unsupported binary protocol flag: IGNITE_BINARY_FLAG_COMPACT_FOOTER: ", 
+                                    IGNITE_BINARY_FLAG_COMPACT_FOOTER);
+                            }
+
                             int32_t typeId = stream->ReadInt32();
                             int32_t hashCode = stream->ReadInt32();
                             int32_t len = stream->ReadInt32();
@@ -783,50 +790,43 @@ namespace ignite
                             int32_t rawOff;
                             int32_t footerBegin;
 
-                            if (flags & IGNITE_BINARY_FLAG_RAW_ONLY)
-                                footerBegin = len;
+                            if (flags & IGNITE_BINARY_FLAG_HAS_SCHEMA)
+                                footerBegin = pos + schemaOrRawOff;
                             else
-                                footerBegin = schemaOrRawOff;
+                                footerBegin = pos + len;
 
                             BinaryOffsetType schemaType;
-                            int32_t trailingBytes;
 
-                            if (flags & IGNITE_BINARY_FLAG_OFFSET_1_BYTE)
-                            {
-                                schemaType = OFFSET_TYPE_1_BYTE;
+                            if (flags & IGNITE_BINARY_FLAG_OFFSET_ONE_BYTE)
+                                schemaType = OFFSET_TYPE_ONE_BYTE;
+                            else if (flags & IGNITE_BINARY_FLAG_OFFSET_TWO_BYTES)
+                                schemaType = OFFSET_TYPE_TWO_BYTES;
+                            else
+                                schemaType = OFFSET_TYPE_FOUR_BYTES;
 
-                                trailingBytes = (len - footerBegin) % 5;
-                            }
-                            else if (flags & IGNITE_BINARY_FLAG_OFFSET_2_BYTE)
+                            int32_t footerEnd;
+
+                            if (flags & IGNITE_BINARY_FLAG_HAS_RAW)
                             {
-                                schemaType = OFFSET_TYPE_2_BYTE;
+                                // 4 is the size of RawOffset field at the end of the packet.
+                                footerEnd = pos + len - 4;
 
-                                trailingBytes = (len - footerBegin) % 6;
+                                rawOff = stream->ReadInt32(footerEnd);
                             }
                             else
                             {
-                                schemaType = OFFSET_TYPE_4_BYTE;
-
-                                trailingBytes = (len - footerBegin) % 8;
-                            }
-
-                            int32_t footerEnd = len - trailingBytes;
+                                footerEnd = pos + len;
 
-                            if (trailingBytes)
-                                rawOff = stream->ReadInt32(pos + len - 4);
-                            else
                                 rawOff = schemaOrRawOff;
+                            }
 
-                            bool usrType = flags & IGNITE_BINARY_FLAG_USER_OBJECT;
-
-                            footerBegin += pos;
-                            footerEnd += pos;
+                            bool usrType = flags & IGNITE_BINARY_FLAG_USER_TYPE;
 
                             ignite::binary::BinaryType<T> type;
                             TemplatedBinaryIdResolver<T> idRslvr(type);
                             BinaryReaderImpl readerImpl(stream, &idRslvr, pos, usrType,
-                                                          typeId, hashCode, len, rawOff,
-                                                          footerBegin, footerEnd, schemaType);
+                                                        typeId, hashCode, len, rawOff,
+                                                        footerBegin, footerEnd, schemaType);
                             ignite::binary::BinaryReader reader(&readerImpl);
 
                             T val = type.Read(reader);
@@ -1038,9 +1038,7 @@ namespace ignite
                     {
                         CheckRawMode(false);
                         CheckSingleMode(true);
-
-                        int32_t pos = stream->Position();
-
+                        
                         int32_t fieldId = idRslvr->GetFieldId(typeId, fieldName);
                         int32_t fieldPos = FindField(fieldId);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/platforms/cpp/core/include/ignite/impl/binary/binary_schema.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/binary/binary_schema.h b/modules/platforms/cpp/core/include/ignite/impl/binary/binary_schema.h
index d0a9f26..b100b8c 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/binary/binary_schema.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/binary/binary_schema.h
@@ -39,13 +39,13 @@ namespace ignite
             enum BinaryOffsetType
             {
                 /** Means all field offsets can be fit in one byte. */
-                OFFSET_TYPE_1_BYTE,
+                OFFSET_TYPE_ONE_BYTE,
 
                 /** Means all field offsets can be fit in two bytes. */
-                OFFSET_TYPE_2_BYTE,
+                OFFSET_TYPE_TWO_BYTES,
 
                 /** Means field offsets should be stored in four bytes. */
-                OFFSET_TYPE_4_BYTE
+                OFFSET_TYPE_FOUR_BYTES
             };
 
             /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/platforms/cpp/core/include/ignite/impl/binary/binary_writer_impl.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/binary/binary_writer_impl.h b/modules/platforms/cpp/core/include/ignite/impl/binary/binary_writer_impl.h
index fe31ece..1b47d9e 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/binary/binary_writer_impl.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/binary/binary_writer_impl.h
@@ -601,7 +601,7 @@ namespace ignite
 
                         stream->WriteInt8(IGNITE_HDR_FULL);
                         stream->WriteInt8(IGNITE_PROTO_VER);
-                        stream->WriteInt16(IGNITE_BINARY_FLAG_USER_OBJECT);
+                        stream->WriteInt16(IGNITE_BINARY_FLAG_USER_TYPE);
                         stream->WriteInt32(idRslvr.GetTypeId());
                         stream->WriteInt32(type.GetHashCode(obj));
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/platforms/cpp/core/src/impl/binary/binary_reader_impl.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/impl/binary/binary_reader_impl.cpp b/modules/platforms/cpp/core/src/impl/binary/binary_reader_impl.cpp
index e189273..5fcbc75 100644
--- a/modules/platforms/cpp/core/src/impl/binary/binary_reader_impl.cpp
+++ b/modules/platforms/cpp/core/src/impl/binary/binary_reader_impl.cpp
@@ -47,7 +47,7 @@ namespace ignite
             BinaryReaderImpl::BinaryReaderImpl(InteropInputStream* stream) :
                 stream(stream), idRslvr(NULL), pos(0), usrType(false), typeId(0), hashCode(0), len(0),
                 rawOff(0), rawMode(true), elemIdGen(0), elemId(0), elemCnt(-1), elemRead(0), footerBegin(-1),
-                footerEnd(-1), schemaType(OFFSET_TYPE_4_BYTE)
+                footerEnd(-1), schemaType(OFFSET_TYPE_FOUR_BYTES)
             {
                 // No-op.
             }
@@ -637,7 +637,7 @@ namespace ignite
 
                 switch (schemaType)
                 {
-                    case OFFSET_TYPE_1_BYTE:
+                    case OFFSET_TYPE_ONE_BYTE:
                     {
                         for (int32_t schemaPos = footerBegin; schemaPos < footerEnd; schemaPos += 5)
                         {
@@ -649,7 +649,7 @@ namespace ignite
                         break;
                     }
 
-                    case OFFSET_TYPE_2_BYTE:
+                    case OFFSET_TYPE_TWO_BYTES:
                     {
                         for (int32_t schemaPos = footerBegin; schemaPos < footerEnd; schemaPos += 6)
                         {
@@ -661,7 +661,7 @@ namespace ignite
                         break;
                     }
 
-                    case OFFSET_TYPE_4_BYTE:
+                    case OFFSET_TYPE_FOUR_BYTES:
                     {
                         for (int32_t schemaPos = footerBegin; schemaPos < footerEnd; schemaPos += 8)
                         {

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/platforms/cpp/core/src/impl/binary/binary_schema.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/impl/binary/binary_schema.cpp b/modules/platforms/cpp/core/src/impl/binary/binary_schema.cpp
index 1596557..0b8025a 100644
--- a/modules/platforms/cpp/core/src/impl/binary/binary_schema.cpp
+++ b/modules/platforms/cpp/core/src/impl/binary/binary_schema.cpp
@@ -70,7 +70,7 @@ namespace ignite
             {
                 switch (GetType())
                 {
-                    case OFFSET_TYPE_1_BYTE:
+                    case OFFSET_TYPE_ONE_BYTE:
                     {
                         for (FieldContainer::const_iterator i = fieldsInfo->begin(); i != fieldsInfo->end(); ++i)
                         {
@@ -80,7 +80,7 @@ namespace ignite
                         break;
                     }
 
-                    case OFFSET_TYPE_2_BYTE:
+                    case OFFSET_TYPE_TWO_BYTES:
                     {
                         for (FieldContainer::const_iterator i = fieldsInfo->begin(); i != fieldsInfo->end(); ++i)
                         {
@@ -90,7 +90,7 @@ namespace ignite
                         break;
                     }
 
-                    case OFFSET_TYPE_4_BYTE:
+                    case OFFSET_TYPE_FOUR_BYTES:
                     {
                         for (FieldContainer::const_iterator i = fieldsInfo->begin(); i != fieldsInfo->end(); ++i)
                         {
@@ -124,11 +124,11 @@ namespace ignite
                 int32_t maxOffset = fieldsInfo->back().offset;
 
                 if (maxOffset < 0x100)
-                    return OFFSET_TYPE_1_BYTE;
+                    return OFFSET_TYPE_ONE_BYTE;
                 else if (maxOffset < 0x10000)
-                    return OFFSET_TYPE_2_BYTE;
+                    return OFFSET_TYPE_TWO_BYTES;
 
-                return OFFSET_TYPE_4_BYTE;
+                return OFFSET_TYPE_FOUR_BYTES;
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/platforms/cpp/core/src/impl/binary/binary_writer_impl.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/impl/binary/binary_writer_impl.cpp b/modules/platforms/cpp/core/src/impl/binary/binary_writer_impl.cpp
index 47df19d..d03d6ba 100644
--- a/modules/platforms/cpp/core/src/impl/binary/binary_writer_impl.cpp
+++ b/modules/platforms/cpp/core/src/impl/binary/binary_writer_impl.cpp
@@ -563,11 +563,15 @@ namespace ignite
                 int32_t lenWithoutSchema = stream->Position() - start;
 
                 int32_t nonRawLen = rawPos == -1 ? lenWithoutSchema : rawPos - start;
-                
-                if (schema.Empty())
+
+                uint16_t flags = IGNITE_BINARY_FLAG_USER_TYPE;
+
+                if (rawPos > 0)
+                    flags |= IGNITE_BINARY_FLAG_HAS_RAW;
+
+                if (!HasSchema())
                 {
-                    stream->WriteInt16(start + IGNITE_OFFSET_FLAGS, IGNITE_BINARY_FLAG_USER_OBJECT | 
-                                                                    IGNITE_BINARY_FLAG_RAW_ONLY);
+                    stream->WriteInt16(start + IGNITE_OFFSET_FLAGS, flags);
                     stream->WriteInt32(start + IGNITE_OFFSET_LEN, lenWithoutSchema);
                     stream->WriteInt32(start + IGNITE_OFFSET_SCHEMA_ID, 0);
                     stream->WriteInt32(start + IGNITE_OFFSET_SCHEMA_OR_RAW_OFF, GetRawPosition() - start);
@@ -584,17 +588,14 @@ namespace ignite
 
                     int32_t length = stream->Position() - start;
 
-                    if (schemaType == OFFSET_TYPE_1_BYTE)
-                    {
-                        stream->WriteInt16(start + IGNITE_OFFSET_FLAGS, 
-                            IGNITE_BINARY_FLAG_USER_OBJECT | IGNITE_BINARY_FLAG_OFFSET_1_BYTE);
-                    }
-                    else if (schemaType == OFFSET_TYPE_2_BYTE)
-                    {
-                        stream->WriteInt16(start + IGNITE_OFFSET_FLAGS, 
-                            IGNITE_BINARY_FLAG_USER_OBJECT | IGNITE_BINARY_FLAG_OFFSET_2_BYTE);
-                    }
+                    flags |= IGNITE_BINARY_FLAG_HAS_SCHEMA;
+                    
+                    if (schemaType == OFFSET_TYPE_ONE_BYTE)
+                        flags |= IGNITE_BINARY_FLAG_OFFSET_ONE_BYTE;
+                    else if (schemaType == OFFSET_TYPE_TWO_BYTES)
+                        flags |= IGNITE_BINARY_FLAG_OFFSET_TWO_BYTES;
 
+                    stream->WriteInt16(start + IGNITE_OFFSET_FLAGS, flags);
                     stream->WriteInt32(start + IGNITE_OFFSET_LEN, length);
                     stream->WriteInt32(start + IGNITE_OFFSET_SCHEMA_ID, schemaId);
                     stream->WriteInt32(start + IGNITE_OFFSET_SCHEMA_OR_RAW_OFF, lenWithoutSchema);

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml
index e373b89..9755033 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml
@@ -70,6 +70,7 @@
         <!-- Binary marshaller configuration -->
         <property name="marshaller">
             <bean class="org.apache.ignite.marshaller.portable.PortableMarshaller">
+                <property name="compactFooter" value="false" />
                 <property name="typeConfigurations">
                     <list>
                         <bean class="org.apache.ignite.binary.BinaryTypeConfiguration">

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/marshaller-explicit.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/marshaller-explicit.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/marshaller-explicit.xml
index 11b87bd..74444a1 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/marshaller-explicit.xml
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/marshaller-explicit.xml
@@ -26,7 +26,9 @@
         <property name="connectorConfiguration"><null/></property>
 
         <property name="marshaller">
-            <bean class="org.apache.ignite.marshaller.portable.PortableMarshaller" />
+            <bean class="org.apache.ignite.marshaller.portable.PortableMarshaller">
+                <property name="compactFooter" value="false"/>
+            </bean>
         </property>
 
         <property name="discoverySpi">

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs
index 1840ab9..97cc381 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs
@@ -678,28 +678,37 @@ namespace Apache.Ignite.Core.Impl.Binary
                                 WriteField(ctx, valEntry.Value);
                             }
 
+                            var flags = inHeader.IsUserType
+                                ? BinaryObjectHeader.Flag.UserType
+                                : BinaryObjectHeader.Flag.None;
+
                             // Write raw data.
                             int outRawOff = outStream.Position - outStartPos;
 
-                            int inRawOff = inHeader.GetRawOffset(inStream, inStartPos);
-                            int inRawLen = inHeader.SchemaOffset - inRawOff;
+                            if (inHeader.HasRaw)
+                            {
+                                var inRawOff = inHeader.GetRawOffset(inStream, inStartPos);
+                                var inRawLen = inHeader.SchemaOffset - inRawOff;
+
+                                flags |= BinaryObjectHeader.Flag.HasRaw;
 
-                            if (inRawLen > 0)
                                 outStream.Write(inStream.InternalArray, inStartPos + inRawOff, inRawLen);
+                            }
 
                             // Write schema
                             int outSchemaOff = outRawOff;
                             var schemaPos = outStream.Position;
                             int outSchemaId;
-                            short flags;
 
-                            var hasSchema = outSchema.WriteSchema(outStream, schemaIdx, out outSchemaId, out flags);
+                            var hasSchema = outSchema.WriteSchema(outStream, schemaIdx, out outSchemaId, ref flags);
 
                             if (hasSchema)
                             {
                                 outSchemaOff = schemaPos - outStartPos;
+                                
+                                flags |= BinaryObjectHeader.Flag.HasSchema;
 
-                                if (inRawLen > 0)
+                                if (inHeader.HasRaw)
                                     outStream.WriteInt(outRawOff);
                             }
 
@@ -707,8 +716,8 @@ namespace Apache.Ignite.Core.Impl.Binary
 
                             var outHash = changeHash ? hash : inHeader.HashCode;
 
-                            var outHeader = new BinaryObjectHeader(inHeader.IsUserType, inHeader.TypeId, outHash,
-                                outLen, outSchemaId, outSchemaOff, !hasSchema, flags);
+                            var outHeader = new BinaryObjectHeader(inHeader.TypeId, outHash, outLen, 
+                                outSchemaId, outSchemaOff, flags);
 
                             BinaryObjectHeader.Write(outHeader, outStream, outStartPos);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs
index 59cb29c1..573e014 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs
@@ -33,22 +33,38 @@ namespace Apache.Ignite.Core.Impl.Binary
         /** Size, equals to sizeof(BinaryObjectHeader). */
         public const int Size = 24;
 
-        /** User type flag. */
-        public const short FlagUserType = 0x1;
+        /// <summary>
+        /// Flags.
+        /// </summary>
+        [Flags]
+        public enum Flag : short
+        {
+            /** No flags. */
+            None            = 0x00,
+
+            /** Flag: user type. */
+            UserType        = 0x01,
+            
+            /** Flag: schema exists. */
+            HasSchema       = 0x02,
+
+            /** Flag indicating that object has raw data. */
+            HasRaw          = 0x04,
 
-        /** Raw only flag. */
-        public const short FlagRawOnly = 0x2;
+            /** Flag: offsets take 1 byte. */
+            OffsetOneByte   = 0x08,
 
-        /** Byte-sized field offsets flag. */
-        public const short FlagByteOffsets = 0x4;
+            /** Flag: offsets take 2 bytes. */
+            OffsetTwoBytes  = 0x10,
 
-        /** Short-sized field offsets flag. */
-        public const short FlagShortOffsets = 0x8;
+            /** Flag: compact footer, no field IDs. */
+            CompactFooter   = 0x20
+        }
 
         /** Actual header layout */
         public readonly byte Header;        // Header code, always 103 (HdrFull)
         public readonly byte Version;       // Protocol version
-        public readonly short Flags;        // Flags
+        public readonly Flag Flags;         // Flags
         public readonly int TypeId;         // Type ID
         public readonly int HashCode;       // Hash code
         public readonly int Length;         // Length, including header
@@ -58,16 +74,13 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <summary>
         /// Initializes a new instance of the <see cref="BinaryObjectHeader" /> struct.
         /// </summary>
-        /// <param name="userType">User type flag.</param>
         /// <param name="typeId">Type ID.</param>
         /// <param name="hashCode">Hash code.</param>
         /// <param name="length">Length.</param>
         /// <param name="schemaId">Schema ID.</param>
         /// <param name="schemaOffset">Schema offset.</param>
-        /// <param name="rawOnly">Raw flag.</param>
         /// <param name="flags">The flags.</param>
-        public BinaryObjectHeader(bool userType, int typeId, int hashCode, int length, int schemaId, int schemaOffset, 
-            bool rawOnly, short flags)
+        public BinaryObjectHeader(int typeId, int hashCode, int length, int schemaId, int schemaOffset, Flag flags)
         {
             Header = BinaryUtils.HdrFull;
             Version = BinaryUtils.ProtoVer;
@@ -75,12 +88,6 @@ namespace Apache.Ignite.Core.Impl.Binary
             Debug.Assert(schemaOffset <= length);
             Debug.Assert(schemaOffset >= Size);
 
-            if (userType)
-                flags |= FlagUserType;
-
-            if (rawOnly)
-                flags |= FlagRawOnly;
-
             Flags = flags;
 
             TypeId = typeId;
@@ -98,7 +105,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         {
             Header = stream.ReadByte();
             Version = stream.ReadByte();
-            Flags = stream.ReadShort();
+            Flags = (Flag) stream.ReadShort();
             Length = stream.ReadInt();
             TypeId = stream.ReadInt();
             HashCode = stream.ReadInt();
@@ -114,7 +121,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         {
             stream.WriteByte(Header);
             stream.WriteByte(Version);
-            stream.WriteShort(Flags);
+            stream.WriteShort((short) Flags);
             stream.WriteInt(Length);
             stream.WriteInt(TypeId);
             stream.WriteInt(HashCode);
@@ -123,31 +130,35 @@ namespace Apache.Ignite.Core.Impl.Binary
         }
 
         /// <summary>
-        /// Gets a user type flag.
+        /// Gets the user type flag.
         /// </summary>
         public bool IsUserType
         {
-            get { return (Flags & FlagUserType) == FlagUserType; }
+            get { return (Flags & Flag.UserType) == Flag.UserType; }
         }
 
         /// <summary>
-        /// Gets a raw-only flag.
+        /// Gets the schema flag.
         /// </summary>
-        public bool IsRawOnly
+        public bool HasSchema
         {
-            get { return (Flags & FlagRawOnly) == FlagRawOnly; }
+            get { return (Flags & Flag.HasSchema) == Flag.HasSchema; }
         }
 
         /// <summary>
-        /// Gets a value indicating whether this instance has raw offset.
+        /// Gets the raw flag.
         /// </summary>
-        public bool HasRawOffset
+        public bool HasRaw
         {
-            get
-            {
-                // Remainder => raw offset is the very last 4 bytes in object.
-                return !IsRawOnly && ((Length - SchemaOffset) % SchemaFieldSize) == 4;
-            }
+            get { return (Flags & Flag.HasRaw) == Flag.HasRaw; }
+        }
+
+        /// <summary>
+        /// Gets the compact footer flag.
+        /// </summary>
+        public bool IsCompactFooter
+        {
+            get { return (Flags & Flag.CompactFooter) == Flag.CompactFooter; }
         }
 
         /// <summary>
@@ -157,10 +168,10 @@ namespace Apache.Ignite.Core.Impl.Binary
         {
             get
             {
-                if ((Flags & FlagByteOffsets) == FlagByteOffsets)
+                if ((Flags & Flag.OffsetOneByte) == Flag.OffsetOneByte)
                     return 1;
 
-                if ((Flags & FlagShortOffsets) == FlagShortOffsets)
+                if ((Flags & Flag.OffsetTwoBytes) == Flag.OffsetTwoBytes)
                     return 2;
 
                 return 4;
@@ -182,7 +193,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         {
             get
             {
-                if (IsRawOnly)
+                if (!HasSchema)
                     return 0;
 
                 var schemaSize = Length - SchemaOffset;
@@ -201,7 +212,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         {
             Debug.Assert(stream != null);
 
-            if (!HasRawOffset)
+            if (!HasRaw || !HasSchema)
                 return SchemaOffset;
 
             stream.Seek(position + Length - 4, SeekOrigin.Begin);
@@ -219,6 +230,8 @@ namespace Apache.Ignite.Core.Impl.Binary
         {
             Debug.Assert(stream != null);
 
+            ThrowIfUnsupported();
+
             var schemaSize = SchemaFieldCount;
 
             if (schemaSize == 0)
@@ -259,6 +272,8 @@ namespace Apache.Ignite.Core.Impl.Binary
         {
             Debug.Assert(stream != null);
 
+            ThrowIfUnsupported();
+
             var schemaSize = SchemaFieldCount;
 
             if (schemaSize == 0)
@@ -297,10 +312,10 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <param name="offset">Offset in the array.</param>
         /// <param name="count">Field count to write.</param>
         /// <returns>
-        /// Flags according to offset sizes: <see cref="BinaryObjectHeader.FlagByteOffsets" />,
-        /// <see cref="BinaryObjectHeader.FlagShortOffsets" />, or 0.
+        /// Flags according to offset sizes: <see cref="Flag.OffsetOneByte" />,
+        /// <see cref="Flag.OffsetTwoBytes" />, or 0.
         /// </returns>
-        public static unsafe short WriteSchema(BinaryObjectSchemaField[] fields, IBinaryStream stream, int offset,
+        public static unsafe Flag WriteSchema(BinaryObjectSchemaField[] fields, IBinaryStream stream, int offset,
             int count)
         {
             Debug.Assert(fields != null);
@@ -324,7 +339,7 @@ namespace Apache.Ignite.Core.Impl.Binary
                         stream.WriteByte((byte)field.Offset);
                     }
 
-                    return FlagByteOffsets;
+                    return Flag.OffsetOneByte;
                 }
 
                 if (maxFieldOffset <= ushort.MaxValue)
@@ -338,7 +353,7 @@ namespace Apache.Ignite.Core.Impl.Binary
                         stream.WriteShort((short)field.Offset);
                     }
 
-                    return FlagShortOffsets;
+                    return Flag.OffsetTwoBytes;
                 }
 
                 if (BitConverter.IsLittleEndian)
@@ -359,9 +374,8 @@ namespace Apache.Ignite.Core.Impl.Binary
                     }
                 }
 
-                return 0;
+                return Flag.None;
             }
-
         }
 
         /// <summary>
@@ -396,24 +410,27 @@ namespace Apache.Ignite.Core.Impl.Binary
 
             stream.Seek(position, SeekOrigin.Begin);
 
+            BinaryObjectHeader hdr;
+
             if (BitConverter.IsLittleEndian)
             {
-                var hdr = new BinaryObjectHeader();
-
                 stream.Read((byte*) &hdr, Size);
 
                 Debug.Assert(hdr.Version == BinaryUtils.ProtoVer);
                 Debug.Assert(hdr.SchemaOffset <= hdr.Length);
                 Debug.Assert(hdr.SchemaOffset >= Size);
 
-                // Only one of the flags can be set
-                var f = hdr.Flags;
-                Debug.Assert((f & (FlagShortOffsets | FlagByteOffsets)) != (FlagShortOffsets | FlagByteOffsets));
-
-                return hdr;
             }
+            else
+                hdr = new BinaryObjectHeader(stream);
+
+            hdr.ThrowIfUnsupported();
 
-            return new BinaryObjectHeader(stream);
+            // Only one of the flags can be set
+            var f = hdr.Flags;
+            Debug.Assert((f & (Flag.OffsetOneByte | Flag.OffsetTwoBytes)) !=
+                         (Flag.OffsetOneByte | Flag.OffsetTwoBytes));
+            return hdr;
         }
 
         /** <inheritdoc> */
@@ -465,5 +482,15 @@ namespace Apache.Ignite.Core.Impl.Binary
         {
             return !left.Equals(right);
         }
+
+        /// <summary>
+        /// Throws an exception if current header represents unsupported mode.
+        /// </summary>
+        private void ThrowIfUnsupported()
+        {
+            // Compact schema is not supported
+            if (IsCompactFooter)
+                throw new NotSupportedException("Compact binary object footer is not supported in Ignite.NET.");
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaHolder.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaHolder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaHolder.cs
index 75ff2c5..65b6fc0 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaHolder.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaHolder.cs
@@ -82,22 +82,21 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <param name="stream">The stream.</param>
         /// <param name="schemaOffset">The schema offset.</param>
         /// <param name="schemaId">The schema identifier.</param>
-        /// <param name="flags">Flags according to offset sizes: <see cref="BinaryObjectHeader.FlagByteOffsets" />,
-        /// <see cref="BinaryObjectHeader.FlagShortOffsets" />, or 0.</param>
+        /// <param name="flags">Flags according to offset sizes.</param>
         /// <returns>
         /// True if current schema was non empty; false otherwise.
         /// </returns>
-        public bool WriteSchema(IBinaryStream stream, int schemaOffset, out int schemaId, out short flags)
+        public bool WriteSchema(IBinaryStream stream, int schemaOffset, out int schemaId, 
+            ref BinaryObjectHeader.Flag flags)
         {
             schemaId = Fnv1Hash.Basis;
-            flags = 0;
 
             var count = _idx - schemaOffset;
 
             if (count == 0) 
                 return false;
 
-            flags = BinaryObjectHeader.WriteSchema(_fields, stream, schemaOffset, count);
+            flags |= BinaryObjectHeader.WriteSchema(_fields, stream, schemaOffset, count);
 
             for (var i = schemaOffset; i < _idx; i++)
                 schemaId = Fnv1Hash.Update(schemaId, _fields[i].Id);

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
index 53f6f4a..9aeb908 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
@@ -898,7 +898,7 @@ namespace Apache.Ignite.Core.Impl.Binary
             if (_curRaw)
                 throw new BinaryObjectException("Cannot read named fields after raw data is read.");
 
-            if (_curHdr.IsRawOnly)
+            if (!_curHdr.HasSchema)
                 return false;
 
             var actionId = _curStruct.CurStructAction;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs
index 58973f7..e09a7f4 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs
@@ -1090,20 +1090,31 @@ namespace Apache.Ignite.Core.Impl.Binary
                     var schemaOffset = _stream.Position - pos;
 
                     int schemaId;
-                    short flags;
-                    var hasSchema = _schema.WriteSchema(_stream, schemaIdx, out schemaId, out flags);
+                    
+                    var flags = desc.UserType
+                        ? BinaryObjectHeader.Flag.UserType
+                        : BinaryObjectHeader.Flag.None;
 
-                    if (!hasSchema)
+                    var hasSchema = _schema.WriteSchema(_stream, schemaIdx, out schemaId, ref flags);
+
+                    if (hasSchema)
+                    {
+                        flags |= BinaryObjectHeader.Flag.HasSchema;
+
+                        // Calculate and write header.
+                        if (_curRawPos > 0)
+                            _stream.WriteInt(_curRawPos - pos); // raw offset is in the last 4 bytes
+                    }
+                    else
                         schemaOffset = BinaryObjectHeader.Size;
 
-                    // Calculate and write header.
-                    if (hasSchema && _curRawPos > 0)
-                        _stream.WriteInt(_curRawPos - pos); // raw offset is in the last 4 bytes
+                    if (_curRawPos > 0)
+                        flags |= BinaryObjectHeader.Flag.HasRaw;
 
                     var len = _stream.Position - pos;
 
-                    var header = new BinaryObjectHeader(desc.UserType, desc.TypeId, obj.GetHashCode(), len,
-                        schemaId, schemaOffset, !hasSchema, flags);
+                    var header = new BinaryObjectHeader(desc.TypeId, obj.GetHashCode(), len,
+                        schemaId, schemaOffset, flags);
 
                     BinaryObjectHeader.Write(header, _stream, pos);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/IgniteBinary.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/IgniteBinary.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/IgniteBinary.cs
index 927ebaf..ecc6807 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/IgniteBinary.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/IgniteBinary.cs
@@ -166,7 +166,8 @@ namespace Apache.Ignite.Core.Impl.Binary
         {
             var len = BinaryObjectHeader.Size;
 
-            var hdr = new BinaryObjectHeader(desc.UserType, desc.TypeId, 0, len, 0, len, true, 0);
+            var hdr = new BinaryObjectHeader(desc.TypeId, 0, len, 0, len,
+                desc.UserType ? BinaryObjectHeader.Flag.UserType : BinaryObjectHeader.Flag.None);
 
             var stream = new BinaryHeapStream(len);
 


[06/13] ignite git commit: IGNITE-1816: Implemented compact footers.

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/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 31f2bf9..95ef9591 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
@@ -98,16 +98,22 @@ public class PortableUtils {
     private static final Collection<Class<?>> PORTABLE_CLS = new HashSet<>();
 
     /** Flag: user type. */
-    public static final short FLAG_USR_TYP = 0x1;
+    public static final short FLAG_USR_TYP = 0x0001;
 
     /** Flag: only raw data exists. */
-    public static final short FLAG_RAW_ONLY = 0x2;
+    public static final short FLAG_HAS_SCHEMA = 0x0002;
+
+    /** Flag indicating that object has raw data. */
+    public static final short FLAG_HAS_RAW = 0x0004;
 
     /** Flag: offsets take 1 byte. */
-    public static final short FLAG_OFFSET_ONE_BYTE = 0x4;
+    public static final short FLAG_OFFSET_ONE_BYTE = 0x0008;
 
     /** Flag: offsets take 2 bytes. */
-    public static final short FLAG_OFFSET_TWO_BYTES = 0x8;
+    public static final short FLAG_OFFSET_TWO_BYTES = 0x0010;
+
+    /** Flag: compact footer, no field IDs. */
+    public static final short FLAG_COMPACT_FOOTER = 0x0020;
 
     /** Offset which fits into 1 byte. */
     public static final int OFFSET_1 = 1;
@@ -118,10 +124,99 @@ public class PortableUtils {
     /** Offset which fits into 4 bytes. */
     public static final int OFFSET_4 = 4;
 
+    /** Field ID length. */
+    public static final int FIELD_ID_LEN = 4;
+
     /** Field type names. */
     private static final String[] FIELD_TYPE_NAMES;
 
+    /** FNV1 hash offset basis. */
+    private static final int FNV1_OFFSET_BASIS = 0x811C9DC5;
+
+    /** FNV1 hash prime. */
+    private static final int FNV1_PRIME = 0x01000193;
+
+    /**
+     * Static class initializer.
+     */
     static {
+        PLAIN_CLASS_TO_FLAG.put(Byte.class, GridPortableMarshaller.BYTE);
+        PLAIN_CLASS_TO_FLAG.put(Short.class, GridPortableMarshaller.SHORT);
+        PLAIN_CLASS_TO_FLAG.put(Integer.class, GridPortableMarshaller.INT);
+        PLAIN_CLASS_TO_FLAG.put(Long.class, GridPortableMarshaller.LONG);
+        PLAIN_CLASS_TO_FLAG.put(Float.class, GridPortableMarshaller.FLOAT);
+        PLAIN_CLASS_TO_FLAG.put(Double.class, GridPortableMarshaller.DOUBLE);
+        PLAIN_CLASS_TO_FLAG.put(Character.class, GridPortableMarshaller.CHAR);
+        PLAIN_CLASS_TO_FLAG.put(Boolean.class, GridPortableMarshaller.BOOLEAN);
+        PLAIN_CLASS_TO_FLAG.put(BigDecimal.class, GridPortableMarshaller.DECIMAL);
+        PLAIN_CLASS_TO_FLAG.put(String.class, GridPortableMarshaller.STRING);
+        PLAIN_CLASS_TO_FLAG.put(UUID.class, GridPortableMarshaller.UUID);
+        PLAIN_CLASS_TO_FLAG.put(Date.class, GridPortableMarshaller.DATE);
+        PLAIN_CLASS_TO_FLAG.put(Timestamp.class, GridPortableMarshaller.TIMESTAMP);
+
+        PLAIN_CLASS_TO_FLAG.put(byte[].class, GridPortableMarshaller.BYTE_ARR);
+        PLAIN_CLASS_TO_FLAG.put(short[].class, GridPortableMarshaller.SHORT_ARR);
+        PLAIN_CLASS_TO_FLAG.put(int[].class, GridPortableMarshaller.INT_ARR);
+        PLAIN_CLASS_TO_FLAG.put(long[].class, GridPortableMarshaller.LONG_ARR);
+        PLAIN_CLASS_TO_FLAG.put(float[].class, GridPortableMarshaller.FLOAT_ARR);
+        PLAIN_CLASS_TO_FLAG.put(double[].class, GridPortableMarshaller.DOUBLE_ARR);
+        PLAIN_CLASS_TO_FLAG.put(char[].class, GridPortableMarshaller.CHAR_ARR);
+        PLAIN_CLASS_TO_FLAG.put(boolean[].class, GridPortableMarshaller.BOOLEAN_ARR);
+        PLAIN_CLASS_TO_FLAG.put(BigDecimal[].class, GridPortableMarshaller.DECIMAL_ARR);
+        PLAIN_CLASS_TO_FLAG.put(String[].class, GridPortableMarshaller.STRING_ARR);
+        PLAIN_CLASS_TO_FLAG.put(UUID[].class, GridPortableMarshaller.UUID_ARR);
+        PLAIN_CLASS_TO_FLAG.put(Date[].class, GridPortableMarshaller.DATE_ARR);
+        PLAIN_CLASS_TO_FLAG.put(Timestamp[].class, GridPortableMarshaller.TIMESTAMP_ARR);
+
+        for (Map.Entry<Class<?>, Byte> entry : PLAIN_CLASS_TO_FLAG.entrySet())
+            FLAG_TO_CLASS.put(entry.getValue(), entry.getKey());
+
+        PLAIN_CLASS_TO_FLAG.put(byte.class, GridPortableMarshaller.BYTE);
+        PLAIN_CLASS_TO_FLAG.put(short.class, GridPortableMarshaller.SHORT);
+        PLAIN_CLASS_TO_FLAG.put(int.class, GridPortableMarshaller.INT);
+        PLAIN_CLASS_TO_FLAG.put(long.class, GridPortableMarshaller.LONG);
+        PLAIN_CLASS_TO_FLAG.put(float.class, GridPortableMarshaller.FLOAT);
+        PLAIN_CLASS_TO_FLAG.put(double.class, GridPortableMarshaller.DOUBLE);
+        PLAIN_CLASS_TO_FLAG.put(char.class, GridPortableMarshaller.CHAR);
+        PLAIN_CLASS_TO_FLAG.put(boolean.class, GridPortableMarshaller.BOOLEAN);
+
+        for (byte b : new byte[] {
+            BYTE, SHORT, INT, LONG, FLOAT, DOUBLE,
+            CHAR, BOOLEAN, DECIMAL, STRING, UUID, DATE, TIMESTAMP,
+            BYTE_ARR, SHORT_ARR, INT_ARR, LONG_ARR, FLOAT_ARR, DOUBLE_ARR,
+            CHAR_ARR, BOOLEAN_ARR, DECIMAL_ARR, STRING_ARR, UUID_ARR, DATE_ARR, TIMESTAMP_ARR,
+            ENUM, ENUM_ARR, NULL}) {
+
+            PLAIN_TYPE_FLAG[b] = true;
+        }
+
+        PORTABLE_CLS.add(Byte.class);
+        PORTABLE_CLS.add(Short.class);
+        PORTABLE_CLS.add(Integer.class);
+        PORTABLE_CLS.add(Long.class);
+        PORTABLE_CLS.add(Float.class);
+        PORTABLE_CLS.add(Double.class);
+        PORTABLE_CLS.add(Character.class);
+        PORTABLE_CLS.add(Boolean.class);
+        PORTABLE_CLS.add(String.class);
+        PORTABLE_CLS.add(UUID.class);
+        PORTABLE_CLS.add(Date.class);
+        PORTABLE_CLS.add(Timestamp.class);
+        PORTABLE_CLS.add(BigDecimal.class);
+        PORTABLE_CLS.add(byte[].class);
+        PORTABLE_CLS.add(short[].class);
+        PORTABLE_CLS.add(int[].class);
+        PORTABLE_CLS.add(long[].class);
+        PORTABLE_CLS.add(float[].class);
+        PORTABLE_CLS.add(double[].class);
+        PORTABLE_CLS.add(char[].class);
+        PORTABLE_CLS.add(boolean[].class);
+        PORTABLE_CLS.add(String[].class);
+        PORTABLE_CLS.add(UUID[].class);
+        PORTABLE_CLS.add(Date[].class);
+        PORTABLE_CLS.add(Timestamp[].class);
+        PORTABLE_CLS.add(BigDecimal[].class);
+
         FIELD_TYPE_NAMES = new String[104];
 
         FIELD_TYPE_NAMES[BYTE] = "byte";
@@ -162,168 +257,113 @@ public class PortableUtils {
     }
 
     /**
-     * @param typeName Field type name.
-     * @return Field type ID;
+     * Check if user type flag is set.
+     *
+     * @param flags Flags.
+     * @return {@code True} if set.
      */
-    @SuppressWarnings("StringEquality")
-    public static int fieldTypeId(String typeName) {
-        for (int i = 0; i < FIELD_TYPE_NAMES.length; i++) {
-            String typeName0 = FIELD_TYPE_NAMES[i];
-
-            if (typeName.equals(typeName0))
-                return i;
-        }
-
-        throw new IllegalArgumentException("Invalid metadata type name: " + typeName);
+    public static boolean isUserType(short flags) {
+        return isFlagSet(flags, FLAG_USR_TYP);
     }
 
     /**
-     * @param typeId Field type ID.
-     * @return Field type name.
+     * Check if raw-only flag is set.
+     *
+     * @param flags Flags.
+     * @return {@code True} if set.
      */
-    public static String fieldTypeName(int typeId) {
-        assert typeId >= 0 && typeId < FIELD_TYPE_NAMES.length : typeId;
-
-        String typeName = FIELD_TYPE_NAMES[typeId];
-
-        assert typeName != null : typeId;
-
-        return typeName;
+    public static boolean hasSchema(short flags) {
+        return isFlagSet(flags, FLAG_HAS_SCHEMA);
     }
 
     /**
-     * @param typeIds Field type IDs.
-     * @return Field type names.
+     * Check if raw-only flag is set.
+     *
+     * @param flags Flags.
+     * @return {@code True} if set.
      */
-    public static Map<String, String> fieldTypeNames(Map<String, Integer> typeIds) {
-        Map<String, String> names = U.newHashMap(typeIds.size());
-
-        for (Map.Entry<String, Integer> e : typeIds.entrySet())
-            names.put(e.getKey(), fieldTypeName(e.getValue()));
-
-        return names;
+    public static boolean hasRaw(short flags) {
+        return isFlagSet(flags, FLAG_HAS_RAW);
     }
 
     /**
-     * Write flags.
+     * Check if "no-field-ids" flag is set.
      *
-     * @param writer Writer.
-     * @param userType User type flag.
+     * @param flags Flags.
+     * @return {@code True} if set.
      */
-    public static void writeFlags(BinaryWriterExImpl writer, boolean userType) {
-        short val = 0;
-
-        if (userType)
-            val |= FLAG_USR_TYP;
-
-        writer.doWriteShort(val);
+    public static boolean isCompactFooter(short flags) {
+        return isFlagSet(flags, FLAG_COMPACT_FOOTER);
     }
 
     /**
-     * Check if user type flag is set.
+     * Check whether particular flag is set.
      *
      * @param flags Flags.
-     * @return {@code True} if set.
+     * @param flag Flag.
+     * @return {@code True} if flag is set in flags.
      */
-    public static boolean isUserType(short flags) {
-        return (flags & FLAG_USR_TYP) == FLAG_USR_TYP;
+    private static boolean isFlagSet(short flags, short flag) {
+        return (flags & flag) == flag;
     }
-
+    
     /**
-     * Check if raw-only flag is set.
+     * Schema initial ID.
      *
-     * @param flags Flags.
-     * @return {@code True} if set.
+     * @return ID.
      */
-    public static boolean isRawOnly(short flags) {
-        return (flags & FLAG_RAW_ONLY) == FLAG_RAW_ONLY;
+    public static int schemaInitialId() {
+        return FNV1_OFFSET_BASIS;
     }
 
     /**
+     * Update schema ID when new field is added.
      *
+     * @param schemaId Current schema ID.
+     * @param fieldId Field ID.
+     * @return New schema ID.
      */
-    static {
-        PORTABLE_CLS.add(Byte.class);
-        PORTABLE_CLS.add(Short.class);
-        PORTABLE_CLS.add(Integer.class);
-        PORTABLE_CLS.add(Long.class);
-        PORTABLE_CLS.add(Float.class);
-        PORTABLE_CLS.add(Double.class);
-        PORTABLE_CLS.add(Character.class);
-        PORTABLE_CLS.add(Boolean.class);
-        PORTABLE_CLS.add(String.class);
-        PORTABLE_CLS.add(UUID.class);
-        PORTABLE_CLS.add(Date.class);
-        PORTABLE_CLS.add(Timestamp.class);
-        PORTABLE_CLS.add(BigDecimal.class);
-        PORTABLE_CLS.add(byte[].class);
-        PORTABLE_CLS.add(short[].class);
-        PORTABLE_CLS.add(int[].class);
-        PORTABLE_CLS.add(long[].class);
-        PORTABLE_CLS.add(float[].class);
-        PORTABLE_CLS.add(double[].class);
-        PORTABLE_CLS.add(char[].class);
-        PORTABLE_CLS.add(boolean[].class);
-        PORTABLE_CLS.add(String[].class);
-        PORTABLE_CLS.add(UUID[].class);
-        PORTABLE_CLS.add(Date[].class);
-        PORTABLE_CLS.add(Timestamp[].class);
-        PORTABLE_CLS.add(BigDecimal[].class);
+    public static int updateSchemaId(int schemaId, int fieldId) {
+        schemaId = schemaId ^ (fieldId & 0xFF);
+        schemaId = schemaId * FNV1_PRIME;
+        schemaId = schemaId ^ ((fieldId >> 8) & 0xFF);
+        schemaId = schemaId * FNV1_PRIME;
+        schemaId = schemaId ^ ((fieldId >> 16) & 0xFF);
+        schemaId = schemaId * FNV1_PRIME;
+        schemaId = schemaId ^ ((fieldId >> 24) & 0xFF);
+        schemaId = schemaId * FNV1_PRIME;
+
+        return schemaId;
     }
 
     /**
-     *
+     * @param typeName Field type name.
+     * @return Field type ID;
      */
-    static {
-        PLAIN_CLASS_TO_FLAG.put(Byte.class, GridPortableMarshaller.BYTE);
-        PLAIN_CLASS_TO_FLAG.put(Short.class, GridPortableMarshaller.SHORT);
-        PLAIN_CLASS_TO_FLAG.put(Integer.class, GridPortableMarshaller.INT);
-        PLAIN_CLASS_TO_FLAG.put(Long.class, GridPortableMarshaller.LONG);
-        PLAIN_CLASS_TO_FLAG.put(Float.class, GridPortableMarshaller.FLOAT);
-        PLAIN_CLASS_TO_FLAG.put(Double.class, GridPortableMarshaller.DOUBLE);
-        PLAIN_CLASS_TO_FLAG.put(Character.class, GridPortableMarshaller.CHAR);
-        PLAIN_CLASS_TO_FLAG.put(Boolean.class, GridPortableMarshaller.BOOLEAN);
-        PLAIN_CLASS_TO_FLAG.put(BigDecimal.class, GridPortableMarshaller.DECIMAL);
-        PLAIN_CLASS_TO_FLAG.put(String.class, GridPortableMarshaller.STRING);
-        PLAIN_CLASS_TO_FLAG.put(UUID.class, GridPortableMarshaller.UUID);
-        PLAIN_CLASS_TO_FLAG.put(Date.class, GridPortableMarshaller.DATE);
-        PLAIN_CLASS_TO_FLAG.put(Timestamp.class, GridPortableMarshaller.TIMESTAMP);
+    @SuppressWarnings("StringEquality")
+    public static int fieldTypeId(String typeName) {
+        for (int i = 0; i < FIELD_TYPE_NAMES.length; i++) {
+            String typeName0 = FIELD_TYPE_NAMES[i];
 
-        PLAIN_CLASS_TO_FLAG.put(byte[].class, GridPortableMarshaller.BYTE_ARR);
-        PLAIN_CLASS_TO_FLAG.put(short[].class, GridPortableMarshaller.SHORT_ARR);
-        PLAIN_CLASS_TO_FLAG.put(int[].class, GridPortableMarshaller.INT_ARR);
-        PLAIN_CLASS_TO_FLAG.put(long[].class, GridPortableMarshaller.LONG_ARR);
-        PLAIN_CLASS_TO_FLAG.put(float[].class, GridPortableMarshaller.FLOAT_ARR);
-        PLAIN_CLASS_TO_FLAG.put(double[].class, GridPortableMarshaller.DOUBLE_ARR);
-        PLAIN_CLASS_TO_FLAG.put(char[].class, GridPortableMarshaller.CHAR_ARR);
-        PLAIN_CLASS_TO_FLAG.put(boolean[].class, GridPortableMarshaller.BOOLEAN_ARR);
-        PLAIN_CLASS_TO_FLAG.put(BigDecimal[].class, GridPortableMarshaller.DECIMAL_ARR);
-        PLAIN_CLASS_TO_FLAG.put(String[].class, GridPortableMarshaller.STRING_ARR);
-        PLAIN_CLASS_TO_FLAG.put(UUID[].class, GridPortableMarshaller.UUID_ARR);
-        PLAIN_CLASS_TO_FLAG.put(Date[].class, GridPortableMarshaller.DATE_ARR);
-        PLAIN_CLASS_TO_FLAG.put(Timestamp[].class, GridPortableMarshaller.TIMESTAMP_ARR);
+            if (typeName.equals(typeName0))
+                return i;
+        }
 
-        for (Map.Entry<Class<?>, Byte> entry : PLAIN_CLASS_TO_FLAG.entrySet())
-            FLAG_TO_CLASS.put(entry.getValue(), entry.getKey());
+        throw new IllegalArgumentException("Invalid metadata type name: " + typeName);
+    }
 
-        PLAIN_CLASS_TO_FLAG.put(byte.class, GridPortableMarshaller.BYTE);
-        PLAIN_CLASS_TO_FLAG.put(short.class, GridPortableMarshaller.SHORT);
-        PLAIN_CLASS_TO_FLAG.put(int.class, GridPortableMarshaller.INT);
-        PLAIN_CLASS_TO_FLAG.put(long.class, GridPortableMarshaller.LONG);
-        PLAIN_CLASS_TO_FLAG.put(float.class, GridPortableMarshaller.FLOAT);
-        PLAIN_CLASS_TO_FLAG.put(double.class, GridPortableMarshaller.DOUBLE);
-        PLAIN_CLASS_TO_FLAG.put(char.class, GridPortableMarshaller.CHAR);
-        PLAIN_CLASS_TO_FLAG.put(boolean.class, GridPortableMarshaller.BOOLEAN);
+    /**
+     * @param typeId Field type ID.
+     * @return Field type name.
+     */
+    public static String fieldTypeName(int typeId) {
+        assert typeId >= 0 && typeId < FIELD_TYPE_NAMES.length : typeId;
 
-        for (byte b : new byte[] {
-            BYTE, SHORT, INT, LONG, FLOAT, DOUBLE,
-            CHAR, BOOLEAN, DECIMAL, STRING, UUID, DATE, TIMESTAMP,
-            BYTE_ARR, SHORT_ARR, INT_ARR, LONG_ARR, FLOAT_ARR, DOUBLE_ARR,
-            CHAR_ARR, BOOLEAN_ARR, DECIMAL_ARR, STRING_ARR, UUID_ARR, DATE_ARR, TIMESTAMP_ARR,
-            ENUM, ENUM_ARR, NULL}) {
+        String typeName = FIELD_TYPE_NAMES[typeId];
 
-            PLAIN_TYPE_FLAG[b] = true;
-        }
+        assert typeName != null : typeId;
+
+        return typeName;
     }
 
     /**
@@ -623,18 +663,16 @@ public class PortableUtils {
      * Write portable header.
      *
      * @param writer Writer.
-     * @param usrTyp User type flag.
      * @param typeId Type ID.
      * @param hashCode Hash code.
      * @param clsName Class name (optional).
      * @return Position where length should be written.
      */
-    public static int writeHeader(BinaryWriterExImpl writer, boolean usrTyp, int typeId, int hashCode,
-        @Nullable String clsName) {
+    public static int writeHeader(BinaryWriterExImpl writer, int typeId, int hashCode, @Nullable String clsName) {
         writer.doWriteByte(GridPortableMarshaller.OBJ);
         writer.doWriteByte(GridPortableMarshaller.PROTO_VER);
 
-        PortableUtils.writeFlags(writer, usrTyp);
+        writer.doWriteShort((short) 0);
 
         writer.doWriteInt(typeId);
         writer.doWriteInt(hashCode);
@@ -668,12 +706,12 @@ public class PortableUtils {
     public static int footerStartRelative(PortablePositionReadable in, int start) {
         short flags = in.readShortPositioned(start + GridPortableMarshaller.FLAGS_POS);
 
-        if (PortableUtils.isRawOnly(flags))
-            // No schema, footer start equals to object end.
-            return length(in, start);
-        else
+        if (hasSchema(flags))
             // Schema exists, use offset.
             return in.readIntPositioned(start + GridPortableMarshaller.SCHEMA_OR_RAW_OFF_POS);
+        else
+            // No schema, footer start equals to object end.
+            return length(in, start);
     }
 
     /**
@@ -692,56 +730,73 @@ public class PortableUtils {
      *
      * @param in Input stream.
      * @param start Start position.
-     * @param fieldOffsetSize Field offset size.
      * @return Footer.
      */
-    public static IgniteBiTuple<Integer, Integer> footerAbsolute(PortablePositionReadable in, int start,
-        int fieldOffsetSize) {
-        int footerStart = footerStartRelative(in, start);
+    public static IgniteBiTuple<Integer, Integer> footerAbsolute(PortablePositionReadable in, int start) {
+        short flags = in.readShortPositioned(start + GridPortableMarshaller.FLAGS_POS);
+
         int footerEnd = length(in, start);
 
-        // Take in count possible raw offset.
-        if ((footerEnd - footerStart) % (4 + fieldOffsetSize) != 0)
-            footerEnd -= 4;
+        if (hasSchema(flags)) {
+            // Schema exists.
+            int footerStart = in.readIntPositioned(start + GridPortableMarshaller.SCHEMA_OR_RAW_OFF_POS);
+
+            if (hasRaw(flags))
+                footerEnd -= 4;
 
-        return F.t(start + footerStart, start + footerEnd);
+            assert footerStart <= footerEnd;
+
+            return F.t(start + footerStart, start + footerEnd);
+        }
+        else
+            // No schema.
+            return F.t(start + footerEnd, start + footerEnd);
     }
 
     /**
-     * Get raw offset of the object.
+     * Get relative raw offset of the object.
      *
      * @param in Input stream.
      * @param start Object start position inside the stream.
-     * @param fieldOffsetSize Field offset size.
      * @return Raw offset.
      */
-    public static int rawOffsetAbsolute(PortablePositionReadable in, int start, int fieldOffsetSize) {
-        int len = length(in, start);
-
+    public static int rawOffsetRelative(PortablePositionReadable in, int start) {
         short flags = in.readShortPositioned(start + GridPortableMarshaller.FLAGS_POS);
 
-        if (PortableUtils.isRawOnly(flags))
-            // No schema, raw offset is located on schema offset position.
-            return start + in.readIntPositioned(start + GridPortableMarshaller.SCHEMA_OR_RAW_OFF_POS);
-        else {
-            // Schema exists.
-            int schemaOff = in.readIntPositioned(start + GridPortableMarshaller.SCHEMA_OR_RAW_OFF_POS);
+        int len = length(in, start);
 
-            if (((len - schemaOff) % (4 + fieldOffsetSize)) == 0x0)
-                // Even amount of records in schema => no raw offset.
-                return start + schemaOff;
+        if (hasSchema(flags)){
+            // Schema exists.
+            if (hasRaw(flags))
+                // Raw offset is set, it is at the very end of the object.
+                return in.readIntPositioned(start + len - 4);
             else
-                // Odd amount of records in schema => raw offset is the very last 4 bytes in object.
-                return start + in.readIntPositioned(start + len - 4);
+                // Raw offset is not set, so just return schema offset.
+                return in.readIntPositioned(start + GridPortableMarshaller.SCHEMA_OR_RAW_OFF_POS);
         }
+        else
+            // No schema, raw offset is located on schema offset position.
+            return in.readIntPositioned(start + GridPortableMarshaller.SCHEMA_OR_RAW_OFF_POS);
+    }
+
+    /**
+     * Get absolute raw offset of the object.
+     *
+     * @param in Input stream.
+     * @param start Object start position inside the stream.
+     * @return Raw offset.
+     */
+    public static int rawOffsetAbsolute(PortablePositionReadable in, int start) {
+        return start + rawOffsetRelative(in, start);
     }
 
     /**
-     * Get offset size for the given flags.
+     * Get offset length for the given flags.
+     *
      * @param flags Flags.
      * @return Offset size.
      */
-    public static int fieldOffsetSize(short flags) {
+    public static int fieldOffsetLength(short flags) {
         if ((flags & FLAG_OFFSET_ONE_BYTE) == FLAG_OFFSET_ONE_BYTE)
             return OFFSET_1;
         else if ((flags & FLAG_OFFSET_TWO_BYTES) == FLAG_OFFSET_TWO_BYTES)
@@ -751,6 +806,16 @@ public class PortableUtils {
     }
 
     /**
+     * Get field ID length.
+     *
+     * @param flags Flags.
+     * @return Field ID length.
+     */
+    public static int fieldIdLength(short flags) {
+        return isCompactFooter(flags) ? 0 : FIELD_ID_LEN;
+    }
+
+    /**
      * Get relative field offset.
      *
      * @param stream Stream.
@@ -770,4 +835,72 @@ public class PortableUtils {
 
         return res;
     }
+
+    /**
+     * Merge old and new metas.
+     *
+     * @param oldMeta Old meta.
+     * @param newMeta New meta.
+     * @return New meta if old meta was null, old meta if no changes detected, merged meta otherwise.
+     * @throws BinaryObjectException If merge failed due to metadata conflict.
+     */
+    public static BinaryMetadata mergeMetadata(@Nullable BinaryMetadata oldMeta, BinaryMetadata newMeta) {
+        assert newMeta != null;
+
+        if (oldMeta == null)
+            return newMeta;
+        else {
+            assert oldMeta.typeId() == newMeta.typeId();
+
+            // Check type name.
+            if (!F.eq(oldMeta.typeName(), newMeta.typeName())) {
+                throw new BinaryObjectException(
+                    "Two portable types have duplicate type ID [" + "typeId=" + oldMeta.typeId() +
+                        ", typeName1=" + oldMeta.typeName() + ", typeName2=" + newMeta.typeName() + ']'
+                );
+            }
+
+            // Check affinity field names.
+            if (!F.eq(oldMeta.affinityKeyFieldName(), newMeta.affinityKeyFieldName())) {
+                throw new BinaryObjectException(
+                    "Binary type has different affinity key fields [" + "typeName=" + newMeta.typeName() +
+                        ", affKeyFieldName1=" + oldMeta.affinityKeyFieldName() +
+                        ", affKeyFieldName2=" + newMeta.affinityKeyFieldName() + ']'
+                );
+            }
+
+            // Check and merge fields.
+            boolean changed = false;
+
+            Map<String, Integer> mergedFields = new HashMap<>(oldMeta.fieldsMap());
+            Map<String, Integer> newFields = newMeta.fieldsMap();
+
+            for (Map.Entry<String, Integer> newField : newFields.entrySet()) {
+                Integer oldFieldType = mergedFields.put(newField.getKey(), newField.getValue());
+
+                if (oldFieldType == null)
+                    changed = true;
+                else if (!F.eq(oldFieldType, newField.getValue())) {
+                    throw new BinaryObjectException(
+                        "Binary type has different field types [" + "typeName=" + oldMeta.typeName() +
+                            ", fieldName=" + newField.getKey() +
+                            ", fieldTypeName1=" + PortableUtils.fieldTypeName(oldFieldType) +
+                            ", fieldTypeName2=" + PortableUtils.fieldTypeName(newField.getValue()) + ']'
+                    );
+                }
+            }
+
+            // Check and merge schemas.
+            Collection<PortableSchema> mergedSchemas = new HashSet<>(oldMeta.schemas());
+
+            for (PortableSchema newSchema : newMeta.schemas()) {
+                if (mergedSchemas.add(newSchema))
+                    changed = true;
+            }
+
+            // Return either old meta if no changes detected, or new merged meta.
+            return changed ? new BinaryMetadata(oldMeta.typeId(), oldMeta.typeName(), mergedFields,
+                oldMeta.affinityKeyFieldName(), mergedSchemas) : oldMeta;
+        }
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/BinaryObjectBuilderImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/BinaryObjectBuilderImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/BinaryObjectBuilderImpl.java
index ca8f09b..dfc2330 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/BinaryObjectBuilderImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/BinaryObjectBuilderImpl.java
@@ -22,11 +22,14 @@ import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryObjectBuilder;
 import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.binary.BinaryType;
+import org.apache.ignite.internal.portable.BinaryMetadata;
 import org.apache.ignite.internal.portable.BinaryObjectImpl;
 import org.apache.ignite.internal.portable.BinaryObjectOffheapImpl;
 import org.apache.ignite.internal.portable.BinaryWriterExImpl;
 import org.apache.ignite.internal.portable.GridPortableMarshaller;
 import org.apache.ignite.internal.portable.PortableContext;
+import org.apache.ignite.internal.portable.PortableSchema;
+import org.apache.ignite.internal.portable.PortableSchemaRegistry;
 import org.apache.ignite.internal.portable.PortableUtils;
 import org.apache.ignite.internal.util.GridArgumentCheck;
 import org.apache.ignite.internal.util.typedef.F;
@@ -176,7 +179,6 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
     /** {@inheritDoc} */
     @Override public BinaryObject build() {
         try (BinaryWriterExImpl writer = new BinaryWriterExImpl(ctx, typeId, false)) {
-
             PortableBuilderSerializer serializationCtx = new PortableBuilderSerializer();
 
             serializationCtx.registerObjectWriting(this, 0);
@@ -196,50 +198,57 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
     void serializeTo(BinaryWriterExImpl writer, PortableBuilderSerializer serializer) {
         try {
             PortableUtils.writeHeader(writer,
-                true,
                 registeredType ? typeId : UNREGISTERED_TYPE_ID,
                 hashCode,
-                registeredType ? null : clsNameToWrite);
+                registeredType ? null : clsNameToWrite
+            );
 
             Set<Integer> remainsFlds = null;
 
             if (reader != null) {
+                PortableSchema schema = reader.schema(start);
+
                 Map<Integer, Object> assignedFldsById;
 
                 if (assignedVals != null) {
                     assignedFldsById = U.newHashMap(assignedVals.size());
 
                     for (Map.Entry<String, Object> entry : assignedVals.entrySet()) {
-                        int fldId = ctx.fieldId(typeId, entry.getKey());
+                        int fieldId = ctx.fieldId(typeId, entry.getKey());
 
-                        assignedFldsById.put(fldId, entry.getValue());
+                        assignedFldsById.put(fieldId, entry.getValue());
                     }
 
                     remainsFlds = assignedFldsById.keySet();
-                } else
+                }
+                else
                     assignedFldsById = Collections.emptyMap();
 
                 // Get footer details.
-                int fieldOffsetSize = PortableUtils.fieldOffsetSize(flags);
+                int fieldIdLen = PortableUtils.fieldIdLength(flags);
+                int fieldOffsetLen = PortableUtils.fieldOffsetLength(flags);
 
-                IgniteBiTuple<Integer, Integer> footer = PortableUtils.footerAbsolute(reader, start, fieldOffsetSize);
+                IgniteBiTuple<Integer, Integer> footer = PortableUtils.footerAbsolute(reader, start);
 
                 int footerPos = footer.get1();
                 int footerEnd = footer.get2();
 
                 // Get raw position.
-                int rawPos = PortableUtils.rawOffsetAbsolute(reader, start, fieldOffsetSize);
+                int rawPos = PortableUtils.rawOffsetAbsolute(reader, start);
 
                 // Position reader on data.
                 reader.position(start + hdrLen);
 
-                while (reader.position() + 4 < rawPos) {
-                    int fieldId = reader.readIntPositioned(footerPos);
-                    int fieldLen = fieldPositionAndLength(footerPos, footerEnd, rawPos, fieldOffsetSize).get2();
+                int idx = 0;
+
+                while (reader.position() < rawPos) {
+                    int fieldId = schema.fieldId(idx++);
+                    int fieldLen =
+                        fieldPositionAndLength(footerPos, footerEnd, rawPos, fieldIdLen, fieldOffsetLen).get2();
 
                     int postPos = reader.position() + fieldLen; // Position where reader will be placed afterwards.
 
-                    footerPos += 4 + fieldOffsetSize;
+                    footerPos += fieldIdLen + fieldOffsetLen;
 
                     if (assignedFldsById.containsKey(fieldId)) {
                         Object assignedVal = assignedFldsById.remove(fieldId);
@@ -281,11 +290,11 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
                 }
             }
 
-            if (assignedVals != null && (remainsFlds == null || !remainsFlds.isEmpty())) {
-                BinaryType metadata = ctx.metaData(typeId);
+            BinaryType meta = ctx.metadata(typeId);
 
-                Map<String, Integer> newFldsMetadata = null;
+            Map<String, Integer> fieldsMeta = null;
 
+            if (assignedVals != null && (remainsFlds == null || !remainsFlds.isEmpty())) {
                 for (Map.Entry<String, Object> entry : assignedVals.entrySet()) {
                     Object val = entry.getValue();
 
@@ -294,16 +303,16 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
 
                     String name = entry.getKey();
 
-                    int fldId = ctx.fieldId(typeId, name);
+                    int fieldId = ctx.fieldId(typeId, name);
 
-                    if (remainsFlds != null && !remainsFlds.contains(fldId))
+                    if (remainsFlds != null && !remainsFlds.contains(fieldId))
                         continue;
 
-                    writer.writeFieldId(fldId);
+                    writer.writeFieldId(fieldId);
 
                     serializer.writeValue(writer, val);
 
-                    String oldFldTypeName = metadata == null ? null : metadata.fieldTypeName(name);
+                    String oldFldTypeName = meta == null ? null : meta.fieldTypeName(name);
 
                     int newFldTypeId;
 
@@ -316,11 +325,10 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
 
                     if (oldFldTypeName == null) {
                         // It's a new field, we have to add it to metadata.
+                        if (fieldsMeta == null)
+                            fieldsMeta = new HashMap<>();
 
-                        if (newFldsMetadata == null)
-                            newFldsMetadata = new HashMap<>();
-
-                        newFldsMetadata.put(name, PortableUtils.fieldTypeId(newFldTypeName));
+                        fieldsMeta.put(name, PortableUtils.fieldTypeId(newFldTypeName));
                     }
                     else {
                         String objTypeName = PortableUtils.fieldTypeName(GridPortableMarshaller.OBJ);
@@ -328,7 +336,7 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
                         if (!objTypeName.equals(oldFldTypeName) && !oldFldTypeName.equals(newFldTypeName)) {
                             throw new BinaryObjectException(
                                 "Wrong value has been set [" +
-                                    "typeName=" + (typeName == null ? metadata.typeName() : typeName) +
+                                    "typeName=" + (typeName == null ? meta.typeName() : typeName) +
                                     ", fieldName=" + name +
                                     ", fieldType=" + oldFldTypeName +
                                     ", assignedValueType=" + newFldTypeName + ']'
@@ -336,25 +344,11 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
                         }
                     }
                 }
-
-                if (newFldsMetadata != null) {
-                    String typeName = this.typeName;
-
-                    if (typeName == null) {
-                        assert metadata != null;
-
-                        typeName = metadata.typeName();
-                    }
-
-                    ctx.updateMetaData(typeId, typeName, newFldsMetadata);
-                }
             }
 
             if (reader != null) {
                 // Write raw data if any.
-                int fieldOffsetSize = PortableUtils.fieldOffsetSize(flags);
-
-                int rawOff = PortableUtils.rawOffsetAbsolute(reader, start, fieldOffsetSize);
+                int rawOff = PortableUtils.rawOffsetAbsolute(reader, start);
                 int footerStart = PortableUtils.footerStartAbsolute(reader, start);
 
                 if (rawOff < footerStart) {
@@ -368,6 +362,28 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
             }
 
             writer.postWrite(true);
+
+            // Update metadata if needed.
+            int schemaId = writer.schemaId();
+
+            PortableSchemaRegistry schemaReg = ctx.schemaRegistry(typeId);
+
+            if (schemaReg.schema(schemaId) == null) {
+                String typeName = this.typeName;
+
+                if (typeName == null) {
+                    assert meta != null;
+
+                    typeName = meta.typeName();
+                }
+
+                PortableSchema curSchema = writer.currentSchema();
+
+                ctx.updateMetadata(typeId, new BinaryMetadata(typeId, typeName, fieldsMeta,
+                    ctx.affinityKeyFieldName(typeId), Collections.singleton(curSchema)));
+
+                schemaReg.addSchema(curSchema.schemaId(), curSchema);
+            }
         }
         finally {
             writer.popSchema();
@@ -387,25 +403,26 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
      * @param footerPos Field position inside the footer (absolute).
      * @param footerEnd Footer end (absolute).
      * @param rawPos Raw data position (absolute).
-     * @param fieldOffsetSize Size of field's offset.
+     * @param fieldIdLen Field ID length.
+     * @param fieldOffsetLen Field offset length.
      * @return Tuple with field position and length.
      */
     private IgniteBiTuple<Integer, Integer> fieldPositionAndLength(int footerPos, int footerEnd, int rawPos,
-        int fieldOffsetSize) {
+        int fieldIdLen, int fieldOffsetLen) {
         // Get field offset first.
-        int fieldOffset = PortableUtils.fieldOffsetRelative(reader, footerPos + 4, fieldOffsetSize);
+        int fieldOffset = PortableUtils.fieldOffsetRelative(reader, footerPos + fieldIdLen, fieldOffsetLen);
         int fieldPos = start + fieldOffset;
 
         // Get field length.
         int fieldLen;
 
-        if (footerPos + 4 + fieldOffsetSize == footerEnd)
+        if (footerPos + fieldIdLen + fieldOffsetLen == footerEnd)
             // This is the last field, compare to raw offset.
             fieldLen = rawPos - fieldPos;
         else {
             // Field is somewhere in the middle, get difference with the next offset.
-            int nextFieldOffset = PortableUtils.fieldOffsetRelative(reader, footerPos + 4 + fieldOffsetSize + 4,
-                fieldOffsetSize);
+            int nextFieldOffset = PortableUtils.fieldOffsetRelative(reader,
+                footerPos + fieldIdLen + fieldOffsetLen + fieldIdLen, fieldOffsetLen);
 
             fieldLen = nextFieldOffset - fieldOffset;
         }
@@ -417,30 +434,37 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
      * Initialize read cache if needed.
      */
     private void ensureReadCacheInit() {
+        assert reader != null;
+
         if (readCache == null) {
-            int fieldOffsetSize = PortableUtils.fieldOffsetSize(flags);
+            int fieldIdLen = PortableUtils.fieldIdLength(flags);
+            int fieldOffsetLen = PortableUtils.fieldOffsetLength(flags);
+
+            PortableSchema schema = reader.schema(start);
 
             Map<Integer, Object> readCache = new HashMap<>();
 
-            IgniteBiTuple<Integer, Integer> footer = PortableUtils.footerAbsolute(reader, start, fieldOffsetSize);
+            IgniteBiTuple<Integer, Integer> footer = PortableUtils.footerAbsolute(reader, start);
 
             int footerPos = footer.get1();
             int footerEnd = footer.get2();
 
-            int rawPos = PortableUtils.rawOffsetAbsolute(reader, start, fieldOffsetSize);
+            int rawPos = PortableUtils.rawOffsetAbsolute(reader, start);
+
+            int idx = 0;
 
-            while (footerPos + 4 < footerEnd) {
-                int fieldId = reader.readIntPositioned(footerPos);
+            while (footerPos + fieldIdLen < footerEnd) {
+                int fieldId = schema.fieldId(idx++);
 
                 IgniteBiTuple<Integer, Integer> posAndLen =
-                    fieldPositionAndLength(footerPos, footerEnd, rawPos, fieldOffsetSize);
+                    fieldPositionAndLength(footerPos, footerEnd, rawPos, fieldIdLen, fieldOffsetLen);
 
                 Object val = reader.getValueQuickly(posAndLen.get1(), posAndLen.get2());
 
                 readCache.put(fieldId, val);
 
                 // Shift current footer position.
-                footerPos += 4 + fieldOffsetSize;
+                footerPos += fieldIdLen + fieldOffsetLen;
             }
 
             this.readCache = readCache;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
index 5c6a131..b6a6b54 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
@@ -27,6 +27,7 @@ import org.apache.ignite.internal.portable.PortablePositionReadable;
 import org.apache.ignite.internal.portable.BinaryObjectImpl;
 import org.apache.ignite.internal.portable.PortablePrimitives;
 import org.apache.ignite.internal.portable.BinaryReaderExImpl;
+import org.apache.ignite.internal.portable.PortableSchema;
 import org.apache.ignite.internal.portable.PortableUtils;
 import org.apache.ignite.internal.portable.BinaryWriterExImpl;
 import org.apache.ignite.binary.BinaryObjectException;
@@ -63,7 +64,7 @@ public class PortableBuilderReader implements PortablePositionReadable {
         pos = objImpl.start();
 
         // TODO: IGNITE-1272 - Is class loader needed here?
-        reader = new BinaryReaderExImpl(portableContext(), arr, pos, null);
+        reader = new BinaryReaderExImpl(ctx, arr, pos, null);
     }
 
     /**
@@ -81,6 +82,24 @@ public class PortableBuilderReader implements PortablePositionReadable {
     }
 
     /**
+     * Get schema of the object, starting at the given position.
+     *
+     * @param start Start position.
+     * @return Object's schema.
+     */
+    public PortableSchema schema(int start) {
+        // We can use current reader in case start is equal to initially recorded position.
+        BinaryReaderExImpl targetReader;
+
+        if (start == pos)
+            targetReader = reader;
+        else
+            targetReader = new BinaryReaderExImpl(ctx, arr, start, null);
+
+        return targetReader.getOrCreateSchema();
+    }
+
+    /**
      * @return Read int value.
      */
     public int readInt() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessor.java
index cac0dcf..e4db77c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessor.java
@@ -59,7 +59,7 @@ public interface CacheObjectBinaryProcessor extends IgniteCacheObjectProcessor {
      * @param fieldTypeIds Fields map.
      * @throws IgniteException In case of error.
      */
-    public void updateMetaData(int typeId, String typeName, @Nullable String affKeyFieldName,
+    public void updateMetadata(int typeId, String typeName, @Nullable String affKeyFieldName,
         Map<String, Integer> fieldTypeIds) throws IgniteException;
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/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 117eece..551ada5 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
@@ -31,12 +31,12 @@ import org.apache.ignite.configuration.CacheConfiguration;
 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.BinaryObjectImpl;
 import org.apache.ignite.internal.portable.BinaryObjectOffheapImpl;
 import org.apache.ignite.internal.portable.BinaryTypeImpl;
 import org.apache.ignite.internal.portable.GridPortableMarshaller;
 import org.apache.ignite.internal.portable.PortableContext;
-import org.apache.ignite.internal.portable.BinaryMetadataHandler;
 import org.apache.ignite.internal.portable.PortableUtils;
 import org.apache.ignite.internal.portable.builder.BinaryObjectBuilderImpl;
 import org.apache.ignite.internal.portable.streams.PortableInputStream;
@@ -88,7 +88,6 @@ import java.io.ObjectInput;
 import java.io.ObjectOutput;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
@@ -168,17 +167,14 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
 
                     if (metaDataCache == null) {
                         BinaryMetadata oldMeta = metaBuf.get(typeId);
+                        BinaryMetadata mergedMeta = PortableUtils.mergeMetadata(oldMeta, newMeta0);
 
-                        if (oldMeta == null || checkMeta(typeId, oldMeta, newMeta0, null)) {
+                        if (oldMeta != mergedMeta) {
                             synchronized (this) {
-                                Map<String, Integer> fields = new HashMap<>();
-
-                                if (checkMeta(typeId, oldMeta, newMeta0, fields)) {
-                                    newMeta0 = new BinaryMetadata(typeId, newMeta0.typeName(), fields,
-                                        newMeta0.affinityKeyFieldName());
+                                mergedMeta = PortableUtils.mergeMetadata(oldMeta, newMeta0);
 
-                                    metaBuf.put(typeId, newMeta0);
-                                }
+                                if (oldMeta != mergedMeta)
+                                    metaBuf.put(typeId, mergedMeta);
                                 else
                                     return;
                             }
@@ -192,6 +188,8 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
                             return;
                     }
 
+                    assert metaDataCache != null;
+
                     CacheObjectBinaryProcessorImpl.this.addMeta(typeId, newMeta0.wrap(portableCtx));
                 }
 
@@ -297,24 +295,22 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
     private void addClientCacheMetaData(PortableMetadataKey key, final BinaryMetadata newMeta) {
         int key0 = key.typeId();
 
-        clientMetaDataCache.compute(key0,
-            new ConcurrentHashMap8.BiFun<Integer, BinaryTypeImpl, BinaryTypeImpl>() {
-                @Override public BinaryTypeImpl apply(Integer key, BinaryTypeImpl oldMeta) {
-                    BinaryMetadata res;
+        clientMetaDataCache.compute(key0, new ConcurrentHashMap8.BiFun<Integer, BinaryTypeImpl, BinaryTypeImpl>() {
+            @Override public BinaryTypeImpl apply(Integer key, BinaryTypeImpl oldMeta) {
+                BinaryMetadata res;
 
-                    BinaryMetadata oldMeta0 = oldMeta != null ? oldMeta.metadata() : null;
+                BinaryMetadata oldMeta0 = oldMeta != null ? oldMeta.metadata() : null;
 
-                    try {
-                        res = checkMeta(key, oldMeta0, newMeta, null) ? newMeta : oldMeta0;
-                    }
-                    catch (BinaryObjectException e) {
-                        res = oldMeta0;
-                    }
-
-                    return res != null ? res.wrap(portableCtx) : null;
+                try {
+                    res = PortableUtils.mergeMetadata(oldMeta0, newMeta);
+                }
+                catch (BinaryObjectException e) {
+                    res = oldMeta0;
                 }
+
+                return res != null ? res.wrap(portableCtx) : null;
             }
-        );
+        });
     }
 
     /** {@inheritDoc} */
@@ -448,9 +444,9 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
     }
 
     /** {@inheritDoc} */
-    @Override public void updateMetaData(int typeId, String typeName, @Nullable String affKeyFieldName,
+    @Override public void updateMetadata(int typeId, String typeName, @Nullable String affKeyFieldName,
         Map<String, Integer> fieldTypeIds) throws BinaryObjectException {
-        portableCtx.updateMetaData(typeId, new BinaryMetadata(typeId, typeName, fieldTypeIds, affKeyFieldName));
+        portableCtx.updateMetadata(typeId, new BinaryMetadata(typeId, typeName, fieldTypeIds, affKeyFieldName, null));
     }
 
     /** {@inheritDoc} */
@@ -464,13 +460,12 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
 
         try {
             BinaryMetadata oldMeta = metaDataCache.localPeek(key);
+            BinaryMetadata mergedMeta = PortableUtils.mergeMetadata(oldMeta, newMeta0);
 
-            if (oldMeta == null || checkMeta(typeId, oldMeta, newMeta0, null)) {
-                BinaryObjectException err = metaDataCache.invoke(key, new MetaDataProcessor(typeId, newMeta0));
+            BinaryObjectException err = metaDataCache.invoke(key, new MetadataProcessor(mergedMeta));
 
-                if (err != null)
-                    throw err;
-            }
+            if (err != null)
+                throw err;
         }
         catch (CacheException e) {
             throw new BinaryObjectException("Failed to update meta data for type: " + newMeta.typeName(), e);
@@ -727,125 +722,44 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
     }
 
     /**
-     * @param typeId Type ID.
-     * @param oldMeta Old meta.
-     * @param newMeta New meta.
-     * @param fields Fields map.
-     * @return Whether meta is changed.
-     * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
-     */
-    private static boolean checkMeta(int typeId, @Nullable BinaryMetadata oldMeta,
-        BinaryMetadata newMeta, @Nullable Map<String, Integer> fields) throws BinaryObjectException {
-        assert newMeta != null;
-
-        Map<String, Integer> oldFields = oldMeta != null ? oldMeta.fieldsMap() : null;
-        Map<String, Integer> newFields = newMeta.fieldsMap();
-
-        boolean changed = false;
-
-        if (oldMeta != null) {
-            if (!oldMeta.typeName().equals(newMeta.typeName())) {
-                throw new BinaryObjectException(
-                    "Two portable types have duplicate type ID [" +
-                        "typeId=" + typeId +
-                        ", typeName1=" + oldMeta.typeName() +
-                        ", typeName2=" + newMeta.typeName() +
-                        ']'
-                );
-            }
-
-            if (!F.eq(oldMeta.affinityKeyFieldName(), newMeta.affinityKeyFieldName())) {
-                throw new BinaryObjectException(
-                    "Portable type has different affinity key fields on different clients [" +
-                        "typeName=" + newMeta.typeName() +
-                        ", affKeyFieldName1=" + oldMeta.affinityKeyFieldName() +
-                        ", affKeyFieldName2=" + newMeta.affinityKeyFieldName() +
-                        ']'
-                );
-            }
-
-            if (fields != null)
-                fields.putAll(oldFields);
-        }
-        else
-            changed = true;
-
-        for (Map.Entry<String, Integer> e : newFields.entrySet()) {
-            Integer oldTypeId = oldFields != null ? oldFields.get(e.getKey()) : null;
-
-            if (oldTypeId != null) {
-                if (!oldTypeId.equals(e.getValue())) {
-                    throw new BinaryObjectException(
-                        "Portable field has different types on different clients [" +
-                            "typeName=" + newMeta.typeName() +
-                            ", fieldName=" + e.getKey() +
-                            ", fieldTypeName1=" + PortableUtils.fieldTypeName(oldTypeId) +
-                            ", fieldTypeName2=" + PortableUtils.fieldTypeName(e.getValue()) +
-                            ']'
-                    );
-                }
-            }
-            else {
-                if (fields != null)
-                    fields.put(e.getKey(), e.getValue());
-
-                changed = true;
-            }
-        }
-
-        return changed;
-    }
-
-    /**
+     * Processor responsible for metadata update.
      */
-    private static class MetaDataProcessor implements
-        EntryProcessor<PortableMetadataKey, BinaryMetadata, BinaryObjectException>, Externalizable {
+    private static class MetadataProcessor
+        implements EntryProcessor<PortableMetadataKey, BinaryMetadata, BinaryObjectException>, Externalizable {
         /** */
         private static final long serialVersionUID = 0L;
 
         /** */
-        private int typeId;
-
-        /** */
         private BinaryMetadata newMeta;
 
         /**
          * For {@link Externalizable}.
          */
-        public MetaDataProcessor() {
+        public MetadataProcessor() {
             // No-op.
         }
 
         /**
-         * @param typeId Type ID.
          * @param newMeta New metadata.
          */
-        private MetaDataProcessor(int typeId, BinaryMetadata newMeta) {
+        private MetadataProcessor(BinaryMetadata newMeta) {
             assert newMeta != null;
 
-            this.typeId = typeId;
             this.newMeta = newMeta;
         }
 
         /** {@inheritDoc} */
-        @Override public BinaryObjectException process(
-            MutableEntry<PortableMetadataKey, BinaryMetadata> entry,
+        @Override public BinaryObjectException process(MutableEntry<PortableMetadataKey, BinaryMetadata> entry,
             Object... args) {
             try {
                 BinaryMetadata oldMeta = entry.getValue();
 
-                Map<String, Integer> fields = new HashMap<>();
+                BinaryMetadata mergedMeta = PortableUtils.mergeMetadata(oldMeta, newMeta);
 
-                if (checkMeta(typeId, oldMeta, newMeta, fields)) {
-                    BinaryMetadata res = new BinaryMetadata(typeId, newMeta.typeName(), fields,
-                        newMeta.affinityKeyFieldName());
+                if (mergedMeta != oldMeta)
+                    entry.setValue(mergedMeta);
 
-                    entry.setValue(res);
-
-                    return null;
-                }
-                else
-                    return null;
+                return null;
             }
             catch (BinaryObjectException e) {
                 return e;
@@ -854,19 +768,17 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
 
         /** {@inheritDoc} */
         @Override public void writeExternal(ObjectOutput out) throws IOException {
-            out.writeInt(typeId);
             out.writeObject(newMeta);
         }
 
         /** {@inheritDoc} */
         @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            typeId = in.readInt();
             newMeta = (BinaryMetadata)in.readObject();
         }
 
         /** {@inheritDoc} */
         @Override public String toString() {
-            return S.toString(MetaDataProcessor.class, this);
+            return S.toString(MetadataProcessor.class, this);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
index 05d3515..d999466 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.platform;
 
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.binary.BinaryType;
 import org.apache.ignite.cluster.ClusterMetrics;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.events.CacheEvent;
@@ -33,10 +34,10 @@ import org.apache.ignite.events.JobEvent;
 import org.apache.ignite.events.SwapSpaceEvent;
 import org.apache.ignite.events.TaskEvent;
 import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.portable.GridPortableMarshaller;
-import org.apache.ignite.internal.portable.BinaryMetadata;
 import org.apache.ignite.internal.portable.BinaryRawReaderEx;
 import org.apache.ignite.internal.portable.BinaryRawWriterEx;
+import org.apache.ignite.internal.portable.BinaryTypeImpl;
+import org.apache.ignite.internal.portable.GridPortableMarshaller;
 import org.apache.ignite.internal.processors.cache.portable.CacheObjectBinaryProcessorImpl;
 import org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilter;
 import org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilterImpl;
@@ -69,7 +70,6 @@ import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T4;
 import org.apache.ignite.lang.IgniteBiTuple;
-import org.apache.ignite.binary.BinaryType;
 import org.jetbrains.annotations.Nullable;
 
 import java.sql.Timestamp;
@@ -359,7 +359,7 @@ public class PlatformContextImpl implements PlatformContext {
         );
 
         for (T4<Integer, String, String, Map<String, Integer>> meta : metas)
-            cacheObjProc.updateMetaData(meta.get1(), meta.get2(), meta.get3(), meta.get4());
+            cacheObjProc.updateMetadata(meta.get1(), meta.get2(), meta.get3(), meta.get4());
     }
 
     /** {@inheritDoc} */
@@ -390,7 +390,7 @@ public class PlatformContextImpl implements PlatformContext {
         else {
             writer.writeBoolean(true);
 
-            Map<String, Integer> fields = ((BinaryMetadata)meta).fieldsMap();
+            Map<String, Integer> fields = ((BinaryTypeImpl)meta).metadata().fieldsMap();
 
             writer.writeInt(typeId);
             writer.writeString(meta.typeName());

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationClosure.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationClosure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationClosure.java
index 9f17bdd..e9cd1e3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationClosure.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationClosure.java
@@ -71,7 +71,11 @@ public class PlatformCppConfigurationClosure extends PlatformAbstractConfigurati
         Marshaller marsh = igniteCfg.getMarshaller();
 
         if (marsh == null) {
-            igniteCfg.setMarshaller(new PortableMarshaller());
+            PortableMarshaller marsh0 = new PortableMarshaller();
+
+            marsh0.setCompactFooter(false);
+
+            igniteCfg.setMarshaller(marsh0);
 
             cppCfg0.warnings(Collections.singleton("Marshaller is automatically set to " +
                 PortableMarshaller.class.getName() + " (other nodes must have the same marshaller type)."));
@@ -79,6 +83,9 @@ public class PlatformCppConfigurationClosure extends PlatformAbstractConfigurati
         else if (!(marsh instanceof PortableMarshaller))
             throw new IgniteException("Unsupported marshaller (only " + PortableMarshaller.class.getName() +
                 " can be used when running Apache Ignite C++): " + marsh.getClass().getName());
+        else if (((PortableMarshaller)marsh).isCompactFooter())
+            throw new IgniteException("Unsupported " + PortableMarshaller.class.getName() +
+                " \"compactFooter\" flag: must be false when running Apache Ignite C++.");
 
         // Set Ignite home so that marshaller context works.
         String ggHome = igniteCfg.getIgniteHome();

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
index d0462e9..a59fd22 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
@@ -92,7 +92,11 @@ public class PlatformDotNetConfigurationClosure extends PlatformAbstractConfigur
         Marshaller marsh = igniteCfg.getMarshaller();
 
         if (marsh == null) {
-            igniteCfg.setMarshaller(new PortableMarshaller());
+            PortableMarshaller marsh0 = new PortableMarshaller();
+
+            marsh0.setCompactFooter(false);
+
+            igniteCfg.setMarshaller(marsh0);
 
             dotNetCfg0.warnings(Collections.singleton("Marshaller is automatically set to " +
                 PortableMarshaller.class.getName() + " (other nodes must have the same marshaller type)."));
@@ -100,6 +104,9 @@ public class PlatformDotNetConfigurationClosure extends PlatformAbstractConfigur
         else if (!(marsh instanceof PortableMarshaller))
             throw new IgniteException("Unsupported marshaller (only " + PortableMarshaller.class.getName() +
                 " can be used when running Apache Ignite.NET): " + marsh.getClass().getName());
+        else if (((PortableMarshaller)marsh).isCompactFooter())
+            throw new IgniteException("Unsupported " + PortableMarshaller.class.getName() +
+                " \"compactFooter\" flag: must be false when running Apache Ignite.NET.");
 
         // Set Ignite home so that marshaller context works.
         String ggHome = igniteCfg.getIgniteHome();

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index 259d8c9..7337378 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -4983,6 +4983,31 @@ public abstract class IgniteUtils {
     }
 
     /**
+     * Read hash map.
+     *
+     * @param in Input.
+     * @return Read map.
+     * @throws IOException If de-serialization failed.
+     * @throws ClassNotFoundException If deserialized class could not be found.
+     */
+    @SuppressWarnings({"unchecked"})
+    @Nullable public static <K, V> HashMap<K, V> readHashMap(ObjectInput in)
+        throws IOException, ClassNotFoundException {
+        int size = in.readInt();
+
+        // Check null flag.
+        if (size == -1)
+            return null;
+
+        HashMap<K, V> map = U.newHashMap(size);
+
+        for (int i = 0; i < size; i++)
+            map.put((K)in.readObject(), (V)in.readObject());
+
+        return map;
+    }
+
+    /**
      *
      * @param in Input.
      * @return Read map.

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java b/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
index 409a893..1704c8a 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
@@ -74,6 +74,12 @@ import org.jetbrains.annotations.Nullable;
  * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
  */
 public class PortableMarshaller extends AbstractMarshaller {
+    /** Default value of "keep deserialized" flag. */
+    public static final boolean DFLT_KEEP_DESERIALIZED = true;
+
+    /** Default value of "compact footer" flag. */
+    public static final boolean DFLT_COMPACT_FOOTER = true;
+
     // TODO ignite-1282 Move to IgniteConfiguration.
     /** Class names. */
     private Collection<String> clsNames;
@@ -88,7 +94,10 @@ public class PortableMarshaller extends AbstractMarshaller {
     private Collection<BinaryTypeConfiguration> typeCfgs;
 
     /** Keep deserialized flag. */
-    private boolean keepDeserialized = true;
+    private boolean keepDeserialized = DFLT_KEEP_DESERIALIZED;
+
+    /** Compact footer. */
+    private boolean compactFooter = DFLT_COMPACT_FOOTER;
 
     /** */
     private GridPortableMarshaller impl;
@@ -192,6 +201,33 @@ public class PortableMarshaller extends AbstractMarshaller {
     }
 
     /**
+     * Get whether to write footers in compact form. When enabled, Ignite will not write fields metadata
+     * when serializing objects, because internally {@code PortableMarshaller} already distribute metadata inside
+     * cluster. This increases serialization performance.
+     * <p>
+     * <b>WARNING!</b> This mode should be disabled when already serialized data can be taken from some external
+     * sources (e.g. cache store which stores data in binary form, data center replication, etc.). Otherwise binary
+     * objects without any associated metadata could appear in the cluster and Ignite will not be able to deserialize
+     * it.
+     * <p>
+     * Defaults to {@link #DFLT_COMPACT_FOOTER}.
+     *
+     * @return Whether to write footers in compact form.
+     */
+    public boolean isCompactFooter() {
+        return compactFooter;
+    }
+
+    /**
+     * Set whether to write footers in compact form. See {@link #isCompactFooter()} for more info.
+     *
+     * @param compactFooter Whether to write footers in compact form.
+     */
+    public void setCompactFooter(boolean compactFooter) {
+        this.compactFooter = compactFooter;
+    }
+
+    /**
      * Returns currently set {@link MarshallerContext}.
      *
      * @return Marshaller context.

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index ae23d0e..45c8e0f 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -135,6 +135,7 @@ import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
 import static org.apache.ignite.events.EventType.EVT_NODE_METRICS_UPDATED;
 import static org.apache.ignite.events.EventType.EVT_NODE_SEGMENTED;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_COMPACT_FOOTER;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_USE_DFLT_SUID;
 import static org.apache.ignite.spi.IgnitePortProtocol.TCP;
 import static org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoverySpiState.AUTH_FAILED;
@@ -3160,8 +3161,8 @@ class ServerImpl extends TcpDiscoveryImpl {
                             " property value differs from remote node's value " +
                             "(to make sure all nodes in topology have identical marshaller settings, " +
                             "configure system property explicitly) " +
-                            "[locMarshUseDfltSuid=" + locMarshUseDfltSuid +
-                            ", rmtMarshUseDfltSuid=" + rmtMarshUseDfltSuid +
+                            "[locMarshUseDfltSuid=" + rmtMarshUseDfltSuid +
+                            ", rmtMarshUseDfltSuid=" + locMarshUseDfltSuid +
                             ", locNodeAddrs=" + U.addressesAsString(node) + ", locPort=" + node.discoveryPort() +
                             ", rmtNodeAddr=" + U.addressesAsString(locNode) + ", locNodeId=" + node.id() +
                             ", rmtNodeId=" + locNode.id() + ']';
@@ -3182,6 +3183,52 @@ class ServerImpl extends TcpDiscoveryImpl {
                     return;
                 }
 
+                // Validate compact footer flags.
+                Boolean locMarshCompactFooter = locNode.attribute(ATTR_MARSHALLER_COMPACT_FOOTER);
+                boolean locMarshCompactFooterBool = locMarshCompactFooter != null ? locMarshCompactFooter : false;
+
+                Boolean rmtMarshCompactFooter = node.attribute(ATTR_MARSHALLER_COMPACT_FOOTER);
+                boolean rmtMarshCompactFooterBool = rmtMarshCompactFooter != null ? rmtMarshCompactFooter : false;
+
+                if (locMarshCompactFooterBool != rmtMarshCompactFooterBool) {
+                    String errMsg = "Local node's portable marshaller \"compactFooter\" property differs from " +
+                        "the same property on remote node (make sure all nodes in topology have the same value " +
+                        "of \"compactFooter\" property) [locMarshallerCompactFooter=" + locMarshCompactFooterBool +
+                        ", rmtMarshallerCompactFooter=" + rmtMarshCompactFooterBool +
+                        ", locNodeAddrs=" + U.addressesAsString(locNode) +
+                        ", rmtNodeAddrs=" + U.addressesAsString(node) +
+                        ", locNodeId=" + locNode.id() + ", rmtNodeId=" + msg.creatorNodeId() + ']';
+
+                    LT.warn(log, null, errMsg);
+
+                    // Always output in debug.
+                    if (log.isDebugEnabled())
+                        log.debug(errMsg);
+
+                    try {
+                        String sndMsg = "Local node's portable marshaller \"compactFooter\" property differs from " +
+                            "the same property on remote node (make sure all nodes in topology have the same value " +
+                            "of \"compactFooter\" property) [locMarshallerCompactFooter=" + rmtMarshCompactFooterBool +
+                            ", rmtMarshallerCompactFooter=" + locMarshCompactFooterBool +
+                            ", locNodeAddrs=" + U.addressesAsString(node) + ", locPort=" + node.discoveryPort() +
+                            ", rmtNodeAddr=" + U.addressesAsString(locNode) + ", locNodeId=" + node.id() +
+                            ", rmtNodeId=" + locNode.id() + ']';
+
+                        trySendMessageDirectly(node, new TcpDiscoveryCheckFailedMessage(locNodeId, sndMsg));
+                    }
+                    catch (IgniteSpiException e) {
+                        if (log.isDebugEnabled())
+                            log.debug("Failed to send marshaller check failed message to node " +
+                                "[node=" + node + ", err=" + e.getMessage() + ']');
+
+                        onException("Failed to send marshaller check failed message to node " +
+                            "[node=" + node + ", err=" + e.getMessage() + ']', e);
+                    }
+
+                    // Ignore join request.
+                    return;
+                }
+
                 // Handle join.
                 node.internalOrder(ring.nextNodeOrder());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/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 14fc6f3..8f79db1 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
@@ -46,11 +46,13 @@ public abstract class BinaryFieldsAbstractSelfTest extends GridCommonAbstractTes
      * @return Portable marshaller.
      * @throws Exception If failed.
      */
-    protected static PortableMarshaller createMarshaller() throws Exception {
-        PortableContext ctx = new PortableContext(new TestCachingMetadataHandler(), new IgniteConfiguration());
+    protected PortableMarshaller createMarshaller() throws Exception {
+        PortableContext ctx = new PortableContext(BinaryCachingMetadataHandler.create(), new IgniteConfiguration());
 
         PortableMarshaller marsh = new PortableMarshaller();
 
+        marsh.setCompactFooter(compactFooter());
+
         marsh.setTypeConfigurations(Arrays.asList(
             new BinaryTypeConfiguration(TestObject.class.getName()),
             new BinaryTypeConfiguration(TestOuterObject.class.getName()),
@@ -65,6 +67,13 @@ public abstract class BinaryFieldsAbstractSelfTest extends GridCommonAbstractTes
     }
 
     /**
+     * @return Whether to use compact footer.
+     */
+    protected boolean compactFooter() {
+        return true;
+    }
+
+    /**
      * Get portable context for the current marshaller.
      *
      * @param marsh Marshaller.

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/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
new file mode 100644
index 0000000..3ec0b83
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsAbstractSelfTest.java
@@ -0,0 +1,199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.portable;
+
+import org.apache.ignite.binary.BinaryField;
+import org.apache.ignite.binary.BinaryTypeConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.marshaller.MarshallerContextTestImpl;
+import org.apache.ignite.marshaller.portable.PortableMarshaller;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import java.util.Arrays;
+
+/**
+ * Contains tests for compact offsets.
+ */
+public abstract class BinaryFooterOffsetsAbstractSelfTest extends GridCommonAbstractTest {
+    /** 2 pow 8. */
+    private static int POW_8 = 1 << 8;
+
+    /** 2 pow 16. */
+    private static int POW_16 = 1 << 16;
+
+    /** Marshaller. */
+    protected PortableMarshaller marsh;
+
+    /** Portable context. */
+    protected PortableContext ctx;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        ctx = new PortableContext(BinaryCachingMetadataHandler.create(), new IgniteConfiguration());
+
+        marsh = new PortableMarshaller();
+
+        marsh.setCompactFooter(compactFooter());
+
+        marsh.setTypeConfigurations(Arrays.asList(new BinaryTypeConfiguration(TestObject.class.getName())));
+        marsh.setContext(new MarshallerContextTestImpl(null));
+
+        IgniteUtils.invoke(PortableMarshaller.class, marsh, "setPortableContext", ctx);
+    }
+
+    /**
+     * @return Whether to use compact footers.
+     */
+    protected boolean compactFooter() {
+        return true;
+    }
+
+    /**
+     * Test 1 byte.
+     *
+     * @throws Exception If failed.
+     */
+    public void test1Byte() throws Exception {
+        check(POW_8 >> 2);
+    }
+
+    /**
+     * Test 1 byte with sign altering.
+     *
+     * @throws Exception If failed.
+     */
+    public void test1ByteSign() throws Exception {
+        check(POW_8 >> 1);
+    }
+
+    /**
+     * Test 2 bytes.
+     *
+     * @throws Exception If failed.
+     */
+    public void test2Bytes() throws Exception {
+        check(POW_16 >> 2);
+    }
+
+    /**
+     * Test 2 bytes with sign altering.
+     *
+     * @throws Exception If failed.
+     */
+    public void test2BytesSign() throws Exception {
+        check(POW_16 >> 1);
+    }
+
+    /**
+     * Test 4 bytes.
+     *
+     * @throws Exception If failed.
+     */
+    public void test4Bytes() throws Exception {
+        check(POW_16 << 2);
+    }
+
+    /**
+     * Main check routine.
+     *
+     * @param len Length of the first field.
+     *
+     * @throws Exception If failed.
+     */
+    private void check(int len) throws Exception {
+        TestObject obj = new TestObject(len);
+
+        BinaryObjectEx portObj = toPortable(marsh, obj);
+
+        // 1. Test portable object content.
+        assert portObj.hasField("field1");
+        assert portObj.hasField("field2");
+
+        byte[] field1 = portObj.field("field1");
+        Integer field2 = portObj.field("field2");
+
+        assert field1 != null;
+        assert field2 != null;
+
+        assert Arrays.equals(obj.field1, field1);
+        assert obj.field2 == field2;
+
+        // 2. Test fields API.
+        BinaryField field1Desc = portObj.type().field("field1");
+        BinaryField field2Desc = portObj.type().field("field2");
+
+        assert field1Desc.exists(portObj);
+        assert field2Desc.exists(portObj);
+
+        assert Arrays.equals(obj.field1, (byte[])field1Desc.value(portObj));
+        assert obj.field2 == (Integer)field2Desc.value(portObj);
+
+        // 3. Test deserialize.
+        TestObject objRestored = portObj.deserialize();
+
+        assert objRestored != null;
+
+        assert Arrays.equals(obj.field1, objRestored.field1);
+        assert obj.field2 == objRestored.field2;
+    }
+
+    /**
+     * Convert object to portable object.
+     *
+     * @param marsh Marshaller.
+     * @param obj Object.
+     * @return Portable object.
+     * @throws Exception If failed.
+     */
+    protected abstract BinaryObjectEx toPortable(PortableMarshaller marsh, Object obj) throws Exception;
+
+    /**
+     * Test object.
+     */
+    public static class TestObject {
+        /** First field with variable length. */
+        public byte[] field1;
+
+        /** Second field. */
+        public int field2;
+
+        /**
+         * Default constructor.
+         */
+        public TestObject() {
+            // No-op.
+        }
+
+        /**
+         * Constructor.
+         *
+         * @param len Array length.
+         */
+        public TestObject(int len) {
+            field1 = new byte[len];
+
+            field1[0] = 1;
+            field1[len - 1] = 2;
+
+            field2 = len;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/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
new file mode 100644
index 0000000..b23f012
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsHeapSelfTest.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.portable;
+
+import org.apache.ignite.marshaller.portable.PortableMarshaller;
+
+/**
+ * Compact offsets tests for heap portable objects.
+ */
+public class BinaryFooterOffsetsHeapSelfTest extends BinaryFooterOffsetsAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected BinaryObjectEx toPortable(PortableMarshaller 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/0b4a8f83/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
new file mode 100644
index 0000000..e52ebe7
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsOffheapSelfTest.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.portable;
+
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.marshaller.portable.PortableMarshaller;
+import org.eclipse.jetty.util.ConcurrentHashSet;
+import sun.misc.Unsafe;
+
+/**
+ * Compact offsets tests for offheap portable objects.
+ */
+public class BinaryFooterOffsetsOffheapSelfTest extends BinaryFooterOffsetsAbstractSelfTest {
+    /** Unsafe instance. */
+    private static final Unsafe UNSAFE = GridUnsafe.unsafe();
+
+    /** Byte array offset for unsafe mechanics. */
+    protected static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** Allocated unsafe pointer. */
+    private final ConcurrentHashSet<Long> ptrs = new ConcurrentHashSet<>();
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        // Cleanup allocated objects.
+        for (Long ptr : ptrs)
+            UNSAFE.freeMemory(ptr);
+
+        ptrs.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected BinaryObjectEx toPortable(PortableMarshaller marsh, Object obj) throws Exception {
+        byte[] arr = marsh.marshal(obj);
+
+        long ptr = UNSAFE.allocateMemory(arr.length);
+
+        ptrs.add(ptr);
+
+        UNSAFE.copyMemory(arr, BYTE_ARR_OFF, null, ptr, arr.length);
+
+        return new BinaryObjectOffheapImpl(ctx, ptr, 0, arr.length);
+    }
+}


[04/13] ignite git commit: IGNITE-1816: Implemented compact footers.

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderAdditionalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderAdditionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderAdditionalSelfTest.java
new file mode 100644
index 0000000..0e31451
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderAdditionalSelfTest.java
@@ -0,0 +1,1291 @@
+/*
+ * 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 com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import java.lang.reflect.Field;
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteBinary;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.portable.builder.PortableBuilderEnum;
+import org.apache.ignite.internal.portable.builder.BinaryObjectBuilderImpl;
+import org.apache.ignite.internal.portable.mutabletest.GridBinaryMarshalerAwareTestClass;
+import org.apache.ignite.internal.processors.cache.portable.CacheObjectBinaryProcessorImpl;
+import org.apache.ignite.internal.processors.cache.portable.IgniteBinaryImpl;
+import org.apache.ignite.internal.util.lang.GridMapEntry;
+import org.apache.ignite.marshaller.portable.PortableMarshaller;
+import org.apache.ignite.binary.BinaryObjectBuilder;
+import org.apache.ignite.binary.BinaryType;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Assert;
+
+import static org.apache.ignite.cache.CacheMode.REPLICATED;
+import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.Address;
+import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.AddressBook;
+import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.Company;
+import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectAllTypes;
+import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectArrayList;
+import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectContainer;
+import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectEnum;
+import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectInner;
+import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectOuter;
+
+/**
+ *
+ */
+public class BinaryObjectBuilderAdditionalSelfTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration cacheCfg = new CacheConfiguration();
+
+        cacheCfg.setCacheMode(REPLICATED);
+
+        cfg.setCacheConfiguration(cacheCfg);
+
+        PortableMarshaller marsh = new PortableMarshaller();
+
+        marsh.setCompactFooter(compactFooter());
+
+        marsh.setClassNames(Arrays.asList("org.apache.ignite.internal.portable.mutabletest.*"));
+
+        cfg.setMarshaller(marsh);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGrids(1);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        jcache(0).clear();
+    }
+
+    /**
+     * @return Compact footer.
+     */
+    protected boolean compactFooter() {
+        return true;
+    }
+
+    /**
+     * @return Portables API.
+     */
+    protected IgniteBinary portables() {
+        return grid(0).binary();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSimpleTypeFieldRead() throws Exception {
+        TestObjectAllTypes exp = new TestObjectAllTypes();
+
+        exp.setDefaultData();
+
+        BinaryObjectBuilder mutPo = wrap(exp);
+
+        for (Field field : TestObjectAllTypes.class.getDeclaredFields()) {
+            Object expVal = field.get(exp);
+            Object actVal = mutPo.getField(field.getName());
+
+            switch (field.getName()) {
+                case "anEnum":
+                    assertEquals(((PortableBuilderEnum)actVal).getOrdinal(), ((Enum)expVal).ordinal());
+                    break;
+
+                case "enumArr": {
+                    PortableBuilderEnum[] actArr = (PortableBuilderEnum[])actVal;
+                    Enum[] expArr = (Enum[])expVal;
+
+                    assertEquals(expArr.length, actArr.length);
+
+                    for (int i = 0; i < actArr.length; i++)
+                        assertEquals(expArr[i].ordinal(), actArr[i].getOrdinal());
+
+                    break;
+                }
+
+                case "entry":
+                    assertEquals(((Map.Entry)expVal).getKey(), ((Map.Entry)actVal).getKey());
+                    assertEquals(((Map.Entry)expVal).getValue(), ((Map.Entry)actVal).getValue());
+                    break;
+
+                default:
+                    assertTrue(field.getName(), Objects.deepEquals(expVal, actVal));
+                    break;
+            }
+        }
+    }
+
+    /**
+     *
+     */
+    public void testSimpleTypeFieldSerialize() {
+        TestObjectAllTypes exp = new TestObjectAllTypes();
+
+        exp.setDefaultData();
+
+        BinaryObjectBuilderImpl mutPo = wrap(exp);
+
+        TestObjectAllTypes res = mutPo.build().deserialize();
+
+        GridTestUtils.deepEquals(exp, res);
+    }
+
+    /**
+     * @throws Exception If any error occurs.
+     */
+    public void testSimpleTypeFieldOverride() throws Exception {
+        TestObjectAllTypes exp = new TestObjectAllTypes();
+
+        exp.setDefaultData();
+
+        BinaryObjectBuilderImpl mutPo = wrap(new TestObjectAllTypes());
+
+        for (Field field : TestObjectAllTypes.class.getDeclaredFields())
+            mutPo.setField(field.getName(), field.get(exp));
+
+        TestObjectAllTypes res = mutPo.build().deserialize();
+
+        GridTestUtils.deepEquals(exp, res);
+    }
+
+    /**
+     * @throws Exception If any error occurs.
+     */
+    public void testSimpleTypeFieldSetNull() throws Exception {
+        TestObjectAllTypes exp = new TestObjectAllTypes();
+
+        exp.setDefaultData();
+
+        BinaryObjectBuilderImpl mutPo = wrap(exp);
+
+        for (Field field : TestObjectAllTypes.class.getDeclaredFields()) {
+            if (!field.getType().isPrimitive())
+                mutPo.setField(field.getName(), null);
+        }
+
+        TestObjectAllTypes res = mutPo.build().deserialize();
+
+        for (Field field : TestObjectAllTypes.class.getDeclaredFields()) {
+            if (!field.getType().isPrimitive())
+                assertNull(field.getName(), field.get(res));
+        }
+    }
+
+    /**
+     * @throws IgniteCheckedException If any error occurs.
+     */
+    public void testMakeCyclicDependency() throws IgniteCheckedException {
+        TestObjectOuter outer = new TestObjectOuter();
+        outer.inner = new TestObjectInner();
+
+        BinaryObjectBuilderImpl mutOuter = wrap(outer);
+
+        BinaryObjectBuilderImpl mutInner = mutOuter.getField("inner");
+
+        mutInner.setField("outer", mutOuter);
+        mutInner.setField("foo", mutInner);
+
+        TestObjectOuter res = mutOuter.build().deserialize();
+
+        assertEquals(res, res.inner.outer);
+        assertEquals(res.inner, res.inner.foo);
+    }
+
+    /**
+     *
+     */
+    public void testDateArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.dateArr =  new Date[] {new Date(11111), new Date(11111), new Date(11111)};
+
+        BinaryObjectBuilderImpl mutObj = wrap(obj);
+
+        Date[] arr = mutObj.getField("dateArr");
+        arr[0] = new Date(22222);
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new Date[] {new Date(22222), new Date(11111), new Date(11111)}, res.dateArr);
+    }
+
+    /**
+     *
+     */
+    public void testTimestampArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.tsArr = new Timestamp[] {new Timestamp(111222333), new Timestamp(222333444)};
+
+        BinaryObjectBuilderImpl mutObj = wrap(obj);
+
+        Timestamp[] arr = mutObj.getField("tsArr");
+        arr[0] = new Timestamp(333444555);
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new Timestamp[] {new Timestamp(333444555), new Timestamp(222333444)}, res.tsArr);
+    }
+
+    /**
+     *
+     */
+    public void testUUIDArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.uuidArr = new UUID[] {new UUID(1, 1), new UUID(1, 1), new UUID(1, 1)};
+
+        BinaryObjectBuilderImpl mutObj = wrap(obj);
+
+        UUID[] arr = mutObj.getField("uuidArr");
+        arr[0] = new UUID(2, 2);
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new UUID[] {new UUID(2, 2), new UUID(1, 1), new UUID(1, 1)}, res.uuidArr);
+    }
+
+    /**
+     *
+     */
+    public void testDecimalArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.bdArr = new BigDecimal[] {new BigDecimal(1000), new BigDecimal(1000), new BigDecimal(1000)};
+
+        BinaryObjectBuilderImpl mutObj = wrap(obj);
+
+        BigDecimal[] arr = mutObj.getField("bdArr");
+        arr[0] = new BigDecimal(2000);
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new BigDecimal[] {new BigDecimal(1000), new BigDecimal(1000), new BigDecimal(1000)},
+            res.bdArr);
+    }
+
+    /**
+     *
+     */
+    public void testBooleanArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.zArr = new boolean[] {false, false, false};
+
+        BinaryObjectBuilderImpl mutObj = wrap(obj);
+
+        boolean[] arr = mutObj.getField("zArr");
+        arr[0] = true;
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        boolean[] expected = new boolean[] {true, false, false};
+
+        assertEquals(expected.length, res.zArr.length);
+
+        for (int i = 0; i < expected.length; i++)
+            assertEquals(expected[i], res.zArr[i]);
+    }
+
+    /**
+     *
+     */
+    public void testCharArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.cArr = new char[] {'a', 'a', 'a'};
+
+        BinaryObjectBuilderImpl mutObj = wrap(obj);
+
+        char[] arr = mutObj.getField("cArr");
+        arr[0] = 'b';
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new char[] {'b', 'a', 'a'}, res.cArr);
+    }
+
+    /**
+     *
+     */
+    public void testDoubleArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.dArr = new double[] {1.0, 1.0, 1.0};
+
+        BinaryObjectBuilderImpl mutObj = wrap(obj);
+
+        double[] arr = mutObj.getField("dArr");
+        arr[0] = 2.0;
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new double[] {2.0, 1.0, 1.0}, res.dArr, 0);
+    }
+
+    /**
+     *
+     */
+    public void testFloatArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.fArr = new float[] {1.0f, 1.0f, 1.0f};
+
+        BinaryObjectBuilderImpl mutObj = wrap(obj);
+
+        float[] arr = mutObj.getField("fArr");
+        arr[0] = 2.0f;
+
+        BinaryObject resBinary = mutObj.build();
+
+        TestObjectAllTypes res = resBinary.deserialize();
+
+        Assert.assertArrayEquals(new float[] {2.0f, 1.0f, 1.0f}, res.fArr, 0);
+    }
+
+    /**
+     *
+     */
+    public void testLongArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.lArr = new long[] {1, 1, 1};
+
+        BinaryObjectBuilderImpl mutObj = wrap(obj);
+
+        long[] arr = mutObj.getField("lArr");
+        arr[0] = 2;
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new long[] {2, 1, 1}, res.lArr);
+    }
+
+    /**
+     *
+     */
+    public void testIntArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.iArr = new int[] {1, 1, 1};
+
+        BinaryObjectBuilderImpl mutObj = wrap(obj);
+
+        int[] arr = mutObj.getField("iArr");
+        arr[0] = 2;
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new int[] {2, 1, 1}, res.iArr);
+    }
+
+    /**
+     *
+     */
+    public void testShortArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.sArr = new short[] {1, 1, 1};
+
+        BinaryObjectBuilderImpl mutObj = wrap(obj);
+
+        short[] arr = mutObj.getField("sArr");
+        arr[0] = 2;
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new short[] {2, 1, 1}, res.sArr);
+    }
+
+    /**
+     *
+     */
+    public void testByteArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.bArr = new byte[] {1, 1, 1};
+
+        BinaryObjectBuilderImpl mutObj = wrap(obj);
+
+        byte[] arr = mutObj.getField("bArr");
+        arr[0] = 2;
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new byte[] {2, 1, 1}, res.bArr);
+    }
+
+    /**
+     *
+     */
+    public void testStringArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.strArr = new String[] {"a", "a", "a"};
+
+        BinaryObjectBuilderImpl mutObj = wrap(obj);
+
+        String[] arr = mutObj.getField("strArr");
+        arr[0] = "b";
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new String[] {"b", "a", "a"}, res.strArr);
+    }
+
+    /**
+     *
+     */
+    public void testModifyObjectArray() {
+        TestObjectContainer obj = new TestObjectContainer();
+        obj.foo = new Object[] {"a"};
+
+        BinaryObjectBuilderImpl mutObj = wrap(obj);
+
+        Object[] arr = mutObj.getField("foo");
+
+        Assert.assertArrayEquals(new Object[] {"a"}, arr);
+
+        arr[0] = "b";
+
+        TestObjectContainer res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new Object[] {"b"}, (Object[])res.foo);
+    }
+
+    /**
+     *
+     */
+    public void testOverrideObjectArrayField() {
+        BinaryObjectBuilderImpl mutObj = wrap(new TestObjectContainer());
+
+        Object[] createdArr = {mutObj, "a", 1, new String[] {"s", "s"}, new byte[] {1, 2}, new UUID(3, 0)};
+
+        mutObj.setField("foo", createdArr.clone());
+
+        TestObjectContainer res = mutObj.build().deserialize();
+
+        createdArr[0] = res;
+
+        assertTrue(Objects.deepEquals(createdArr, res.foo));
+    }
+
+    /**
+     *
+     */
+    public void testDeepArray() {
+        TestObjectContainer obj = new TestObjectContainer();
+        obj.foo = new Object[] {new Object[] {"a", obj}};
+
+        BinaryObjectBuilderImpl mutObj = wrap(obj);
+
+        Object[] arr = (Object[])mutObj.<Object[]>getField("foo")[0];
+
+        assertEquals("a", arr[0]);
+        assertSame(mutObj, arr[1]);
+
+        arr[0] = mutObj;
+
+        TestObjectContainer res = mutObj.build().deserialize();
+
+        arr = (Object[])((Object[])res.foo)[0];
+
+        assertSame(arr[0], res);
+        assertSame(arr[0], arr[1]);
+    }
+
+    /**
+     *
+     */
+    public void testArrayListRead() {
+        TestObjectContainer obj = new TestObjectContainer();
+        obj.foo = Lists.newArrayList(obj, "a");
+
+        BinaryObjectBuilderImpl mutObj = wrap(obj);
+
+        List<Object> list = mutObj.getField("foo");
+
+        assert list.equals(Lists.newArrayList(mutObj, "a"));
+    }
+
+    /**
+     *
+     */
+    public void testArrayListOverride() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        BinaryObjectBuilderImpl mutObj = wrap(obj);
+
+        ArrayList<Object> list = Lists.newArrayList(mutObj, "a", Lists.newArrayList(1, 2));
+
+        mutObj.setField("foo", list);
+
+        TestObjectContainer res = mutObj.build().deserialize();
+
+        list.set(0, res);
+
+        assertNotSame(list, res.foo);
+        assertEquals(list, res.foo);
+    }
+
+    /**
+     *
+     */
+    public void testArrayListModification() {
+        TestObjectContainer obj = new TestObjectContainer();
+        obj.foo = Lists.newArrayList("a", "b", "c");
+
+        BinaryObjectBuilderImpl mutObj = wrap(obj);
+
+        List<String> list = mutObj.getField("foo");
+
+        list.add("!"); // "a", "b", "c", "!"
+        list.add(0, "_"); // "_", "a", "b", "c", "!"
+
+        String s = list.remove(1); // "_", "b", "c", "!"
+        assertEquals("a", s);
+
+        assertEquals(Arrays.asList("c", "!"), list.subList(2, 4));
+        assertEquals(1, list.indexOf("b"));
+        assertEquals(1, list.lastIndexOf("b"));
+
+        TestObjectContainer res = mutObj.build().deserialize();
+
+        assertTrue(res.foo instanceof ArrayList);
+        assertEquals(Arrays.asList("_", "b", "c", "!"), res.foo);
+    }
+
+    /**
+     *
+     */
+    public void testArrayListClear() {
+        TestObjectContainer obj = new TestObjectContainer();
+        obj.foo = Lists.newArrayList("a", "b", "c");
+
+        BinaryObjectBuilderImpl mutObj = wrap(obj);
+
+        List<String> list = mutObj.getField("foo");
+
+        list.clear();
+
+        assertEquals(Collections.emptyList(), mutObj.build().<TestObjectContainer>deserialize().foo);
+    }
+
+    /**
+     *
+     */
+    public void testArrayListWriteUnmodifiable() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        ArrayList<Object> src = Lists.newArrayList(obj, "a", "b", "c");
+
+        obj.foo = src;
+
+        BinaryObjectBuilderImpl mutObj = wrap(obj);
+
+        TestObjectContainer deserialized = mutObj.build().deserialize();
+
+        List<Object> res = (List<Object>)deserialized.foo;
+
+        src.set(0, deserialized);
+
+        assertEquals(src, res);
+    }
+
+    /**
+     *
+     */
+    public void testLinkedListRead() {
+        TestObjectContainer obj = new TestObjectContainer();
+        obj.foo = Lists.newLinkedList(Arrays.asList(obj, "a"));
+
+        BinaryObjectBuilderImpl mutObj = wrap(obj);
+
+        List<Object> list = mutObj.getField("foo");
+
+        assert list.equals(Lists.newLinkedList(Arrays.asList(mutObj, "a")));
+    }
+
+    /**
+     *
+     */
+    public void testLinkedListOverride() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        BinaryObjectBuilderImpl mutObj = wrap(obj);
+
+        List<Object> list = Lists.newLinkedList(Arrays.asList(mutObj, "a", Lists.newLinkedList(Arrays.asList(1, 2))));
+
+        mutObj.setField("foo", list);
+
+        TestObjectContainer res = mutObj.build().deserialize();
+
+        list.set(0, res);
+
+        assertNotSame(list, res.foo);
+        assertEquals(list, res.foo);
+    }
+
+    /**
+     *
+     */
+    public void testLinkedListModification() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        obj.foo = Lists.newLinkedList(Arrays.asList("a", "b", "c"));
+
+        BinaryObjectBuilderImpl mutObj = wrap(obj);
+
+        List<String> list = mutObj.getField("foo");
+
+        list.add("!"); // "a", "b", "c", "!"
+        list.add(0, "_"); // "_", "a", "b", "c", "!"
+
+        String s = list.remove(1); // "_", "b", "c", "!"
+        assertEquals("a", s);
+
+        assertEquals(Arrays.asList("c", "!"), list.subList(2, 4));
+        assertEquals(1, list.indexOf("b"));
+        assertEquals(1, list.lastIndexOf("b"));
+
+        TestObjectContainer res = mutObj.build().deserialize();
+
+        assertTrue(res.foo instanceof LinkedList);
+        assertEquals(Arrays.asList("_", "b", "c", "!"), res.foo);
+    }
+
+    /**
+     *
+     */
+    public void testLinkedListWriteUnmodifiable() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        LinkedList<Object> src = Lists.newLinkedList(Arrays.asList(obj, "a", "b", "c"));
+
+        obj.foo = src;
+
+        BinaryObjectBuilderImpl mutObj = wrap(obj);
+
+        TestObjectContainer deserialized = mutObj.build().deserialize();
+
+        List<Object> res = (List<Object>)deserialized.foo;
+
+        src.set(0, deserialized);
+
+        assertEquals(src, res);
+    }
+
+    /**
+     *
+     */
+    public void testHashSetRead() {
+        TestObjectContainer obj = new TestObjectContainer();
+        obj.foo = Sets.newHashSet(obj, "a");
+
+        BinaryObjectBuilderImpl mutObj = wrap(obj);
+
+        Set<Object> set = mutObj.getField("foo");
+
+        assert set.equals(Sets.newHashSet(mutObj, "a"));
+    }
+
+    /**
+     *
+     */
+    public void testHashSetOverride() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        BinaryObjectBuilderImpl mutObj = wrap(obj);
+
+        Set<Object> c = Sets.newHashSet(mutObj, "a", Sets.newHashSet(1, 2));
+
+        mutObj.setField("foo", c);
+
+        TestObjectContainer res = mutObj.build().deserialize();
+
+        c.remove(mutObj);
+        c.add(res);
+
+        assertNotSame(c, res.foo);
+        assertEquals(c, res.foo);
+    }
+
+    /**
+     *
+     */
+    public void testHashSetModification() {
+        TestObjectContainer obj = new TestObjectContainer();
+        obj.foo = Sets.newHashSet("a", "b", "c");
+
+        BinaryObjectBuilderImpl mutObj = wrap(obj);
+
+        Set<String> set = mutObj.getField("foo");
+
+        set.remove("b");
+        set.add("!");
+
+        assertEquals(Sets.newHashSet("a", "!", "c"), set);
+        assertTrue(set.contains("a"));
+        assertTrue(set.contains("!"));
+
+        TestObjectContainer res = mutObj.build().deserialize();
+
+        assertTrue(res.foo instanceof HashSet);
+        assertEquals(Sets.newHashSet("a", "!", "c"), res.foo);
+    }
+
+    /**
+     *
+     */
+    public void testHashSetWriteUnmodifiable() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        Set<Object> src = Sets.newHashSet(obj, "a", "b", "c");
+
+        obj.foo = src;
+
+        TestObjectContainer deserialized = wrap(obj).build().deserialize();
+
+        Set<Object> res = (Set<Object>)deserialized.foo;
+
+        src.remove(obj);
+        src.add(deserialized);
+
+        assertEquals(src, res);
+    }
+
+    /**
+     *
+     */
+    public void testMapRead() {
+        TestObjectContainer obj = new TestObjectContainer();
+        obj.foo = Maps.newHashMap(ImmutableMap.of(obj, "a", "b", obj));
+
+        BinaryObjectBuilderImpl mutObj = wrap(obj);
+
+        Map<Object, Object> map = mutObj.getField("foo");
+
+        assert map.equals(ImmutableMap.of(mutObj, "a", "b", mutObj));
+    }
+
+    /**
+     *
+     */
+    public void testMapOverride() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        BinaryObjectBuilderImpl mutObj = wrap(obj);
+
+        Map<Object, Object> map = Maps.newHashMap(ImmutableMap.of(mutObj, "a", "b", mutObj));
+
+        mutObj.setField("foo", map);
+
+        TestObjectContainer res = mutObj.build().deserialize();
+
+        assertEquals(ImmutableMap.of(res, "a", "b", res), res.foo);
+    }
+
+    /**
+     *
+     */
+    public void testMapModification() {
+        TestObjectContainer obj = new TestObjectContainer();
+        obj.foo = Maps.newHashMap(ImmutableMap.of(1, "a", 2, "b"));
+
+        BinaryObjectBuilderImpl mutObj = wrap(obj);
+
+        Map<Object, Object> map = mutObj.getField("foo");
+
+        map.put(3, mutObj);
+        Object rmv = map.remove(1);
+
+        assertEquals("a", rmv);
+
+        TestObjectContainer res = mutObj.build().deserialize();
+
+        assertEquals(ImmutableMap.of(2, "b", 3, res), res.foo);
+    }
+
+    /**
+     *
+     */
+    public void testEnumArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.enumArr = new TestObjectEnum[] {TestObjectEnum.A, TestObjectEnum.B};
+
+        BinaryObjectBuilderImpl mutObj = wrap(obj);
+
+        PortableBuilderEnum[] arr = mutObj.getField("enumArr");
+        arr[0] = new PortableBuilderEnum(mutObj.typeId(), TestObjectEnum.B);
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new TestObjectEnum[] {TestObjectEnum.A, TestObjectEnum.B}, res.enumArr);
+    }
+
+    /**
+     *
+     */
+    public void testEditObjectWithRawData() {
+        GridBinaryMarshalerAwareTestClass obj = new GridBinaryMarshalerAwareTestClass();
+
+        obj.s = "a";
+        obj.sRaw = "aa";
+
+        BinaryObjectBuilderImpl mutableObj = wrap(obj);
+
+        mutableObj.setField("s", "z");
+
+        GridBinaryMarshalerAwareTestClass res = mutableObj.build().deserialize();
+        assertEquals("z", res.s);
+        assertEquals("aa", res.sRaw);
+    }
+
+    /**
+     *
+     */
+    public void testHashCode() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        BinaryObjectBuilderImpl mutableObj = wrap(obj);
+
+        assertEquals(obj.hashCode(), mutableObj.build().hashCode());
+
+        mutableObj.hashCode(25);
+
+        assertEquals(25, mutableObj.build().hashCode());
+    }
+
+    /**
+     *
+     */
+    public void testCollectionsInCollection() {
+        TestObjectContainer obj = new TestObjectContainer();
+        obj.foo = Lists.newArrayList(
+            Lists.newArrayList(1, 2),
+            Lists.newLinkedList(Arrays.asList(1, 2)),
+            Sets.newHashSet("a", "b"),
+            Sets.newLinkedHashSet(Arrays.asList("a", "b")),
+            Maps.newHashMap(ImmutableMap.of(1, "a", 2, "b")));
+
+        TestObjectContainer deserialized = wrap(obj).build().deserialize();
+
+        assertEquals(obj.foo, deserialized.foo);
+    }
+
+    /**
+     *
+     */
+    public void testMapEntryModification() {
+        TestObjectContainer obj = new TestObjectContainer();
+        obj.foo = ImmutableMap.of(1, "a").entrySet().iterator().next();
+
+        BinaryObjectBuilderImpl mutableObj = wrap(obj);
+
+        Map.Entry<Object, Object> entry = mutableObj.getField("foo");
+
+        assertEquals(1, entry.getKey());
+        assertEquals("a", entry.getValue());
+
+        entry.setValue("b");
+
+        TestObjectContainer res = mutableObj.build().deserialize();
+
+        assertEquals(new GridMapEntry<>(1, "b"), res.foo);
+    }
+
+    /**
+     *
+     */
+    public void testMapEntryOverride() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        BinaryObjectBuilderImpl mutableObj = wrap(obj);
+
+        mutableObj.setField("foo", new GridMapEntry<>(1, "a"));
+
+        TestObjectContainer res = mutableObj.build().deserialize();
+
+        assertEquals(new GridMapEntry<>(1, "a"), res.foo);
+    }
+
+    /**
+     *
+     */
+    public void testMetadataChangingDoublePut() {
+        BinaryObjectBuilderImpl mutableObj = wrap(new TestObjectContainer());
+
+        mutableObj.setField("xx567", "a");
+        mutableObj.setField("xx567", "b");
+
+        mutableObj.build();
+
+        BinaryType metadata = portables().metadata(TestObjectContainer.class);
+
+        assertEquals("String", metadata.fieldTypeName("xx567"));
+    }
+
+    /**
+     *
+     */
+    public void testMetadataChangingDoublePut2() {
+        BinaryObjectBuilderImpl mutableObj = wrap(new TestObjectContainer());
+
+        mutableObj.setField("xx567", "a");
+        mutableObj.setField("xx567", "b");
+
+        mutableObj.build();
+
+        BinaryType metadata = portables().metadata(TestObjectContainer.class);
+
+        assertEquals("String", metadata.fieldTypeName("xx567"));
+    }
+
+    /**
+     *
+     */
+    public void testMetadataChanging() {
+        TestObjectContainer c = new TestObjectContainer();
+
+        BinaryObjectBuilderImpl mutableObj = wrap(c);
+
+        mutableObj.setField("intField", 1);
+        mutableObj.setField("intArrField", new int[] {1});
+        mutableObj.setField("arrField", new String[] {"1"});
+        mutableObj.setField("strField", "1");
+        mutableObj.setField("colField", Lists.newArrayList("1"));
+        mutableObj.setField("mapField", Maps.newHashMap(ImmutableMap.of(1, "1")));
+        mutableObj.setField("enumField", TestObjectEnum.A);
+        mutableObj.setField("enumArrField", new Enum[] {TestObjectEnum.A});
+
+        mutableObj.build();
+
+        BinaryType metadata = portables().metadata(c.getClass());
+
+        assertTrue(metadata.fieldNames().containsAll(Arrays.asList("intField", "intArrField", "arrField", "strField",
+            "colField", "mapField", "enumField", "enumArrField")));
+
+        assertEquals("int", metadata.fieldTypeName("intField"));
+        assertEquals("int[]", metadata.fieldTypeName("intArrField"));
+        assertEquals("String[]", metadata.fieldTypeName("arrField"));
+        assertEquals("String", metadata.fieldTypeName("strField"));
+        assertEquals("Collection", metadata.fieldTypeName("colField"));
+        assertEquals("Map", metadata.fieldTypeName("mapField"));
+        assertEquals("Enum", metadata.fieldTypeName("enumField"));
+        assertEquals("Enum[]", metadata.fieldTypeName("enumArrField"));
+    }
+
+    /**
+     *
+     */
+    public void testDateInObjectField() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        obj.foo = new Date();
+
+        BinaryObjectBuilderImpl mutableObj = wrap(obj);
+
+        assertEquals(Date.class, mutableObj.getField("foo").getClass());
+    }
+
+    /**
+     *
+     */
+    public void testTimestampInObjectField() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        obj.foo = new Timestamp(100020003);
+
+        BinaryObjectBuilderImpl mutableObj = wrap(obj);
+
+        assertEquals(Timestamp.class, mutableObj.getField("foo").getClass());
+    }
+
+    /**
+     *
+     */
+    public void testDateInCollection() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        obj.foo = Lists.newArrayList(new Date());
+
+        BinaryObjectBuilderImpl mutableObj = wrap(obj);
+
+        assertEquals(Date.class, ((List<?>)mutableObj.getField("foo")).get(0).getClass());
+    }
+
+    /**
+     *
+     */
+    public void testTimestampInCollection() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        obj.foo = Lists.newArrayList(new Timestamp(100020003));
+
+        BinaryObjectBuilderImpl mutableObj = wrap(obj);
+
+        assertEquals(Timestamp.class, ((List<?>)mutableObj.getField("foo")).get(0).getClass());
+    }
+
+    /**
+     *
+     */
+    @SuppressWarnings("AssertEqualsBetweenInconvertibleTypes")
+    public void testDateArrayOverride() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        BinaryObjectBuilderImpl mutableObj = wrap(obj);
+
+        Date[] arr = { new Date() };
+
+        mutableObj.setField("foo", arr);
+
+        TestObjectContainer res = mutableObj.build().deserialize();
+
+        assertEquals(Date[].class, res.foo.getClass());
+        assertTrue(Objects.deepEquals(arr, res.foo));
+    }
+
+    /**
+     *
+     */
+    @SuppressWarnings("AssertEqualsBetweenInconvertibleTypes")
+    public void testTimestampArrayOverride() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        BinaryObjectBuilderImpl mutableObj = wrap(obj);
+
+        Timestamp[] arr = { new Timestamp(100020003) };
+
+        mutableObj.setField("foo", arr);
+
+        TestObjectContainer res = mutableObj.build().deserialize();
+
+        assertEquals(Timestamp[].class, res.foo.getClass());
+        assertTrue(Objects.deepEquals(arr, res.foo));
+    }
+
+    /**
+     *
+     */
+    public void testChangeMap() {
+        AddressBook addrBook = new AddressBook();
+
+        addrBook.addCompany(new Company(1, "Google inc", 100, new Address("Saint-Petersburg", "Torzhkovskya", 1, 53), "occupation"));
+        addrBook.addCompany(new Company(2, "Apple inc", 100, new Address("Saint-Petersburg", "Torzhkovskya", 1, 54), "occupation"));
+        addrBook.addCompany(new Company(3, "Microsoft", 100, new Address("Saint-Petersburg", "Torzhkovskya", 1, 55), "occupation"));
+        addrBook.addCompany(new Company(4, "Oracle", 100, new Address("Saint-Petersburg", "Nevskiy", 1, 1), "occupation"));
+
+        BinaryObjectBuilderImpl mutableObj = wrap(addrBook);
+
+        Map<String, List<BinaryObjectBuilderImpl>> map = mutableObj.getField("companyByStreet");
+
+        List<BinaryObjectBuilderImpl> list = map.get("Torzhkovskya");
+
+        BinaryObjectBuilderImpl company = list.get(0);
+
+        assert "Google inc".equals(company.<String>getField("name"));
+
+        list.remove(0);
+
+        AddressBook res = mutableObj.build().deserialize();
+
+        assertEquals(Arrays.asList("Nevskiy", "Torzhkovskya"), new ArrayList<>(res.getCompanyByStreet().keySet()));
+
+        List<Company> torzhkovskyaCompanies = res.getCompanyByStreet().get("Torzhkovskya");
+
+        assertEquals(2, torzhkovskyaCompanies.size());
+        assertEquals("Apple inc", torzhkovskyaCompanies.get(0).name);
+    }
+
+    /**
+     *
+     */
+    public void testSavingObjectWithNotZeroStart() {
+        TestObjectOuter out = new TestObjectOuter();
+        TestObjectInner inner = new TestObjectInner();
+
+        out.inner = inner;
+        inner.outer = out;
+
+        BinaryObjectBuilderImpl builder = wrap(out);
+
+        BinaryObjectBuilderImpl innerBuilder = builder.getField("inner");
+
+        TestObjectInner res = innerBuilder.build().deserialize();
+
+        assertSame(res, res.outer.inner);
+    }
+
+    /**
+     *
+     */
+    public void testPortableObjectField() {
+        TestObjectContainer container = new TestObjectContainer(toPortable(new TestObjectArrayList()));
+
+        BinaryObjectBuilderImpl wrapper = wrap(container);
+
+        assertTrue(wrapper.getField("foo") instanceof BinaryObject);
+
+        TestObjectContainer deserialized = wrapper.build().deserialize();
+        assertTrue(deserialized.foo instanceof BinaryObject);
+    }
+
+    /**
+     *
+     */
+    public void testAssignPortableObject() {
+        TestObjectContainer container = new TestObjectContainer();
+
+        BinaryObjectBuilderImpl wrapper = wrap(container);
+
+        wrapper.setField("foo", toPortable(new TestObjectArrayList()));
+
+        TestObjectContainer deserialized = wrapper.build().deserialize();
+        assertTrue(deserialized.foo instanceof TestObjectArrayList);
+    }
+
+    /**
+     *
+     */
+    public void testRemoveFromNewObject() {
+        BinaryObjectBuilderImpl wrapper = newWrapper(TestObjectAllTypes.class);
+
+        wrapper.setField("str", "a");
+
+        wrapper.removeField("str");
+
+        assertNull(wrapper.build().<TestObjectAllTypes>deserialize().str);
+    }
+
+    /**
+     *
+     */
+    public void testRemoveFromExistingObject() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+        obj.setDefaultData();
+
+        BinaryObjectBuilderImpl wrapper = wrap(toPortable(obj));
+
+        wrapper.removeField("str");
+
+        assertNull(wrapper.build().<TestObjectAllTypes>deserialize().str);
+    }
+
+    /**
+     *
+     */
+    public void testCyclicArrays() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        Object[] arr1 = new Object[1];
+        Object[] arr2 = new Object[] {arr1};
+
+        arr1[0] = arr2;
+
+        obj.foo = arr1;
+
+        TestObjectContainer res = toPortable(obj).deserialize();
+
+        Object[] resArr = (Object[])res.foo;
+
+        assertSame(((Object[])resArr[0])[0], resArr);
+    }
+
+    /**
+     *
+     */
+    @SuppressWarnings("TypeMayBeWeakened")
+    public void testCyclicArrayList() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        List<Object> arr1 = new ArrayList<>();
+        List<Object> arr2 = new ArrayList<>();
+
+        arr1.add(arr2);
+        arr2.add(arr1);
+
+        obj.foo = arr1;
+
+        TestObjectContainer res = toPortable(obj).deserialize();
+
+        List<?> resArr = (List<?>)res.foo;
+
+        assertSame(((List<Object>)resArr.get(0)).get(0), resArr);
+    }
+
+    /**
+     * @param obj Object.
+     * @return Object in portable format.
+     */
+    private BinaryObject toPortable(Object obj) {
+        return portables().toBinary(obj);
+    }
+
+    /**
+     * @param obj Object.
+     * @return GridMutablePortableObject.
+     */
+    private BinaryObjectBuilderImpl wrap(Object obj) {
+        return BinaryObjectBuilderImpl.wrap(toPortable(obj));
+    }
+
+    /**
+     * @param aCls Class.
+     * @return Wrapper.
+     */
+    private BinaryObjectBuilderImpl newWrapper(Class<?> aCls) {
+        CacheObjectBinaryProcessorImpl processor = (CacheObjectBinaryProcessorImpl)(
+            (IgniteBinaryImpl)portables()).processor();
+
+        return new BinaryObjectBuilderImpl(processor.portableContext(), processor.typeId(aCls.getName()),
+            aCls.getSimpleName());
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/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
new file mode 100644
index 0000000..e88db99
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderSelfTest.java
@@ -0,0 +1,1066 @@
+/*
+ * 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.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteBinary;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.portable.builder.BinaryObjectBuilderImpl;
+import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectAllTypes;
+import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectContainer;
+import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectInner;
+import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectOuter;
+import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectPlainPortable;
+import org.apache.ignite.internal.processors.cache.portable.CacheObjectBinaryProcessorImpl;
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.marshaller.portable.PortableMarshaller;
+import org.apache.ignite.binary.BinaryObjectBuilder;
+import org.apache.ignite.binary.BinaryIdMapper;
+import org.apache.ignite.binary.BinaryType;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.binary.BinaryTypeConfiguration;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import sun.misc.Unsafe;
+
+/**
+ * Portable builder test.
+ */
+public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final Unsafe UNSAFE = GridUnsafe.unsafe();
+
+    /** */
+    protected static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        PortableMarshaller marsh = new PortableMarshaller();
+
+        marsh.setCompactFooter(compactFooter());
+
+        marsh.setClassNames(Arrays.asList(Key.class.getName(), Value.class.getName(),
+            "org.gridgain.grid.internal.util.portable.mutabletest.*"));
+
+        BinaryTypeConfiguration customIdMapper = new BinaryTypeConfiguration();
+
+        customIdMapper.setClassName(CustomIdMapper.class.getName());
+        customIdMapper.setIdMapper(new BinaryIdMapper() {
+            @Override public int typeId(String clsName) {
+                return ~PortableContext.DFLT_ID_MAPPER.typeId(clsName);
+            }
+
+            @Override public int fieldId(int typeId, String fieldName) {
+                return typeId + ~PortableContext.DFLT_ID_MAPPER.fieldId(typeId, fieldName);
+            }
+        });
+
+        marsh.setTypeConfigurations(Collections.singleton(customIdMapper));
+
+        cfg.setMarshaller(marsh);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGrids(1);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @return Whether to use compact footer.
+     */
+    protected boolean compactFooter() {
+        return true;
+    }
+
+    /**
+     *
+     */
+    public void testAllFieldsSerialization() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+        obj.setDefaultData();
+        obj.enumArr = null;
+
+        TestObjectAllTypes deserialized = builder(toPortable(obj)).build().deserialize();
+
+        GridTestUtils.deepEquals(obj, deserialized);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testByteField() throws Exception {
+        BinaryObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("byteField", (byte)1);
+
+        BinaryObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertEquals((byte) 1, po.<Byte>field("byteField").byteValue());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testShortField() throws Exception {
+        BinaryObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("shortField", (short)1);
+
+        BinaryObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertEquals((short)1, po.<Short>field("shortField").shortValue());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIntField() throws Exception {
+        BinaryObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("intField", 1);
+
+        BinaryObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertEquals(1, po.<Integer>field("intField").intValue());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLongField() throws Exception {
+        BinaryObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("longField", 1L);
+
+        BinaryObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertEquals(1L, po.<Long>field("longField").longValue());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testFloatField() throws Exception {
+        BinaryObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("floatField", 1.0f);
+
+        BinaryObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertEquals(1.0f, po.<Float>field("floatField").floatValue(), 0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDoubleField() throws Exception {
+        BinaryObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("doubleField", 1.0d);
+
+        BinaryObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertEquals(1.0d, po.<Double>field("doubleField").doubleValue(), 0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCharField() throws Exception {
+        BinaryObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("charField", (char)1);
+
+        BinaryObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertEquals((char)1, po.<Character>field("charField").charValue());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBooleanField() throws Exception {
+        BinaryObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("booleanField", true);
+
+        BinaryObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertTrue(po.<Boolean>field("booleanField"));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDecimalField() throws Exception {
+        BinaryObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("decimalField", BigDecimal.TEN);
+
+        BinaryObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertEquals(BigDecimal.TEN, po.<String>field("decimalField"));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStringField() throws Exception {
+        BinaryObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("stringField", "str");
+
+        BinaryObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertEquals("str", po.<String>field("stringField"));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDateField() throws Exception {
+        Date date = new Date();
+
+        assertEquals(date, builder("C").setField("d", date).build().<Date>field("d"));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTimestampField() throws Exception {
+        Timestamp ts = new Timestamp(new Date().getTime());
+        ts.setNanos(1000);
+
+        assertEquals(ts, builder("C").setField("t", ts).build().<Timestamp>field("t"));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testUuidField() throws Exception {
+        BinaryObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        UUID uuid = UUID.randomUUID();
+
+        builder.setField("uuidField", uuid);
+
+        BinaryObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertEquals(uuid, po.<UUID>field("uuidField"));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testByteArrayField() throws Exception {
+        BinaryObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("byteArrayField", new byte[] {1, 2, 3});
+
+        BinaryObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertTrue(Arrays.equals(new byte[] {1, 2, 3}, po.<byte[]>field("byteArrayField")));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testShortArrayField() throws Exception {
+        BinaryObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("shortArrayField", new short[] {1, 2, 3});
+
+        BinaryObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertTrue(Arrays.equals(new short[] {1, 2, 3}, po.<short[]>field("shortArrayField")));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIntArrayField() throws Exception {
+        BinaryObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("intArrayField", new int[] {1, 2, 3});
+
+        BinaryObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertTrue(Arrays.equals(new int[] {1, 2, 3}, po.<int[]>field("intArrayField")));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLongArrayField() throws Exception {
+        BinaryObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("longArrayField", new long[] {1, 2, 3});
+
+        BinaryObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertTrue(Arrays.equals(new long[] {1, 2, 3}, po.<long[]>field("longArrayField")));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testFloatArrayField() throws Exception {
+        BinaryObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("floatArrayField", new float[] {1, 2, 3});
+
+        BinaryObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertTrue(Arrays.equals(new float[] {1, 2, 3}, po.<float[]>field("floatArrayField")));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDoubleArrayField() throws Exception {
+        BinaryObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("doubleArrayField", new double[] {1, 2, 3});
+
+        BinaryObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertTrue(Arrays.equals(new double[] {1, 2, 3}, po.<double[]>field("doubleArrayField")));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCharArrayField() throws Exception {
+        BinaryObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("charArrayField", new char[] {1, 2, 3});
+
+        BinaryObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertTrue(Arrays.equals(new char[] {1, 2, 3}, po.<char[]>field("charArrayField")));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBooleanArrayField() throws Exception {
+        BinaryObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("booleanArrayField", new boolean[] {true, false});
+
+        BinaryObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        boolean[] arr = po.field("booleanArrayField");
+
+        assertEquals(2, arr.length);
+
+        assertTrue(arr[0]);
+        assertFalse(arr[1]);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDecimalArrayField() throws Exception {
+        BinaryObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("decimalArrayField", new BigDecimal[] {BigDecimal.ONE, BigDecimal.TEN});
+
+        BinaryObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertTrue(Arrays.equals(new BigDecimal[] {BigDecimal.ONE, BigDecimal.TEN}, po.<String[]>field("decimalArrayField")));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStringArrayField() throws Exception {
+        BinaryObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("stringArrayField", new String[] {"str1", "str2", "str3"});
+
+        BinaryObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertTrue(Arrays.equals(new String[] {"str1", "str2", "str3"}, po.<String[]>field("stringArrayField")));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDateArrayField() throws Exception {
+        Date date1 = new Date();
+        Date date2 = new Date(date1.getTime() + 1000);
+
+        Date[] dateArr = new Date[] { date1, date2 };
+
+        assertTrue(Arrays.equals(dateArr, builder("C").setField("da", dateArr).build().<Date[]>field("da")));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTimestampArrayField() throws Exception {
+        Timestamp ts1 = new Timestamp(new Date().getTime());
+        Timestamp ts2 = new Timestamp(new Date().getTime() + 1000);
+
+        ts1.setNanos(1000);
+        ts2.setNanos(2000);
+
+        Timestamp[] tsArr = new Timestamp[] { ts1, ts2 };
+
+        assertTrue(Arrays.equals(tsArr, builder("C").setField("ta", tsArr).build().<Timestamp[]>field("ta")));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testUuidArrayField() throws Exception {
+        BinaryObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        UUID[] arr = new UUID[] {UUID.randomUUID(), UUID.randomUUID()};
+
+        builder.setField("uuidArrayField", arr);
+
+        BinaryObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertTrue(Arrays.equals(arr, po.<UUID[]>field("uuidArrayField")));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testObjectField() throws Exception {
+        BinaryObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("objectField", new Value(1));
+
+        BinaryObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertEquals(1, po.<BinaryObject>field("objectField").<Value>deserialize().i);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testObjectArrayField() throws Exception {
+        BinaryObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("objectArrayField", new Value[] {new Value(1), new Value(2)});
+
+        BinaryObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        Object[] arr = po.field("objectArrayField");
+
+        assertEquals(2, arr.length);
+
+        assertEquals(1, ((BinaryObject)arr[0]).<Value>deserialize().i);
+        assertEquals(2, ((BinaryObject)arr[1]).<Value>deserialize().i);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCollectionField() throws Exception {
+        BinaryObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("collectionField", Arrays.asList(new Value(1), new Value(2)));
+
+        BinaryObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        List<BinaryObject> list = po.field("collectionField");
+
+        assertEquals(2, list.size());
+
+        assertEquals(1, list.get(0).<Value>deserialize().i);
+        assertEquals(2, list.get(1).<Value>deserialize().i);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMapField() throws Exception {
+        BinaryObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("mapField", F.asMap(new Key(1), new Value(1), new Key(2), new Value(2)));
+
+        BinaryObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        Map<BinaryObject, BinaryObject> map = po.field("mapField");
+
+        assertEquals(2, map.size());
+
+        for (Map.Entry<BinaryObject, BinaryObject> e : map.entrySet())
+            assertEquals(e.getKey().<Key>deserialize().i, e.getValue().<Value>deserialize().i);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSeveralFields() throws Exception {
+        BinaryObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("i", 111);
+        builder.setField("f", 111.111f);
+        builder.setField("iArr", new int[] {1, 2, 3});
+        builder.setField("obj", new Key(1));
+        builder.setField("col", Arrays.asList(new Value(1), new Value(2)));
+
+        BinaryObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertEquals(111, po.<Integer>field("i").intValue());
+        assertEquals(111.111f, po.<Float>field("f").floatValue(), 0);
+        assertTrue(Arrays.equals(new int[] {1, 2, 3}, po.<int[]>field("iArr")));
+        assertEquals(1, po.<BinaryObject>field("obj").<Key>deserialize().i);
+
+        List<BinaryObject> list = po.field("col");
+
+        assertEquals(2, list.size());
+
+        assertEquals(1, list.get(0).<Value>deserialize().i);
+        assertEquals(2, list.get(1).<Value>deserialize().i);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testOffheapPortable() throws Exception {
+        BinaryObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("i", 111);
+        builder.setField("f", 111.111f);
+        builder.setField("iArr", new int[] {1, 2, 3});
+        builder.setField("obj", new Key(1));
+        builder.setField("col", Arrays.asList(new Value(1), new Value(2)));
+
+        BinaryObject po = builder.build();
+
+        byte[] arr = ((CacheObjectBinaryProcessorImpl)(grid(0)).context().cacheObjects()).marshal(po);
+
+        long ptr = UNSAFE.allocateMemory(arr.length + 5);
+
+        try {
+            long ptr0 = ptr;
+
+            UNSAFE.putBoolean(null, ptr0++, false);
+
+            UNSAFE.putInt(ptr0, arr.length);
+
+            UNSAFE.copyMemory(arr, BYTE_ARR_OFF, null, ptr0 + 4, arr.length);
+
+            BinaryObject offheapObj = (BinaryObject)
+                ((CacheObjectBinaryProcessorImpl)(grid(0)).context().cacheObjects()).unmarshal(ptr, false);
+
+            assertEquals(BinaryObjectOffheapImpl.class, offheapObj.getClass());
+
+            assertEquals("class".hashCode(), offheapObj.typeId());
+            assertEquals(100, offheapObj.hashCode());
+
+            assertEquals(111, offheapObj.<Integer>field("i").intValue());
+            assertEquals(111.111f, offheapObj.<Float>field("f").floatValue(), 0);
+            assertTrue(Arrays.equals(new int[] {1, 2, 3}, offheapObj.<int[]>field("iArr")));
+            assertEquals(1, offheapObj.<BinaryObject>field("obj").<Key>deserialize().i);
+
+            List<BinaryObject> list = offheapObj.field("col");
+
+            assertEquals(2, list.size());
+
+            assertEquals(1, list.get(0).<Value>deserialize().i);
+            assertEquals(2, list.get(1).<Value>deserialize().i);
+
+            assertEquals(po, offheapObj);
+            assertEquals(offheapObj, po);
+        }
+        finally {
+            UNSAFE.freeMemory(ptr);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBuildAndDeserialize() throws Exception {
+        BinaryObjectBuilder builder = builder(Value.class.getName());
+
+        builder.hashCode(100);
+
+        builder.setField("i", 1);
+
+        BinaryObject po = builder.build();
+
+        assertEquals("value".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertEquals(1, po.<Value>deserialize().i);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMetaData2() throws Exception {
+        BinaryObjectBuilder builder = builder("org.test.MetaTest2");
+
+        builder.setField("objectField", "a", Object.class);
+
+        BinaryObject po = builder.build();
+
+        BinaryType meta = po.type();
+
+        assertEquals("MetaTest2", meta.typeName());
+        assertEquals("Object", meta.fieldTypeName("objectField"));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMetaData() throws Exception {
+        BinaryObjectBuilder builder = builder("org.test.MetaTest");
+
+        builder.hashCode(100);
+
+        builder.setField("intField", 1);
+        builder.setField("byteArrayField", new byte[] {1, 2, 3});
+
+        BinaryObject po = builder.build();
+
+        BinaryType meta = po.type();
+
+        assertEquals("MetaTest", meta.typeName());
+
+        Collection<String> fields = meta.fieldNames();
+
+        assertEquals(2, fields.size());
+
+        assertTrue(fields.contains("intField"));
+        assertTrue(fields.contains("byteArrayField"));
+
+        assertEquals("int", meta.fieldTypeName("intField"));
+        assertEquals("byte[]", meta.fieldTypeName("byteArrayField"));
+
+        builder = builder("org.test.MetaTest");
+
+        builder.hashCode(100);
+
+        builder.setField("intField", 2);
+        builder.setField("uuidField", UUID.randomUUID());
+
+        po = builder.build();
+
+        meta = po.type();
+
+        assertEquals("MetaTest", meta.typeName());
+
+        fields = meta.fieldNames();
+
+        assertEquals(3, fields.size());
+
+        assertTrue(fields.contains("intField"));
+        assertTrue(fields.contains("byteArrayField"));
+        assertTrue(fields.contains("uuidField"));
+
+        assertEquals("int", meta.fieldTypeName("intField"));
+        assertEquals("byte[]", meta.fieldTypeName("byteArrayField"));
+        assertEquals("UUID", meta.fieldTypeName("uuidField"));
+    }
+
+    /**
+     *
+     */
+    public void testGetFromCopiedObj() {
+        BinaryObject objStr = builder(TestObjectAllTypes.class.getName()).setField("str", "aaa").build();
+
+        BinaryObjectBuilderImpl builder = builder(objStr);
+        assertEquals("aaa", builder.getField("str"));
+
+        builder.setField("str", "bbb");
+        assertEquals("bbb", builder.getField("str"));
+
+        assertNull(builder.getField("i_"));
+        assertEquals("bbb", builder.build().<TestObjectAllTypes>deserialize().str);
+    }
+
+    /**
+     *
+     */
+    public void testCopyFromInnerObjects() {
+        ArrayList<Object> list = new ArrayList<>();
+        list.add(new TestObjectAllTypes());
+        list.add(list.get(0));
+
+        TestObjectContainer c = new TestObjectContainer(list);
+
+        BinaryObjectBuilderImpl builder = builder(toPortable(c));
+        builder.<List>getField("foo").add("!!!");
+
+        BinaryObject res = builder.build();
+
+        TestObjectContainer deserialized = res.deserialize();
+
+        List deserializedList = (List)deserialized.foo;
+
+        assertSame(deserializedList.get(0), deserializedList.get(1));
+        assertEquals("!!!", deserializedList.get(2));
+        assertTrue(deserializedList.get(0) instanceof TestObjectAllTypes);
+    }
+
+    /**
+     *
+     */
+    public void testSetPortableObject() {
+        BinaryObject portableObj = builder(TestObjectContainer.class.getName())
+            .setField("foo", toPortable(new TestObjectAllTypes()))
+            .build();
+
+        assertTrue(portableObj.<TestObjectContainer>deserialize().foo instanceof TestObjectAllTypes);
+    }
+
+    /**
+     *
+     */
+    public void testPlainPortableObjectCopyFrom() {
+        TestObjectPlainPortable obj = new TestObjectPlainPortable(toPortable(new TestObjectAllTypes()));
+
+        BinaryObjectBuilderImpl builder = builder(toPortable(obj));
+        assertTrue(builder.getField("plainPortable") instanceof BinaryObject);
+
+        TestObjectPlainPortable deserialized = builder.build().deserialize();
+        assertTrue(deserialized.plainPortable instanceof BinaryObject);
+    }
+
+    /**
+     *
+     */
+    public void testRemoveFromNewObject() {
+        BinaryObjectBuilder builder = builder(TestObjectAllTypes.class.getName());
+
+        builder.setField("str", "a");
+
+        builder.removeField("str");
+
+        assertNull(builder.build().<TestObjectAllTypes>deserialize().str);
+    }
+
+    /**
+     *
+     */
+    public void testRemoveFromExistingObject() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+        obj.setDefaultData();
+        obj.enumArr = null;
+
+        BinaryObjectBuilder builder = builder(toPortable(obj));
+
+        builder.removeField("str");
+
+        BinaryObject binary = builder.build();
+
+        TestObjectAllTypes deserialzied = binary.deserialize();
+
+        assertNull(deserialzied.str);
+    }
+
+    /**
+     *
+     */
+    public void testRemoveFromExistingObjectAfterGet() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+        obj.setDefaultData();
+        obj.enumArr = null;
+
+        BinaryObjectBuilderImpl builder = builder(toPortable(obj));
+
+        builder.getField("i_");
+
+        builder.removeField("str");
+
+        assertNull(builder.build().<TestObjectAllTypes>deserialize().str);
+    }
+
+    /**
+     * @throws IgniteCheckedException If any error occurs.
+     */
+    public void testDontBrokeCyclicDependency() throws IgniteCheckedException {
+        TestObjectOuter outer = new TestObjectOuter();
+        outer.inner = new TestObjectInner();
+        outer.inner.outer = outer;
+        outer.foo = "a";
+
+        BinaryObjectBuilder builder = builder(toPortable(outer));
+
+        builder.setField("foo", "b");
+
+        TestObjectOuter res = builder.build().deserialize();
+
+        assertEquals("b", res.foo);
+        assertSame(res, res.inner.outer);
+    }
+
+    /**
+     * @return Portables.
+     */
+    private IgniteBinary portables() {
+        return grid(0).binary();
+    }
+
+    /**
+     * @param obj Object.
+     * @return Portable object.
+     */
+    private BinaryObject toPortable(Object obj) {
+        return portables().toBinary(obj);
+    }
+
+    /**
+     * @return Builder.
+     */
+    private <T> BinaryObjectBuilder builder(String clsName) {
+        return portables().builder(clsName);
+    }
+
+    /**
+     * @return Builder.
+     */
+    private <T> BinaryObjectBuilderImpl builder(BinaryObject obj) {
+        return (BinaryObjectBuilderImpl)portables().builder(obj);
+    }
+
+    /**
+     *
+     */
+    private static class CustomIdMapper {
+        /** */
+        private String str = "a";
+
+        /** */
+        private int i = 10;
+    }
+
+    /**
+     */
+    private static class Key {
+        /** */
+        private int i;
+
+        /**
+         */
+        private Key() {
+            // No-op.
+        }
+
+        /**
+         * @param i Index.
+         */
+        private Key(int i) {
+            this.i = i;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            Key key = (Key)o;
+
+            return i == key.i;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return i;
+        }
+    }
+
+    /**
+     */
+    private static class Value {
+        /** */
+        private int i;
+
+        /**
+         */
+        private Value() {
+            // No-op.
+        }
+
+        /**
+         * @param i Index.
+         */
+        private Value(int i) {
+            this.i = i;
+        }
+    }
+}
\ No newline at end of file


[07/13] ignite git commit: IGNITE-1816: Implemented compact footers.

Posted by ag...@apache.org.
IGNITE-1816: Implemented compact footers.


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

Branch: refs/heads/ignite-1945
Commit: 0b4a8f831fe2d183e6e831c90da0d3fc86ac2ed0
Parents: 66c84ea
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Nov 18 14:54:38 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Nov 18 14:54:38 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/IgniteKernal.java    |    6 +
 .../ignite/internal/IgniteNodeAttributes.java   |    3 +
 .../portable/BinaryCachingMetadataHandler.java  |   70 +
 .../internal/portable/BinaryMetadata.java       |   16 +-
 .../portable/BinaryMetadataCollector.java       |   49 +-
 .../internal/portable/BinaryObjectEx.java       |    2 +-
 .../internal/portable/BinaryObjectImpl.java     |   14 +-
 .../portable/BinaryObjectOffheapImpl.java       |   14 +-
 .../internal/portable/BinaryReaderExImpl.java   |  132 +-
 .../internal/portable/BinaryTypeImpl.java       |    7 +-
 .../internal/portable/BinaryWriterExImpl.java   |  161 +-
 .../portable/PortableClassDescriptor.java       |  102 +-
 .../internal/portable/PortableContext.java      |   66 +-
 .../internal/portable/PortableSchema.java       |  296 +-
 .../ignite/internal/portable/PortableUtils.java |  457 ++-
 .../builder/BinaryObjectBuilderImpl.java        |  130 +-
 .../portable/builder/PortableBuilderReader.java |   21 +-
 .../portable/CacheObjectBinaryProcessor.java    |    2 +-
 .../CacheObjectBinaryProcessorImpl.java         |  162 +-
 .../platform/PlatformContextImpl.java           |   10 +-
 .../cpp/PlatformCppConfigurationClosure.java    |    9 +-
 .../PlatformDotNetConfigurationClosure.java     |    9 +-
 .../ignite/internal/util/IgniteUtils.java       |   25 +
 .../marshaller/portable/PortableMarshaller.java |   38 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |   51 +-
 .../portable/BinaryFieldsAbstractSelfTest.java  |   13 +-
 .../BinaryFooterOffsetsAbstractSelfTest.java    |  199 +
 .../BinaryFooterOffsetsHeapSelfTest.java        |   32 +
 .../BinaryFooterOffsetsOffheapSelfTest.java     |   61 +
 .../portable/BinaryMarshallerSelfTest.java      | 3795 ++++++++++++++++++
 .../BinaryObjectBuilderAdditionalSelfTest.java  | 1291 ++++++
 .../portable/BinaryObjectBuilderSelfTest.java   | 1066 +++++
 ...idBinaryObjectBuilderAdditionalSelfTest.java | 1280 ------
 .../GridBinaryObjectBuilderSelfTest.java        | 1053 -----
 ...idPortableMarshallerCtxDisabledSelfTest.java |    2 +-
 .../GridPortableMarshallerSelfTest.java         | 3760 -----------------
 .../PortableCompactOffsetsAbstractSelfTest.java |  190 -
 .../PortableCompactOffsetsHeapSelfTest.java     |   32 -
 .../PortableCompactOffsetsOffheapSelfTest.java  |   61 -
 .../BinaryFieldsHeapNonCompactSelfTest.java     |   34 +
 .../BinaryFieldsOffheapNonCompactSelfTest.java  |   30 +
 ...naryFooterOffsetsHeapNonCompactSelfTest.java |   30 +
 ...yFooterOffsetsOffheapNonCompactSelfTest.java |   30 +
 .../BinaryMarshallerNonCompactSelfTest.java     |   30 +
 ...jectBuilderAdditionalNonCompactSelfTest.java |   30 +
 .../BinaryObjectBuilderNonCompactSelfTest.java  |   30 +
 .../IgnitePortableObjectsTestSuite.java         |   36 +-
 .../core-test/src/binary_reader_writer_test.cpp |   64 +-
 .../include/ignite/impl/binary/binary_common.h  |   22 +-
 .../ignite/impl/binary/binary_reader_impl.h     |   58 +-
 .../include/ignite/impl/binary/binary_schema.h  |    6 +-
 .../ignite/impl/binary/binary_writer_impl.h     |    2 +-
 .../core/src/impl/binary/binary_reader_impl.cpp |    8 +-
 .../cpp/core/src/impl/binary/binary_schema.cpp  |   12 +-
 .../core/src/impl/binary/binary_writer_impl.cpp |   29 +-
 .../Config/Compute/compute-grid1.xml            |    1 +
 .../Config/marshaller-explicit.xml              |    4 +-
 .../Impl/Binary/BinaryObjectBuilder.cs          |   25 +-
 .../Impl/Binary/BinaryObjectHeader.cs           |  131 +-
 .../Impl/Binary/BinaryObjectSchemaHolder.cs     |    9 +-
 .../Impl/Binary/BinaryReader.cs                 |    2 +-
 .../Impl/Binary/BinaryWriter.cs                 |   27 +-
 .../Impl/Binary/IgniteBinary.cs                 |    3 +-
 63 files changed, 8154 insertions(+), 7186 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index c4829a4..2b6eaad 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -157,6 +157,7 @@ import org.apache.ignite.lifecycle.LifecycleBean;
 import org.apache.ignite.lifecycle.LifecycleEventType;
 import org.apache.ignite.marshaller.MarshallerExclusions;
 import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
+import org.apache.ignite.marshaller.portable.PortableMarshaller;
 import org.apache.ignite.mxbean.ClusterLocalNodeMetricsMXBean;
 import org.apache.ignite.mxbean.IgniteMXBean;
 import org.apache.ignite.mxbean.ThreadPoolMXBean;
@@ -201,6 +202,7 @@ import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_JVM_PID;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_LANG_RUNTIME;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MACS;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_COMPACT_FOOTER;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_USE_DFLT_SUID;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_NODE_CONSISTENT_ID;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PEER_CLASSLOADING;
@@ -1272,6 +1274,10 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         add(ATTR_MARSHALLER, cfg.getMarshaller().getClass().getName());
         add(ATTR_MARSHALLER_USE_DFLT_SUID,
             getBoolean(IGNITE_OPTIMIZED_MARSHALLER_USE_DEFAULT_SUID, OptimizedMarshaller.USE_DFLT_SUID));
+
+        if (cfg.getMarshaller() instanceof PortableMarshaller)
+            add(ATTR_MARSHALLER_COMPACT_FOOTER, ((PortableMarshaller)cfg.getMarshaller()).isCompactFooter());
+
         add(ATTR_USER_NAME, System.getProperty("user.name"));
         add(ATTR_GRID_NAME, gridName);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
index 86a460d..946b686 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
@@ -36,6 +36,9 @@ public final class IgniteNodeAttributes {
     /** Internal attribute name constant. */
     public static final String ATTR_MARSHALLER_USE_DFLT_SUID = ATTR_PREFIX + ".marshaller.useDefaultSUID";
 
+    /** Attribute for marshaller compact footers. */
+    public static final String ATTR_MARSHALLER_COMPACT_FOOTER = ATTR_PREFIX + ".marshaller.compactFooter";
+
     /** Internal attribute name constant. */
     public static final String ATTR_JIT_NAME = ATTR_PREFIX + ".jit.name";
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryCachingMetadataHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryCachingMetadataHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryCachingMetadataHandler.java
new file mode 100644
index 0000000..a3c846b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryCachingMetadataHandler.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.portable;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryType;
+
+import java.util.HashMap;
+
+/**
+ * Simple caching metadata handler. Used mainly in tests.
+ */
+public class BinaryCachingMetadataHandler implements BinaryMetadataHandler {
+    /** Cached metadatas. */
+    private final HashMap<Integer, BinaryType> metas = new HashMap<>();
+
+    /**
+     * Create new handler instance.
+     *
+     * @return New handler.
+     */
+    public static BinaryCachingMetadataHandler create() {
+        return new BinaryCachingMetadataHandler();
+    }
+
+    /**
+     * Private constructor.
+     */
+    private BinaryCachingMetadataHandler() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized void addMeta(int typeId, BinaryType type) throws BinaryObjectException {
+        synchronized (this) {
+            BinaryType oldType = metas.put(typeId, type);
+
+            if (oldType != null) {
+                BinaryMetadata oldMeta = ((BinaryTypeImpl)oldType).metadata();
+                BinaryMetadata newMeta = ((BinaryTypeImpl)type).metadata();
+
+                BinaryMetadata mergedMeta = PortableUtils.mergeMetadata(oldMeta, newMeta);
+
+                BinaryType mergedType = mergedMeta.wrap(((BinaryTypeImpl)oldType).context());
+
+                metas.put(typeId, mergedType);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized BinaryType metadata(int typeId) throws BinaryObjectException {
+        return metas.get(typeId);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadata.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadata.java
index fe88d11..a464d6e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadata.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadata.java
@@ -49,6 +49,9 @@ public class BinaryMetadata implements Externalizable {
     /** Affinity key field name. */
     private String affKeyFieldName;
 
+    /** Schemas associated with type. */
+    private Collection<PortableSchema> schemas;
+
     /**
      * For {@link Externalizable}.
      */
@@ -63,15 +66,17 @@ public class BinaryMetadata implements Externalizable {
      * @param typeName Type name.
      * @param fields Fields map.
      * @param affKeyFieldName Affinity key field name.
+     * @param schemas Schemas.
      */
     public BinaryMetadata(int typeId, String typeName, @Nullable Map<String, Integer> fields,
-        @Nullable String affKeyFieldName) {
+        @Nullable String affKeyFieldName, @Nullable Collection<PortableSchema> schemas) {
         assert typeName != null;
 
         this.typeId = typeId;
         this.typeName = typeName;
         this.fields = fields;
         this.affKeyFieldName = affKeyFieldName;
+        this.schemas = schemas;
     }
 
     /**
@@ -120,6 +125,13 @@ public class BinaryMetadata implements Externalizable {
     }
 
     /**
+     * @return Schemas.
+     */
+    public Collection<PortableSchema> schemas() {
+        return schemas != null ? schemas : Collections.<PortableSchema>emptyList();
+    }
+
+    /**
      * Wrap metadata into binary type.
      *
      * @param ctx Portable context.
@@ -135,6 +147,7 @@ public class BinaryMetadata implements Externalizable {
         U.writeString(out, typeName);
         U.writeMap(out, fields);
         U.writeString(out, affKeyFieldName);
+        U.writeCollection(out, schemas);
     }
 
     /** {@inheritDoc} */
@@ -143,6 +156,7 @@ public class BinaryMetadata implements Externalizable {
         typeName = U.readString(in);
         fields = U.readMap(in);
         affKeyFieldName = U.readString(in);
+        schemas = U.readCollection(in);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadataCollector.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadataCollector.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadataCollector.java
index 67e1a0d..28eb1d0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadataCollector.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadataCollector.java
@@ -17,6 +17,12 @@
 
 package org.apache.ignite.internal.portable;
 
+import org.apache.ignite.binary.BinaryIdMapper;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryRawWriter;
+import org.apache.ignite.binary.BinaryWriter;
+import org.jetbrains.annotations.Nullable;
+
 import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.Method;
 import java.lang.reflect.Proxy;
@@ -27,26 +33,37 @@ import java.util.Date;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.UUID;
-import org.apache.ignite.binary.BinaryObjectException;
-import org.apache.ignite.binary.BinaryRawWriter;
-import org.apache.ignite.binary.BinaryWriter;
-import org.jetbrains.annotations.Nullable;
 
 /**
  * Writer for meta data collection.
  */
 class BinaryMetadataCollector implements BinaryWriter {
-    /** */
-    private final Map<String, Integer> meta = new HashMap<>();
+    /** Type ID. */
+    private final int typeId;
 
-    /** */
+    /** Type name. */
     private final String typeName;
 
+    /** ID mapper. */
+    private final BinaryIdMapper idMapper;
+
+    /** Collected metadata. */
+    private final Map<String, Integer> meta = new HashMap<>();
+
+    /** Schema builder. */
+    private PortableSchema.Builder schemaBuilder = PortableSchema.Builder.newBuilder();
+
     /**
+     * Constructor.
+     *
+     * @param typeId Type ID.
      * @param typeName Type name.
+     * @param idMapper ID mapper.
      */
-    BinaryMetadataCollector(String typeName) {
+    BinaryMetadataCollector(int typeId, String typeName, BinaryIdMapper idMapper) {
+        this.typeId = typeId;
         this.typeName = typeName;
+        this.idMapper = idMapper;
     }
 
     /**
@@ -56,6 +73,13 @@ class BinaryMetadataCollector implements BinaryWriter {
         return meta;
     }
 
+    /**
+     * @return Schemas.
+     */
+    PortableSchema schema() {
+        return schemaBuilder.build();
+    }
+
     /** {@inheritDoc} */
     @Override public void writeByte(String fieldName, byte val) throws BinaryObjectException {
         add(fieldName, PortableClassDescriptor.Mode.BYTE);
@@ -242,13 +266,12 @@ class BinaryMetadataCollector implements BinaryWriter {
 
         if (oldFieldTypeId != null && !oldFieldTypeId.equals(fieldTypeId)) {
             throw new BinaryObjectException(
-                "Field is written twice with different types [" +
-                "typeName=" + typeName +
-                ", fieldName=" + name +
+                "Field is written twice with different types [" + "typeName=" + typeName + ", fieldName=" + name +
                 ", fieldTypeName1=" + PortableUtils.fieldTypeName(oldFieldTypeId) +
-                ", fieldTypeName2=" + PortableUtils.fieldTypeName(fieldTypeId) +
-                ']'
+                ", fieldTypeName2=" + PortableUtils.fieldTypeName(fieldTypeId) + ']'
             );
         }
+
+        schemaBuilder.addField(idMapper.fieldId(typeId, name));
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/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 b3512ce..6902675 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
@@ -170,7 +170,7 @@ public abstract class BinaryObjectEx implements BinaryObject {
         }
 
         if (meta == null)
-            return "PortableObject [hash=" + idHash + ", typeId=" + typeId() + ']';
+            return BinaryObject.class.getSimpleName() +  " [hash=" + idHash + ", typeId=" + typeId() + ']';
 
         handles.put(this, idHash);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/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 d432ea0..d9339f8 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
@@ -251,7 +251,7 @@ public final class BinaryObjectImpl extends BinaryObjectEx implements Externaliz
         if (ctx == null)
             throw new BinaryObjectException("PortableContext is not set for the object.");
 
-        return ctx.metaData(typeId());
+        return ctx.metadata(typeId());
     }
 
     /** {@inheritDoc} */
@@ -279,15 +279,17 @@ public final class BinaryObjectImpl extends BinaryObjectEx implements Externaliz
         int schemaOffset = PortablePrimitives.readInt(arr, start + GridPortableMarshaller.SCHEMA_OR_RAW_OFF_POS);
 
         short flags = PortablePrimitives.readShort(arr, start + GridPortableMarshaller.FLAGS_POS);
-        int fieldOffsetSize = PortableUtils.fieldOffsetSize(flags);
 
-        int fieldOffsetPos = start + schemaOffset + order * (4 + fieldOffsetSize) + 4;
+        int fieldIdLen = PortableUtils.isCompactFooter(flags) ? 0 : PortableUtils.FIELD_ID_LEN;
+        int fieldOffsetLen = PortableUtils.fieldOffsetLength(flags);
+
+        int fieldOffsetPos = start + schemaOffset + order * (fieldIdLen + fieldOffsetLen) + fieldIdLen;
 
         int fieldPos;
 
-        if (fieldOffsetSize == PortableUtils.OFFSET_1)
+        if (fieldOffsetLen == PortableUtils.OFFSET_1)
             fieldPos = start + ((int)PortablePrimitives.readByte(arr, fieldOffsetPos) & 0xFF);
-        else if (fieldOffsetSize == PortableUtils.OFFSET_2)
+        else if (fieldOffsetLen == PortableUtils.OFFSET_2)
             fieldPos = start + ((int)PortablePrimitives.readShort(arr, fieldOffsetPos) & 0xFFFF);
         else
             fieldPos = start + PortablePrimitives.readInt(arr, fieldOffsetPos);
@@ -458,7 +460,7 @@ public final class BinaryObjectImpl extends BinaryObjectEx implements Externaliz
     @Override protected PortableSchema createSchema() {
         BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx, arr, start, null);
 
-        return reader.createSchema();
+        return reader.getOrCreateSchema();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/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 f7cb844..a71c98a 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
@@ -136,7 +136,7 @@ public class BinaryObjectOffheapImpl extends BinaryObjectEx implements Externali
             start,
             null);
 
-        return reader.createSchema();
+        return reader.getOrCreateSchema();
     }
 
     /** {@inheritDoc} */
@@ -164,7 +164,7 @@ public class BinaryObjectOffheapImpl extends BinaryObjectEx implements Externali
         if (ctx == null)
             throw new BinaryObjectException("PortableContext is not set for the object.");
 
-        return ctx.metaData(typeId());
+        return ctx.metadata(typeId());
     }
 
     /** {@inheritDoc} */
@@ -198,15 +198,17 @@ public class BinaryObjectOffheapImpl extends BinaryObjectEx implements Externali
         int schemaOffset = PortablePrimitives.readInt(ptr, start + GridPortableMarshaller.SCHEMA_OR_RAW_OFF_POS);
 
         short flags = PortablePrimitives.readShort(ptr, start + GridPortableMarshaller.FLAGS_POS);
-        int fieldOffsetSize = PortableUtils.fieldOffsetSize(flags);
 
-        int fieldOffsetPos = start + schemaOffset + order * (4 + fieldOffsetSize) + 4;
+        int fieldIdLen = PortableUtils.isCompactFooter(flags) ? 0 : PortableUtils.FIELD_ID_LEN;
+        int fieldOffsetLen = PortableUtils.fieldOffsetLength(flags);
+
+        int fieldOffsetPos = start + schemaOffset + order * (fieldIdLen + fieldOffsetLen) + fieldIdLen;
 
         int fieldPos;
 
-        if (fieldOffsetSize == PortableUtils.OFFSET_1)
+        if (fieldOffsetLen == PortableUtils.OFFSET_1)
             fieldPos = start + ((int)PortablePrimitives.readByte(ptr, fieldOffsetPos) & 0xFF);
-        else if (fieldOffsetSize == PortableUtils.OFFSET_2)
+        else if (fieldOffsetLen == PortableUtils.OFFSET_2)
             fieldPos = start + ((int)PortablePrimitives.readShort(ptr, fieldOffsetPos) & 0xFFFF);
         else
             fieldPos = start + PortablePrimitives.readInt(ptr, fieldOffsetPos);

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
index 669ba01..6ff3047 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
@@ -17,6 +17,23 @@
 
 package org.apache.ignite.internal.portable;
 
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.binary.BinaryIdMapper;
+import org.apache.ignite.binary.BinaryInvalidTypeException;
+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.portable.streams.PortableHeapInputStream;
+import org.apache.ignite.internal.portable.streams.PortableInputStream;
+import org.apache.ignite.internal.util.GridEnumCache;
+import org.apache.ignite.internal.util.lang.GridMapEntry;
+import org.apache.ignite.internal.util.typedef.internal.SB;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
 import java.io.ByteArrayInputStream;
 import java.io.EOFException;
 import java.io.IOException;
@@ -30,7 +47,6 @@ import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Date;
-import java.util.LinkedHashMap;
 import java.util.LinkedList;
 import java.util.Map;
 import java.util.Properties;
@@ -39,22 +55,6 @@ import java.util.TreeSet;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentSkipListSet;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.binary.BinaryObject;
-import org.apache.ignite.binary.BinaryObjectException;
-import org.apache.ignite.binary.BinaryIdMapper;
-import org.apache.ignite.binary.BinaryInvalidTypeException;
-import org.apache.ignite.binary.BinaryRawReader;
-import org.apache.ignite.binary.BinaryReader;
-import org.apache.ignite.internal.portable.streams.PortableHeapInputStream;
-import org.apache.ignite.internal.portable.streams.PortableInputStream;
-import org.apache.ignite.internal.util.GridEnumCache;
-import org.apache.ignite.internal.util.lang.GridMapEntry;
-import org.apache.ignite.internal.util.typedef.internal.SB;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.lang.IgniteBiTuple;
-import org.jetbrains.annotations.NotNull;
-import org.jetbrains.annotations.Nullable;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.ARR_LIST;
@@ -117,9 +117,6 @@ import static org.apache.ignite.internal.portable.GridPortableMarshaller.UUID_AR
  */
 @SuppressWarnings("unchecked")
 public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, ObjectInput {
-    /** Length of a single field descriptor. */
-    private static final int FIELD_DESC_LEN = 16;
-
     /** */
     private final PortableContext ctx;
 
@@ -162,8 +159,14 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Obje
     /** Schema Id. */
     private int schemaId;
 
+    /** Whether this is user type or not. */
+    private boolean userType;
+
+    /** Whether field IDs exist. */
+    private int fieldIdLen;
+
     /** Offset size in bytes. */
-    private int offsetSize;
+    private int fieldOffsetLen;
 
     /** Object schema. */
     private PortableSchema schema;
@@ -225,18 +228,21 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Obje
 
         short flags = in.readShort();
 
-        offsetSize = PortableUtils.fieldOffsetSize(flags);
+        userType = PortableUtils.isUserType(flags);
+
+        fieldIdLen = PortableUtils.fieldIdLength(flags);
+        fieldOffsetLen = PortableUtils.fieldOffsetLength(flags);
 
         typeId = in.readIntPositioned(start + GridPortableMarshaller.TYPE_ID_POS);
 
-        IgniteBiTuple<Integer, Integer> footer = PortableUtils.footerAbsolute(in, start, offsetSize);
+        IgniteBiTuple<Integer, Integer> footer = PortableUtils.footerAbsolute(in, start);
 
         footerStart = footer.get1();
         footerLen = footer.get2() - footerStart;
 
         schemaId = in.readIntPositioned(start + GridPortableMarshaller.SCHEMA_ID_POS);
 
-        rawOff = PortableUtils.rawOffsetAbsolute(in, start, offsetSize);
+        rawOff = PortableUtils.rawOffsetAbsolute(in, start);
 
         if (typeId == UNREGISTERED_TYPE_ID) {
             // Skip to the class name position.
@@ -2555,29 +2561,68 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Obje
     }
 
     /**
-     * Create schema.
+     * Get or create object schema.
      *
      * @return Schema.
      */
-    public PortableSchema createSchema() {
+    public PortableSchema getOrCreateSchema() {
         parseHeaderIfNeeded();
 
-        LinkedHashMap<Integer, Integer> fields = new LinkedHashMap<>();
+        PortableSchema schema = ctx.schemaRegistry(typeId).schema(schemaId);
+
+        if (schema == null) {
+            if (fieldIdLen != PortableUtils.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()) {
+                    if (schemaId == typeSchema.schemaId()) {
+                        schema = typeSchema;
+
+                        break;
+                    }
+                }
+
+                if (schema == null)
+                    throw new BinaryObjectException("Cannot find schema for object with compact footer [" +
+                        "typeId=" + typeId + ", schemaId=" + schemaId + ']');
+            }
+            else
+                schema = createSchema();
+
+            assert schema != null;
+
+            ctx.schemaRegistry(typeId).addSchema(schemaId, schema);
+        }
+
+        return schema;
+    }
+
+    /**
+     * Create schema.
+     *
+     * @return Schema.
+     */
+    private PortableSchema createSchema() {
+        assert fieldIdLen == PortableUtils.FIELD_ID_LEN;
+
+        PortableSchema.Builder builder = PortableSchema.Builder.newBuilder();
 
         int searchPos = footerStart;
         int searchEnd = searchPos + footerLen;
 
-        int idx = 0;
-
         while (searchPos < searchEnd) {
             int fieldId = in.readIntPositioned(searchPos);
 
-            fields.put(fieldId, idx++);
+            builder.addField(fieldId);
 
-            searchPos += 4 + offsetSize;
+            searchPos += PortableUtils.FIELD_ID_LEN + fieldOffsetLen;
         }
 
-        return new PortableSchema(fields);
+        return builder.build();
     }
 
     /**
@@ -2593,7 +2638,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Obje
         int searchPos = footerStart;
         int searchTail = searchPos + footerLen;
 
-        if (hasLowFieldsCount(footerLen)) {
+        if (!userType || (fieldIdLen != 0 && hasLowFieldsCount(footerLen))) {
             while (true) {
                 if (searchPos >= searchTail)
                     return 0;
@@ -2601,37 +2646,32 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Obje
                 int id0 = in.readIntPositioned(searchPos);
 
                 if (id0 == id) {
-                    int pos = start + PortableUtils.fieldOffsetRelative(in, searchPos + 4, offsetSize);
+                    int pos = start + PortableUtils.fieldOffsetRelative(in, searchPos + PortableUtils.FIELD_ID_LEN,
+                        fieldOffsetLen);
 
                     in.position(pos);
 
                     return pos;
                 }
 
-                searchPos += 4 + offsetSize;
+                searchPos += PortableUtils.FIELD_ID_LEN + fieldOffsetLen;
             }
         }
         else {
             PortableSchema schema0 = schema;
 
             if (schema0 == null) {
-                schema0 = ctx.schemaRegistry(typeId).schema(schemaId);
-
-                if (schema0 == null) {
-                    schema0 = createSchema();
-
-                    ctx.schemaRegistry(typeId).addSchema(schemaId, schema0);
-                }
+                schema0 = getOrCreateSchema();
 
                 schema = schema0;
             }
 
-            int order = schema.order(id);
+            int order = schema0.order(id);
 
             if (order != PortableSchema.ORDER_NOT_FOUND) {
-                int offsetPos = footerStart + order * (4 + offsetSize) + 4;
+                int offsetPos = footerStart + order * (fieldIdLen + fieldOffsetLen) + fieldIdLen;
 
-                int pos = start + PortableUtils.fieldOffsetRelative(in, offsetPos, offsetSize);
+                int pos = start + PortableUtils.fieldOffsetRelative(in, offsetPos, fieldOffsetLen);
 
                 in.position(pos);
 
@@ -2650,7 +2690,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Obje
     private boolean hasLowFieldsCount(int footerLen) {
         assert hdrParsed;
 
-        return footerLen < (FIELD_DESC_LEN << 4);
+        return footerLen < ((fieldOffsetLen + fieldIdLen) << 3);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/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 40b6252..60c135d 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
@@ -62,10 +62,15 @@ public class BinaryTypeImpl implements BinaryType {
         return ctx.createField(meta.typeId(), fieldName);
     }
 
-    public String affinityKeyFieldName() {
+    /** {@inheritDoc} */
+    @Override public String affinityKeyFieldName() {
         return meta.affinityKeyFieldName();
     }
 
+    /** {@inheritDoc} */
+    public PortableContext context() {
+        return ctx;
+    }
     /**
      * @return Metadata.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriterExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriterExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriterExImpl.java
index cedf1c8..6cb18fb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriterExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriterExImpl.java
@@ -90,15 +90,9 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
     /** Length: integer. */
     private static final int LEN_INT = 4;
 
-    /** */
+    /** Initial capacity. */
     private static final int INIT_CAP = 1024;
 
-    /** FNV1 hash offset basis. */
-    private static final int FNV1_OFFSET_BASIS = 0x811C9DC5;
-
-    /** FNV1 hash prime. */
-    private static final int FNV1_PRIME = 0x01000193;
-
     /** Maximum offset which fits in 1 byte. */
     private static final int MAX_OFFSET_1 = 1 << 8;
 
@@ -139,7 +133,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
     private SchemaHolder schema;
 
     /** Schema ID. */
-    private int schemaId;
+    private int schemaId = PortableUtils.schemaInitialId();
 
     /** Amount of written fields. */
     private int fieldCnt;
@@ -332,6 +326,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
 
     /**
      * Perform post-write activity. This includes:
+     * - writing flags;
      * - writing object length;
      * - writing schema offset;
      * - writing schema to the tail.
@@ -339,7 +334,16 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
      * @param userType User type flag.
      */
     public void postWrite(boolean userType) {
+        short flags = userType ? PortableUtils.FLAG_USR_TYP : 0;
+
+        boolean useCompactFooter = ctx.isCompactFooter() && userType;
+
+        if (useCompactFooter)
+            flags |= PortableUtils.FLAG_COMPACT_FOOTER;
+        
         if (schema != null) {
+            flags |= PortableUtils.FLAG_HAS_SCHEMA;
+
             // Write schema ID.
             out.writeInt(start + SCHEMA_ID_POS, schemaId);
 
@@ -347,34 +351,35 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
             out.writeInt(start + SCHEMA_OR_RAW_OFF_POS, out.position() - start);
 
             // Write the schema.
-            int offsetByteCnt = schema.write(this, fieldCnt);
+            int offsetByteCnt = schema.write(this, fieldCnt, useCompactFooter);
 
+            if (offsetByteCnt == PortableUtils.OFFSET_1)
+                flags |= PortableUtils.FLAG_OFFSET_ONE_BYTE;
+            else if (offsetByteCnt == PortableUtils.OFFSET_2)
+                flags |= PortableUtils.FLAG_OFFSET_TWO_BYTES;
+            
             // Write raw offset if needed.
-            if (rawOffPos != 0)
-                out.writeInt(rawOffPos - start);
-
-            if (offsetByteCnt == PortableUtils.OFFSET_1) {
-                int flags = (userType ? PortableUtils.FLAG_USR_TYP : 0) | PortableUtils.FLAG_OFFSET_ONE_BYTE;
-
-                out.writeShort(start + FLAGS_POS, (short)flags);
-            }
-            else if (offsetByteCnt == PortableUtils.OFFSET_2) {
-                int flags = (userType ? PortableUtils.FLAG_USR_TYP : 0) | PortableUtils.FLAG_OFFSET_TWO_BYTES;
+            if (rawOffPos != 0) {
+                flags |= PortableUtils.FLAG_HAS_RAW;
 
-                out.writeShort(start + FLAGS_POS, (short)flags);
+                out.writeInt(rawOffPos - start);
             }
         }
         else {
-            // Write raw-only flag is needed.
-            int flags = (userType ? PortableUtils.FLAG_USR_TYP : 0) | PortableUtils.FLAG_RAW_ONLY;
-
-            out.writeShort(start + FLAGS_POS, (short)flags);
+            if (rawOffPos != 0) {
+                // If there are no schema, we are free to write raw offset to schema offset.
+                flags |= PortableUtils.FLAG_HAS_RAW;
 
-            // If there are no schema, we are free to write raw offset to schema offset.
-            out.writeInt(start + SCHEMA_OR_RAW_OFF_POS, (rawOffPos == 0 ? out.position() : rawOffPos) - start);
+                out.writeInt(start + SCHEMA_OR_RAW_OFF_POS, rawOffPos - start);
+            }
+            else
+                out.writeInt(start + SCHEMA_OR_RAW_OFF_POS, 0);
         }
 
-        // 5. Write length.
+        // Write flags.
+        out.writeShort(start + FLAGS_POS, flags);
+
+        // Write length.
         out.writeInt(start + TOTAL_LEN_POS, out.position() - start);
     }
 
@@ -1737,22 +1742,9 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
 
                 SCHEMA.set(schema);
             }
-
-            // Initialize offset when the first field is written.
-            schemaId = FNV1_OFFSET_BASIS;
         }
 
-        // Advance schema hash.
-        int schemaId0 = schemaId ^ (fieldId & 0xFF);
-        schemaId0 = schemaId0 * FNV1_PRIME;
-        schemaId0 = schemaId0 ^ ((fieldId >> 8) & 0xFF);
-        schemaId0 = schemaId0 * FNV1_PRIME;
-        schemaId0 = schemaId0 ^ ((fieldId >> 16) & 0xFF);
-        schemaId0 = schemaId0 * FNV1_PRIME;
-        schemaId0 = schemaId0 ^ ((fieldId >> 24) & 0xFF);
-        schemaId0 = schemaId0 * FNV1_PRIME;
-
-        schemaId = schemaId0;
+        schemaId = PortableUtils.updateSchemaId(schemaId, fieldId);
 
         schema.push(fieldId, fieldOff);
 
@@ -1760,6 +1752,25 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
     }
 
     /**
+     * @return Current schema ID.
+     */
+    public int schemaId() {
+        return schemaId;
+    }
+
+    /**
+     * @return Current writer's schema.
+     */
+    public PortableSchema currentSchema() {
+        PortableSchema.Builder builder = PortableSchema.Builder.newBuilder();
+
+        if (schema != null)
+            schema.build(builder, fieldCnt);
+
+        return builder.build();
+    }
+
+    /**
      * Attempts to write the object as a handle.
      *
      * @param obj Object to write.
@@ -1844,13 +1855,25 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
         }
 
         /**
+         * Build the schema.
+         *
+         * @param builder Builder.
+         * @param fieldCnt Fields count.
+         */
+        public void build(PortableSchema.Builder builder, int fieldCnt) {
+            for (int curIdx = idx - fieldCnt * 2; curIdx < idx; curIdx += 2)
+                builder.addField(data[curIdx]);
+        }
+
+        /**
          * Write collected frames and pop them.
          *
          * @param writer Writer.
          * @param fieldCnt Count.
-         * @return Amount of bytes dedicated to
+         * @param compactFooter Whether footer should be written in compact form.
+         * @return Amount of bytes dedicated to each field offset. Could be 1, 2 or 4.
          */
-        public int write(BinaryWriterExImpl writer, int fieldCnt) {
+        public int write(BinaryWriterExImpl writer, int fieldCnt, boolean compactFooter) {
             int startIdx = idx - fieldCnt * 2;
 
             assert startIdx >= 0;
@@ -1859,29 +1882,51 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
 
             int res;
 
-            if (lastOffset < MAX_OFFSET_1) {
-                for (int idx0 = startIdx; idx0 < idx; ) {
-                    writer.writeInt(data[idx0++]);
-                    writer.writeByte((byte) data[idx0++]);
+            if (compactFooter) {
+                if (lastOffset < MAX_OFFSET_1) {
+                    for (int curIdx = startIdx + 1; curIdx < idx; curIdx += 2)
+                        writer.writeByte((byte)data[curIdx]);
+
+                    res = PortableUtils.OFFSET_1;
                 }
+                else if (lastOffset < MAX_OFFSET_2) {
+                    for (int curIdx = startIdx + 1; curIdx < idx; curIdx += 2)
+                        writer.writeShort((short)data[curIdx]);
 
-                res = PortableUtils.OFFSET_1;
-            }
-            else if (lastOffset < MAX_OFFSET_2) {
-                for (int idx0 = startIdx; idx0 < idx; ) {
-                    writer.writeInt(data[idx0++]);
-                    writer.writeShort((short)data[idx0++]);
+                    res = PortableUtils.OFFSET_2;
                 }
+                else {
+                    for (int curIdx = startIdx + 1; curIdx < idx; curIdx += 2)
+                        writer.writeInt(data[curIdx]);
 
-                res = PortableUtils.OFFSET_2;
+                    res = PortableUtils.OFFSET_4;
+                }
             }
             else {
-                for (int idx0 = startIdx; idx0 < idx; ) {
-                    writer.writeInt(data[idx0++]);
-                    writer.writeInt(data[idx0++]);
+                if (lastOffset < MAX_OFFSET_1) {
+                    for (int curIdx = startIdx; curIdx < idx;) {
+                        writer.writeInt(data[curIdx++]);
+                        writer.writeByte((byte) data[curIdx++]);
+                    }
+
+                    res = PortableUtils.OFFSET_1;
+                }
+                else if (lastOffset < MAX_OFFSET_2) {
+                    for (int curIdx = startIdx; curIdx < idx;) {
+                        writer.writeInt(data[curIdx++]);
+                        writer.writeShort((short)data[curIdx++]);
+                    }
+
+                    res = PortableUtils.OFFSET_2;
                 }
+                else {
+                    for (int curIdx = startIdx; curIdx < idx;) {
+                        writer.writeInt(data[curIdx++]);
+                        writer.writeInt(data[curIdx++]);
+                    }
 
-                res = PortableUtils.OFFSET_4;
+                    res = PortableUtils.OFFSET_4;
+                }
             }
 
             return res;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
index 225e0ba..8543ce6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
@@ -38,6 +38,7 @@ import java.math.BigDecimal;
 import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Date;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -79,6 +80,9 @@ public class PortableClassDescriptor {
     /** */
     private final String typeName;
 
+    /** Affinity key field name. */
+    private final String affKeyFieldName;
+
     /** */
     private final Constructor<?> ctor;
 
@@ -92,7 +96,13 @@ public class PortableClassDescriptor {
     private final Method readResolveMtd;
 
     /** */
-    private final Map<String, Integer> fieldsMeta;
+    private final Map<String, Integer> stableFieldsMeta;
+
+    /** Object schemas. Initialized only for serializable classes and contains only 1 entry. */
+    private final Collection<PortableSchema> stableSchemas;
+
+    /** Schema registry. */
+    private final PortableSchemaRegistry schemaReg;
 
     /** */
     private final boolean keepDeserialized;
@@ -112,13 +122,14 @@ public class PortableClassDescriptor {
      * @param userType User type flag.
      * @param typeId Type ID.
      * @param typeName Type name.
+     * @param affKeyFieldName Affinity key field name.
      * @param idMapper ID mapper.
      * @param serializer Serializer.
      * @param metaDataEnabled Metadata enabled flag.
      * @param keepDeserialized Keep deserialized flag.
      * @param registered Whether typeId has been successfully registered by MarshallerContext or not.
      * @param predefined Whether the class is predefined or not.
-     * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
+     * @throws BinaryObjectException In case of error.
      */
     PortableClassDescriptor(
         PortableContext ctx,
@@ -126,6 +137,7 @@ public class PortableClassDescriptor {
         boolean userType,
         int typeId,
         String typeName,
+        @Nullable String affKeyFieldName,
         @Nullable BinaryIdMapper idMapper,
         @Nullable BinarySerializer serializer,
         boolean metaDataEnabled,
@@ -135,17 +147,21 @@ public class PortableClassDescriptor {
     ) throws BinaryObjectException {
         assert ctx != null;
         assert cls != null;
+        assert idMapper != null;
 
         this.ctx = ctx;
         this.cls = cls;
-        this.userType = userType;
         this.typeId = typeId;
+        this.userType = userType;
         this.typeName = typeName;
+        this.affKeyFieldName = affKeyFieldName;
         this.serializer = serializer;
         this.idMapper = idMapper;
         this.keepDeserialized = keepDeserialized;
         this.registered = registered;
 
+        schemaReg = ctx.schemaRegistry(typeId);
+
         excluded = MarshallerExclusions.isExcluded(cls);
 
         useOptMarshaller = !predefined && initUseOptimizedMarshallerFlag();
@@ -193,7 +209,8 @@ public class PortableClassDescriptor {
             case EXCLUSION:
                 ctor = null;
                 fields = null;
-                fieldsMeta = null;
+                stableFieldsMeta = null;
+                stableSchemas = null;
 
                 break;
 
@@ -201,16 +218,17 @@ public class PortableClassDescriptor {
             case EXTERNALIZABLE:
                 ctor = constructor(cls);
                 fields = null;
-                fieldsMeta = null;
+                stableFieldsMeta = null;
+                stableSchemas = null;
 
                 break;
 
             case OBJECT:
-                assert idMapper != null;
-
                 ctor = constructor(cls);
                 fields = new ArrayList<>();
-                fieldsMeta = metaDataEnabled ? new HashMap<String, Integer>() : null;
+                stableFieldsMeta = metaDataEnabled ? new HashMap<String, Integer>() : null;
+
+                PortableSchema.Builder schemaBuilder = PortableSchema.Builder.newBuilder();
 
                 Collection<String> names = new HashSet<>();
                 Collection<Integer> ids = new HashSet<>();
@@ -236,12 +254,16 @@ public class PortableClassDescriptor {
 
                             fields.add(fieldInfo);
 
+                            schemaBuilder.addField(fieldId);
+
                             if (metaDataEnabled)
-                                fieldsMeta.put(name, fieldInfo.fieldMode().typeId());
+                                stableFieldsMeta.put(name, fieldInfo.fieldMode().typeId());
                         }
                     }
                 }
 
+                stableSchemas = Collections.singleton(schemaBuilder.build());
+
                 break;
 
             default:
@@ -284,7 +306,14 @@ public class PortableClassDescriptor {
      * @return Fields meta data.
      */
     Map<String, Integer> fieldsMeta() {
-        return fieldsMeta;
+        return stableFieldsMeta;
+    }
+
+    /**
+     * @return Schemas.
+     */
+    Collection<PortableSchema> schemas() {
+        return stableSchemas;
     }
 
     /**
@@ -345,7 +374,7 @@ public class PortableClassDescriptor {
     /**
      * @param obj Object.
      * @param writer Writer.
-     * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
+     * @throws BinaryObjectException In case of error.
      */
     void write(Object obj, BinaryWriterExImpl writer) throws BinaryObjectException {
         assert obj != null;
@@ -539,21 +568,34 @@ public class PortableClassDescriptor {
                             ((Binarylizable)obj).writeBinary(writer);
 
                         writer.postWrite(userType);
-                    }
-                    finally {
-                        writer.popSchema();
-                    }
 
-                    if (obj.getClass() != BinaryMetadata.class
-                        && ctx.isMetaDataChanged(typeId, writer.metaDataHashSum())) {
-                        BinaryMetadataCollector metaCollector = new BinaryMetadataCollector(typeName);
+                        // Check whether we need to update metadata.
+                        if (obj.getClass() != BinaryMetadata.class) {
+                            int schemaId = writer.schemaId();
 
-                        if (serializer != null)
-                            serializer.writeBinary(obj, metaCollector);
-                        else
-                            ((Binarylizable)obj).writeBinary(metaCollector);
+                            if (schemaReg.schema(schemaId) == null) {
+                                // This is new schema, let's update metadata.
+                                BinaryMetadataCollector collector =
+                                    new BinaryMetadataCollector(typeId, typeName, idMapper);
+
+                                if (serializer != null)
+                                    serializer.writeBinary(obj, collector);
+                                else
+                                    ((Binarylizable)obj).writeBinary(collector);
+
+                                PortableSchema newSchema = collector.schema();
 
-                        ctx.updateMetaData(typeId, typeName, metaCollector.meta());
+                                BinaryMetadata meta = new BinaryMetadata(typeId, typeName, collector.meta(),
+                                    affKeyFieldName, Collections.singleton(newSchema));
+
+                                ctx.updateMetadata(typeId, meta);
+
+                                schemaReg.addSchema(newSchema.schemaId(), newSchema);
+                            }
+                        }
+                    }
+                    finally {
+                        writer.popSchema();
                     }
                 }
 
@@ -601,7 +643,7 @@ public class PortableClassDescriptor {
     /**
      * @param reader Reader.
      * @return Object.
-     * @throws org.apache.ignite.binary.BinaryObjectException If failed.
+     * @throws BinaryObjectException If failed.
      */
     Object read(BinaryReaderExImpl reader) throws BinaryObjectException {
         assert reader != null;
@@ -683,7 +725,6 @@ public class PortableClassDescriptor {
 
         PortableUtils.writeHeader(
             writer,
-            userType,
             registered ? typeId : GridPortableMarshaller.UNREGISTERED_TYPE_ID,
             obj instanceof CacheObjectImpl ? 0 : obj.hashCode(),
             registered ? null : cls.getName()
@@ -694,7 +735,7 @@ public class PortableClassDescriptor {
 
     /**
      * @return Instance.
-     * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
+     * @throws BinaryObjectException In case of error.
      */
     private Object newInstance() throws BinaryObjectException {
         assert ctor != null;
@@ -710,7 +751,7 @@ public class PortableClassDescriptor {
     /**
      * @param cls Class.
      * @return Constructor.
-     * @throws org.apache.ignite.binary.BinaryObjectException If constructor doesn't exist.
+     * @throws BinaryObjectException If constructor doesn't exist.
      */
     @SuppressWarnings("ConstantConditions")
     @Nullable private static Constructor<?> constructor(Class<?> cls) throws BinaryObjectException {
@@ -719,6 +760,9 @@ public class PortableClassDescriptor {
         try {
             Constructor<?> ctor = U.forceEmptyConstructor(cls);
 
+            if (ctor == null)
+                throw new BinaryObjectException("Failed to find empty constructor for class: " + cls.getName());
+
             ctor.setAccessible(true);
 
             return ctor;
@@ -871,7 +915,7 @@ public class PortableClassDescriptor {
         /**
          * @param obj Object.
          * @param writer Writer.
-         * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
+         * @throws BinaryObjectException In case of error.
          */
         public void write(Object obj, BinaryWriterExImpl writer) throws BinaryObjectException {
             assert obj != null;
@@ -1074,7 +1118,7 @@ public class PortableClassDescriptor {
         /**
          * @param obj Object.
          * @param reader Reader.
-         * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
+         * @throws BinaryObjectException In case of error.
          */
         public void read(Object obj, BinaryReaderExImpl reader) throws BinaryObjectException {
             Object val = null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
index 86578ad..afc23e1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
@@ -61,7 +61,6 @@ import org.apache.ignite.binary.BinarySerializer;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.IgnitionEx;
 import org.apache.ignite.internal.processors.cache.portable.CacheObjectBinaryProcessorImpl;
-import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.lang.GridMapEntry;
 import org.apache.ignite.internal.util.typedef.F;
@@ -107,9 +106,6 @@ public class PortableContext implements Externalizable {
     }
 
     /** */
-    private final ConcurrentMap<Integer, Collection<Integer>> metaDataCache = new ConcurrentHashMap8<>();
-
-    /** */
     private final ConcurrentMap<Class<?>, PortableClassDescriptor> descByCls = new ConcurrentHashMap8<>();
 
     /** Holds classes loaded by default class loader only. */
@@ -130,6 +126,9 @@ public class PortableContext implements Externalizable {
     /** */
     private final ConcurrentMap<Integer, BinaryIdMapper> mappers = new ConcurrentHashMap8<>(0);
 
+    /** Affinity key field names. */
+    private final ConcurrentMap<Integer, String> affKeyFieldNames = new ConcurrentHashMap8<>(0);
+
     /** */
     private final Map<String, BinaryIdMapper> typeMappers = new ConcurrentHashMap8<>(0);
 
@@ -151,6 +150,9 @@ public class PortableContext implements Externalizable {
     /** */
     private boolean keepDeserialized;
 
+    /** Compact footer flag. */
+    private boolean compactFooter;
+
     /** Object schemas. */
     private volatile Map<Integer, PortableSchemaRegistry> schemas;
 
@@ -262,6 +264,8 @@ public class PortableContext implements Externalizable {
             marsh.getClassNames(),
             marsh.getTypeConfigurations()
         );
+
+        compactFooter = marsh.isCompactFooter();
     }
 
     /**
@@ -504,6 +508,7 @@ public class PortableContext implements Externalizable {
                 false,
                 clsName.hashCode(),
                 clsName,
+                null,
                 BASIC_CLS_ID_MAPPER,
                 null,
                 false,
@@ -550,6 +555,7 @@ public class PortableContext implements Externalizable {
             true,
             typeId,
             typeName,
+            null,
             idMapper,
             null,
             true,
@@ -567,7 +573,8 @@ public class PortableContext implements Externalizable {
 
         mappers.putIfAbsent(typeId, idMapper);
 
-        metaHnd.addMeta(typeId, new BinaryMetadata(typeId, typeName, desc.fieldsMeta(), null).wrap(this));
+        metaHnd.addMeta(typeId,
+            new BinaryMetadata(typeId, typeName, desc.fieldsMeta(), null, desc.schemas()).wrap(this));
 
         return desc;
     }
@@ -694,6 +701,7 @@ public class PortableContext implements Externalizable {
             false,
             id,
             typeName,
+            null,
             DFLT_ID_MAPPER,
             null,
             false,
@@ -745,11 +753,17 @@ public class PortableContext implements Externalizable {
         if (mappers.put(id, idMapper) != null)
             throw new BinaryObjectException("Duplicate type ID [clsName=" + clsName + ", id=" + id + ']');
 
+        if (affKeyFieldName != null) {
+            if (affKeyFieldNames.put(id, affKeyFieldName) != null)
+                throw new BinaryObjectException("Duplicate type ID [clsName=" + clsName + ", id=" + id + ']');
+        }
+
         String typeName = typeName(clsName);
 
         typeMappers.put(typeName, idMapper);
 
         Map<String, Integer> fieldsMeta = null;
+        Collection<PortableSchema> schemas = null;
 
         if (cls != null) {
             PortableClassDescriptor desc = new PortableClassDescriptor(
@@ -758,6 +772,7 @@ public class PortableContext implements Externalizable {
                 true,
                 id,
                 typeName,
+                affKeyFieldName,
                 idMapper,
                 serializer,
                 true,
@@ -767,6 +782,7 @@ public class PortableContext implements Externalizable {
             );
 
             fieldsMeta = desc.fieldsMeta();
+            schemas = desc.schemas();
 
             if (IgniteUtils.detectClassLoader(cls).equals(dfltLdr))
                 userTypes.put(id, desc);
@@ -774,7 +790,7 @@ public class PortableContext implements Externalizable {
             descByCls.put(cls, desc);
         }
 
-        metaHnd.addMeta(id, new BinaryMetadata(id, typeName, fieldsMeta, affKeyFieldName).wrap(this));
+        metaHnd.addMeta(id, new BinaryMetadata(id, typeName, fieldsMeta, affKeyFieldName, schemas).wrap(this));
     }
 
     /**
@@ -797,48 +813,32 @@ public class PortableContext implements Externalizable {
      * @return Meta data.
      * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
      */
-    @Nullable public BinaryType metaData(int typeId) throws BinaryObjectException {
+    @Nullable public BinaryType metadata(int typeId) throws BinaryObjectException {
         return metaHnd != null ? metaHnd.metadata(typeId) : null;
     }
 
     /**
      * @param typeId Type ID.
-     * @param metaHashSum Meta data hash sum.
-     * @return Whether meta is changed.
+     * @return Affinity key field name.
      */
-    boolean isMetaDataChanged(int typeId, @Nullable Integer metaHashSum) {
-        if (metaHashSum == null)
-            return false;
-
-        Collection<Integer> hist = metaDataCache.get(typeId);
-
-        if (hist == null) {
-            Collection<Integer> old = metaDataCache.putIfAbsent(typeId, hist = new GridConcurrentHashSet<>());
-
-            if (old != null)
-                hist = old;
-        }
-
-        return hist.add(metaHashSum);
+    public String affinityKeyFieldName(int typeId) {
+        return affKeyFieldNames.get(typeId);
     }
 
     /**
      * @param typeId Type ID.
-     * @param typeName Type name.
-     * @param fields Fields map.
-     * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
+     * @param meta Meta data.
+     * @throws BinaryObjectException In case of error.
      */
-    public void updateMetaData(int typeId, String typeName, Map<String, Integer> fields) throws BinaryObjectException {
-        updateMetaData(typeId, new BinaryMetadata(typeId, typeName, fields, null));
+    public void updateMetadata(int typeId, BinaryMetadata meta) throws BinaryObjectException {
+        metaHnd.addMeta(typeId, meta.wrap(this));
     }
 
     /**
-     * @param typeId Type ID.
-     * @param meta Meta data.
-     * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
+     * @return Whether field IDs should be skipped in footer or not.
      */
-    public void updateMetaData(int typeId, BinaryMetadata meta) throws BinaryObjectException {
-        metaHnd.addMeta(typeId, meta.wrap(this));
+    public boolean isCompactFooter() {
+        return compactFooter;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableSchema.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableSchema.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableSchema.java
index 96a93f4..86ca5f8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableSchema.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableSchema.java
@@ -17,10 +17,16 @@
 
 package org.apache.ignite.internal.portable;
 
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.Map;
+import java.util.List;
 
 /**
  * Schema describing portable object content. We rely on the following assumptions:
@@ -28,130 +34,150 @@ import java.util.Map;
  * for quick comparisons performed within already fetched L1 cache line.
  * - When there are more fields, we store them inside a hash map.
  */
-public class PortableSchema {
+public class PortableSchema implements Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
     /** Order returned if field is not found. */
     public static final int ORDER_NOT_FOUND = -1;
 
     /** Inline flag. */
-    private final boolean inline;
+    private boolean inline;
+
+    /** Map with ID to order. */
+    private HashMap<Integer, Integer> idToOrder;
 
-    /** Map with offsets. */
-    private final HashMap<Integer, Integer> map;
+    /** IDs depending on order. */
+    private ArrayList<Integer> ids;
 
     /** ID 1. */
-    private final int id0;
+    private int id0;
 
     /** ID 2. */
-    private final int id1;
+    private int id1;
 
     /** ID 3. */
-    private final int id2;
+    private int id2;
 
     /** ID 4. */
-    private final int id3;
+    private int id3;
 
     /** ID 1. */
-    private final int id4;
+    private int id4;
 
     /** ID 2. */
-    private final int id5;
+    private int id5;
 
     /** ID 3. */
-    private final int id6;
+    private int id6;
 
     /** ID 4. */
-    private final int id7;
+    private int id7;
+
+    /** Schema ID. */
+    private int schemaId;
+
+    /**
+     * {@link Externalizable} support.
+     */
+    public PortableSchema() {
+        // No-op.
+    }
 
     /**
      * Constructor.
      *
-     * @param vals Values.
+     * @param schemaId Schema ID.
+     * @param fieldIds Field IDs.
      */
-    public PortableSchema(LinkedHashMap<Integer, Integer> vals) {
-        if (vals.size() <= 8) {
-            inline = true;
-
-            Iterator<Map.Entry<Integer, Integer>> iter = vals.entrySet().iterator();
+    private PortableSchema(int schemaId, List<Integer> fieldIds) {
+        this.schemaId = schemaId;
 
-            Map.Entry<Integer, Integer> entry = iter.hasNext() ? iter.next() : null;
-
-            if (entry != null) {
-                id0 = entry.getKey();
+        if (fieldIds.size() <= 8) {
+            inline = true;
 
-                assert entry.getValue() == 0;
-            }
-            else
-                id0 = 0;
+            Iterator<Integer> iter = fieldIds.iterator();
 
-            if ((entry = iter.hasNext() ? iter.next() : null) != null) {
-                id1 = entry.getKey();
+            id0 = iter.hasNext() ? iter.next() : 0;
+            id1 = iter.hasNext() ? iter.next() : 0;
+            id2 = iter.hasNext() ? iter.next() : 0;
+            id3 = iter.hasNext() ? iter.next() : 0;
+            id4 = iter.hasNext() ? iter.next() : 0;
+            id5 = iter.hasNext() ? iter.next() : 0;
+            id6 = iter.hasNext() ? iter.next() : 0;
+            id7 = iter.hasNext() ? iter.next() : 0;
 
-                assert entry.getValue() == 1;
-            }
-            else
-                id1 = 0;
+            idToOrder = null;
+        }
+        else {
+            inline = false;
 
-            if ((entry = iter.hasNext() ? iter.next() : null) != null) {
-                id2 = entry.getKey();
+            id0 = id1 = id2 = id3 = id4 = id5 = id6 = id7 = 0;
 
-                assert entry.getValue() == 2;
-            }
-            else
-                id2 = 0;
+            ids = new ArrayList<>();
+            idToOrder = new HashMap<>();
 
-            if ((entry = iter.hasNext() ? iter.next() : null) != null) {
-                id3 = entry.getKey();
+            for (int i = 0; i < fieldIds.size(); i++) {
+                int fieldId = fieldIds.get(i);
 
-                assert entry.getValue() == 3;
+                ids.add(fieldId);
+                idToOrder.put(fieldId, i);
             }
-            else
-                id3 = 0;
+        }
+    }
 
-            if ((entry = iter.hasNext() ? iter.next() : null) != null) {
-                id4 = entry.getKey();
+    /**
+     * @return Schema ID.
+     */
+    public int schemaId() {
+        return schemaId;
+    }
 
-                assert entry.getValue() == 4;
-            }
-            else
-                id4 = 0;
+    /**
+     * Get field ID by order in footer.
+     *
+     * @param order Order.
+     * @return Field ID.
+     */
+    public int fieldId(int order) {
+        if (inline) {
+            switch (order) {
+                case 0:
+                    return id0;
 
-            if ((entry = iter.hasNext() ? iter.next() : null) != null) {
-                id5 = entry.getKey();
+                case 1:
+                    return id1;
 
-                assert entry.getValue() == 5;
-            }
-            else
-                id5 = 0;
+                case 2:
+                    return id2;
 
-            if ((entry = iter.hasNext() ? iter.next() : null) != null) {
-                id6 = entry.getKey();
+                case 3:
+                    return id3;
 
-                assert entry.getValue() == 6;
-            }
-            else
-                id6 = 0;
+                case 4:
+                    return id4;
 
-            if ((entry = iter.hasNext() ? iter.next() : null) != null) {
-                id7 = entry.getKey();
+                case 5:
+                    return id5;
 
-                assert entry.getValue() == 7;
-            }
-            else
-                id7 = 0;
+                case 6:
+                    return id6;
 
-            map = null;
-        }
-        else {
-            inline = false;
+                case 7:
+                    return id7;
 
-            id0 = id1 = id2 = id3 = id4 = id5 = id6 = id7 = 0;
+                default:
+                    assert false : "Should not reach here.";
 
-            map = new HashMap<>(vals);
+                    return 0;
+            }
         }
+        else
+            return ids.get(order);
     }
 
     /**
-     * Get field position in footer by schema ID.
+     * Get field order in footer by field ID.
      *
      * @param id Field ID.
      * @return Offset or {@code 0} if there is no such field.
@@ -185,9 +211,125 @@ public class PortableSchema {
             return ORDER_NOT_FOUND;
         }
         else {
-            Integer order = map.get(id);
+            Integer order = idToOrder.get(id);
 
             return order != null ? order : ORDER_NOT_FOUND;
         }
     }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return schemaId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        return o != null && o instanceof PortableSchema && schemaId == ((PortableSchema)o).schemaId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeInt(schemaId);
+
+        if (inline) {
+            out.writeBoolean(true);
+
+            out.writeInt(id0);
+            out.writeInt(id1);
+            out.writeInt(id2);
+            out.writeInt(id3);
+            out.writeInt(id4);
+            out.writeInt(id5);
+            out.writeInt(id6);
+            out.writeInt(id7);
+        }
+        else {
+            out.writeBoolean(false);
+
+            out.writeInt(ids.size());
+
+            for (Integer id : ids)
+                out.writeInt(id);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        schemaId = in.readInt();
+
+        if (in.readBoolean()) {
+            inline = true;
+
+            id0 = in.readInt();
+            id1 = in.readInt();
+            id2 = in.readInt();
+            id3 = in.readInt();
+            id4 = in.readInt();
+            id5 = in.readInt();
+            id6 = in.readInt();
+            id7 = in.readInt();
+        }
+        else {
+            inline = false;
+
+            int size = in.readInt();
+
+            ids = new ArrayList<>(size);
+            idToOrder = U.newHashMap(size);
+
+            for (int i = 0; i < size; i++) {
+                int fieldId = in.readInt();
+
+                ids.add(fieldId);
+                idToOrder.put(fieldId, i);
+            }
+        }
+    }
+
+    /**
+     * Schema builder.
+     */
+    public static class Builder {
+        /** Schema ID. */
+        private int schemaId = PortableUtils.schemaInitialId();
+
+        /** Fields. */
+        private final ArrayList<Integer> fields = new ArrayList<>();
+
+        /**
+         * Create new schema builder.
+         *
+         * @return Schema builder.
+         */
+        public static Builder newBuilder() {
+            return new Builder();
+        }
+
+        /**
+         * Private constructor.
+         */
+        private Builder() {
+            // No-op.
+        }
+
+        /**
+         * Add field.
+         *
+         * @param fieldId Field ID.
+         */
+        public void addField(int fieldId) {
+            fields.add(fieldId);
+
+            schemaId = PortableUtils.updateSchemaId(schemaId, fieldId);
+        }
+
+        /**
+         * Build schema.
+         *
+         * @return Schema.
+         */
+        public PortableSchema build() {
+            return new PortableSchema(schemaId, fields);
+        }
+    }
 }


[12/13] ignite git commit: Merge branch ignite-1282 into ignite-1945

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a511fa17/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryMarshallerSelfTest.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryMarshallerSelfTest.java
index 0000000,9809a7e..989b41b
mode 000000,100644..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
@@@ -1,0 -1,3795 +1,3687 @@@
+ /*
+  * 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 junit.framework.Assert;
+ import org.apache.ignite.IgniteCheckedException;
+ import org.apache.ignite.binary.BinaryIdMapper;
+ import org.apache.ignite.binary.BinaryObject;
+ import org.apache.ignite.binary.BinaryObjectBuilder;
+ import org.apache.ignite.binary.BinaryObjectException;
+ import org.apache.ignite.binary.BinaryRawReader;
+ import org.apache.ignite.binary.BinaryRawWriter;
+ import org.apache.ignite.binary.BinaryReader;
+ import org.apache.ignite.binary.BinarySerializer;
+ import org.apache.ignite.binary.BinaryTypeConfiguration;
+ import org.apache.ignite.binary.BinaryWriter;
+ import org.apache.ignite.binary.Binarylizable;
++import org.apache.ignite.configuration.BinaryConfiguration;
+ import org.apache.ignite.configuration.IgniteConfiguration;
+ import org.apache.ignite.internal.portable.builder.BinaryObjectBuilderImpl;
++import org.apache.ignite.internal.processors.cache.CacheObjectContext;
+ import org.apache.ignite.internal.util.GridUnsafe;
+ import org.apache.ignite.internal.util.IgniteUtils;
+ import org.apache.ignite.internal.util.lang.GridMapEntry;
+ import org.apache.ignite.internal.util.typedef.F;
+ import org.apache.ignite.internal.util.typedef.internal.S;
+ import org.apache.ignite.internal.util.typedef.internal.U;
+ import org.apache.ignite.marshaller.MarshallerContextTestImpl;
 -import org.apache.ignite.marshaller.portable.PortableMarshaller;
++import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+ import org.apache.ignite.testframework.GridTestUtils;
+ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+ import org.jsr166.ConcurrentHashMap8;
+ import sun.misc.Unsafe;
+ 
+ import java.lang.reflect.Field;
+ import java.lang.reflect.InvocationTargetException;
+ import java.lang.reflect.Method;
+ import java.math.BigDecimal;
+ import java.math.BigInteger;
+ import java.net.InetSocketAddress;
+ import java.sql.Timestamp;
+ import java.util.ArrayList;
+ import java.util.Arrays;
+ import java.util.Collection;
+ import java.util.Collections;
+ import java.util.Date;
+ import java.util.HashMap;
+ import java.util.HashSet;
+ import java.util.LinkedHashMap;
+ import java.util.LinkedHashSet;
+ import java.util.Map;
+ import java.util.TreeMap;
+ import java.util.TreeSet;
+ import java.util.UUID;
+ import java.util.concurrent.ConcurrentHashMap;
+ import java.util.concurrent.ConcurrentSkipListSet;
+ 
+ import static org.apache.ignite.internal.portable.PortableThreadLocalMemoryAllocator.THREAD_LOCAL_ALLOC;
+ import static org.junit.Assert.assertArrayEquals;
+ 
+ /**
+  * Portable marshaller tests.
+  */
+ @SuppressWarnings({"OverlyStrongTypeCast", "ArrayHashCode", "ConstantConditions"})
+ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
+     /** */
+     private static final Unsafe UNSAFE = GridUnsafe.unsafe();
+ 
+     /** */
+     protected static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testNull() throws Exception {
+         assertNull(marshalUnmarshal(null));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testByte() throws Exception {
+         assertEquals((byte)100, marshalUnmarshal((byte)100).byteValue());
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testShort() throws Exception {
+         assertEquals((short)100, marshalUnmarshal((short)100).shortValue());
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testInt() throws Exception {
+         assertEquals(100, marshalUnmarshal(100).intValue());
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testLong() throws Exception {
+         assertEquals(100L, marshalUnmarshal(100L).longValue());
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testFloat() throws Exception {
+         assertEquals(100.001f, marshalUnmarshal(100.001f).floatValue(), 0);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testDouble() throws Exception {
+         assertEquals(100.001d, marshalUnmarshal(100.001d).doubleValue(), 0);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testChar() throws Exception {
+         assertEquals((char)100, marshalUnmarshal((char)100).charValue());
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testBoolean() throws Exception {
+         assertEquals(true, marshalUnmarshal(true).booleanValue());
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testDecimal() throws Exception {
+         BigDecimal val;
+ 
+         assertEquals((val = BigDecimal.ZERO), marshalUnmarshal(val));
+         assertEquals((val = BigDecimal.valueOf(Long.MAX_VALUE, 0)), marshalUnmarshal(val));
+         assertEquals((val = BigDecimal.valueOf(Long.MIN_VALUE, 0)), marshalUnmarshal(val));
+         assertEquals((val = BigDecimal.valueOf(Long.MAX_VALUE, 8)), marshalUnmarshal(val));
+         assertEquals((val = BigDecimal.valueOf(Long.MIN_VALUE, 8)), marshalUnmarshal(val));
+ 
+         assertEquals((val = new BigDecimal(new BigInteger("-79228162514264337593543950336"))), marshalUnmarshal(val));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testString() throws Exception {
+         assertEquals("str", marshalUnmarshal("str"));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testUuid() throws Exception {
+         UUID uuid = UUID.randomUUID();
+ 
+         assertEquals(uuid, marshalUnmarshal(uuid));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testDate() throws Exception {
+         Date date = new Date();
+ 
+         Date val = marshalUnmarshal(date);
+ 
+         assertEquals(date, val);
+         assertEquals(Date.class, val.getClass());
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testTimestamp() throws Exception {
+         Timestamp ts = new Timestamp(System.currentTimeMillis());
+ 
+         ts.setNanos(999999999);
+ 
+         assertEquals(ts, marshalUnmarshal(ts));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testByteArray() throws Exception {
+         byte[] arr = new byte[] {10, 20, 30};
+ 
+         assertArrayEquals(arr, marshalUnmarshal(arr));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testShortArray() throws Exception {
+         short[] arr = new short[] {10, 20, 30};
+ 
+         assertArrayEquals(arr, marshalUnmarshal(arr));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testIntArray() throws Exception {
+         int[] arr = new int[] {10, 20, 30};
+ 
+         assertArrayEquals(arr, marshalUnmarshal(arr));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testLongArray() throws Exception {
+         long[] arr = new long[] {10, 20, 30};
+ 
+         assertArrayEquals(arr, marshalUnmarshal(arr));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testFloatArray() throws Exception {
+         float[] arr = new float[] {10.1f, 20.1f, 30.1f};
+ 
+         assertArrayEquals(arr, marshalUnmarshal(arr), 0);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testDoubleArray() throws Exception {
+         double[] arr = new double[] {10.1d, 20.1d, 30.1d};
+ 
+         assertArrayEquals(arr, marshalUnmarshal(arr), 0);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testCharArray() throws Exception {
+         char[] arr = new char[] {10, 20, 30};
+ 
+         assertArrayEquals(arr, marshalUnmarshal(arr));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testBooleanArray() throws Exception {
+         boolean[] arr = new boolean[] {true, false, true};
+ 
+         assertBooleanArrayEquals(arr, marshalUnmarshal(arr));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testDecimalArray() throws Exception {
+         BigDecimal[] arr = new BigDecimal[] { BigDecimal.ZERO, BigDecimal.ONE, BigDecimal.TEN } ;
+ 
+         assertArrayEquals(arr, marshalUnmarshal(arr));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testStringArray() throws Exception {
+         String[] arr = new String[] {"str1", "str2", "str3"};
+ 
+         assertArrayEquals(arr, marshalUnmarshal(arr));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testUuidArray() throws Exception {
+         UUID[] arr = new UUID[] {UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID()};
+ 
+         assertArrayEquals(arr, marshalUnmarshal(arr));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testDateArray() throws Exception {
+         Date[] arr = new Date[] {new Date(11111), new Date(22222), new Date(33333)};
+ 
+         assertArrayEquals(arr, marshalUnmarshal(arr));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testObjectArray() throws Exception {
+         Object[] arr = new Object[] {1, 2, 3};
+ 
+         assertArrayEquals(arr, marshalUnmarshal(arr));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testCollection() throws Exception {
+         testCollection(new ArrayList<Integer>(3));
+         testCollection(new LinkedHashSet<Integer>());
+         testCollection(new HashSet<Integer>());
+         testCollection(new TreeSet<Integer>());
+         testCollection(new ConcurrentSkipListSet<Integer>());
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     private void testCollection(Collection<Integer> col) throws Exception {
+         col.add(1);
+         col.add(2);
+         col.add(3);
+ 
+         assertEquals(col, marshalUnmarshal(col));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testMap() throws Exception {
+         testMap(new HashMap<Integer, String>());
+         testMap(new LinkedHashMap());
+         testMap(new TreeMap<Integer, String>());
+         testMap(new ConcurrentHashMap8<Integer, String>());
+         testMap(new ConcurrentHashMap<Integer, String>());
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     private void testMap(Map<Integer, String> map) throws Exception {
+         map.put(1, "str1");
+         map.put(2, "str2");
+         map.put(3, "str3");
+ 
+         assertEquals(map, marshalUnmarshal(map));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testMapEntry() throws Exception {
+         Map.Entry<Integer, String> e = new GridMapEntry<>(1, "str1");
+ 
+         assertEquals(e, marshalUnmarshal(e));
+ 
+         Map<Integer, String> map = new HashMap<>(1);
+ 
+         map.put(2, "str2");
+ 
+         e = F.firstEntry(map);
+ 
+         Map.Entry<Integer, String> e0 = marshalUnmarshal(e);
+ 
+         assertEquals(2, e0.getKey().intValue());
+         assertEquals("str2", e0.getValue());
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testBinaryObject() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
 -
 -        marsh.setTypeConfigurations(Arrays.asList(new BinaryTypeConfiguration(SimpleObject.class.getName())));
++        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(new BinaryTypeConfiguration(SimpleObject.class.getName())));
+ 
+         SimpleObject obj = simpleObject();
+ 
+         BinaryObject po = marshal(obj, marsh);
+ 
+         BinaryObject po0 = marshalUnmarshal(po, marsh);
+ 
+         assertTrue(po.hasField("b"));
+         assertTrue(po.hasField("s"));
+         assertTrue(po.hasField("i"));
+         assertTrue(po.hasField("l"));
+         assertTrue(po.hasField("f"));
+         assertTrue(po.hasField("d"));
+         assertTrue(po.hasField("c"));
+         assertTrue(po.hasField("bool"));
+ 
+         assertFalse(po.hasField("no_such_field"));
+ 
+         assertEquals(obj, po.deserialize());
+         assertEquals(obj, po0.deserialize());
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testEnum() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
 -
 -        marsh.setClassNames(Arrays.asList(TestEnum.class.getName()));
++        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(new BinaryTypeConfiguration(TestEnum.class.getName())));
+ 
+         assertEquals(TestEnum.B, marshalUnmarshal(TestEnum.B, marsh));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testDateAndTimestampInSingleObject() throws Exception {
+         BinaryTypeConfiguration cfg1 = new BinaryTypeConfiguration(DateClass1.class.getName());
+ 
 -        PortableMarshaller marsh = createMarshaller();
 -
 -        marsh.setTypeConfigurations(Arrays.asList(cfg1));
++        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(cfg1));
+ 
+         Date date = new Date();
+         Timestamp ts = new Timestamp(System.currentTimeMillis());
+ 
+         DateClass1 obj1 = new DateClass1();
+         obj1.date = date;
+         obj1.ts = ts;
+ 
+         BinaryObject po1 = marshal(obj1, marsh);
+ 
+         assertEquals(date, po1.field("date"));
+         assertEquals(Date.class, po1.field("date").getClass());
+         assertEquals(ts, po1.field("ts"));
+         assertEquals(Timestamp.class, po1.field("ts").getClass());
+ 
+         obj1 = po1.deserialize();
+         assertEquals(date, obj1.date);
+         assertEquals(ts, obj1.ts);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testSimpleObject() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
 -
 -        marsh.setTypeConfigurations(Arrays.asList(
++        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(
+             new BinaryTypeConfiguration(SimpleObject.class.getName())
+         ));
+ 
+         SimpleObject obj = simpleObject();
+ 
+         BinaryObject po = marshal(obj, marsh);
+ 
+         assertEquals(obj.hashCode(), po.hashCode());
+ 
+         assertEquals(obj, po.deserialize());
+ 
+         assertEquals(obj.b, (byte)po.field("b"));
+         assertEquals(obj.s, (short)po.field("s"));
+         assertEquals(obj.i, (int)po.field("i"));
+         assertEquals(obj.l, (long)po.field("l"));
+         assertEquals(obj.f, (float)po.field("f"), 0);
+         assertEquals(obj.d, (double)po.field("d"), 0);
+         assertEquals(obj.c, (char)po.field("c"));
+         assertEquals(obj.bool, (boolean)po.field("bool"));
+         assertEquals(obj.str, po.field("str"));
+         assertEquals(obj.uuid, po.field("uuid"));
+         assertEquals(obj.date, po.field("date"));
+         assertEquals(Date.class, obj.date.getClass());
+         assertEquals(obj.ts, po.field("ts"));
+         assertArrayEquals(obj.bArr, (byte[])po.field("bArr"));
+         assertArrayEquals(obj.sArr, (short[])po.field("sArr"));
+         assertArrayEquals(obj.iArr, (int[])po.field("iArr"));
+         assertArrayEquals(obj.lArr, (long[])po.field("lArr"));
+         assertArrayEquals(obj.fArr, (float[])po.field("fArr"), 0);
+         assertArrayEquals(obj.dArr, (double[])po.field("dArr"), 0);
+         assertArrayEquals(obj.cArr, (char[])po.field("cArr"));
+         assertBooleanArrayEquals(obj.boolArr, (boolean[])po.field("boolArr"));
+         assertArrayEquals(obj.strArr, (String[])po.field("strArr"));
+         assertArrayEquals(obj.uuidArr, (UUID[])po.field("uuidArr"));
+         assertArrayEquals(obj.dateArr, (Date[])po.field("dateArr"));
+         assertArrayEquals(obj.objArr, (Object[])po.field("objArr"));
+         assertEquals(obj.col, po.field("col"));
+         assertEquals(obj.map, po.field("map"));
+         assertEquals(new Integer(obj.enumVal.ordinal()), new Integer(((Enum<?>)po.field("enumVal")).ordinal()));
+         assertArrayEquals(ordinals(obj.enumArr), ordinals((Enum<?>[])po.field("enumArr")));
+         assertNull(po.field("unknown"));
+ 
+         BinaryObject innerPo = po.field("inner");
+ 
+         assertEquals(obj.inner, innerPo.deserialize());
+ 
+         assertEquals(obj.inner.b, (byte)innerPo.field("b"));
+         assertEquals(obj.inner.s, (short)innerPo.field("s"));
+         assertEquals(obj.inner.i, (int)innerPo.field("i"));
+         assertEquals(obj.inner.l, (long)innerPo.field("l"));
+         assertEquals(obj.inner.f, (float)innerPo.field("f"), 0);
+         assertEquals(obj.inner.d, (double)innerPo.field("d"), 0);
+         assertEquals(obj.inner.c, (char)innerPo.field("c"));
+         assertEquals(obj.inner.bool, (boolean)innerPo.field("bool"));
+         assertEquals(obj.inner.str, innerPo.field("str"));
+         assertEquals(obj.inner.uuid, innerPo.field("uuid"));
+         assertEquals(obj.inner.date, innerPo.field("date"));
+         assertEquals(Date.class, obj.inner.date.getClass());
+         assertEquals(obj.inner.ts, innerPo.field("ts"));
+         assertArrayEquals(obj.inner.bArr, (byte[])innerPo.field("bArr"));
+         assertArrayEquals(obj.inner.sArr, (short[])innerPo.field("sArr"));
+         assertArrayEquals(obj.inner.iArr, (int[])innerPo.field("iArr"));
+         assertArrayEquals(obj.inner.lArr, (long[])innerPo.field("lArr"));
+         assertArrayEquals(obj.inner.fArr, (float[])innerPo.field("fArr"), 0);
+         assertArrayEquals(obj.inner.dArr, (double[])innerPo.field("dArr"), 0);
+         assertArrayEquals(obj.inner.cArr, (char[])innerPo.field("cArr"));
+         assertBooleanArrayEquals(obj.inner.boolArr, (boolean[])innerPo.field("boolArr"));
+         assertArrayEquals(obj.inner.strArr, (String[])innerPo.field("strArr"));
+         assertArrayEquals(obj.inner.uuidArr, (UUID[])innerPo.field("uuidArr"));
+         assertArrayEquals(obj.inner.dateArr, (Date[])innerPo.field("dateArr"));
+         assertArrayEquals(obj.inner.objArr, (Object[])innerPo.field("objArr"));
+         assertEquals(obj.inner.col, innerPo.field("col"));
+         assertEquals(obj.inner.map, innerPo.field("map"));
+         assertEquals(new Integer(obj.inner.enumVal.ordinal()),
+             new Integer(((Enum<?>)innerPo.field("enumVal")).ordinal()));
+         assertArrayEquals(ordinals(obj.inner.enumArr), ordinals((Enum<?>[])innerPo.field("enumArr")));
+         assertNull(innerPo.field("inner"));
+         assertNull(innerPo.field("unknown"));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testPortable() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
 -
 -        marsh.setTypeConfigurations(Arrays.asList(
++        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(
+             new BinaryTypeConfiguration(SimpleObject.class.getName()),
+             new BinaryTypeConfiguration(TestBinary.class.getName())
+         ));
+ 
+         TestBinary obj = binaryObject();
+ 
+         BinaryObject po = marshal(obj, marsh);
+ 
+         assertEquals(obj.hashCode(), po.hashCode());
+ 
+         assertEquals(obj, po.deserialize());
+ 
+         assertEquals(obj.b, (byte)po.field("_b"));
+         assertEquals(obj.s, (short)po.field("_s"));
+         assertEquals(obj.i, (int)po.field("_i"));
+         assertEquals(obj.l, (long)po.field("_l"));
+         assertEquals(obj.f, (float)po.field("_f"), 0);
+         assertEquals(obj.d, (double)po.field("_d"), 0);
+         assertEquals(obj.c, (char)po.field("_c"));
+         assertEquals(obj.bool, (boolean)po.field("_bool"));
+         assertEquals(obj.str, po.field("_str"));
+         assertEquals(obj.uuid, po.field("_uuid"));
+         assertEquals(obj.date, po.field("_date"));
+         assertEquals(obj.ts, po.field("_ts"));
+         assertArrayEquals(obj.bArr, (byte[])po.field("_bArr"));
+         assertArrayEquals(obj.sArr, (short[])po.field("_sArr"));
+         assertArrayEquals(obj.iArr, (int[])po.field("_iArr"));
+         assertArrayEquals(obj.lArr, (long[])po.field("_lArr"));
+         assertArrayEquals(obj.fArr, (float[])po.field("_fArr"), 0);
+         assertArrayEquals(obj.dArr, (double[])po.field("_dArr"), 0);
+         assertArrayEquals(obj.cArr, (char[])po.field("_cArr"));
+         assertBooleanArrayEquals(obj.boolArr, (boolean[])po.field("_boolArr"));
+         assertArrayEquals(obj.strArr, (String[])po.field("_strArr"));
+         assertArrayEquals(obj.uuidArr, (UUID[])po.field("_uuidArr"));
+         assertArrayEquals(obj.dateArr, (Date[])po.field("_dateArr"));
+         assertArrayEquals(obj.objArr, (Object[])po.field("_objArr"));
+         assertEquals(obj.col, po.field("_col"));
+         assertEquals(obj.map, po.field("_map"));
+         assertEquals(new Integer(obj.enumVal.ordinal()), new Integer(((Enum<?>)po.field("_enumVal")).ordinal()));
+         assertArrayEquals(ordinals(obj.enumArr), ordinals((Enum<?>[])po.field("_enumArr")));
+         assertNull(po.field("unknown"));
+ 
+         BinaryObject simplePo = po.field("_simple");
+ 
+         assertEquals(obj.simple, simplePo.deserialize());
+ 
+         assertEquals(obj.simple.b, (byte)simplePo.field("b"));
+         assertEquals(obj.simple.s, (short)simplePo.field("s"));
+         assertEquals(obj.simple.i, (int)simplePo.field("i"));
+         assertEquals(obj.simple.l, (long)simplePo.field("l"));
+         assertEquals(obj.simple.f, (float)simplePo.field("f"), 0);
+         assertEquals(obj.simple.d, (double)simplePo.field("d"), 0);
+         assertEquals(obj.simple.c, (char)simplePo.field("c"));
+         assertEquals(obj.simple.bool, (boolean)simplePo.field("bool"));
+         assertEquals(obj.simple.str, simplePo.field("str"));
+         assertEquals(obj.simple.uuid, simplePo.field("uuid"));
+         assertEquals(obj.simple.date, simplePo.field("date"));
+         assertEquals(Date.class, obj.simple.date.getClass());
+         assertEquals(obj.simple.ts, simplePo.field("ts"));
+         assertArrayEquals(obj.simple.bArr, (byte[])simplePo.field("bArr"));
+         assertArrayEquals(obj.simple.sArr, (short[])simplePo.field("sArr"));
+         assertArrayEquals(obj.simple.iArr, (int[])simplePo.field("iArr"));
+         assertArrayEquals(obj.simple.lArr, (long[])simplePo.field("lArr"));
+         assertArrayEquals(obj.simple.fArr, (float[])simplePo.field("fArr"), 0);
+         assertArrayEquals(obj.simple.dArr, (double[])simplePo.field("dArr"), 0);
+         assertArrayEquals(obj.simple.cArr, (char[])simplePo.field("cArr"));
+         assertBooleanArrayEquals(obj.simple.boolArr, (boolean[])simplePo.field("boolArr"));
+         assertArrayEquals(obj.simple.strArr, (String[])simplePo.field("strArr"));
+         assertArrayEquals(obj.simple.uuidArr, (UUID[])simplePo.field("uuidArr"));
+         assertArrayEquals(obj.simple.dateArr, (Date[])simplePo.field("dateArr"));
+         assertArrayEquals(obj.simple.objArr, (Object[])simplePo.field("objArr"));
+         assertEquals(obj.simple.col, simplePo.field("col"));
+         assertEquals(obj.simple.map, simplePo.field("map"));
+         assertEquals(new Integer(obj.simple.enumVal.ordinal()),
+             new Integer(((Enum<?>)simplePo.field("enumVal")).ordinal()));
+         assertArrayEquals(ordinals(obj.simple.enumArr), ordinals((Enum<?>[])simplePo.field("enumArr")));
+         assertNull(simplePo.field("simple"));
+         assertNull(simplePo.field("portable"));
+         assertNull(simplePo.field("unknown"));
+ 
+         BinaryObject portablePo = po.field("_portable");
+ 
+         assertEquals(obj.portable, portablePo.deserialize());
+ 
+         assertEquals(obj.portable.b, (byte)portablePo.field("_b"));
+         assertEquals(obj.portable.s, (short)portablePo.field("_s"));
+         assertEquals(obj.portable.i, (int)portablePo.field("_i"));
+         assertEquals(obj.portable.l, (long)portablePo.field("_l"));
+         assertEquals(obj.portable.f, (float)portablePo.field("_f"), 0);
+         assertEquals(obj.portable.d, (double)portablePo.field("_d"), 0);
+         assertEquals(obj.portable.c, (char)portablePo.field("_c"));
+         assertEquals(obj.portable.bool, (boolean)portablePo.field("_bool"));
+         assertEquals(obj.portable.str, portablePo.field("_str"));
+         assertEquals(obj.portable.uuid, portablePo.field("_uuid"));
+         assertEquals(obj.portable.date, portablePo.field("_date"));
+         assertEquals(obj.portable.ts, portablePo.field("_ts"));
+         assertArrayEquals(obj.portable.bArr, (byte[])portablePo.field("_bArr"));
+         assertArrayEquals(obj.portable.sArr, (short[])portablePo.field("_sArr"));
+         assertArrayEquals(obj.portable.iArr, (int[])portablePo.field("_iArr"));
+         assertArrayEquals(obj.portable.lArr, (long[])portablePo.field("_lArr"));
+         assertArrayEquals(obj.portable.fArr, (float[])portablePo.field("_fArr"), 0);
+         assertArrayEquals(obj.portable.dArr, (double[])portablePo.field("_dArr"), 0);
+         assertArrayEquals(obj.portable.cArr, (char[])portablePo.field("_cArr"));
+         assertBooleanArrayEquals(obj.portable.boolArr, (boolean[])portablePo.field("_boolArr"));
+         assertArrayEquals(obj.portable.strArr, (String[])portablePo.field("_strArr"));
+         assertArrayEquals(obj.portable.uuidArr, (UUID[])portablePo.field("_uuidArr"));
+         assertArrayEquals(obj.portable.dateArr, (Date[])portablePo.field("_dateArr"));
+         assertArrayEquals(obj.portable.objArr, (Object[])portablePo.field("_objArr"));
+         assertEquals(obj.portable.col, portablePo.field("_col"));
+         assertEquals(obj.portable.map, portablePo.field("_map"));
+         assertEquals(new Integer(obj.portable.enumVal.ordinal()),
+             new Integer(((Enum<?>)portablePo.field("_enumVal")).ordinal()));
+         assertArrayEquals(ordinals(obj.portable.enumArr), ordinals((Enum<?>[])portablePo.field("_enumArr")));
+         assertNull(portablePo.field("_simple"));
+         assertNull(portablePo.field("_portable"));
+         assertNull(portablePo.field("unknown"));
+     }
+ 
+     /**
+      * @param obj Simple object.
+      * @param po Portable object.
+      */
+     private void checkSimpleObjectData(SimpleObject obj, BinaryObject po) {
+         assertEquals(obj.b, (byte)po.field("b"));
+         assertEquals(obj.s, (short)po.field("s"));
+         assertEquals(obj.i, (int)po.field("i"));
+         assertEquals(obj.l, (long)po.field("l"));
+         assertEquals(obj.f, (float)po.field("f"), 0);
+         assertEquals(obj.d, (double)po.field("d"), 0);
+         assertEquals(obj.c, (char)po.field("c"));
+         assertEquals(obj.bool, (boolean)po.field("bool"));
+         assertEquals(obj.str, po.field("str"));
+         assertEquals(obj.uuid, po.field("uuid"));
+         assertEquals(obj.date, po.field("date"));
+         assertEquals(Date.class, obj.date.getClass());
+         assertEquals(obj.ts, po.field("ts"));
+         assertArrayEquals(obj.bArr, (byte[])po.field("bArr"));
+         assertArrayEquals(obj.sArr, (short[])po.field("sArr"));
+         assertArrayEquals(obj.iArr, (int[])po.field("iArr"));
+         assertArrayEquals(obj.lArr, (long[])po.field("lArr"));
+         assertArrayEquals(obj.fArr, (float[])po.field("fArr"), 0);
+         assertArrayEquals(obj.dArr, (double[])po.field("dArr"), 0);
+         assertArrayEquals(obj.cArr, (char[])po.field("cArr"));
+         assertBooleanArrayEquals(obj.boolArr, (boolean[])po.field("boolArr"));
+         assertArrayEquals(obj.strArr, (String[])po.field("strArr"));
+         assertArrayEquals(obj.uuidArr, (UUID[])po.field("uuidArr"));
+         assertArrayEquals(obj.dateArr, (Date[])po.field("dateArr"));
+         assertArrayEquals(obj.objArr, (Object[])po.field("objArr"));
+         assertEquals(obj.col, po.field("col"));
+         assertEquals(obj.map, po.field("map"));
+         assertEquals(new Integer(obj.enumVal.ordinal()), new Integer(((Enum<?>)po.field("enumVal")).ordinal()));
+         assertArrayEquals(ordinals(obj.enumArr), ordinals((Enum<?>[])po.field("enumArr")));
+         assertNull(po.field("unknown"));
+ 
+         assertEquals(obj, po.deserialize());
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testClassWithoutPublicConstructor() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
 -
 -        marsh.setTypeConfigurations(Arrays.asList(
++        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(
+                 new BinaryTypeConfiguration(NoPublicConstructor.class.getName()),
+                 new BinaryTypeConfiguration(NoPublicDefaultConstructor.class.getName()),
+                 new BinaryTypeConfiguration(ProtectedConstructor.class.getName()))
+         );
+ 
+         NoPublicConstructor npc = new NoPublicConstructor();
+         BinaryObject npc2 = marshal(npc, marsh);
+ 
+         assertEquals("test", npc2.<NoPublicConstructor>deserialize().val);
+ 
+         NoPublicDefaultConstructor npdc = new NoPublicDefaultConstructor(239);
+         BinaryObject npdc2 = marshal(npdc, marsh);
+ 
+         assertEquals(239, npdc2.<NoPublicDefaultConstructor>deserialize().val);
+ 
+         ProtectedConstructor pc = new ProtectedConstructor();
+         BinaryObject pc2 = marshal(pc, marsh);
+ 
+         assertEquals(ProtectedConstructor.class, pc2.<ProtectedConstructor>deserialize().getClass());
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testCustomSerializer() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
 -
+         BinaryTypeConfiguration type =
+             new BinaryTypeConfiguration(CustomSerializedObject1.class.getName());
+ 
+         type.setSerializer(new CustomSerializer1());
+ 
 -        marsh.setTypeConfigurations(Arrays.asList(type));
++        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(type));
+ 
+         CustomSerializedObject1 obj1 = new CustomSerializedObject1(10);
+ 
+         BinaryObject po1 = marshal(obj1, marsh);
+ 
+         assertEquals(20, po1.<CustomSerializedObject1>deserialize().val);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testCustomSerializerWithGlobal() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
 -
 -        marsh.setSerializer(new CustomSerializer1());
 -
+         BinaryTypeConfiguration type1 =
+             new BinaryTypeConfiguration(CustomSerializedObject1.class.getName());
+         BinaryTypeConfiguration type2 =
+             new BinaryTypeConfiguration(CustomSerializedObject2.class.getName());
+ 
+         type2.setSerializer(new CustomSerializer2());
+ 
 -        marsh.setTypeConfigurations(Arrays.asList(type1, type2));
++        BinaryMarshaller marsh = binaryMarshaller(new CustomSerializer1(), Arrays.asList(type1, type2));
+ 
+         CustomSerializedObject1 obj1 = new CustomSerializedObject1(10);
+ 
+         BinaryObject po1 = marshal(obj1, marsh);
+ 
+         assertEquals(20, po1.<CustomSerializedObject1>deserialize().val);
+ 
+         CustomSerializedObject2 obj2 = new CustomSerializedObject2(10);
+ 
+         BinaryObject po2 = marshal(obj2, marsh);
+ 
+         assertEquals(30, po2.<CustomSerializedObject2>deserialize().val);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testCustomIdMapper() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
 -
+         BinaryTypeConfiguration type =
+             new BinaryTypeConfiguration(CustomMappedObject1.class.getName());
+ 
+         type.setIdMapper(new BinaryIdMapper() {
+             @Override public int typeId(String clsName) {
+                 return 11111;
+             }
+ 
+             @Override public int fieldId(int typeId, String fieldName) {
+                 assert typeId == 11111;
+ 
+                 if ("val1".equals(fieldName))
+                     return 22222;
+                 else if ("val2".equals(fieldName))
+                     return 33333;
+ 
+                 assert false : "Unknown field: " + fieldName;
+ 
+                 return 0;
+             }
+         });
+ 
 -        marsh.setTypeConfigurations(Arrays.asList(type));
++        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(type));
+ 
+         CustomMappedObject1 obj1 = new CustomMappedObject1(10, "str");
+ 
+         BinaryObjectEx po1 = marshal(obj1, marsh);
+ 
+         assertEquals(11111, po1.typeId());
+         assertEquals(10, po1.field(22222));
+         assertEquals("str", po1.field(33333));
+ 
+         assertEquals(10, po1.<CustomMappedObject1>deserialize().val1);
+         assertEquals("str", po1.<CustomMappedObject1>deserialize().val2);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testCustomIdMapperWithGlobal() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
++        BinaryTypeConfiguration type1 =
++            new BinaryTypeConfiguration(CustomMappedObject1.class.getName());
++        BinaryTypeConfiguration type2 =
++            new BinaryTypeConfiguration(CustomMappedObject2.class.getName());
+ 
 -        marsh.setIdMapper(new BinaryIdMapper() {
++        type2.setIdMapper(new BinaryIdMapper() {
+             @Override public int typeId(String clsName) {
 -                return 11111;
++                return 44444;
+             }
+ 
+             @Override public int fieldId(int typeId, String fieldName) {
 -                assert typeId == 11111;
++                assert typeId == 44444;
+ 
 -                if ("val1".equals(fieldName)) return 22222;
 -                else if ("val2".equals(fieldName)) return 33333;
++                if ("val1".equals(fieldName)) return 55555;
++                else if ("val2".equals(fieldName)) return 66666;
+ 
+                 assert false : "Unknown field: " + fieldName;
+ 
+                 return 0;
+             }
+         });
+ 
 -        BinaryTypeConfiguration type1 =
 -            new BinaryTypeConfiguration(CustomMappedObject1.class.getName());
 -        BinaryTypeConfiguration type2 =
 -            new BinaryTypeConfiguration(CustomMappedObject2.class.getName());
 -
 -        type2.setIdMapper(new BinaryIdMapper() {
++        BinaryMarshaller marsh = binaryMarshaller(new BinaryIdMapper() {
+             @Override public int typeId(String clsName) {
 -                return 44444;
++                return 11111;
+             }
+ 
+             @Override public int fieldId(int typeId, String fieldName) {
 -                assert typeId == 44444;
++                assert typeId == 11111;
+ 
 -                if ("val1".equals(fieldName)) return 55555;
 -                else if ("val2".equals(fieldName)) return 66666;
++                if ("val1".equals(fieldName))
++                    return 22222;
++                else if ("val2".equals(fieldName))
++                    return 33333;
+ 
+                 assert false : "Unknown field: " + fieldName;
+ 
+                 return 0;
+             }
 -        });
 -
 -        marsh.setTypeConfigurations(Arrays.asList(type1, type2));
++        }, Arrays.asList(type1, type2));
+ 
+         CustomMappedObject1 obj1 = new CustomMappedObject1(10, "str1");
+ 
+         BinaryObjectEx po1 = marshal(obj1, marsh);
+ 
+         assertEquals(11111, po1.typeId());
+         assertEquals(10, po1.field(22222));
+         assertEquals("str1", po1.field(33333));
+ 
+         assertEquals(10, po1.<CustomMappedObject1>deserialize().val1);
+         assertEquals("str1", po1.<CustomMappedObject1>deserialize().val2);
+ 
+         CustomMappedObject2 obj2 = new CustomMappedObject2(20, "str2");
+ 
+         BinaryObjectEx po2 = marshal(obj2, marsh);
+ 
+         assertEquals(44444, po2.typeId());
+         assertEquals(20, po2.field(55555));
+         assertEquals("str2", po2.field(66666));
+ 
+         assertEquals(20, po2.<CustomMappedObject2>deserialize().val1);
+         assertEquals("str2", po2.<CustomMappedObject2>deserialize().val2);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testDynamicObject() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
 -
 -        marsh.setTypeConfigurations(Arrays.asList(
++        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(
+             new BinaryTypeConfiguration(DynamicObject.class.getName())
+         ));
+ 
+         initializePortableContext(marsh);
+ 
+         BinaryObject po1 = marshal(new DynamicObject(0, 10, 20, 30), marsh);
+ 
+         assertEquals(new Integer(10), po1.field("val1"));
+         assertEquals(null, po1.field("val2"));
+         assertEquals(null, po1.field("val3"));
+ 
+         DynamicObject do1 = po1.deserialize();
+ 
+         assertEquals(10, do1.val1);
+         assertEquals(0, do1.val2);
+         assertEquals(0, do1.val3);
+ 
+         BinaryObject po2 = marshal(new DynamicObject(1, 10, 20, 30), marsh);
+ 
+         assertEquals(new Integer(10), po2.field("val1"));
+         assertEquals(new Integer(20), po2.field("val2"));
+         assertEquals(null, po2.field("val3"));
+ 
+         DynamicObject do2 = po2.deserialize();
+ 
+         assertEquals(10, do2.val1);
+         assertEquals(20, do2.val2);
+         assertEquals(0, do2.val3);
+ 
+         BinaryObject po3 = marshal(new DynamicObject(2, 10, 20, 30), marsh);
+ 
+         assertEquals(new Integer(10), po3.field("val1"));
+         assertEquals(new Integer(20), po3.field("val2"));
+         assertEquals(new Integer(30), po3.field("val3"));
+ 
+         DynamicObject do3 = po3.deserialize();
+ 
+         assertEquals(10, do3.val1);
+         assertEquals(20, do3.val2);
+         assertEquals(30, do3.val3);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testCycleLink() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
 -
 -        marsh.setTypeConfigurations(Arrays.asList(
++        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(
+             new BinaryTypeConfiguration(CycleLinkObject.class.getName())
+         ));
+ 
+         CycleLinkObject obj = new CycleLinkObject();
+ 
+         obj.self = obj;
+ 
+         BinaryObject po = marshal(obj, marsh);
+ 
+         CycleLinkObject obj0 = po.deserialize();
+ 
+         assert obj0.self == obj0;
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testDetached() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
 -
 -        marsh.setTypeConfigurations(Arrays.asList(
++        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(
+             new BinaryTypeConfiguration(DetachedTestObject.class.getName()),
+             new BinaryTypeConfiguration(DetachedInnerTestObject.class.getName())
+         ));
+ 
+         UUID id = UUID.randomUUID();
+ 
+         DetachedTestObject obj = marshal(new DetachedTestObject(
+             new DetachedInnerTestObject(null, id)), marsh).deserialize();
+ 
+         assertEquals(id, obj.inner1.id);
+         assertEquals(id, obj.inner4.id);
+ 
+         assert obj.inner1 == obj.inner4;
+ 
+         BinaryObjectImpl innerPo = (BinaryObjectImpl)obj.inner2;
+ 
+         assert innerPo.detached();
+ 
+         DetachedInnerTestObject inner = innerPo.deserialize();
+ 
+         assertEquals(id, inner.id);
+ 
+         BinaryObjectImpl detachedPo = (BinaryObjectImpl)innerPo.detach();
+ 
+         assert detachedPo.detached();
+ 
+         inner = detachedPo.deserialize();
+ 
+         assertEquals(id, inner.id);
+ 
+         innerPo = (BinaryObjectImpl)obj.inner3;
+ 
+         assert innerPo.detached();
+ 
+         inner = innerPo.deserialize();
+ 
+         assertEquals(id, inner.id);
+         assertNotNull(inner.inner);
+ 
+         detachedPo = (BinaryObjectImpl)innerPo.detach();
+ 
+         assert detachedPo.detached();
+ 
+         inner = innerPo.deserialize();
+ 
+         assertEquals(id, inner.id);
+         assertNotNull(inner.inner);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testCollectionFields() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
 -
 -        marsh.setTypeConfigurations(Arrays.asList(
++        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(
+             new BinaryTypeConfiguration(CollectionFieldsObject.class.getName()),
+             new BinaryTypeConfiguration(Key.class.getName()),
+             new BinaryTypeConfiguration(Value.class.getName())
+         ));
+ 
+         Object[] arr = new Object[] {new Value(1), new Value(2), new Value(3)};
+         Collection<Value> col = Arrays.asList(new Value(4), new Value(5), new Value(6));
+         Map<Key, Value> map = F.asMap(new Key(10), new Value(10), new Key(20), new Value(20), new Key(30), new Value(30));
+ 
+         CollectionFieldsObject obj = new CollectionFieldsObject(arr, col, map);
+ 
+         BinaryObject po = marshal(obj, marsh);
+ 
+         Object[] arr0 = po.field("arr");
+ 
+         assertEquals(3, arr0.length);
+ 
+         int i = 1;
+ 
+         for (Object valPo : arr0)
+             assertEquals(i++, ((BinaryObject)valPo).<Value>deserialize().val);
+ 
+         Collection<BinaryObject> col0 = po.field("col");
+ 
+         i = 4;
+ 
+         for (BinaryObject valPo : col0)
+             assertEquals(i++, valPo.<Value>deserialize().val);
+ 
+         Map<BinaryObject, BinaryObject> map0 = po.field("map");
+ 
+         for (Map.Entry<BinaryObject, BinaryObject> e : map0.entrySet())
+             assertEquals(e.getKey().<Key>deserialize().key, e.getValue().<Value>deserialize().val);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
 -    // TODO: Only with full headers.
+     public void _testDefaultMapping() throws Exception {
 -        PortableMarshaller marsh1 = createMarshaller();
 -
+         BinaryTypeConfiguration customMappingType =
+             new BinaryTypeConfiguration(TestBinary.class.getName());
+ 
+         customMappingType.setIdMapper(new BinaryIdMapper() {
+             @Override public int typeId(String clsName) {
+                 String typeName;
+ 
+                 try {
+                     Method mtd = PortableContext.class.getDeclaredMethod("typeName", String.class);
+ 
+                     mtd.setAccessible(true);
+ 
+                     typeName = (String)mtd.invoke(null, clsName);
+                 }
+                 catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) {
+                     throw new RuntimeException(e);
+                 }
+ 
+                 return typeName.toLowerCase().hashCode();
+             }
+ 
+             @Override public int fieldId(int typeId, String fieldName) {
+                 return fieldName.toLowerCase().hashCode();
+             }
+         });
+ 
 -        marsh1.setTypeConfigurations(Arrays.asList(
++        BinaryMarshaller marsh1 = binaryMarshaller(Arrays.asList(
+             new BinaryTypeConfiguration(SimpleObject.class.getName()),
+             customMappingType
+         ));
+ 
+         TestBinary obj = binaryObject();
+ 
+         BinaryObjectImpl po = marshal(obj, marsh1);
+ 
 -        PortableMarshaller marsh2 = createMarshaller();
 -
 -        marsh2.setTypeConfigurations(Arrays.asList(
++        BinaryMarshaller marsh2 = binaryMarshaller(Arrays.asList(
+             new BinaryTypeConfiguration(SimpleObject.class.getName()),
+             new BinaryTypeConfiguration(TestBinary.class.getName())
+         ));
+ 
 -        PortableContext ctx = initializePortableContext(marsh2);
 -
 -        po.context(ctx);
++        po = marshal(obj, marsh2);
+ 
+         assertEquals(obj, po.deserialize());
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testTypeNames() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
 -
+         BinaryTypeConfiguration customType1 = new BinaryTypeConfiguration(Value.class.getName());
+ 
+         customType1.setIdMapper(new BinaryIdMapper() {
+             @Override public int typeId(String clsName) {
+                 return 300;
+             }
+ 
+             @Override public int fieldId(int typeId, String fieldName) {
+                 return 0;
+             }
+         });
+ 
+         BinaryTypeConfiguration customType2 = new BinaryTypeConfiguration("org.gridgain.NonExistentClass1");
+ 
+         customType2.setIdMapper(new BinaryIdMapper() {
+             @Override public int typeId(String clsName) {
+                 return 400;
+             }
+ 
+             @Override public int fieldId(int typeId, String fieldName) {
+                 return 0;
+             }
+         });
+ 
+         BinaryTypeConfiguration customType3 = new BinaryTypeConfiguration("NonExistentClass2");
+ 
+         customType3.setIdMapper(new BinaryIdMapper() {
+             @Override public int typeId(String clsName) {
+                 return 500;
+             }
+ 
+             @Override public int fieldId(int typeId, String fieldName) {
+                 return 0;
+             }
+         });
+ 
+         BinaryTypeConfiguration customType4 = new BinaryTypeConfiguration("NonExistentClass5");
+ 
+         customType4.setIdMapper(new BinaryIdMapper() {
+             @Override public int typeId(String clsName) {
+                 return 0;
+             }
+ 
+             @Override public int fieldId(int typeId, String fieldName) {
+                 return 0;
+             }
+         });
+ 
 -        marsh.setTypeConfigurations(Arrays.asList(
++        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(
+             new BinaryTypeConfiguration(Key.class.getName()),
+             new BinaryTypeConfiguration("org.gridgain.NonExistentClass3"),
+             new BinaryTypeConfiguration("NonExistentClass4"),
+             customType1,
+             customType2,
+             customType3,
+             customType4
+         ));
+ 
 -        PortableContext ctx = initializePortableContext(marsh);
++        PortableContext ctx = portableContext(marsh);
+ 
+         assertEquals("notconfiguredclass".hashCode(), ctx.typeId("NotConfiguredClass"));
+         assertEquals("key".hashCode(), ctx.typeId("Key"));
+         assertEquals("nonexistentclass3".hashCode(), ctx.typeId("NonExistentClass3"));
+         assertEquals("nonexistentclass4".hashCode(), ctx.typeId("NonExistentClass4"));
+         assertEquals(300, ctx.typeId(getClass().getSimpleName() + "$Value"));
+         assertEquals(400, ctx.typeId("NonExistentClass1"));
+         assertEquals(500, ctx.typeId("NonExistentClass2"));
+         assertEquals("nonexistentclass5".hashCode(), ctx.typeId("NonExistentClass5"));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testFieldIdMapping() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
 -
+         BinaryTypeConfiguration customType1 = new BinaryTypeConfiguration(Value.class.getName());
+ 
+         customType1.setIdMapper(new BinaryIdMapper() {
+             @Override public int typeId(String clsName) {
+                 return 300;
+             }
+ 
+             @Override public int fieldId(int typeId, String fieldName) {
+                 switch (fieldName) {
+                     case "val1":
+                         return 301;
+ 
+                     case "val2":
+                         return 302;
+ 
+                     default:
+                         return 0;
+                 }
+             }
+         });
+ 
+         BinaryTypeConfiguration customType2 = new BinaryTypeConfiguration("NonExistentClass1");
+ 
+         customType2.setIdMapper(new BinaryIdMapper() {
+             @Override public int typeId(String clsName) {
+                 return 400;
+             }
+ 
+             @Override public int fieldId(int typeId, String fieldName) {
+                 switch (fieldName) {
+                     case "val1":
+                         return 401;
+ 
+                     case "val2":
+                         return 402;
+ 
+                     default:
+                         return 0;
+                 }
+             }
+         });
+ 
 -        marsh.setTypeConfigurations(Arrays.asList(new BinaryTypeConfiguration(Key.class.getName()),
++        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(new BinaryTypeConfiguration(Key.class.getName()),
+             new BinaryTypeConfiguration("NonExistentClass2"),
+             customType1,
+             customType2));
+ 
 -        PortableContext ctx = initializePortableContext(marsh);
++        PortableContext ctx = portableContext(marsh);
+ 
+         assertEquals("val".hashCode(), ctx.fieldId("key".hashCode(), "val"));
+         assertEquals("val".hashCode(), ctx.fieldId("nonexistentclass2".hashCode(), "val"));
+         assertEquals("val".hashCode(), ctx.fieldId("notconfiguredclass".hashCode(), "val"));
+         assertEquals(301, ctx.fieldId(300, "val1"));
+         assertEquals(302, ctx.fieldId(300, "val2"));
+         assertEquals("val3".hashCode(), ctx.fieldId(300, "val3"));
+         assertEquals(401, ctx.fieldId(400, "val1"));
+         assertEquals(402, ctx.fieldId(400, "val2"));
+         assertEquals("val3".hashCode(), ctx.fieldId(400, "val3"));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testDuplicateTypeId() throws Exception {
 -        final PortableMarshaller marsh = createMarshaller();
 -
+         BinaryTypeConfiguration customType1 = new BinaryTypeConfiguration("org.gridgain.Class1");
+ 
+         customType1.setIdMapper(new BinaryIdMapper() {
+             @Override public int typeId(String clsName) {
+                 return 100;
+             }
+ 
+             @Override public int fieldId(int typeId, String fieldName) {
+                 return 0;
+             }
+         });
+ 
+         BinaryTypeConfiguration customType2 = new BinaryTypeConfiguration("org.gridgain.Class2");
+ 
+         customType2.setIdMapper(new BinaryIdMapper() {
+             @Override public int typeId(String clsName) {
+                 return 100;
+             }
+ 
+             @Override public int fieldId(int typeId, String fieldName) {
+                 return 0;
+             }
+         });
+ 
 -        marsh.setTypeConfigurations(Arrays.asList(customType1, customType2));
 -
+         try {
 -            initializePortableContext(marsh);
++            binaryMarshaller(Arrays.asList(customType1, customType2));
+         }
+         catch (IgniteCheckedException e) {
 -            assertEquals("Duplicate type ID [clsName=org.gridgain.Class1, id=100]",
++            assertEquals("Duplicate type ID [clsName=org.gridgain.Class2, id=100]",
+                 e.getCause().getCause().getMessage());
+ 
+             return;
+         }
+ 
+         assert false;
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testPortableCopy() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
 -
 -        marsh.setTypeConfigurations(Arrays.asList(
++        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(
+             new BinaryTypeConfiguration(SimpleObject.class.getName())
+         ));
+ 
+         initializePortableContext(marsh);
+ 
+         SimpleObject obj = simpleObject();
+ 
+         final BinaryObject po = marshal(obj, marsh);
+ 
+         assertEquals(obj, po.deserialize());
+ 
+         BinaryObject copy = copy(po, null);
+ 
+         assertEquals(obj, copy.deserialize());
+ 
+         copy = copy(po, new HashMap<String, Object>());
+ 
+         assertEquals(obj, copy.deserialize());
+ 
+         Map<String, Object> map = new HashMap<>(1, 1.0f);
+ 
+         map.put("i", 3);
+ 
+         copy = copy(po, map);
+ 
+         assertEquals((byte)2, copy.<Byte>field("b").byteValue());
+         assertEquals((short)2, copy.<Short>field("s").shortValue());
+         assertEquals(3, copy.<Integer>field("i").intValue());
+         assertEquals(2L, copy.<Long>field("l").longValue());
+         assertEquals(2.2f, copy.<Float>field("f").floatValue(), 0);
+         assertEquals(2.2d, copy.<Double>field("d").doubleValue(), 0);
+         assertEquals((char)2, copy.<Character>field("c").charValue());
+         assertEquals(false, copy.<Boolean>field("bool").booleanValue());
+ 
+         SimpleObject obj0 = copy.deserialize();
+ 
+         assertEquals((byte)2, obj0.b);
+         assertEquals((short)2, obj0.s);
+         assertEquals(3, obj0.i);
+         assertEquals(2L, obj0.l);
+         assertEquals(2.2f, obj0.f, 0);
+         assertEquals(2.2d, obj0.d, 0);
+         assertEquals((char)2, obj0.c);
+         assertEquals(false, obj0.bool);
+ 
+         map = new HashMap<>(3, 1.0f);
+ 
+         map.put("b", (byte)3);
+         map.put("l", 3L);
+         map.put("bool", true);
+ 
+         copy = copy(po, map);
+ 
+         assertEquals((byte)3, copy.<Byte>field("b").byteValue());
+         assertEquals((short)2, copy.<Short>field("s").shortValue());
+         assertEquals(2, copy.<Integer>field("i").intValue());
+         assertEquals(3L, copy.<Long>field("l").longValue());
+         assertEquals(2.2f, copy.<Float>field("f").floatValue(), 0);
+         assertEquals(2.2d, copy.<Double>field("d").doubleValue(), 0);
+         assertEquals((char)2, copy.<Character>field("c").charValue());
+         assertEquals(true, copy.<Boolean>field("bool").booleanValue());
+ 
+         obj0 = copy.deserialize();
+ 
+         assertEquals((byte)3, obj0.b);
+         assertEquals((short)2, obj0.s);
+         assertEquals(2, obj0.i);
+         assertEquals(3L, obj0.l);
+         assertEquals(2.2f, obj0.f, 0);
+         assertEquals(2.2d, obj0.d, 0);
+         assertEquals((char)2, obj0.c);
+         assertEquals(true, obj0.bool);
+ 
+         map = new HashMap<>(8, 1.0f);
+ 
+         map.put("b", (byte)3);
+         map.put("s", (short)3);
+         map.put("i", 3);
+         map.put("l", 3L);
+         map.put("f", 3.3f);
+         map.put("d", 3.3d);
+         map.put("c", (char)3);
+         map.put("bool", true);
+ 
+         copy = copy(po, map);
+ 
+         assertEquals((byte)3, copy.<Byte>field("b").byteValue());
+         assertEquals((short)3, copy.<Short>field("s").shortValue());
+         assertEquals(3, copy.<Integer>field("i").intValue());
+         assertEquals(3L, copy.<Long>field("l").longValue());
+         assertEquals(3.3f, copy.<Float>field("f").floatValue(), 0);
+         assertEquals(3.3d, copy.<Double>field("d").doubleValue(), 0);
+         assertEquals((char)3, copy.<Character>field("c").charValue());
+         assertEquals(true, copy.<Boolean>field("bool").booleanValue());
+ 
+         obj0 = copy.deserialize();
+ 
+         assertEquals((byte)3, obj0.b);
+         assertEquals((short)3, obj0.s);
+         assertEquals(3, obj0.i);
+         assertEquals(3L, obj0.l);
+         assertEquals(3.3f, obj0.f, 0);
+         assertEquals(3.3d, obj0.d, 0);
+         assertEquals((char)3, obj0.c);
+         assertEquals(true, obj0.bool);
+ 
+ //        GridTestUtils.assertThrows(
+ //            log,
+ //            new Callable<Object>() {
+ //                @Override public Object call() throws Exception {
+ //                    po.copy(F.<String, Object>asMap("i", false));
+ //
+ //                    return null;
+ //                }
+ //            },
+ //            PortableException.class,
+ //            "Invalid value type for field: i"
+ //        );
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testPortableCopyString() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
 -
 -        marsh.setTypeConfigurations(Arrays.asList(
++        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(
+             new BinaryTypeConfiguration(SimpleObject.class.getName())
+         ));
+ 
+         SimpleObject obj = simpleObject();
+ 
+         BinaryObject po = marshal(obj, marsh);
+ 
+         BinaryObject copy = copy(po, F.<String, Object>asMap("str", "str3"));
+ 
+         assertEquals("str3", copy.<String>field("str"));
+ 
+         SimpleObject obj0 = copy.deserialize();
+ 
+         assertEquals("str3", obj0.str);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testPortableCopyUuid() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
 -
 -        marsh.setTypeConfigurations(Arrays.asList(
++        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(
+             new BinaryTypeConfiguration(SimpleObject.class.getName())
+         ));
+ 
+         SimpleObject obj = simpleObject();
+ 
+         BinaryObject po = marshal(obj, marsh);
+ 
+         UUID uuid = UUID.randomUUID();
+ 
+         BinaryObject copy = copy(po, F.<String, Object>asMap("uuid", uuid));
+ 
+         assertEquals(uuid, copy.<UUID>field("uuid"));
+ 
+         SimpleObject obj0 = copy.deserialize();
+ 
+         assertEquals(uuid, obj0.uuid);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testPortableCopyByteArray() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
 -
 -        marsh.setTypeConfigurations(Arrays.asList(
++        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(
+             new BinaryTypeConfiguration(SimpleObject.class.getName())
+         ));
+ 
+         SimpleObject obj = simpleObject();
+ 
+         BinaryObject po = marshal(obj, marsh);
+ 
+         BinaryObject copy = copy(po, F.<String, Object>asMap("bArr", new byte[]{1, 2, 3}));
+ 
+         assertArrayEquals(new byte[] {1, 2, 3}, copy.<byte[]>field("bArr"));
+ 
+         SimpleObject obj0 = copy.deserialize();
+ 
+         assertArrayEquals(new byte[] {1, 2, 3}, obj0.bArr);
+     }
+ 
+     /**
+      * @param po Portable object.
+      * @param fields Fields.
+      * @return Copy.
+      */
+     private BinaryObject copy(BinaryObject po, Map<String, Object> fields) {
+         BinaryObjectBuilder builder = BinaryObjectBuilderImpl.wrap(po);
+ 
+         if (fields != null) {
+             for (Map.Entry<String, Object> e : fields.entrySet())
+                 builder.setField(e.getKey(), e.getValue());
+         }
+ 
+         return builder.build();
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testPortableCopyShortArray() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
 -
 -        marsh.setTypeConfigurations(Arrays.asList(
++        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(
+             new BinaryTypeConfiguration(SimpleObject.class.getName())
+         ));
+ 
+         SimpleObject obj = simpleObject();
+ 
+         BinaryObject po = marshal(obj, marsh);
+ 
+         BinaryObject copy = copy(po, F.<String, Object>asMap("sArr", new short[]{1, 2, 3}));
+ 
+         assertArrayEquals(new short[] {1, 2, 3}, copy.<short[]>field("sArr"));
+ 
+         SimpleObject obj0 = copy.deserialize();
+ 
+         assertArrayEquals(new short[] {1, 2, 3}, obj0.sArr);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testPortableCopyIntArray() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
 -
 -        marsh.setTypeConfigurations(Arrays.asList(
++        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(
+             new BinaryTypeConfiguration(SimpleObject.class.getName())
+         ));
+ 
+         SimpleObject obj = simpleObject();
+ 
+         BinaryObject po = marshal(obj, marsh);
+ 
+         BinaryObject copy = copy(po, F.<String, Object>asMap("iArr", new int[]{1, 2, 3}));
+ 
+         assertArrayEquals(new int[] {1, 2, 3}, copy.<int[]>field("iArr"));
+ 
+         SimpleObject obj0 = copy.deserialize();
+ 
+         assertArrayEquals(new int[] {1, 2, 3}, obj0.iArr);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testPortableCopyLongArray() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
 -
 -        marsh.setTypeConfigurations(Arrays.asList(
++        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(
+             new BinaryTypeConfiguration(SimpleObject.class.getName())
+         ));
+ 
+         SimpleObject obj = simpleObject();
+ 
+         BinaryObject po = marshal(obj, marsh);
+ 
+         BinaryObject copy = copy(po, F.<String, Object>asMap("lArr", new long[]{1, 2, 3}));
+ 
+         assertArrayEquals(new long[] {1, 2, 3}, copy.<long[]>field("lArr"));
+ 
+         SimpleObject obj0 = copy.deserialize();
+ 
+         assertArrayEquals(new long[] {1, 2, 3}, obj0.lArr);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testPortableCopyFloatArray() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
 -
 -        marsh.setTypeConfigurations(Arrays.asList(
++        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(
+             new BinaryTypeConfiguration(SimpleObject.class.getName())
+         ));
+ 
+         SimpleObject obj = simpleObject();
+ 
+         BinaryObject po = marshal(obj, marsh);
+ 
+         BinaryObject copy = copy(po, F.<String, Object>asMap("fArr", new float[]{1, 2, 3}));
+ 
+         assertArrayEquals(new float[] {1, 2, 3}, copy.<float[]>field("fArr"), 0);
+ 
+         SimpleObject obj0 = copy.deserialize();
+ 
+         assertArrayEquals(new float[] {1, 2, 3}, obj0.fArr, 0);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testPortableCopyDoubleArray() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
 -
 -        marsh.setTypeConfigurations(Arrays.asList(
++        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(
+             new BinaryTypeConfiguration(SimpleObject.class.getName())
+         ));
+ 
+         SimpleObject obj = simpleObject();
+ 
+         BinaryObject po = marshal(obj, marsh);
+ 
+         BinaryObject copy = copy(po, F.<String, Object>asMap("dArr", new double[]{1, 2, 3}));
+ 
+         assertArrayEquals(new double[] {1, 2, 3}, copy.<double[]>field("dArr"), 0);
+ 
+         SimpleObject obj0 = copy.deserialize();
+ 
+         assertArrayEquals(new double[] {1, 2, 3}, obj0.dArr, 0);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testPortableCopyCharArray() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
 -
 -        marsh.setTypeConfigurations(Arrays.asList(
++        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(
+             new BinaryTypeConfiguration(SimpleObject.class.getName())
+         ));
+ 
+         SimpleObject obj = simpleObject();
+ 
+         BinaryObject po = marshal(obj, marsh);
+ 
+         BinaryObject copy = copy(po, F.<String, Object>asMap("cArr", new char[]{1, 2, 3}));
+ 
+         assertArrayEquals(new char[]{1, 2, 3}, copy.<char[]>field("cArr"));
+ 
+         SimpleObject obj0 = copy.deserialize();
+ 
+         assertArrayEquals(new char[]{1, 2, 3}, obj0.cArr);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testPortableCopyStringArray() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
 -
 -        marsh.setTypeConfigurations(Arrays.asList(
++        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(
+             new BinaryTypeConfiguration(SimpleObject.class.getName())
+         ));
+ 
+         SimpleObject obj = simpleObject();
+ 
+         BinaryObject po = marshal(obj, marsh);
+ 
+         BinaryObject copy = copy(po, F.<String, Object>asMap("strArr", new String[]{"str1", "str2"}));
+ 
+         assertArrayEquals(new String[]{"str1", "str2"}, copy.<String[]>field("strArr"));
+ 
+         SimpleObject obj0 = copy.deserialize();
+ 
+         assertArrayEquals(new String[]{"str1", "str2"}, obj0.strArr);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testPortableCopyObject() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
 -
 -        marsh.setTypeConfigurations(Arrays.asList(
++        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(
+             new BinaryTypeConfiguration(SimpleObject.class.getName())
+         ));
+ 
+         SimpleObject obj = simpleObject();
+ 
+         BinaryObject po = marshal(obj, marsh);
+ 
+         SimpleObject newObj = new SimpleObject();
+ 
+         newObj.i = 12345;
+         newObj.fArr = new float[] {5, 8, 0};
+         newObj.str = "newStr";
+ 
+         BinaryObject copy = copy(po, F.<String, Object>asMap("inner", newObj));
+ 
+         assertEquals(newObj, copy.<BinaryObject>field("inner").deserialize());
+ 
+         SimpleObject obj0 = copy.deserialize();
+ 
+         assertEquals(newObj, obj0.inner);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testPortableCopyNonPrimitives() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
 -
 -        marsh.setTypeConfigurations(Arrays.asList(
++        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(
+             new BinaryTypeConfiguration(SimpleObject.class.getName())
+         ));
+ 
+         SimpleObject obj = simpleObject();
+ 
+         BinaryObject po = marshal(obj, marsh);
+ 
+         Map<String, Object> map = new HashMap<>(3, 1.0f);
+ 
+         SimpleObject newObj = new SimpleObject();
+ 
+         newObj.i = 12345;
+         newObj.fArr = new float[] {5, 8, 0};
+         newObj.str = "newStr";
+ 
+         map.put("str", "str555");
+         map.put("inner", newObj);
+         map.put("bArr", new byte[]{6, 7, 9});
+ 
+         BinaryObject copy = copy(po, map);
+ 
+         assertEquals("str555", copy.<String>field("str"));
+         assertEquals(newObj, copy.<BinaryObject>field("inner").deserialize());
+         assertArrayEquals(new byte[]{6, 7, 9}, copy.<byte[]>field("bArr"));
+ 
+         SimpleObject obj0 = copy.deserialize();
+ 
+         assertEquals("str555", obj0.str);
+         assertEquals(newObj, obj0.inner);
+         assertArrayEquals(new byte[] {6, 7, 9}, obj0.bArr);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testPortableCopyMixed() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
 -
 -        marsh.setTypeConfigurations(Arrays.asList(new BinaryTypeConfiguration(SimpleObject.class.getName())));
++        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(new BinaryTypeConfiguration(SimpleObject.class.getName())));
+ 
+         SimpleObject obj = simpleObject();
+ 
+         BinaryObject po = marshal(obj, marsh);
+ 
+         Map<String, Object> map = new HashMap<>(3, 1.0f);
+ 
+         SimpleObject newObj = new SimpleObject();
+ 
+         newObj.i = 12345;
+         newObj.fArr = new float[] {5, 8, 0};
+         newObj.str = "newStr";
+ 
+         map.put("i", 1234);
+         map.put("str", "str555");
+         map.put("inner", newObj);
+         map.put("s", (short)2323);
+         map.put("bArr", new byte[]{6, 7, 9});
+         map.put("b", (byte)111);
+ 
+         BinaryObject copy = copy(po, map);
+ 
+         assertEquals(1234, copy.<Integer>field("i").intValue());
+         assertEquals("str555", copy.<String>field("str"));
+         assertEquals(newObj, copy.<BinaryObject>field("inner").deserialize());
+         assertEquals((short)2323, copy.<Short>field("s").shortValue());
+         assertArrayEquals(new byte[] {6, 7, 9}, copy.<byte[]>field("bArr"));
+         assertEquals((byte)111, copy.<Byte>field("b").byteValue());
+ 
+         SimpleObject obj0 = copy.deserialize();
+ 
+         assertEquals(1234, obj0.i);
+         assertEquals("str555", obj0.str);
+         assertEquals(newObj, obj0.inner);
+         assertEquals((short)2323, obj0.s);
+         assertArrayEquals(new byte[] {6, 7, 9}, obj0.bArr);
+         assertEquals((byte)111, obj0.b);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testKeepDeserialized() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
 -
 -        marsh.setClassNames(Arrays.asList(SimpleObject.class.getName()));
 -        marsh.setKeepDeserialized(true);
 -
 -        BinaryObject po = marshal(simpleObject(), marsh);
++        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(new BinaryTypeConfiguration(SimpleObject.class.getName())));
+ 
 -        assert po.deserialize() == po.deserialize();
++        BinaryObjectImpl po = marshal(simpleObject(), marsh);
+ 
 -        marsh = createMarshaller();
++        CacheObjectContext coCtx = new CacheObjectContext(newContext(), null, false, true, false);
+ 
 -        marsh.setClassNames(Arrays.asList(SimpleObject.class.getName()));
 -        marsh.setKeepDeserialized(false);
++        assert po.value(coCtx, false) == po.value(coCtx, false);
+ 
+         po = marshal(simpleObject(), marsh);
+ 
+         assert po.deserialize() != po.deserialize();
 -
 -        marsh = createMarshaller();
 -
 -        marsh.setKeepDeserialized(true);
 -        marsh.setTypeConfigurations(Arrays.asList(
 -            new BinaryTypeConfiguration(SimpleObject.class.getName())));
 -
 -        po = marshal(simpleObject(), marsh);
 -
 -        assert po.deserialize() == po.deserialize();
 -
 -        marsh = createMarshaller();
 -
 -        marsh.setKeepDeserialized(false);
 -        marsh.setTypeConfigurations(Arrays.asList(
 -            new BinaryTypeConfiguration(SimpleObject.class.getName())));
 -
 -        po = marshal(simpleObject(), marsh);
 -
 -        assert po.deserialize() != po.deserialize();
 -
 -        marsh = createMarshaller();
 -
 -        marsh.setKeepDeserialized(true);
 -
 -        BinaryTypeConfiguration typeCfg = new BinaryTypeConfiguration(SimpleObject.class.getName());
 -
 -        typeCfg.setKeepDeserialized(false);
 -
 -        marsh.setTypeConfigurations(Arrays.asList(typeCfg));
 -
 -        po = marshal(simpleObject(), marsh);
 -
 -        assert po.deserialize() != po.deserialize();
 -
 -        marsh = createMarshaller();
 -
 -        marsh.setKeepDeserialized(false);
 -
 -        typeCfg = new BinaryTypeConfiguration(SimpleObject.class.getName());
 -
 -        typeCfg.setKeepDeserialized(true);
 -
 -        marsh.setTypeConfigurations(Arrays.asList(typeCfg));
 -
 -        po = marshal(simpleObject(), marsh);
 -
 -        assert po.deserialize() == po.deserialize();
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testOffheapPortable() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
 -
 -        marsh.setTypeConfigurations(Arrays.asList(new BinaryTypeConfiguration(SimpleObject.class.getName())));
++        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(new BinaryTypeConfiguration(SimpleObject.class.getName())));
+ 
 -        PortableContext ctx = initializePortableContext(marsh);
++        PortableContext ctx = portableContext(marsh);
+ 
+         SimpleObject simpleObj = simpleObject();
+ 
+         BinaryObjectImpl obj = marshal(simpleObj, marsh);
+ 
+         long ptr = 0;
+ 
+         long ptr1 = 0;
+ 
+         long ptr2 = 0;
+ 
+         try {
+             ptr = copyOffheap(obj);
+ 
+             BinaryObjectOffheapImpl offheapObj = new BinaryObjectOffheapImpl(ctx,
+                 ptr,
+                 0,
+                 obj.array().length);
+ 
+             assertTrue(offheapObj.equals(offheapObj));
+             assertFalse(offheapObj.equals(null));
+             assertFalse(offheapObj.equals("str"));
+             assertTrue(offheapObj.equals(obj));
+             assertTrue(obj.equals(offheapObj));
+ 
+             ptr1 = copyOffheap(obj);
+ 
+             BinaryObjectOffheapImpl offheapObj1 = new BinaryObjectOffheapImpl(ctx,
+                 ptr1,
+                 0,
+                 obj.array().length);
+ 
+             assertTrue(offheapObj.equals(offheapObj1));
+             assertTrue(offheapObj1.equals(offheapObj));
+ 
+             assertEquals(obj.typeId(), offheapObj.typeId());
+             assertEquals(obj.hashCode(), offheapObj.hashCode());
+ 
+             checkSimpleObjectData(simpleObj, offheapObj);
+ 
+             BinaryObjectOffheapImpl innerOffheapObj = offheapObj.field("inner");
+ 
+             assertNotNull(innerOffheapObj);
+ 
+             checkSimpleObjectData(simpleObj.inner, innerOffheapObj);
+ 
+             obj = (BinaryObjectImpl)offheapObj.heapCopy();
+ 
+             assertEquals(obj.typeId(), offheapObj.typeId());
+             assertEquals(obj.hashCode(), offheapObj.hashCode());
+ 
+             checkSimpleObjectData(simpleObj, obj);
+ 
+             BinaryObjectImpl innerObj = obj.field("inner");
+ 
+             assertNotNull(innerObj);
+ 
+             checkSimpleObjectData(simpleObj.inner, innerObj);
+ 
+             simpleObj.d = 0;
+ 
+             obj = marshal(simpleObj, marsh);
+ 
+             assertFalse(offheapObj.equals(obj));
+             assertFalse(obj.equals(offheapObj));
+ 
+             ptr2 = copyOffheap(obj);
+ 
+             BinaryObjectOffheapImpl offheapObj2 = new BinaryObjectOffheapImpl(ctx,
+                 ptr2,
+                 0,
+                 obj.array().length);
+ 
+             assertFalse(offheapObj.equals(offheapObj2));
+             assertFalse(offheapObj2.equals(offheapObj));
+         }
+         finally {
+             UNSAFE.freeMemory(ptr);
+ 
+             if (ptr1 > 0)
+                 UNSAFE.freeMemory(ptr1);
+ 
+             if (ptr2 > 0)
+                 UNSAFE.freeMemory(ptr2);
+         }
+     }
+ 
+     /**
+      *
+      */
+     public void testReadResolve() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
 -
 -        marsh.setClassNames(
 -            Arrays.asList(MySingleton.class.getName(), SingletonMarker.class.getName()));
++        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(
++            new BinaryTypeConfiguration(MySingleton.class.getName()),
++            new BinaryTypeConfiguration(SingletonMarker.class.getName())));
+ 
+         BinaryObjectImpl portableObj = marshal(MySingleton.INSTANCE, marsh);
+ 
+         assertTrue(portableObj.array().length <= 1024); // Check that big string was not serialized.
+ 
+         MySingleton singleton = portableObj.deserialize();
+ 
+         assertSame(MySingleton.INSTANCE, singleton);
+     }
+ 
+     /**
+      *
+      */
+     public void testReadResolveOnPortableAware() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
 -
 -        marsh.setClassNames(Collections.singletonList(MyTestClass.class.getName()));
++        BinaryMarshaller marsh = binaryMarshaller(Collections.singletonList(
++            new BinaryTypeConfiguration(MyTestClass.class.getName())));
+ 
+         BinaryObjectImpl portableObj = marshal(new MyTestClass(), marsh);
+ 
+         MyTestClass obj = portableObj.deserialize();
+ 
+         assertEquals("readResolve", obj.s);
+     }
+ 
+     /**
+      * @throws Exception If ecxeption thrown.
+      */
+     public void testDeclareReadResolveInParent() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
 -
 -        marsh.setClassNames(Arrays.asList(ChildPortable.class.getName()));
++        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(new BinaryTypeConfiguration(ChildPortable.class.getName())));
+ 
+         BinaryObjectImpl portableObj = marshal(new ChildPortable(), marsh);
+ 
+         ChildPortable singleton = portableObj.deserialize();
+ 
+         assertNotNull(singleton.s);
+     }
+ 
+     /**
+      *
+      */
+     public void testDecimalFields() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
 -
 -        Collection<String> clsNames = new ArrayList<>();
++        Collection<BinaryTypeConfiguration> clsNames = new ArrayList<>();
+ 
 -        clsNames.add(DecimalReflective.class.getName());
 -        clsNames.add(DecimalMarshalAware.class.getName());
++        clsNames.add(new BinaryTypeConfiguration(DecimalReflective.class.getName()));
++        clsNames.add(new BinaryTypeConfiguration(DecimalMarshalAware.class.getName()));
+ 
 -        marsh.setClassNames(clsNames);
++        BinaryMarshaller marsh = binaryMarshaller(clsNames);
+ 
+         // 1. Test reflective stuff.
+         DecimalReflective obj1 = new DecimalReflective();
+ 
+         obj1.val = BigDecimal.ZERO;
+         obj1.valArr = new BigDecimal[] { BigDecimal.ONE, BigDecimal.TEN };
+ 
+         BinaryObjectImpl portObj = marshal(obj1, marsh);
+ 
+         assertEquals(obj1.val, portObj.field("val"));
+         assertArrayEquals(obj1.valArr, portObj.<BigDecimal[]>field("valArr"));
+ 
+         assertEquals(obj1.val, portObj.<DecimalReflective>deserialize().val);
+         assertArrayEquals(obj1.valArr, portObj.<DecimalReflective>deserialize().valArr);
+ 
+         // 2. Test marshal aware stuff.
+         DecimalMarshalAware obj2 = new DecimalMarshalAware();
+ 
+         obj2.val = BigDecimal.ZERO;
+         obj2.valArr = new BigDecimal[] { BigDecimal.ONE, BigDecimal.TEN.negate() };
+         obj2.rawVal = BigDecimal.TEN;
+         obj2.rawValArr = new BigDecimal[] { BigDecimal.ZERO, BigDecimal.ONE };
+ 
+         portObj = marshal(obj2, marsh);
+ 
+         assertEquals(obj2.val, portObj.field("val"));
+         assertArrayEquals(obj2.valArr, portObj.<BigDecimal[]>field("valArr"));
+ 
+         assertEquals(obj2.val, portObj.<DecimalMarshalAware>deserialize().val);
+         assertArrayEquals(obj2.valArr, portObj.<DecimalMarshalAware>deserialize().valArr);
+         assertEquals(obj2.rawVal, portObj.<DecimalMarshalAware>deserialize().rawVal);
+         assertArrayEquals(obj2.rawValArr, portObj.<DecimalMarshalAware>deserialize().rawValArr);
+     }
+ 
+     /**
+      * @throws IgniteCheckedException If failed.
+      */
 -    public void testFinalField() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
++    public void testFinalField() throws IgniteCheckedException {
++        BinaryMarshaller marsh = binaryMarshaller();
+ 
+         SimpleObjectWithFinal obj = new SimpleObjectWithFinal();
+ 
+         SimpleObjectWithFinal po0 = marshalUnmarshal(obj, marsh);
+ 
+         assertEquals(obj.time, po0.time);
+     }
+ 
+     /**
+      * @throws IgniteCheckedException If failed.
+      */
+     public void testThreadLocalArrayReleased() throws Exception {
+         // Checking the writer directly.
+         assertEquals(false, THREAD_LOCAL_ALLOC.isThreadLocalArrayAcquired());
+ 
 -        PortableMarshaller marsh0 = createMarshaller();
++        BinaryMarshaller marsh = binaryMarshaller();
+ 
 -        try (BinaryWriterExImpl writer = new BinaryWriterExImpl(portableContext(marsh0))) {
++        try (BinaryWriterExImpl writer = new BinaryWriterExImpl(portableContext(marsh))) {
+             assertEquals(true, THREAD_LOCAL_ALLOC.isThreadLocalArrayAcquired());
+ 
+             writer.writeString("Thread local test");
+ 
+             writer.array();
+ 
+             assertEquals(true, THREAD_LOCAL_ALLOC.isThreadLocalArrayAcquired());
+         }
+ 
+         // Checking the portable marshaller.
+         assertEquals(false, THREAD_LOCAL_ALLOC.isThreadLocalArrayAcquired());
+ 
 -        PortableMarshaller marsh = createMarshaller();
++        marsh = binaryMarshaller();
+ 
+         marsh.marshal(new SimpleObject());
+ 
+         assertEquals(false, THREAD_LOCAL_ALLOC.isThreadLocalArrayAcquired());
+ 
 -        // Checking the builder.
 -        PortableMarshaller marsh2 = createMarshaller();
++        marsh = binaryMarshaller();
+ 
 -        BinaryObjectBuilder builder = new BinaryObjectBuilderImpl(portableContext(marsh2),
++        // Checking the builder.
++        BinaryObjectBuilder builder = new BinaryObjectBuilderImpl(portableContext(marsh),
+             "org.gridgain.foo.bar.TestClass");
+ 
+         builder.setField("a", "1");
+ 
+         BinaryObject portableObj = builder.build();
+ 
+         assertEquals(false, THREAD_LOCAL_ALLOC.isThreadLocalArrayAcquired());
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testDuplicateName() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
++        BinaryMarshaller marsh = binaryMarshaller();
+ 
+         Test1.Job job1 = new Test1().new Job();
+         Test2.Job job2 = new Test2().new Job();
+ 
+         marsh.marshal(job1);
+ 
+         try {
+             marsh.marshal(job2);
+         }
+         catch (BinaryObjectException e) {
+             assertEquals(true, e.getMessage().contains("Failed to register class"));
+ 
+             return;
+         }
+ 
+         assert false;
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testClass() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
++        BinaryMarshaller marsh = binaryMarshaller();
+ 
+         Class cls = BinaryMarshallerSelfTest.class;
+ 
+         Class unmarshalledCls = marshalUnmarshal(cls, marsh);
+ 
+         Assert.assertEquals(cls, unmarshalledCls);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testClassFieldsMarshalling() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
++        BinaryMarshaller marsh = binaryMarshaller();
+ 
+         ObjectWithClassFields obj = new ObjectWithClassFields();
+         obj.cls1 = BinaryMarshallerSelfTest.class;
+ 
+         byte[] marshal = marsh.marshal(obj);
+ 
+         ObjectWithClassFields obj2 = marsh.unmarshal(marshal, null);
+ 
+         assertEquals(obj.cls1, obj2.cls1);
+         assertNull(obj2.cls2);
+ 
+         BinaryObject portObj = marshal(obj, marsh);
+ 
+         Class cls1 = portObj.field("cls1");
+ 
+         assertEquals(obj.cls1, cls1);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testMarshallingThroughJdk() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
++        BinaryMarshaller marsh = binaryMarshaller();
+ 
+         InetSocketAddress addr = new InetSocketAddress("192.168.0.2", 4545);
+ 
+         byte[] arr = marsh.marshal(addr);
+ 
+         InetSocketAddress addr2 = marsh.unmarshal(arr, null);
+ 
+         assertEquals(addr.getHostString(), addr2.getHostString());
+         assertEquals(addr.getPort(), addr2.getPort());
+ 
+         TestAddress testAddr = new TestAddress();
+         testAddr.addr = addr;
+         testAddr.str1 = "Hello World";
+ 
+         SimpleObject simpleObj = new SimpleObject();
+         simpleObj.c = 'g';
+         simpleObj.date = new Date();
+ 
+         testAddr.obj = simpleObj;
+ 
+         arr = marsh.marshal(testAddr);
+ 
+         TestAddress testAddr2 = marsh.unmarshal(arr, null);
+ 
+         assertEquals(testAddr.addr.getHostString(), testAddr2.addr.getHostString());
+         assertEquals(testAddr.addr.getPort(), testAddr2.addr.getPort());
+         assertEquals(testAddr.str1, testAddr2.str1);
+         assertEquals(testAddr.obj.c, testAddr2.obj.c);
+         assertEquals(testAddr.obj.date, testAddr2.obj.date);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testPredefinedTypeIds() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
++        BinaryMarshaller marsh = binaryMarshaller();
+ 
 -        PortableContext pCtx = initializePortableContext(marsh);
++        PortableContext pCtx = portableContext(marsh);
+ 
+         Field field = pCtx.getClass().getDeclaredField("predefinedTypeNames");
+ 
+         field.setAccessible(true);
+ 
+         Map<String, Integer> map = (Map<String, Integer>)field.get(pCtx);
+ 
+         assertTrue(map.size() > 0);
+ 
+         for (Map.Entry<String, Integer> entry : map.entrySet()) {
+             int id = entry.getValue();
+ 
+             if (id == GridPortableMarshaller.UNREGISTERED_TYPE_ID)
+                 continue;
+ 
+             PortableClassDescriptor desc = pCtx.descriptorForTypeId(false, entry.getValue(), null);
+ 
+             assertEquals(desc.typeId(), pCtx.typeId(desc.describedClass().getName()));
+             assertEquals(desc.typeId(), pCtx.typeId(pCtx.typeName(desc.describedClass().getName())));
+         }
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testCyclicReferencesMarshalling() throws Exception {
 -        PortableMarshaller marsh = createMarshaller();
++        BinaryMarshaller marsh = binaryMarshaller();
+ 
+         SimpleObject obj = simpleObject();
+ 
+         obj.bArr = obj.inner.bArr;
+         obj.cArr = obj.inner.cArr;
+         obj.boolArr = obj.inner.boolArr;
+         obj.sArr = obj.inner.sArr;
+         obj.strArr = obj.inner.strArr;
+         obj.iArr = obj.inner.iArr;
+         obj.lArr = obj.inner.lArr;
+         obj.fArr = obj.inner.fArr;
+         obj.dArr = obj.inner.dArr;
+         obj.dateArr = obj.inner.dateArr;
+         obj.uuidArr = obj.inner.uuidArr;
+         obj.objArr = obj.inner.objArr;
+         obj.bdArr = obj.inner.bdArr;
+         obj.map = obj.inner.map;
+         obj.col = obj.inner.col;
+         obj.mEntry = obj.inner.mEntry;
+ 
+         SimpleObject res = (SimpleObject)marshalUnmarshal(obj, marsh);
+ 
+         assertEquals(obj, res);
+ 
+         assertTrue(res.bArr == res.inner.bArr);
+         assertTrue(res.cArr == res.inner.cArr);
+         assertTrue(res.boolArr == res.inner.boolArr);
+         assertTrue(res.sArr == res.inner.sArr);
+         assertTrue(res.strArr == res.inner.strArr);
+         assertTrue(res.iArr == res.inner.iArr);
+         assertTrue(res.lArr == res.inner.lArr);
+         assertTrue(res.fArr == res.inner.fArr);
+         assertTrue(res.dArr == res.inner.dArr);
+         assertTrue(res.dateArr == res.inner.dateArr);
+         assertTrue(res.uuidArr == res.inner.uuidArr);
+         assertTrue(res.objArr == res.inner.objArr);
+         assertTrue(res.bdArr == res.inner.bdArr);
+         assertTrue(res.map == res.inner.map);
+         assertTrue(res.col == res.inner.col);
+         assertTrue(res.mEntry == res.inner.mEntry);
+     }
+ 
+     /**
+      * Object with class fields.
+      */
+     private static class ObjectWithClassFields {
+         /** */
+         private Class<?> cls1;
+ 
+         /** */
+         private Class<?> cls2;
+     }
+ 
+     /**
+      *
+      */
+     private static class TestAddress {
+         /** */
+         private SimpleObject obj;
+ 
+         /** */
+         private InetSocketAddress addr;
+ 
+         /** */
+         private String str1;
+     }
+ 
+     /**
+      *
+      */
+     private static class Test1 {
+         /**
+          *
+          */
+         private class Job {
+ 
+         }
+     }
+ 
+     /**
+      *
+      */
+     private static class Test2 {
+         /**
+          *
+          */
+         private class Job {
+ 
+         }
+     }
+ 
+     /**
+      * @param obj Object.
+      * @return Offheap address.
+      */
+     private long copyOffheap(BinaryObjectImpl obj) {
+         byte[] arr = obj.array();
+ 
+         long ptr = UNSAFE.allocateMemory(arr.length);
+ 
+         UNSAFE.copyMemory(arr, BYTE_ARR_OFF, null, ptr, arr.length);
+ 
+         return ptr;
+     }
+ 
+     /**
+      * @param enumArr Enum array.
+      * @return Ordinals.
+      */
+     private <T extends Enum<?>> Integer[] ordinals(T[] enumArr) {

<TRUNCATED>

[09/13] ignite git commit: IGNITE-1949: C++ platform library for Linux: make uninstall now delete directories.

Posted by ag...@apache.org.
IGNITE-1949: C++ platform library for Linux: make uninstall now delete directories.


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

Branch: refs/heads/ignite-1945
Commit: 94afe3e2eb421323d1602c5f9845c93b23a96548
Parents: a7b22f8
Author: Igor Sapego <is...@gridgain.com>
Authored: Thu Nov 19 13:41:37 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Nov 19 13:41:37 2015 +0300

----------------------------------------------------------------------
 modules/platforms/cpp/common/include/Makefile.am          | 3 +++
 modules/platforms/cpp/common/os/linux/include/Makefile.am | 3 +++
 modules/platforms/cpp/core/include/Makefile.am            | 3 +++
 modules/platforms/cpp/core/os/linux/include/Makefile.am   | 3 +++
 4 files changed, 12 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/94afe3e2/modules/platforms/cpp/common/include/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/include/Makefile.am b/modules/platforms/cpp/common/include/Makefile.am
index 7a02225..0df9741 100644
--- a/modules/platforms/cpp/common/include/Makefile.am
+++ b/modules/platforms/cpp/common/include/Makefile.am
@@ -21,3 +21,6 @@ nobase_include_HEADERS = ignite/common/concurrent.h \
                          ignite/common/java.h \
                          ignite/common/exports.h \
                          ignite/common/utils.h
+
+uninstall-hook:
+	find ${includedir}/ignite -type d -empty -delete

http://git-wip-us.apache.org/repos/asf/ignite/blob/94afe3e2/modules/platforms/cpp/common/os/linux/include/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/os/linux/include/Makefile.am b/modules/platforms/cpp/common/os/linux/include/Makefile.am
index 68e45e6..f6e2803 100644
--- a/modules/platforms/cpp/common/os/linux/include/Makefile.am
+++ b/modules/platforms/cpp/common/os/linux/include/Makefile.am
@@ -19,3 +19,6 @@ ACLOCAL_AMFLAGS = "-Im4"
 
 nobase_include_HEADERS = ignite/common/common.h \
                          ignite/common/concurrent_os.h
+
+uninstall-hook:
+	find ${includedir}/ignite -type d -empty -delete

http://git-wip-us.apache.org/repos/asf/ignite/blob/94afe3e2/modules/platforms/cpp/core/include/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/Makefile.am b/modules/platforms/cpp/core/include/Makefile.am
index 1e7bf7d..b28caed 100644
--- a/modules/platforms/cpp/core/include/Makefile.am
+++ b/modules/platforms/cpp/core/include/Makefile.am
@@ -64,3 +64,6 @@ nobase_include_HEADERS = ignite/cache/cache.h \
                          ignite/ignite_error.h \
                          ignite/ignition.h \
                          ignite/guid.h
+
+uninstall-hook:
+	find ${includedir}/ignite -type d -empty -delete

http://git-wip-us.apache.org/repos/asf/ignite/blob/94afe3e2/modules/platforms/cpp/core/os/linux/include/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/os/linux/include/Makefile.am b/modules/platforms/cpp/core/os/linux/include/Makefile.am
index 2ee13eff..9fa5242 100644
--- a/modules/platforms/cpp/core/os/linux/include/Makefile.am
+++ b/modules/platforms/cpp/core/os/linux/include/Makefile.am
@@ -18,3 +18,6 @@
 ACLOCAL_AMFLAGS = "-Im4"
 
 nobase_include_HEADERS = ignite/impl/utils.h
+
+uninstall-hook:
+	find ${includedir}/ignite -type d -empty -delete


[02/13] ignite git commit: IGNITE-1816: Implemented compact footers.

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
deleted file mode 100644
index d0a5709..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
+++ /dev/null
@@ -1,3760 +0,0 @@
-/*
- * 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.lang.reflect.Field;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.net.InetSocketAddress;
-import java.sql.Timestamp;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
-import java.util.LinkedHashSet;
-import java.util.Map;
-import java.util.TreeMap;
-import java.util.TreeSet;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentSkipListSet;
-import junit.framework.Assert;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.binary.BinaryObject;
-import org.apache.ignite.binary.BinaryObjectBuilder;
-import org.apache.ignite.binary.BinaryObjectException;
-import org.apache.ignite.binary.BinaryRawReader;
-import org.apache.ignite.binary.BinaryRawWriter;
-import org.apache.ignite.binary.BinaryReader;
-import org.apache.ignite.binary.BinarySerializer;
-import org.apache.ignite.binary.BinaryType;
-import org.apache.ignite.binary.BinaryTypeConfiguration;
-import org.apache.ignite.binary.BinaryIdMapper;
-import org.apache.ignite.binary.BinaryWriter;
-import org.apache.ignite.binary.Binarylizable;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.portable.builder.BinaryObjectBuilderImpl;
-import org.apache.ignite.internal.util.GridUnsafe;
-import org.apache.ignite.internal.util.IgniteUtils;
-import org.apache.ignite.internal.util.lang.GridMapEntry;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.marshaller.MarshallerContextTestImpl;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.testframework.GridTestUtils;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.jsr166.ConcurrentHashMap8;
-import sun.misc.Unsafe;
-
-import static org.apache.ignite.internal.portable.PortableThreadLocalMemoryAllocator.THREAD_LOCAL_ALLOC;
-import static org.junit.Assert.assertArrayEquals;
-
-/**
- * Portable marshaller tests.
- */
-@SuppressWarnings({"OverlyStrongTypeCast", "ArrayHashCode", "ConstantConditions"})
-public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
-    /** */
-    private static final Unsafe UNSAFE = GridUnsafe.unsafe();
-
-    /** */
-    protected static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testNull() throws Exception {
-        assertNull(marshalUnmarshal(null));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testByte() throws Exception {
-        assertEquals((byte)100, marshalUnmarshal((byte)100).byteValue());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testShort() throws Exception {
-        assertEquals((short)100, marshalUnmarshal((short)100).shortValue());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testInt() throws Exception {
-        assertEquals(100, marshalUnmarshal(100).intValue());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testLong() throws Exception {
-        assertEquals(100L, marshalUnmarshal(100L).longValue());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testFloat() throws Exception {
-        assertEquals(100.001f, marshalUnmarshal(100.001f).floatValue(), 0);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDouble() throws Exception {
-        assertEquals(100.001d, marshalUnmarshal(100.001d).doubleValue(), 0);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testChar() throws Exception {
-        assertEquals((char)100, marshalUnmarshal((char)100).charValue());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testBoolean() throws Exception {
-        assertEquals(true, marshalUnmarshal(true).booleanValue());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDecimal() throws Exception {
-        BigDecimal val;
-
-        assertEquals((val = BigDecimal.ZERO), marshalUnmarshal(val));
-        assertEquals((val = BigDecimal.valueOf(Long.MAX_VALUE, 0)), marshalUnmarshal(val));
-        assertEquals((val = BigDecimal.valueOf(Long.MIN_VALUE, 0)), marshalUnmarshal(val));
-        assertEquals((val = BigDecimal.valueOf(Long.MAX_VALUE, 8)), marshalUnmarshal(val));
-        assertEquals((val = BigDecimal.valueOf(Long.MIN_VALUE, 8)), marshalUnmarshal(val));
-
-        assertEquals((val = new BigDecimal(new BigInteger("-79228162514264337593543950336"))), marshalUnmarshal(val));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testString() throws Exception {
-        assertEquals("str", marshalUnmarshal("str"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testUuid() throws Exception {
-        UUID uuid = UUID.randomUUID();
-
-        assertEquals(uuid, marshalUnmarshal(uuid));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDate() throws Exception {
-        Date date = new Date();
-
-        Date val = marshalUnmarshal(date);
-
-        assertEquals(date, val);
-        assertEquals(Date.class, val.getClass());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testTimestamp() throws Exception {
-        Timestamp ts = new Timestamp(System.currentTimeMillis());
-
-        ts.setNanos(999999999);
-
-        assertEquals(ts, marshalUnmarshal(ts));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testByteArray() throws Exception {
-        byte[] arr = new byte[] {10, 20, 30};
-
-        assertArrayEquals(arr, marshalUnmarshal(arr));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testShortArray() throws Exception {
-        short[] arr = new short[] {10, 20, 30};
-
-        assertArrayEquals(arr, marshalUnmarshal(arr));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testIntArray() throws Exception {
-        int[] arr = new int[] {10, 20, 30};
-
-        assertArrayEquals(arr, marshalUnmarshal(arr));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testLongArray() throws Exception {
-        long[] arr = new long[] {10, 20, 30};
-
-        assertArrayEquals(arr, marshalUnmarshal(arr));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testFloatArray() throws Exception {
-        float[] arr = new float[] {10.1f, 20.1f, 30.1f};
-
-        assertArrayEquals(arr, marshalUnmarshal(arr), 0);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDoubleArray() throws Exception {
-        double[] arr = new double[] {10.1d, 20.1d, 30.1d};
-
-        assertArrayEquals(arr, marshalUnmarshal(arr), 0);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCharArray() throws Exception {
-        char[] arr = new char[] {10, 20, 30};
-
-        assertArrayEquals(arr, marshalUnmarshal(arr));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testBooleanArray() throws Exception {
-        boolean[] arr = new boolean[] {true, false, true};
-
-        assertBooleanArrayEquals(arr, marshalUnmarshal(arr));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDecimalArray() throws Exception {
-        BigDecimal[] arr = new BigDecimal[] { BigDecimal.ZERO, BigDecimal.ONE, BigDecimal.TEN } ;
-
-        assertArrayEquals(arr, marshalUnmarshal(arr));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testStringArray() throws Exception {
-        String[] arr = new String[] {"str1", "str2", "str3"};
-
-        assertArrayEquals(arr, marshalUnmarshal(arr));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testUuidArray() throws Exception {
-        UUID[] arr = new UUID[] {UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID()};
-
-        assertArrayEquals(arr, marshalUnmarshal(arr));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDateArray() throws Exception {
-        Date[] arr = new Date[] {new Date(11111), new Date(22222), new Date(33333)};
-
-        assertArrayEquals(arr, marshalUnmarshal(arr));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testObjectArray() throws Exception {
-        Object[] arr = new Object[] {1, 2, 3};
-
-        assertArrayEquals(arr, marshalUnmarshal(arr));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCollection() throws Exception {
-        testCollection(new ArrayList<Integer>(3));
-        testCollection(new LinkedHashSet<Integer>());
-        testCollection(new HashSet<Integer>());
-        testCollection(new TreeSet<Integer>());
-        testCollection(new ConcurrentSkipListSet<Integer>());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    private void testCollection(Collection<Integer> col) throws Exception {
-        col.add(1);
-        col.add(2);
-        col.add(3);
-
-        assertEquals(col, marshalUnmarshal(col));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMap() throws Exception {
-        testMap(new HashMap<Integer, String>());
-        testMap(new LinkedHashMap());
-        testMap(new TreeMap<Integer, String>());
-        testMap(new ConcurrentHashMap8<Integer, String>());
-        testMap(new ConcurrentHashMap<Integer, String>());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    private void testMap(Map<Integer, String> map) throws Exception {
-        map.put(1, "str1");
-        map.put(2, "str2");
-        map.put(3, "str3");
-
-        assertEquals(map, marshalUnmarshal(map));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMapEntry() throws Exception {
-        Map.Entry<Integer, String> e = new GridMapEntry<>(1, "str1");
-
-        assertEquals(e, marshalUnmarshal(e));
-
-        Map<Integer, String> map = new HashMap<>(1);
-
-        map.put(2, "str2");
-
-        e = F.firstEntry(map);
-
-        Map.Entry<Integer, String> e0 = marshalUnmarshal(e);
-
-        assertEquals(2, e0.getKey().intValue());
-        assertEquals("str2", e0.getValue());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testBinaryObject() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(new BinaryTypeConfiguration(SimpleObject.class.getName())));
-
-        SimpleObject obj = simpleObject();
-
-        BinaryObject po = marshal(obj, marsh);
-
-        BinaryObject po0 = marshalUnmarshal(po, marsh);
-
-        assertTrue(po.hasField("b"));
-        assertTrue(po.hasField("s"));
-        assertTrue(po.hasField("i"));
-        assertTrue(po.hasField("l"));
-        assertTrue(po.hasField("f"));
-        assertTrue(po.hasField("d"));
-        assertTrue(po.hasField("c"));
-        assertTrue(po.hasField("bool"));
-
-        assertFalse(po.hasField("no_such_field"));
-
-        assertEquals(obj, po.deserialize());
-        assertEquals(obj, po0.deserialize());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testEnum() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setClassNames(Arrays.asList(TestEnum.class.getName()));
-
-        assertEquals(TestEnum.B, marshalUnmarshal(TestEnum.B, marsh));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDateAndTimestampInSingleObject() throws Exception {
-        BinaryTypeConfiguration cfg1 = new BinaryTypeConfiguration(DateClass1.class.getName());
-
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(cfg1));
-
-        Date date = new Date();
-        Timestamp ts = new Timestamp(System.currentTimeMillis());
-
-        DateClass1 obj1 = new DateClass1();
-        obj1.date = date;
-        obj1.ts = ts;
-
-        BinaryObject po1 = marshal(obj1, marsh);
-
-        assertEquals(date, po1.field("date"));
-        assertEquals(Date.class, po1.field("date").getClass());
-        assertEquals(ts, po1.field("ts"));
-        assertEquals(Timestamp.class, po1.field("ts").getClass());
-
-        obj1 = po1.deserialize();
-        assertEquals(date, obj1.date);
-        assertEquals(ts, obj1.ts);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testSimpleObject() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new BinaryTypeConfiguration(SimpleObject.class.getName())
-        ));
-
-        SimpleObject obj = simpleObject();
-
-        BinaryObject po = marshal(obj, marsh);
-
-        assertEquals(obj.hashCode(), po.hashCode());
-
-        assertEquals(obj, po.deserialize());
-
-        assertEquals(obj.b, (byte)po.field("b"));
-        assertEquals(obj.s, (short)po.field("s"));
-        assertEquals(obj.i, (int)po.field("i"));
-        assertEquals(obj.l, (long)po.field("l"));
-        assertEquals(obj.f, (float)po.field("f"), 0);
-        assertEquals(obj.d, (double)po.field("d"), 0);
-        assertEquals(obj.c, (char)po.field("c"));
-        assertEquals(obj.bool, (boolean)po.field("bool"));
-        assertEquals(obj.str, po.field("str"));
-        assertEquals(obj.uuid, po.field("uuid"));
-        assertEquals(obj.date, po.field("date"));
-        assertEquals(Date.class, obj.date.getClass());
-        assertEquals(obj.ts, po.field("ts"));
-        assertArrayEquals(obj.bArr, (byte[])po.field("bArr"));
-        assertArrayEquals(obj.sArr, (short[])po.field("sArr"));
-        assertArrayEquals(obj.iArr, (int[])po.field("iArr"));
-        assertArrayEquals(obj.lArr, (long[])po.field("lArr"));
-        assertArrayEquals(obj.fArr, (float[])po.field("fArr"), 0);
-        assertArrayEquals(obj.dArr, (double[])po.field("dArr"), 0);
-        assertArrayEquals(obj.cArr, (char[])po.field("cArr"));
-        assertBooleanArrayEquals(obj.boolArr, (boolean[])po.field("boolArr"));
-        assertArrayEquals(obj.strArr, (String[])po.field("strArr"));
-        assertArrayEquals(obj.uuidArr, (UUID[])po.field("uuidArr"));
-        assertArrayEquals(obj.dateArr, (Date[])po.field("dateArr"));
-        assertArrayEquals(obj.objArr, (Object[])po.field("objArr"));
-        assertEquals(obj.col, po.field("col"));
-        assertEquals(obj.map, po.field("map"));
-        assertEquals(new Integer(obj.enumVal.ordinal()), new Integer(((Enum<?>)po.field("enumVal")).ordinal()));
-        assertArrayEquals(ordinals(obj.enumArr), ordinals((Enum<?>[])po.field("enumArr")));
-        assertNull(po.field("unknown"));
-
-        BinaryObject innerPo = po.field("inner");
-
-        assertEquals(obj.inner, innerPo.deserialize());
-
-        assertEquals(obj.inner.b, (byte)innerPo.field("b"));
-        assertEquals(obj.inner.s, (short)innerPo.field("s"));
-        assertEquals(obj.inner.i, (int)innerPo.field("i"));
-        assertEquals(obj.inner.l, (long)innerPo.field("l"));
-        assertEquals(obj.inner.f, (float)innerPo.field("f"), 0);
-        assertEquals(obj.inner.d, (double)innerPo.field("d"), 0);
-        assertEquals(obj.inner.c, (char)innerPo.field("c"));
-        assertEquals(obj.inner.bool, (boolean)innerPo.field("bool"));
-        assertEquals(obj.inner.str, innerPo.field("str"));
-        assertEquals(obj.inner.uuid, innerPo.field("uuid"));
-        assertEquals(obj.inner.date, innerPo.field("date"));
-        assertEquals(Date.class, obj.inner.date.getClass());
-        assertEquals(obj.inner.ts, innerPo.field("ts"));
-        assertArrayEquals(obj.inner.bArr, (byte[])innerPo.field("bArr"));
-        assertArrayEquals(obj.inner.sArr, (short[])innerPo.field("sArr"));
-        assertArrayEquals(obj.inner.iArr, (int[])innerPo.field("iArr"));
-        assertArrayEquals(obj.inner.lArr, (long[])innerPo.field("lArr"));
-        assertArrayEquals(obj.inner.fArr, (float[])innerPo.field("fArr"), 0);
-        assertArrayEquals(obj.inner.dArr, (double[])innerPo.field("dArr"), 0);
-        assertArrayEquals(obj.inner.cArr, (char[])innerPo.field("cArr"));
-        assertBooleanArrayEquals(obj.inner.boolArr, (boolean[])innerPo.field("boolArr"));
-        assertArrayEquals(obj.inner.strArr, (String[])innerPo.field("strArr"));
-        assertArrayEquals(obj.inner.uuidArr, (UUID[])innerPo.field("uuidArr"));
-        assertArrayEquals(obj.inner.dateArr, (Date[])innerPo.field("dateArr"));
-        assertArrayEquals(obj.inner.objArr, (Object[])innerPo.field("objArr"));
-        assertEquals(obj.inner.col, innerPo.field("col"));
-        assertEquals(obj.inner.map, innerPo.field("map"));
-        assertEquals(new Integer(obj.inner.enumVal.ordinal()),
-            new Integer(((Enum<?>)innerPo.field("enumVal")).ordinal()));
-        assertArrayEquals(ordinals(obj.inner.enumArr), ordinals((Enum<?>[])innerPo.field("enumArr")));
-        assertNull(innerPo.field("inner"));
-        assertNull(innerPo.field("unknown"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortable() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new BinaryTypeConfiguration(SimpleObject.class.getName()),
-            new BinaryTypeConfiguration(TestBinary.class.getName())
-        ));
-
-        TestBinary obj = BinaryObject();
-
-        BinaryObject po = marshal(obj, marsh);
-
-        assertEquals(obj.hashCode(), po.hashCode());
-
-        assertEquals(obj, po.deserialize());
-
-        assertEquals(obj.b, (byte)po.field("_b"));
-        assertEquals(obj.s, (short)po.field("_s"));
-        assertEquals(obj.i, (int)po.field("_i"));
-        assertEquals(obj.l, (long)po.field("_l"));
-        assertEquals(obj.f, (float)po.field("_f"), 0);
-        assertEquals(obj.d, (double)po.field("_d"), 0);
-        assertEquals(obj.c, (char)po.field("_c"));
-        assertEquals(obj.bool, (boolean)po.field("_bool"));
-        assertEquals(obj.str, po.field("_str"));
-        assertEquals(obj.uuid, po.field("_uuid"));
-        assertEquals(obj.date, po.field("_date"));
-        assertEquals(obj.ts, po.field("_ts"));
-        assertArrayEquals(obj.bArr, (byte[])po.field("_bArr"));
-        assertArrayEquals(obj.sArr, (short[])po.field("_sArr"));
-        assertArrayEquals(obj.iArr, (int[])po.field("_iArr"));
-        assertArrayEquals(obj.lArr, (long[])po.field("_lArr"));
-        assertArrayEquals(obj.fArr, (float[])po.field("_fArr"), 0);
-        assertArrayEquals(obj.dArr, (double[])po.field("_dArr"), 0);
-        assertArrayEquals(obj.cArr, (char[])po.field("_cArr"));
-        assertBooleanArrayEquals(obj.boolArr, (boolean[])po.field("_boolArr"));
-        assertArrayEquals(obj.strArr, (String[])po.field("_strArr"));
-        assertArrayEquals(obj.uuidArr, (UUID[])po.field("_uuidArr"));
-        assertArrayEquals(obj.dateArr, (Date[])po.field("_dateArr"));
-        assertArrayEquals(obj.objArr, (Object[])po.field("_objArr"));
-        assertEquals(obj.col, po.field("_col"));
-        assertEquals(obj.map, po.field("_map"));
-        assertEquals(new Integer(obj.enumVal.ordinal()), new Integer(((Enum<?>)po.field("_enumVal")).ordinal()));
-        assertArrayEquals(ordinals(obj.enumArr), ordinals((Enum<?>[])po.field("_enumArr")));
-        assertNull(po.field("unknown"));
-
-        BinaryObject simplePo = po.field("_simple");
-
-        assertEquals(obj.simple, simplePo.deserialize());
-
-        assertEquals(obj.simple.b, (byte)simplePo.field("b"));
-        assertEquals(obj.simple.s, (short)simplePo.field("s"));
-        assertEquals(obj.simple.i, (int)simplePo.field("i"));
-        assertEquals(obj.simple.l, (long)simplePo.field("l"));
-        assertEquals(obj.simple.f, (float)simplePo.field("f"), 0);
-        assertEquals(obj.simple.d, (double)simplePo.field("d"), 0);
-        assertEquals(obj.simple.c, (char)simplePo.field("c"));
-        assertEquals(obj.simple.bool, (boolean)simplePo.field("bool"));
-        assertEquals(obj.simple.str, simplePo.field("str"));
-        assertEquals(obj.simple.uuid, simplePo.field("uuid"));
-        assertEquals(obj.simple.date, simplePo.field("date"));
-        assertEquals(Date.class, obj.simple.date.getClass());
-        assertEquals(obj.simple.ts, simplePo.field("ts"));
-        assertArrayEquals(obj.simple.bArr, (byte[])simplePo.field("bArr"));
-        assertArrayEquals(obj.simple.sArr, (short[])simplePo.field("sArr"));
-        assertArrayEquals(obj.simple.iArr, (int[])simplePo.field("iArr"));
-        assertArrayEquals(obj.simple.lArr, (long[])simplePo.field("lArr"));
-        assertArrayEquals(obj.simple.fArr, (float[])simplePo.field("fArr"), 0);
-        assertArrayEquals(obj.simple.dArr, (double[])simplePo.field("dArr"), 0);
-        assertArrayEquals(obj.simple.cArr, (char[])simplePo.field("cArr"));
-        assertBooleanArrayEquals(obj.simple.boolArr, (boolean[])simplePo.field("boolArr"));
-        assertArrayEquals(obj.simple.strArr, (String[])simplePo.field("strArr"));
-        assertArrayEquals(obj.simple.uuidArr, (UUID[])simplePo.field("uuidArr"));
-        assertArrayEquals(obj.simple.dateArr, (Date[])simplePo.field("dateArr"));
-        assertArrayEquals(obj.simple.objArr, (Object[])simplePo.field("objArr"));
-        assertEquals(obj.simple.col, simplePo.field("col"));
-        assertEquals(obj.simple.map, simplePo.field("map"));
-        assertEquals(new Integer(obj.simple.enumVal.ordinal()),
-            new Integer(((Enum<?>)simplePo.field("enumVal")).ordinal()));
-        assertArrayEquals(ordinals(obj.simple.enumArr), ordinals((Enum<?>[])simplePo.field("enumArr")));
-        assertNull(simplePo.field("simple"));
-        assertNull(simplePo.field("portable"));
-        assertNull(simplePo.field("unknown"));
-
-        BinaryObject portablePo = po.field("_portable");
-
-        assertEquals(obj.portable, portablePo.deserialize());
-
-        assertEquals(obj.portable.b, (byte)portablePo.field("_b"));
-        assertEquals(obj.portable.s, (short)portablePo.field("_s"));
-        assertEquals(obj.portable.i, (int)portablePo.field("_i"));
-        assertEquals(obj.portable.l, (long)portablePo.field("_l"));
-        assertEquals(obj.portable.f, (float)portablePo.field("_f"), 0);
-        assertEquals(obj.portable.d, (double)portablePo.field("_d"), 0);
-        assertEquals(obj.portable.c, (char)portablePo.field("_c"));
-        assertEquals(obj.portable.bool, (boolean)portablePo.field("_bool"));
-        assertEquals(obj.portable.str, portablePo.field("_str"));
-        assertEquals(obj.portable.uuid, portablePo.field("_uuid"));
-        assertEquals(obj.portable.date, portablePo.field("_date"));
-        assertEquals(obj.portable.ts, portablePo.field("_ts"));
-        assertArrayEquals(obj.portable.bArr, (byte[])portablePo.field("_bArr"));
-        assertArrayEquals(obj.portable.sArr, (short[])portablePo.field("_sArr"));
-        assertArrayEquals(obj.portable.iArr, (int[])portablePo.field("_iArr"));
-        assertArrayEquals(obj.portable.lArr, (long[])portablePo.field("_lArr"));
-        assertArrayEquals(obj.portable.fArr, (float[])portablePo.field("_fArr"), 0);
-        assertArrayEquals(obj.portable.dArr, (double[])portablePo.field("_dArr"), 0);
-        assertArrayEquals(obj.portable.cArr, (char[])portablePo.field("_cArr"));
-        assertBooleanArrayEquals(obj.portable.boolArr, (boolean[])portablePo.field("_boolArr"));
-        assertArrayEquals(obj.portable.strArr, (String[])portablePo.field("_strArr"));
-        assertArrayEquals(obj.portable.uuidArr, (UUID[])portablePo.field("_uuidArr"));
-        assertArrayEquals(obj.portable.dateArr, (Date[])portablePo.field("_dateArr"));
-        assertArrayEquals(obj.portable.objArr, (Object[])portablePo.field("_objArr"));
-        assertEquals(obj.portable.col, portablePo.field("_col"));
-        assertEquals(obj.portable.map, portablePo.field("_map"));
-        assertEquals(new Integer(obj.portable.enumVal.ordinal()),
-            new Integer(((Enum<?>)portablePo.field("_enumVal")).ordinal()));
-        assertArrayEquals(ordinals(obj.portable.enumArr), ordinals((Enum<?>[])portablePo.field("_enumArr")));
-        assertNull(portablePo.field("_simple"));
-        assertNull(portablePo.field("_portable"));
-        assertNull(portablePo.field("unknown"));
-    }
-
-    /**
-     * @param obj Simple object.
-     * @param po Portable object.
-     */
-    private void checkSimpleObjectData(SimpleObject obj, BinaryObject po) {
-        assertEquals(obj.b, (byte)po.field("b"));
-        assertEquals(obj.s, (short)po.field("s"));
-        assertEquals(obj.i, (int)po.field("i"));
-        assertEquals(obj.l, (long)po.field("l"));
-        assertEquals(obj.f, (float)po.field("f"), 0);
-        assertEquals(obj.d, (double)po.field("d"), 0);
-        assertEquals(obj.c, (char)po.field("c"));
-        assertEquals(obj.bool, (boolean)po.field("bool"));
-        assertEquals(obj.str, po.field("str"));
-        assertEquals(obj.uuid, po.field("uuid"));
-        assertEquals(obj.date, po.field("date"));
-        assertEquals(Date.class, obj.date.getClass());
-        assertEquals(obj.ts, po.field("ts"));
-        assertArrayEquals(obj.bArr, (byte[])po.field("bArr"));
-        assertArrayEquals(obj.sArr, (short[])po.field("sArr"));
-        assertArrayEquals(obj.iArr, (int[])po.field("iArr"));
-        assertArrayEquals(obj.lArr, (long[])po.field("lArr"));
-        assertArrayEquals(obj.fArr, (float[])po.field("fArr"), 0);
-        assertArrayEquals(obj.dArr, (double[])po.field("dArr"), 0);
-        assertArrayEquals(obj.cArr, (char[])po.field("cArr"));
-        assertBooleanArrayEquals(obj.boolArr, (boolean[])po.field("boolArr"));
-        assertArrayEquals(obj.strArr, (String[])po.field("strArr"));
-        assertArrayEquals(obj.uuidArr, (UUID[])po.field("uuidArr"));
-        assertArrayEquals(obj.dateArr, (Date[])po.field("dateArr"));
-        assertArrayEquals(obj.objArr, (Object[])po.field("objArr"));
-        assertEquals(obj.col, po.field("col"));
-        assertEquals(obj.map, po.field("map"));
-        assertEquals(new Integer(obj.enumVal.ordinal()), new Integer(((Enum<?>)po.field("enumVal")).ordinal()));
-        assertArrayEquals(ordinals(obj.enumArr), ordinals((Enum<?>[])po.field("enumArr")));
-        assertNull(po.field("unknown"));
-
-        assertEquals(obj, po.deserialize());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClassWithoutPublicConstructor() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-                new BinaryTypeConfiguration(NoPublicConstructor.class.getName()),
-                new BinaryTypeConfiguration(NoPublicDefaultConstructor.class.getName()),
-                new BinaryTypeConfiguration(ProtectedConstructor.class.getName()))
-        );
-
-        initPortableContext(marsh);
-
-        NoPublicConstructor npc = new NoPublicConstructor();
-        BinaryObject npc2 = marshal(npc, marsh);
-
-        assertEquals("test", npc2.<NoPublicConstructor>deserialize().val);
-
-        NoPublicDefaultConstructor npdc = new NoPublicDefaultConstructor(239);
-        BinaryObject npdc2 = marshal(npdc, marsh);
-
-        assertEquals(239, npdc2.<NoPublicDefaultConstructor>deserialize().val);
-
-        ProtectedConstructor pc = new ProtectedConstructor();
-        BinaryObject pc2 = marshal(pc, marsh);
-
-        assertEquals(ProtectedConstructor.class, pc2.<ProtectedConstructor>deserialize().getClass());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCustomSerializer() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        BinaryTypeConfiguration type =
-            new BinaryTypeConfiguration(CustomSerializedObject1.class.getName());
-
-        type.setSerializer(new CustomSerializer1());
-
-        marsh.setTypeConfigurations(Arrays.asList(type));
-
-        CustomSerializedObject1 obj1 = new CustomSerializedObject1(10);
-
-        BinaryObject po1 = marshal(obj1, marsh);
-
-        assertEquals(20, po1.<CustomSerializedObject1>deserialize().val);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCustomSerializerWithGlobal() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setSerializer(new CustomSerializer1());
-
-        BinaryTypeConfiguration type1 =
-            new BinaryTypeConfiguration(CustomSerializedObject1.class.getName());
-        BinaryTypeConfiguration type2 =
-            new BinaryTypeConfiguration(CustomSerializedObject2.class.getName());
-
-        type2.setSerializer(new CustomSerializer2());
-
-        marsh.setTypeConfigurations(Arrays.asList(type1, type2));
-
-        CustomSerializedObject1 obj1 = new CustomSerializedObject1(10);
-
-        BinaryObject po1 = marshal(obj1, marsh);
-
-        assertEquals(20, po1.<CustomSerializedObject1>deserialize().val);
-
-        CustomSerializedObject2 obj2 = new CustomSerializedObject2(10);
-
-        BinaryObject po2 = marshal(obj2, marsh);
-
-        assertEquals(30, po2.<CustomSerializedObject2>deserialize().val);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCustomIdMapper() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        BinaryTypeConfiguration type =
-            new BinaryTypeConfiguration(CustomMappedObject1.class.getName());
-
-        type.setIdMapper(new BinaryIdMapper() {
-            @Override public int typeId(String clsName) {
-                return 11111;
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                assert typeId == 11111;
-
-                if ("val1".equals(fieldName))
-                    return 22222;
-                else if ("val2".equals(fieldName))
-                    return 33333;
-
-                assert false : "Unknown field: " + fieldName;
-
-                return 0;
-            }
-        });
-
-        marsh.setTypeConfigurations(Arrays.asList(type));
-
-        CustomMappedObject1 obj1 = new CustomMappedObject1(10, "str");
-
-        BinaryObjectEx po1 = marshal(obj1, marsh);
-
-        assertEquals(11111, po1.typeId());
-        assertEquals(10, po1.field(22222));
-        assertEquals("str", po1.field(33333));
-
-        assertEquals(10, po1.<CustomMappedObject1>deserialize().val1);
-        assertEquals("str", po1.<CustomMappedObject1>deserialize().val2);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCustomIdMapperWithGlobal() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setIdMapper(new BinaryIdMapper() {
-            @Override public int typeId(String clsName) {
-                return 11111;
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                assert typeId == 11111;
-
-                if ("val1".equals(fieldName)) return 22222;
-                else if ("val2".equals(fieldName)) return 33333;
-
-                assert false : "Unknown field: " + fieldName;
-
-                return 0;
-            }
-        });
-
-        BinaryTypeConfiguration type1 =
-            new BinaryTypeConfiguration(CustomMappedObject1.class.getName());
-        BinaryTypeConfiguration type2 =
-            new BinaryTypeConfiguration(CustomMappedObject2.class.getName());
-
-        type2.setIdMapper(new BinaryIdMapper() {
-            @Override public int typeId(String clsName) {
-                return 44444;
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                assert typeId == 44444;
-
-                if ("val1".equals(fieldName)) return 55555;
-                else if ("val2".equals(fieldName)) return 66666;
-
-                assert false : "Unknown field: " + fieldName;
-
-                return 0;
-            }
-        });
-
-        marsh.setTypeConfigurations(Arrays.asList(type1, type2));
-
-        CustomMappedObject1 obj1 = new CustomMappedObject1(10, "str1");
-
-        BinaryObjectEx po1 = marshal(obj1, marsh);
-
-        assertEquals(11111, po1.typeId());
-        assertEquals(10, po1.field(22222));
-        assertEquals("str1", po1.field(33333));
-
-        assertEquals(10, po1.<CustomMappedObject1>deserialize().val1);
-        assertEquals("str1", po1.<CustomMappedObject1>deserialize().val2);
-
-        CustomMappedObject2 obj2 = new CustomMappedObject2(20, "str2");
-
-        BinaryObjectEx po2 = marshal(obj2, marsh);
-
-        assertEquals(44444, po2.typeId());
-        assertEquals(20, po2.field(55555));
-        assertEquals("str2", po2.field(66666));
-
-        assertEquals(20, po2.<CustomMappedObject2>deserialize().val1);
-        assertEquals("str2", po2.<CustomMappedObject2>deserialize().val2);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDynamicObject() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new BinaryTypeConfiguration(DynamicObject.class.getName())
-        ));
-
-        BinaryObject po1 = marshal(new DynamicObject(0, 10, 20, 30), marsh);
-
-        assertEquals(new Integer(10), po1.field("val1"));
-        assertEquals(null, po1.field("val2"));
-        assertEquals(null, po1.field("val3"));
-
-        DynamicObject do1 = po1.deserialize();
-
-        assertEquals(10, do1.val1);
-        assertEquals(0, do1.val2);
-        assertEquals(0, do1.val3);
-
-        BinaryObject po2 = marshal(new DynamicObject(1, 10, 20, 30), marsh);
-
-        assertEquals(new Integer(10), po2.field("val1"));
-        assertEquals(new Integer(20), po2.field("val2"));
-        assertEquals(null, po2.field("val3"));
-
-        DynamicObject do2 = po2.deserialize();
-
-        assertEquals(10, do2.val1);
-        assertEquals(20, do2.val2);
-        assertEquals(0, do2.val3);
-
-        BinaryObject po3 = marshal(new DynamicObject(2, 10, 20, 30), marsh);
-
-        assertEquals(new Integer(10), po3.field("val1"));
-        assertEquals(new Integer(20), po3.field("val2"));
-        assertEquals(new Integer(30), po3.field("val3"));
-
-        DynamicObject do3 = po3.deserialize();
-
-        assertEquals(10, do3.val1);
-        assertEquals(20, do3.val2);
-        assertEquals(30, do3.val3);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCycleLink() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new BinaryTypeConfiguration(CycleLinkObject.class.getName())
-        ));
-
-        CycleLinkObject obj = new CycleLinkObject();
-
-        obj.self = obj;
-
-        BinaryObject po = marshal(obj, marsh);
-
-        CycleLinkObject obj0 = po.deserialize();
-
-        assert obj0.self == obj0;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDetached() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new BinaryTypeConfiguration(DetachedTestObject.class.getName()),
-            new BinaryTypeConfiguration(DetachedInnerTestObject.class.getName())
-        ));
-
-        UUID id = UUID.randomUUID();
-
-        DetachedTestObject obj = marshal(new DetachedTestObject(
-            new DetachedInnerTestObject(null, id)), marsh).deserialize();
-
-        assertEquals(id, obj.inner1.id);
-        assertEquals(id, obj.inner4.id);
-
-        assert obj.inner1 == obj.inner4;
-
-        BinaryObjectImpl innerPo = (BinaryObjectImpl)obj.inner2;
-
-        assert innerPo.detached();
-
-        DetachedInnerTestObject inner = innerPo.deserialize();
-
-        assertEquals(id, inner.id);
-
-        BinaryObjectImpl detachedPo = (BinaryObjectImpl)innerPo.detach();
-
-        assert detachedPo.detached();
-
-        inner = detachedPo.deserialize();
-
-        assertEquals(id, inner.id);
-
-        innerPo = (BinaryObjectImpl)obj.inner3;
-
-        assert innerPo.detached();
-
-        inner = innerPo.deserialize();
-
-        assertEquals(id, inner.id);
-        assertNotNull(inner.inner);
-
-        detachedPo = (BinaryObjectImpl)innerPo.detach();
-
-        assert detachedPo.detached();
-
-        inner = innerPo.deserialize();
-
-        assertEquals(id, inner.id);
-        assertNotNull(inner.inner);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCollectionFields() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new BinaryTypeConfiguration(CollectionFieldsObject.class.getName()),
-            new BinaryTypeConfiguration(Key.class.getName()),
-            new BinaryTypeConfiguration(Value.class.getName())
-        ));
-
-        Object[] arr = new Object[] {new Value(1), new Value(2), new Value(3)};
-        Collection<Value> col = Arrays.asList(new Value(4), new Value(5), new Value(6));
-        Map<Key, Value> map = F.asMap(new Key(10), new Value(10), new Key(20), new Value(20), new Key(30), new Value(30));
-
-        CollectionFieldsObject obj = new CollectionFieldsObject(arr, col, map);
-
-        BinaryObject po = marshal(obj, marsh);
-
-        Object[] arr0 = po.field("arr");
-
-        assertEquals(3, arr0.length);
-
-        int i = 1;
-
-        for (Object valPo : arr0)
-            assertEquals(i++, ((BinaryObject)valPo).<Value>deserialize().val);
-
-        Collection<BinaryObject> col0 = po.field("col");
-
-        i = 4;
-
-        for (BinaryObject valPo : col0)
-            assertEquals(i++, valPo.<Value>deserialize().val);
-
-        Map<BinaryObject, BinaryObject> map0 = po.field("map");
-
-        for (Map.Entry<BinaryObject, BinaryObject> e : map0.entrySet())
-            assertEquals(e.getKey().<Key>deserialize().key, e.getValue().<Value>deserialize().val);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDefaultMapping() throws Exception {
-        PortableMarshaller marsh1 = new PortableMarshaller();
-
-        BinaryTypeConfiguration customMappingType =
-            new BinaryTypeConfiguration(TestBinary.class.getName());
-
-        customMappingType.setIdMapper(new BinaryIdMapper() {
-            @Override public int typeId(String clsName) {
-                String typeName;
-
-                try {
-                    Method mtd = PortableContext.class.getDeclaredMethod("typeName", String.class);
-
-                    mtd.setAccessible(true);
-
-                    typeName = (String)mtd.invoke(null, clsName);
-                }
-                catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) {
-                    throw new RuntimeException(e);
-                }
-
-                return typeName.toLowerCase().hashCode();
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                return fieldName.toLowerCase().hashCode();
-            }
-        });
-
-        marsh1.setTypeConfigurations(Arrays.asList(
-            new BinaryTypeConfiguration(SimpleObject.class.getName()),
-            customMappingType
-        ));
-
-        TestBinary obj = BinaryObject();
-
-        BinaryObjectImpl po = marshal(obj, marsh1);
-
-        PortableMarshaller marsh2 = new PortableMarshaller();
-
-        marsh2.setTypeConfigurations(Arrays.asList(
-            new BinaryTypeConfiguration(SimpleObject.class.getName()),
-            new BinaryTypeConfiguration(TestBinary.class.getName())
-        ));
-
-        PortableContext ctx = initPortableContext(marsh2);
-
-        po.context(ctx);
-
-        assertEquals(obj, po.deserialize());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testTypeNames() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        BinaryTypeConfiguration customType1 = new BinaryTypeConfiguration(Value.class.getName());
-
-        customType1.setIdMapper(new BinaryIdMapper() {
-            @Override public int typeId(String clsName) {
-                return 300;
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                return 0;
-            }
-        });
-
-        BinaryTypeConfiguration customType2 = new BinaryTypeConfiguration("org.gridgain.NonExistentClass1");
-
-        customType2.setIdMapper(new BinaryIdMapper() {
-            @Override public int typeId(String clsName) {
-                return 400;
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                return 0;
-            }
-        });
-
-        BinaryTypeConfiguration customType3 = new BinaryTypeConfiguration("NonExistentClass2");
-
-        customType3.setIdMapper(new BinaryIdMapper() {
-            @Override public int typeId(String clsName) {
-                return 500;
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                return 0;
-            }
-        });
-
-        BinaryTypeConfiguration customType4 = new BinaryTypeConfiguration("NonExistentClass5");
-
-        customType4.setIdMapper(new BinaryIdMapper() {
-            @Override public int typeId(String clsName) {
-                return 0;
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                return 0;
-            }
-        });
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new BinaryTypeConfiguration(Key.class.getName()),
-            new BinaryTypeConfiguration("org.gridgain.NonExistentClass3"),
-            new BinaryTypeConfiguration("NonExistentClass4"),
-            customType1,
-            customType2,
-            customType3,
-            customType4
-        ));
-
-        PortableContext ctx = initPortableContext(marsh);
-
-        assertEquals("notconfiguredclass".hashCode(), ctx.typeId("NotConfiguredClass"));
-        assertEquals("key".hashCode(), ctx.typeId("Key"));
-        assertEquals("nonexistentclass3".hashCode(), ctx.typeId("NonExistentClass3"));
-        assertEquals("nonexistentclass4".hashCode(), ctx.typeId("NonExistentClass4"));
-        assertEquals(300, ctx.typeId(getClass().getSimpleName() + "$Value"));
-        assertEquals(400, ctx.typeId("NonExistentClass1"));
-        assertEquals(500, ctx.typeId("NonExistentClass2"));
-        assertEquals("nonexistentclass5".hashCode(), ctx.typeId("NonExistentClass5"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testFieldIdMapping() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        BinaryTypeConfiguration customType1 = new BinaryTypeConfiguration(Value.class.getName());
-
-        customType1.setIdMapper(new BinaryIdMapper() {
-            @Override public int typeId(String clsName) {
-                return 300;
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                switch (fieldName) {
-                    case "val1":
-                        return 301;
-
-                    case "val2":
-                        return 302;
-
-                    default:
-                        return 0;
-                }
-            }
-        });
-
-        BinaryTypeConfiguration customType2 = new BinaryTypeConfiguration("NonExistentClass1");
-
-        customType2.setIdMapper(new BinaryIdMapper() {
-            @Override public int typeId(String clsName) {
-                return 400;
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                switch (fieldName) {
-                    case "val1":
-                        return 401;
-
-                    case "val2":
-                        return 402;
-
-                    default:
-                        return 0;
-                }
-            }
-        });
-
-        marsh.setTypeConfigurations(Arrays.asList(new BinaryTypeConfiguration(Key.class.getName()),
-            new BinaryTypeConfiguration("NonExistentClass2"),
-            customType1,
-            customType2));
-
-        PortableContext ctx = initPortableContext(marsh);
-
-        assertEquals("val".hashCode(), ctx.fieldId("key".hashCode(), "val"));
-        assertEquals("val".hashCode(), ctx.fieldId("nonexistentclass2".hashCode(), "val"));
-        assertEquals("val".hashCode(), ctx.fieldId("notconfiguredclass".hashCode(), "val"));
-        assertEquals(301, ctx.fieldId(300, "val1"));
-        assertEquals(302, ctx.fieldId(300, "val2"));
-        assertEquals("val3".hashCode(), ctx.fieldId(300, "val3"));
-        assertEquals(401, ctx.fieldId(400, "val1"));
-        assertEquals(402, ctx.fieldId(400, "val2"));
-        assertEquals("val3".hashCode(), ctx.fieldId(400, "val3"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDuplicateTypeId() throws Exception {
-        final PortableMarshaller marsh = new PortableMarshaller();
-
-        BinaryTypeConfiguration customType1 = new BinaryTypeConfiguration("org.gridgain.Class1");
-
-        customType1.setIdMapper(new BinaryIdMapper() {
-            @Override public int typeId(String clsName) {
-                return 100;
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                return 0;
-            }
-        });
-
-        BinaryTypeConfiguration customType2 = new BinaryTypeConfiguration("org.gridgain.Class2");
-
-        customType2.setIdMapper(new BinaryIdMapper() {
-            @Override public int typeId(String clsName) {
-                return 100;
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                return 0;
-            }
-        });
-
-        marsh.setTypeConfigurations(Arrays.asList(customType1, customType2));
-
-        try {
-            initPortableContext(marsh);
-        }
-        catch (IgniteCheckedException e) {
-            assertEquals("Duplicate type ID [clsName=org.gridgain.Class1, id=100]",
-                e.getCause().getCause().getMessage());
-
-            return;
-        }
-
-        assert false;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortableCopy() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new BinaryTypeConfiguration(SimpleObject.class.getName())
-        ));
-
-        SimpleObject obj = simpleObject();
-
-        final BinaryObject po = marshal(obj, marsh);
-
-        BinaryObject copy = copy(po, null);
-
-        assertEquals(obj, copy.deserialize());
-
-        copy = copy(po, new HashMap<String, Object>());
-
-        assertEquals(obj, copy.deserialize());
-
-        Map<String, Object> map = new HashMap<>(1, 1.0f);
-
-        map.put("i", 3);
-
-        copy = copy(po, map);
-
-        assertEquals((byte)2, copy.<Byte>field("b").byteValue());
-        assertEquals((short)2, copy.<Short>field("s").shortValue());
-        assertEquals(3, copy.<Integer>field("i").intValue());
-        assertEquals(2L, copy.<Long>field("l").longValue());
-        assertEquals(2.2f, copy.<Float>field("f").floatValue(), 0);
-        assertEquals(2.2d, copy.<Double>field("d").doubleValue(), 0);
-        assertEquals((char)2, copy.<Character>field("c").charValue());
-        assertEquals(false, copy.<Boolean>field("bool").booleanValue());
-
-        SimpleObject obj0 = copy.deserialize();
-
-        assertEquals((byte)2, obj0.b);
-        assertEquals((short)2, obj0.s);
-        assertEquals(3, obj0.i);
-        assertEquals(2L, obj0.l);
-        assertEquals(2.2f, obj0.f, 0);
-        assertEquals(2.2d, obj0.d, 0);
-        assertEquals((char)2, obj0.c);
-        assertEquals(false, obj0.bool);
-
-        map = new HashMap<>(3, 1.0f);
-
-        map.put("b", (byte)3);
-        map.put("l", 3L);
-        map.put("bool", true);
-
-        copy = copy(po, map);
-
-        assertEquals((byte)3, copy.<Byte>field("b").byteValue());
-        assertEquals((short)2, copy.<Short>field("s").shortValue());
-        assertEquals(2, copy.<Integer>field("i").intValue());
-        assertEquals(3L, copy.<Long>field("l").longValue());
-        assertEquals(2.2f, copy.<Float>field("f").floatValue(), 0);
-        assertEquals(2.2d, copy.<Double>field("d").doubleValue(), 0);
-        assertEquals((char)2, copy.<Character>field("c").charValue());
-        assertEquals(true, copy.<Boolean>field("bool").booleanValue());
-
-        obj0 = copy.deserialize();
-
-        assertEquals((byte)3, obj0.b);
-        assertEquals((short)2, obj0.s);
-        assertEquals(2, obj0.i);
-        assertEquals(3L, obj0.l);
-        assertEquals(2.2f, obj0.f, 0);
-        assertEquals(2.2d, obj0.d, 0);
-        assertEquals((char)2, obj0.c);
-        assertEquals(true, obj0.bool);
-
-        map = new HashMap<>(8, 1.0f);
-
-        map.put("b", (byte)3);
-        map.put("s", (short)3);
-        map.put("i", 3);
-        map.put("l", 3L);
-        map.put("f", 3.3f);
-        map.put("d", 3.3d);
-        map.put("c", (char)3);
-        map.put("bool", true);
-
-        copy = copy(po, map);
-
-        assertEquals((byte)3, copy.<Byte>field("b").byteValue());
-        assertEquals((short)3, copy.<Short>field("s").shortValue());
-        assertEquals(3, copy.<Integer>field("i").intValue());
-        assertEquals(3L, copy.<Long>field("l").longValue());
-        assertEquals(3.3f, copy.<Float>field("f").floatValue(), 0);
-        assertEquals(3.3d, copy.<Double>field("d").doubleValue(), 0);
-        assertEquals((char)3, copy.<Character>field("c").charValue());
-        assertEquals(true, copy.<Boolean>field("bool").booleanValue());
-
-        obj0 = copy.deserialize();
-
-        assertEquals((byte)3, obj0.b);
-        assertEquals((short)3, obj0.s);
-        assertEquals(3, obj0.i);
-        assertEquals(3L, obj0.l);
-        assertEquals(3.3f, obj0.f, 0);
-        assertEquals(3.3d, obj0.d, 0);
-        assertEquals((char)3, obj0.c);
-        assertEquals(true, obj0.bool);
-
-//        GridTestUtils.assertThrows(
-//            log,
-//            new Callable<Object>() {
-//                @Override public Object call() throws Exception {
-//                    po.copy(F.<String, Object>asMap("i", false));
-//
-//                    return null;
-//                }
-//            },
-//            PortableException.class,
-//            "Invalid value type for field: i"
-//        );
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortableCopyString() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new BinaryTypeConfiguration(SimpleObject.class.getName())
-        ));
-
-        SimpleObject obj = simpleObject();
-
-        BinaryObject po = marshal(obj, marsh);
-
-        BinaryObject copy = copy(po, F.<String, Object>asMap("str", "str3"));
-
-        assertEquals("str3", copy.<String>field("str"));
-
-        SimpleObject obj0 = copy.deserialize();
-
-        assertEquals("str3", obj0.str);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortableCopyUuid() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new BinaryTypeConfiguration(SimpleObject.class.getName())
-        ));
-
-        SimpleObject obj = simpleObject();
-
-        BinaryObject po = marshal(obj, marsh);
-
-        UUID uuid = UUID.randomUUID();
-
-        BinaryObject copy = copy(po, F.<String, Object>asMap("uuid", uuid));
-
-        assertEquals(uuid, copy.<UUID>field("uuid"));
-
-        SimpleObject obj0 = copy.deserialize();
-
-        assertEquals(uuid, obj0.uuid);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortableCopyByteArray() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new BinaryTypeConfiguration(SimpleObject.class.getName())
-        ));
-
-        SimpleObject obj = simpleObject();
-
-        BinaryObject po = marshal(obj, marsh);
-
-        BinaryObject copy = copy(po, F.<String, Object>asMap("bArr", new byte[]{1, 2, 3}));
-
-        assertArrayEquals(new byte[] {1, 2, 3}, copy.<byte[]>field("bArr"));
-
-        SimpleObject obj0 = copy.deserialize();
-
-        assertArrayEquals(new byte[] {1, 2, 3}, obj0.bArr);
-    }
-
-    /**
-     * @param po Portable object.
-     * @param fields Fields.
-     * @return Copy.
-     */
-    private BinaryObject copy(BinaryObject po, Map<String, Object> fields) {
-        BinaryObjectBuilder builder = BinaryObjectBuilderImpl.wrap(po);
-
-        if (fields != null) {
-            for (Map.Entry<String, Object> e : fields.entrySet())
-                builder.setField(e.getKey(), e.getValue());
-        }
-
-        return builder.build();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortableCopyShortArray() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new BinaryTypeConfiguration(SimpleObject.class.getName())
-        ));
-
-        SimpleObject obj = simpleObject();
-
-        BinaryObject po = marshal(obj, marsh);
-
-        BinaryObject copy = copy(po, F.<String, Object>asMap("sArr", new short[]{1, 2, 3}));
-
-        assertArrayEquals(new short[] {1, 2, 3}, copy.<short[]>field("sArr"));
-
-        SimpleObject obj0 = copy.deserialize();
-
-        assertArrayEquals(new short[] {1, 2, 3}, obj0.sArr);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortableCopyIntArray() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new BinaryTypeConfiguration(SimpleObject.class.getName())
-        ));
-
-        SimpleObject obj = simpleObject();
-
-        BinaryObject po = marshal(obj, marsh);
-
-        BinaryObject copy = copy(po, F.<String, Object>asMap("iArr", new int[]{1, 2, 3}));
-
-        assertArrayEquals(new int[] {1, 2, 3}, copy.<int[]>field("iArr"));
-
-        SimpleObject obj0 = copy.deserialize();
-
-        assertArrayEquals(new int[] {1, 2, 3}, obj0.iArr);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortableCopyLongArray() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new BinaryTypeConfiguration(SimpleObject.class.getName())
-        ));
-
-        SimpleObject obj = simpleObject();
-
-        BinaryObject po = marshal(obj, marsh);
-
-        BinaryObject copy = copy(po, F.<String, Object>asMap("lArr", new long[]{1, 2, 3}));
-
-        assertArrayEquals(new long[] {1, 2, 3}, copy.<long[]>field("lArr"));
-
-        SimpleObject obj0 = copy.deserialize();
-
-        assertArrayEquals(new long[] {1, 2, 3}, obj0.lArr);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortableCopyFloatArray() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new BinaryTypeConfiguration(SimpleObject.class.getName())
-        ));
-
-        SimpleObject obj = simpleObject();
-
-        BinaryObject po = marshal(obj, marsh);
-
-        BinaryObject copy = copy(po, F.<String, Object>asMap("fArr", new float[]{1, 2, 3}));
-
-        assertArrayEquals(new float[] {1, 2, 3}, copy.<float[]>field("fArr"), 0);
-
-        SimpleObject obj0 = copy.deserialize();
-
-        assertArrayEquals(new float[] {1, 2, 3}, obj0.fArr, 0);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortableCopyDoubleArray() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new BinaryTypeConfiguration(SimpleObject.class.getName())
-        ));
-
-        SimpleObject obj = simpleObject();
-
-        BinaryObject po = marshal(obj, marsh);
-
-        BinaryObject copy = copy(po, F.<String, Object>asMap("dArr", new double[]{1, 2, 3}));
-
-        assertArrayEquals(new double[] {1, 2, 3}, copy.<double[]>field("dArr"), 0);
-
-        SimpleObject obj0 = copy.deserialize();
-
-        assertArrayEquals(new double[] {1, 2, 3}, obj0.dArr, 0);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortableCopyCharArray() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new BinaryTypeConfiguration(SimpleObject.class.getName())
-        ));
-
-        SimpleObject obj = simpleObject();
-
-        BinaryObject po = marshal(obj, marsh);
-
-        BinaryObject copy = copy(po, F.<String, Object>asMap("cArr", new char[]{1, 2, 3}));
-
-        assertArrayEquals(new char[]{1, 2, 3}, copy.<char[]>field("cArr"));
-
-        SimpleObject obj0 = copy.deserialize();
-
-        assertArrayEquals(new char[]{1, 2, 3}, obj0.cArr);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortableCopyStringArray() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new BinaryTypeConfiguration(SimpleObject.class.getName())
-        ));
-
-        SimpleObject obj = simpleObject();
-
-        BinaryObject po = marshal(obj, marsh);
-
-        BinaryObject copy = copy(po, F.<String, Object>asMap("strArr", new String[]{"str1", "str2"}));
-
-        assertArrayEquals(new String[]{"str1", "str2"}, copy.<String[]>field("strArr"));
-
-        SimpleObject obj0 = copy.deserialize();
-
-        assertArrayEquals(new String[]{"str1", "str2"}, obj0.strArr);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortableCopyObject() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new BinaryTypeConfiguration(SimpleObject.class.getName())
-        ));
-
-        SimpleObject obj = simpleObject();
-
-        BinaryObject po = marshal(obj, marsh);
-
-        SimpleObject newObj = new SimpleObject();
-
-        newObj.i = 12345;
-        newObj.fArr = new float[] {5, 8, 0};
-        newObj.str = "newStr";
-
-        BinaryObject copy = copy(po, F.<String, Object>asMap("inner", newObj));
-
-        assertEquals(newObj, copy.<BinaryObject>field("inner").deserialize());
-
-        SimpleObject obj0 = copy.deserialize();
-
-        assertEquals(newObj, obj0.inner);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortableCopyNonPrimitives() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new BinaryTypeConfiguration(SimpleObject.class.getName())
-        ));
-
-        SimpleObject obj = simpleObject();
-
-        BinaryObject po = marshal(obj, marsh);
-
-        Map<String, Object> map = new HashMap<>(3, 1.0f);
-
-        SimpleObject newObj = new SimpleObject();
-
-        newObj.i = 12345;
-        newObj.fArr = new float[] {5, 8, 0};
-        newObj.str = "newStr";
-
-        map.put("str", "str555");
-        map.put("inner", newObj);
-        map.put("bArr", new byte[]{6, 7, 9});
-
-        BinaryObject copy = copy(po, map);
-
-        assertEquals("str555", copy.<String>field("str"));
-        assertEquals(newObj, copy.<BinaryObject>field("inner").deserialize());
-        assertArrayEquals(new byte[]{6, 7, 9}, copy.<byte[]>field("bArr"));
-
-        SimpleObject obj0 = copy.deserialize();
-
-        assertEquals("str555", obj0.str);
-        assertEquals(newObj, obj0.inner);
-        assertArrayEquals(new byte[] {6, 7, 9}, obj0.bArr);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortableCopyMixed() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(new BinaryTypeConfiguration(SimpleObject.class.getName())));
-
-        SimpleObject obj = simpleObject();
-
-        BinaryObject po = marshal(obj, marsh);
-
-        Map<String, Object> map = new HashMap<>(3, 1.0f);
-
-        SimpleObject newObj = new SimpleObject();
-
-        newObj.i = 12345;
-        newObj.fArr = new float[] {5, 8, 0};
-        newObj.str = "newStr";
-
-        map.put("i", 1234);
-        map.put("str", "str555");
-        map.put("inner", newObj);
-        map.put("s", (short)2323);
-        map.put("bArr", new byte[]{6, 7, 9});
-        map.put("b", (byte)111);
-
-        BinaryObject copy = copy(po, map);
-
-        assertEquals(1234, copy.<Integer>field("i").intValue());
-        assertEquals("str555", copy.<String>field("str"));
-        assertEquals(newObj, copy.<BinaryObject>field("inner").deserialize());
-        assertEquals((short)2323, copy.<Short>field("s").shortValue());
-        assertArrayEquals(new byte[] {6, 7, 9}, copy.<byte[]>field("bArr"));
-        assertEquals((byte)111, copy.<Byte>field("b").byteValue());
-
-        SimpleObject obj0 = copy.deserialize();
-
-        assertEquals(1234, obj0.i);
-        assertEquals("str555", obj0.str);
-        assertEquals(newObj, obj0.inner);
-        assertEquals((short)2323, obj0.s);
-        assertArrayEquals(new byte[] {6, 7, 9}, obj0.bArr);
-        assertEquals((byte)111, obj0.b);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testKeepDeserialized() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setClassNames(Arrays.asList(SimpleObject.class.getName()));
-        marsh.setKeepDeserialized(true);
-
-        BinaryObject po = marshal(simpleObject(), marsh);
-
-        assert po.deserialize() == po.deserialize();
-
-        marsh = new PortableMarshaller();
-
-        marsh.setClassNames(Arrays.asList(SimpleObject.class.getName()));
-        marsh.setKeepDeserialized(false);
-
-        po = marshal(simpleObject(), marsh);
-
-        assert po.deserialize() != po.deserialize();
-
-        marsh = new PortableMarshaller();
-
-        marsh.setKeepDeserialized(true);
-        marsh.setTypeConfigurations(Arrays.asList(
-            new BinaryTypeConfiguration(SimpleObject.class.getName())));
-
-        po = marshal(simpleObject(), marsh);
-
-        assert po.deserialize() == po.deserialize();
-
-        marsh = new PortableMarshaller();
-
-        marsh.setKeepDeserialized(false);
-        marsh.setTypeConfigurations(Arrays.asList(
-            new BinaryTypeConfiguration(SimpleObject.class.getName())));
-
-        po = marshal(simpleObject(), marsh);
-
-        assert po.deserialize() != po.deserialize();
-
-        marsh = new PortableMarshaller();
-
-        marsh.setKeepDeserialized(true);
-
-        BinaryTypeConfiguration typeCfg = new BinaryTypeConfiguration(SimpleObject.class.getName());
-
-        typeCfg.setKeepDeserialized(false);
-
-        marsh.setTypeConfigurations(Arrays.asList(typeCfg));
-
-        po = marshal(simpleObject(), marsh);
-
-        assert po.deserialize() != po.deserialize();
-
-        marsh = new PortableMarshaller();
-
-        marsh.setKeepDeserialized(false);
-
-        typeCfg = new BinaryTypeConfiguration(SimpleObject.class.getName());
-
-        typeCfg.setKeepDeserialized(true);
-
-        marsh.setTypeConfigurations(Arrays.asList(typeCfg));
-
-        po = marshal(simpleObject(), marsh);
-
-        assert po.deserialize() == po.deserialize();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testOffheapPortable() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(new BinaryTypeConfiguration(SimpleObject.class.getName())));
-
-        PortableContext ctx = initPortableContext(marsh);
-
-        SimpleObject simpleObj = simpleObject();
-
-        BinaryObjectImpl obj = marshal(simpleObj, marsh);
-
-        long ptr = 0;
-
-        long ptr1 = 0;
-
-        long ptr2 = 0;
-
-        try {
-            ptr = copyOffheap(obj);
-
-            BinaryObjectOffheapImpl offheapObj = new BinaryObjectOffheapImpl(ctx,
-                ptr,
-                0,
-                obj.array().length);
-
-            assertTrue(offheapObj.equals(offheapObj));
-            assertFalse(offheapObj.equals(null));
-            assertFalse(offheapObj.equals("str"));
-            assertTrue(offheapObj.equals(obj));
-            assertTrue(obj.equals(offheapObj));
-
-            ptr1 = copyOffheap(obj);
-
-            BinaryObjectOffheapImpl offheapObj1 = new BinaryObjectOffheapImpl(ctx,
-                ptr1,
-                0,
-                obj.array().length);
-
-            assertTrue(offheapObj.equals(offheapObj1));
-            assertTrue(offheapObj1.equals(offheapObj));
-
-            assertEquals(obj.typeId(), offheapObj.typeId());
-            assertEquals(obj.hashCode(), offheapObj.hashCode());
-
-            checkSimpleObjectData(simpleObj, offheapObj);
-
-            BinaryObjectOffheapImpl innerOffheapObj = offheapObj.field("inner");
-
-            assertNotNull(innerOffheapObj);
-
-            checkSimpleObjectData(simpleObj.inner, innerOffheapObj);
-
-            obj = (BinaryObjectImpl)offheapObj.heapCopy();
-
-            assertEquals(obj.typeId(), offheapObj.typeId());
-            assertEquals(obj.hashCode(), offheapObj.hashCode());
-
-            checkSimpleObjectData(simpleObj, obj);
-
-            BinaryObjectImpl innerObj = obj.field("inner");
-
-            assertNotNull(innerObj);
-
-            checkSimpleObjectData(simpleObj.inner, innerObj);
-
-            simpleObj.d = 0;
-
-            obj = marshal(simpleObj, marsh);
-
-            assertFalse(offheapObj.equals(obj));
-            assertFalse(obj.equals(offheapObj));
-
-            ptr2 = copyOffheap(obj);
-
-            BinaryObjectOffheapImpl offheapObj2 = new BinaryObjectOffheapImpl(ctx,
-                ptr2,
-                0,
-                obj.array().length);
-
-            assertFalse(offheapObj.equals(offheapObj2));
-            assertFalse(offheapObj2.equals(offheapObj));
-        }
-        finally {
-            UNSAFE.freeMemory(ptr);
-
-            if (ptr1 > 0)
-                UNSAFE.freeMemory(ptr1);
-
-            if (ptr2 > 0)
-                UNSAFE.freeMemory(ptr2);
-        }
-    }
-
-    /**
-     *
-     */
-    public void testReadResolve() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setClassNames(
-            Arrays.asList(MySingleton.class.getName(), SingletonMarker.class.getName()));
-
-        BinaryObjectImpl portableObj = marshal(MySingleton.INSTANCE, marsh);
-
-        assertTrue(portableObj.array().length <= 1024); // Check that big string was not serialized.
-
-        MySingleton singleton = portableObj.deserialize();
-
-        assertSame(MySingleton.INSTANCE, singleton);
-    }
-
-    /**
-     *
-     */
-    public void testReadResolveOnPortableAware() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setClassNames(Collections.singletonList(MyTestClass.class.getName()));
-
-        BinaryObjectImpl portableObj = marshal(new MyTestClass(), marsh);
-
-        MyTestClass obj = portableObj.deserialize();
-
-        assertEquals("readResolve", obj.s);
-    }
-
-    /**
-     * @throws Exception If ecxeption thrown.
-     */
-    public void testDeclareReadResolveInParent() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setClassNames(Arrays.asList(ChildPortable.class.getName()));
-
-        BinaryObjectImpl portableObj = marshal(new ChildPortable(), marsh);
-
-        ChildPortable singleton = portableObj.deserialize();
-
-        assertNotNull(singleton.s);
-    }
-
-    /**
-     *
-     */
-    public void testDecimalFields() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        Collection<String> clsNames = new ArrayList<>();
-
-        clsNames.add(DecimalReflective.class.getName());
-        clsNames.add(DecimalMarshalAware.class.getName());
-
-        marsh.setClassNames(clsNames);
-
-        // 1. Test reflective stuff.
-        DecimalReflective obj1 = new DecimalReflective();
-
-        obj1.val = BigDecimal.ZERO;
-        obj1.valArr = new BigDecimal[] { BigDecimal.ONE, BigDecimal.TEN };
-
-        BinaryObjectImpl portObj = marshal(obj1, marsh);
-
-        assertEquals(obj1.val, portObj.field("val"));
-        assertArrayEquals(obj1.valArr, portObj.<BigDecimal[]>field("valArr"));
-
-        assertEquals(obj1.val, portObj.<DecimalReflective>deserialize().val);
-        assertArrayEquals(obj1.valArr, portObj.<DecimalReflective>deserialize().valArr);
-
-        // 2. Test marshal aware stuff.
-        DecimalMarshalAware obj2 = new DecimalMarshalAware();
-
-        obj2.val = BigDecimal.ZERO;
-        obj2.valArr = new BigDecimal[] { BigDecimal.ONE, BigDecimal.TEN.negate() };
-        obj2.rawVal = BigDecimal.TEN;
-        obj2.rawValArr = new BigDecimal[] { BigDecimal.ZERO, BigDecimal.ONE };
-
-        portObj = marshal(obj2, marsh);
-
-        assertEquals(obj2.val, portObj.field("val"));
-        assertArrayEquals(obj2.valArr, portObj.<BigDecimal[]>field("valArr"));
-
-        assertEquals(obj2.val, portObj.<DecimalMarshalAware>deserialize().val);
-        assertArrayEquals(obj2.valArr, portObj.<DecimalMarshalAware>deserialize().valArr);
-        assertEquals(obj2.rawVal, portObj.<DecimalMarshalAware>deserialize().rawVal);
-        assertArrayEquals(obj2.rawValArr, portObj.<DecimalMarshalAware>deserialize().rawValArr);
-    }
-
-    /**
-     * @throws IgniteCheckedException If failed.
-     */
-    public void testFinalField() throws IgniteCheckedException {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        SimpleObjectWithFinal obj = new SimpleObjectWithFinal();
-
-        SimpleObjectWithFinal po0 = marshalUnmarshal(obj, marsh);
-
-        assertEquals(obj.time, po0.time);
-    }
-
-    /**
-     * @throws IgniteCheckedException If failed.
-     */
-    public void testThreadLocalArrayReleased() throws IgniteCheckedException {
-        // Checking the writer directly.
-        assertEquals(false, THREAD_LOCAL_ALLOC.isThreadLocalArrayAcquired());
-
-        try (BinaryWriterExImpl writer = new BinaryWriterExImpl(initPortableContext(new PortableMarshaller()))) {
-            assertEquals(true, THREAD_LOCAL_ALLOC.isThreadLocalArrayAcquired());
-
-            writer.writeString("Thread local test");
-
-            writer.array();
-
-            assertEquals(true, THREAD_LOCAL_ALLOC.isThreadLocalArrayAcquired());
-        }
-
-        // Checking the portable marshaller.
-        assertEquals(false, THREAD_LOCAL_ALLOC.isThreadLocalArrayAcquired());
-
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        initPortableContext(marsh);
-
-        marsh.marshal(new SimpleObject());
-
-        assertEquals(false, THREAD_LOCAL_ALLOC.isThreadLocalArrayAcquired());
-
-        // Checking the builder.
-        BinaryObjectBuilder builder = new BinaryObjectBuilderImpl(initPortableContext(new PortableMarshaller()),
-            "org.gridgain.foo.bar.TestClass");
-
-        builder.setField("a", "1");
-
-        BinaryObject portableObj = builder.build();
-
-        assertEquals(false, THREAD_LOCAL_ALLOC.isThreadLocalArrayAcquired());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDuplicateName() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        initPortableContext(marsh);
-
-        Test1.Job job1 = new Test1().new Job();
-        Test2.Job job2 = new Test2().new Job();
-
-        marsh.marshal(job1);
-
-        try {
-            marsh.marshal(job2);
-        } catch (BinaryObjectException e) {
-            assertEquals(true, e.getMessage().contains("Failed to register class"));
-            return;
-        }
-
-        assert false;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClass() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        initPortableContext(marsh);
-
-        Class cls = GridPortableMarshallerSelfTest.class;
-
-        Class unmarshalledCls = marshalUnmarshal(cls, marsh);
-
-        Assert.assertEquals(cls, unmarshalledCls);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClassFieldsMarshalling() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        initPortableContext(marsh);
-
-        ObjectWithClassFields obj = new ObjectWithClassFields();
-        obj.cls1 = GridPortableMarshallerSelfTest.class;
-
-        byte[] marshal = marsh.marshal(obj);
-
-        ObjectWithClassFields obj2 = marsh.unmarshal(marshal, null);
-
-        assertEquals(obj.cls1, obj2.cls1);
-        assertNull(obj2.cls2);
-
-        BinaryObject portObj = marshal(obj, marsh);
-
-        Class cls1 = portObj.field("cls1");
-
-        assertEquals(obj.cls1, cls1);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMarshallingThroughJdk() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        initPortableContext(marsh);
-
-        InetSocketAddress addr = new InetSocketAddress("192.168.0.2", 4545);
-
-        byte[] arr = marsh.marshal(addr);
-
-        InetSocketAddress addr2 = marsh.unmarshal(arr, null);
-
-        assertEquals(addr.getHostString(), addr2.getHostString());
-        assertEquals(addr.getPort(), addr2.getPort());
-
-        TestAddress testAddr = new TestAddress();
-        testAddr.addr = addr;
-        testAddr.str1 = "Hello World";
-
-        SimpleObject simpleObj = new SimpleObject();
-        simpleObj.c = 'g';
-        simpleObj.date = new Date();
-
-        testAddr.obj = simpleObj;
-
-        arr = marsh.marshal(testAddr);
-
-        TestAddress testAddr2 = marsh.unmarshal(arr, null);
-
-        assertEquals(testAddr.addr.getHostString(), testAddr2.addr.getHostString());
-        assertEquals(testAddr.addr.getPort(), testAddr2.addr.getPort());
-        assertEquals(testAddr.str1, testAddr2.str1);
-        assertEquals(testAddr.obj.c, testAddr2.obj.c);
-        assertEquals(testAddr.obj.date, testAddr2.obj.date);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPredefinedTypeIds() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        PortableContext pCtx = initPortableContext(marsh);
-
-        Field field = pCtx.getClass().getDeclaredField("predefinedTypeNames");
-
-        field.setAccessible(true);
-
-        Map<String, Integer> map = (Map<String, Integer>)field.get(pCtx);
-
-        assertTrue(map.size() > 0);
-
-        for (Map.Entry<String, Integer> entry : map.entrySet()) {
-            int id = entry.getValue();
-
-            if (id == GridPortableMarshaller.UNREGISTERED_TYPE_ID)
-                continue;
-
-            PortableClassDescriptor desc = pCtx.descriptorForTypeId(false, entry.getValue(), null);
-
-            assertEquals(desc.typeId(), pCtx.typeId(desc.describedClass().getName()));
-            assertEquals(desc.typeId(), pCtx.typeId(pCtx.typeName(desc.describedClass().getName())));
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCyclicReferencesMarshalling() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        SimpleObject obj = simpleObject();
-
-        obj.bArr = obj.inner.bArr;
-        obj.cArr = obj.inner.cArr;
-        obj.boolArr = obj.inner.boolArr;
-        obj.sArr = obj.inner.sArr;
-        obj.strArr = obj.inner.strArr;
-        obj.iArr = obj.inner.iArr;
-        obj.lArr = obj.inner.lArr;
-        obj.fArr = obj.inner.fArr;
-        obj.dArr = obj.inner.dArr;
-        obj.dateArr = obj.inner.dateArr;
-        obj.uuidArr = obj.inner.uuidArr;
-        obj.objArr = obj.inner.objArr;
-        obj.bdArr = obj.inner.bdArr;
-        obj.map = obj.inner.map;
-        obj.col = obj.inner.col;
-        obj.mEntry = obj.inner.mEntry;
-
-        SimpleObject res = (SimpleObject)marshalUnmarshal(obj, marsh);
-
-        assertEquals(obj, res);
-
-        assertTrue(res.bArr == res.inner.bArr);
-        assertTrue(res.cArr == res.inner.cArr);
-        assertTrue(res.boolArr == res.inner.boolArr);
-        assertTrue(res.sArr == res.inner.sArr);
-        assertTrue(res.strArr == res.inner.strArr);
-        assertTrue(res.iArr == res.inner.iArr);
-        assertTrue(res.lArr == res.inner.lArr);
-        assertTrue(res.fArr == res.inner.fArr);
-        assertTrue(res.dArr == res.inner.dArr);
-        assertTrue(res.dateArr == res.inner.dateArr);
-        assertTrue(res.uuidArr == res.inner.uuidArr);
-        assertTrue(res.objArr == res.inner.objArr);
-        assertTrue(res.bdArr == res.inner.bdArr);
-        assertTrue(res.map == res.inner.map);
-        assertTrue(res.col == res.inner.col);
-        assertTrue(res.mEntry == res.inner.mEntry);
-    }
-
-    /**
-     * Object with class fields.
-     */
-    private static class ObjectWithClassFields {
-        /** */
-        private Class<?> cls1;
-
-        /** */
-        private Class<?> cls2;
-    }
-
-    /**
-     *
-     */
-    private static class TestAddress {
-        /** */
-        private SimpleObject obj;
-
-        /** */
-        private InetSocketAddress addr;
-
-        /** */
-        private String str1;
-    }
-
-    /**
-     *
-     */
-    private static class Test1 {
-        /**
-         *
-         */
-        private class Job {
-
-        }
-    }
-
-    /**
-     *
-     */
-    private static class Test2 {
-        /**
-         *
-         */
-        private class Job {
-
-        }
-    }
-
-    /**
-     * @param obj Object.
-     * @return Offheap address.
-     */
-    private long copyOffheap(BinaryObjectImpl obj) {
-        byte[] arr = obj.array();
-
-        long ptr = UNSAFE.allocateMemory(arr.length);
-
-        UNSAFE.copyMemory(arr, BYTE_ARR_OFF, null, ptr, arr.length);
-
-        return ptr;
-    }
-
-    /**
-     * @param enumArr Enum array.
-     * @return Ordinals.
-     */
-    private <T extends Enum<?>> Integer[] ordinals(T[] enumArr) {
-        Integer[] ords = new Integer[enumArr.length];
-
-        for (int i = 0; i < enumArr.length; i++)
-            ords[i] = enumArr[i].ordinal();
-
-        return ords;
-    }
-
-    /**
-     * @param po Portable object.
-     * @param off Offset.
-     * @return Value.
-     */
-    private int intFromPortable(BinaryObject po, int off) {
-        byte[] arr = U.field(po, "arr");
-
-        return Integer.reverseBytes(U.bytesToInt(arr, off));
-    }
-
-    /**
-     * @param obj Original object.
-     * @return Result object.
-     */
-    private <T> T marshalUnmarshal(T obj) throws IgniteCheckedException {
-        return marshalUnmarshal(obj, new PortableMarshaller());
-    }
-
-    /**
-     * @param obj Original object.
-     * @param marsh Marshaller.
-     * @return Result object.
-     */
-    private <T> T marshalUnmarshal(Object obj, PortableMarshaller marsh) throws IgniteCheckedException {
-        initPortableContext(marsh);
-
-        byte[] bytes = marsh.marshal(obj);
-
-        return marsh.unmarshal(bytes, null);
-    }
-
-    /**
-     * @param obj Object.
-     * @param marsh Marshaller.
-     * @return Portable object.
-     */
-    private <T> BinaryObjectImpl marshal(T obj, PortableMarshaller marsh) throws IgniteCheckedException {
-        initPortableContext(marsh);
-
-        byte[] bytes = marsh.marshal(obj);
-
-        return new BinaryObjectImpl(U.<GridPortableMarshaller>field(marsh, "impl").context(),
-            bytes, 0);
-    }
-
-    /**
-     * @return Portable context.
-     */
-    protected PortableContext initPortableContext(PortableMarshaller marsh) throws IgniteCheckedException {
-        IgniteConfiguration iCfg = new IgniteConfiguration();
-
-        PortableContext ctx = new PortableContext(BinaryNoopMetadataHandler.instance(), iCfg);
-
-        marsh.setContext(new MarshallerContextTestImpl(null));
-
-        IgniteUtils.invoke(PortableMarshaller.class, marsh, "setPortableContext", ctx);
-
-        return ctx;
-    }
-
-    /**
-     * @param exp Expected.
-     * @param act Actual.
-     */
-    private void assertBooleanArrayEquals(boolean[] exp, boolean[] act) {
-        assertEquals(exp.length, act.length);
-
-        for (int i = 0; i < act.length; i++)
-            assertEquals(exp[i], act[i]);
-    }
-
-    /**
-     *
-     */
-    private static class SimpleObjectWithFinal {
-        /** */
-        private final long time = System.currentTimeMillis();
-    }
-
-    /**
-     * @return Simple object.
-     */
-    private SimpleObject simpleObject() {
-        SimpleObject inner = new SimpleObject();
-
-        inner.b = 1;
-        inner.s = 1;
-        inner.i = 1;
-        inner.l = 1;
-        inner.f = 1.1f;
-        inner.d = 1.1d;
-        inner.c = 1;
-        inner.bool = true;
-        inner.str = "str1";
-        inner.uuid = UUID.randomUUID();
-        inner.date = new Date();
-        inner.ts = new Timestamp(System.currentTimeMillis());
-        inner.bArr = new byte[] {1, 2, 3};
-        inner.sArr = new short[] {1, 2, 3};
-        inner.iArr = new int[] {1, 2, 3};
-        inner.lArr = new long[] {1, 2, 3};
-        inner.fArr = new float[] {1.1f, 2.2f, 3.3f};
-        inner.dArr = new double[] {1.1d, 2.2d, 3.3d};
-        inner.cArr = new char[] {1, 2, 3};
-        inner.boolArr = new boolean[] {true, false, true};
-        inner.strArr = new String[] {"str1", "str2", "str3"};
-        inner.uuidArr = new UUID[] {UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID()};
-        inner.dateArr = new Date[] {new Date(11111), new Date(22222), new Date(33333)};
-        inner.objArr = new Object[] {UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID()};
-        inner.col = new ArrayList<>();
-        inner.map = new HashMap<>();
-        inner.enumVal = TestEnum.A;
-        inner.enumArr = new TestEnum[] {TestEnum.A, TestEnum.B};
-        inner.bdArr = new BigDecimal[] {new BigDecimal(1000), BigDecimal.ONE};
-
-        inner.col.add("str1");
-        inner.col.add("str2");
-        inner.col.add("str3");
-
-        inner.map.put(1, "str1");
-        inner.map.put(2, "str2");
-        inner.map.put(3, "str3");
-
-        inner.mEntry = inner.map.entrySet().iterator().next();
-
-        SimpleObject outer = new SimpleObject();
-
-        outer.b = 2;
-        outer.s = 2;
-        outer.i = 2;
-        outer.l = 2;
-        outer.f = 2.2f;
-        outer.d = 2.2d;
-        outer.c = 2;
-        outer.bool = false;
-        outer.str = "str2";
-        outer.uuid = UUID.randomUUID();
-        outer.date = new Date();
-        outer.ts = new Timestamp(System.currentTimeMillis());
-        outer.bArr = new byte[] {10, 20, 30};
-        outer.sArr = new short[] {10, 20, 30};
-        outer.iArr = new int[] {10, 20, 30};
-        outer.lArr = new long[] {10, 20, 30};
-        outer.fArr = new float[] {10.01f, 20.02f, 30.03f};
-        outer.dArr = new double[] {10.01d, 20.02d, 30.03d};
-        outer.cArr = new char[] {10, 20, 30};
-        outer.boolArr = new boolean[] {false, true, false};
-        outer.strArr = new String[] {"str10", "str20", "str30"};
-        outer.uuidArr = new UUID[] {UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID()};
-        outer.dateArr = new Date[] {new Date(44444), new Date(55555), new Date(66666)};
-        outer.objArr = new Object[] {UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID()};
-        outer.col = new ArrayList<>();
-        outer.map = new HashMap<>();
-        outer.enumVal = TestEnum.B;
-        outer.enumArr = new TestEnum[] {TestEnum.B, TestEnum.C};
-        outer.inner = inner;
-        outer.bdArr = new BigDecimal[] {new BigDecimal(5000), BigDecimal.TEN};
-
-
-        outer.col.add("str4");
-        outer.col.add("str5");
-        outer.col.add("str6");
-
-        outer.map.put(4, "str4");
-        outer.map.put(5, "str5");
-        outer.map.put(6, "str6");
-
-        outer.mEntry = outer.map.entrySet().iterator().next();
-
-        return outer;
-    }
-
-    /**
-     * @return Portable object.
-     */
-    private TestBinary BinaryObject() {
-        SimpleObject innerSimple = new SimpleObject();
-
-        innerSimple.b = 1;
-        innerSimple.s = 1;
-        innerSimple.i = 1;
-        innerSimple.l = 1;
-        innerSimple.f = 1.1f;
-        innerSimple.d = 1.1d;
-        innerSimple.c = 1;
-        innerSimple.bool = true;
-        innerSimple.str = "str1";
-        innerSimple.uuid = UUID.randomUUID();
-        innerSimple.date = new Date();
-        innerSimple.ts = new Timestamp(System.currentTimeMillis());
-        innerSimple.bArr = new byte[] {1, 2, 3};
-        innerSimple.sArr = new short[] {1, 2, 3};
-        innerSimple.iArr = new int[] {1, 2, 3};
-        innerSimple.lArr = new long[] {1, 2, 3};
-        innerSimple.fArr = new float[] {1.1f, 2.2f, 3.3f};
-        innerSimple.dArr = new double[] {1.1d, 2.2d, 3.3d};
-        innerSimple.cArr = new char[] {1, 2, 3};
-        innerSimple.boolArr = new boolean[] {true, false, true};
-        innerSimple.strArr = new String[] {"str1", "str2", "str3"};
-        innerSimple.uuidArr = new UUID[] {UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID()};
-        innerSimple.dateArr = new Date[] {new Date(11111), new Date(22222), new Date(33333)};
-        innerSimple.objArr = new UUID[] {UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID()};
-        innerSimple.col = new ArrayList<>();
-        innerSimple.map = new HashMap<>();
-        innerSimple.enumVal = TestEnum.A;
-        innerSimple.enumArr = new TestEnum[] {TestEnum.A, TestEnum.B};
-
-        innerSimple.col.add("str1");
-        innerSimple.col.add("str2");
-        innerSimple.col.add("str3");
-
-        innerSimple.map.put(1, "str1");
-        innerSimple.map.put(2, "str2");
-        innerSimple.map.put(3, "str3");
-
-        TestBinary innerPortable = new TestBinary();
-
-        innerPortable.b = 2;
-        innerPortable.s = 2;
-    

<TRUNCATED>

[03/13] ignite git commit: IGNITE-1816: Implemented compact footers.

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/test/java/org/apache/ignite/internal/portable/GridBinaryObjectBuilderAdditionalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridBinaryObjectBuilderAdditionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridBinaryObjectBuilderAdditionalSelfTest.java
deleted file mode 100644
index 11b54ae..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridBinaryObjectBuilderAdditionalSelfTest.java
+++ /dev/null
@@ -1,1280 +0,0 @@
-/*
- * 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 com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-import java.lang.reflect.Field;
-import java.math.BigDecimal;
-import java.sql.Timestamp;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-import java.util.UUID;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.IgniteBinary;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.portable.builder.PortableBuilderEnum;
-import org.apache.ignite.internal.portable.builder.BinaryObjectBuilderImpl;
-import org.apache.ignite.internal.portable.mutabletest.GridBinaryMarshalerAwareTestClass;
-import org.apache.ignite.internal.processors.cache.portable.CacheObjectBinaryProcessorImpl;
-import org.apache.ignite.internal.processors.cache.portable.IgniteBinaryImpl;
-import org.apache.ignite.internal.util.lang.GridMapEntry;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.binary.BinaryObjectBuilder;
-import org.apache.ignite.binary.BinaryType;
-import org.apache.ignite.binary.BinaryObject;
-import org.apache.ignite.testframework.GridTestUtils;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.junit.Assert;
-
-import static org.apache.ignite.cache.CacheMode.REPLICATED;
-import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.Address;
-import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.AddressBook;
-import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.Company;
-import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectAllTypes;
-import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectArrayList;
-import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectContainer;
-import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectEnum;
-import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectInner;
-import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectOuter;
-
-/**
- *
- */
-public class GridBinaryObjectBuilderAdditionalSelfTest extends GridCommonAbstractTest {
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        CacheConfiguration cacheCfg = new CacheConfiguration();
-
-        cacheCfg.setCacheMode(REPLICATED);
-
-        cfg.setCacheConfiguration(cacheCfg);
-
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setClassNames(Arrays.asList("org.apache.ignite.internal.portable.mutabletest.*"));
-
-        cfg.setMarshaller(marsh);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        startGrids(1);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopAllGrids();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        jcache(0).clear();
-    }
-
-    /**
-     * @return Portables API.
-     */
-    protected IgniteBinary portables() {
-        return grid(0).binary();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testSimpleTypeFieldRead() throws Exception {
-        TestObjectAllTypes exp = new TestObjectAllTypes();
-
-        exp.setDefaultData();
-
-        BinaryObjectBuilder mutPo = wrap(exp);
-
-        for (Field field : TestObjectAllTypes.class.getDeclaredFields()) {
-            Object expVal = field.get(exp);
-            Object actVal = mutPo.getField(field.getName());
-
-            switch (field.getName()) {
-                case "anEnum":
-                    assertEquals(((PortableBuilderEnum)actVal).getOrdinal(), ((Enum)expVal).ordinal());
-                    break;
-
-                case "enumArr": {
-                    PortableBuilderEnum[] actArr = (PortableBuilderEnum[])actVal;
-                    Enum[] expArr = (Enum[])expVal;
-
-                    assertEquals(expArr.length, actArr.length);
-
-                    for (int i = 0; i < actArr.length; i++)
-                        assertEquals(expArr[i].ordinal(), actArr[i].getOrdinal());
-
-                    break;
-                }
-
-                case "entry":
-                    assertEquals(((Map.Entry)expVal).getKey(), ((Map.Entry)actVal).getKey());
-                    assertEquals(((Map.Entry)expVal).getValue(), ((Map.Entry)actVal).getValue());
-                    break;
-
-                default:
-                    assertTrue(field.getName(), Objects.deepEquals(expVal, actVal));
-                    break;
-            }
-        }
-    }
-
-    /**
-     *
-     */
-    public void testSimpleTypeFieldSerialize() {
-        TestObjectAllTypes exp = new TestObjectAllTypes();
-
-        exp.setDefaultData();
-
-        BinaryObjectBuilderImpl mutPo = wrap(exp);
-
-        TestObjectAllTypes res = mutPo.build().deserialize();
-
-        GridTestUtils.deepEquals(exp, res);
-    }
-
-    /**
-     * @throws Exception If any error occurs.
-     */
-    public void testSimpleTypeFieldOverride() throws Exception {
-        TestObjectAllTypes exp = new TestObjectAllTypes();
-
-        exp.setDefaultData();
-
-        BinaryObjectBuilderImpl mutPo = wrap(new TestObjectAllTypes());
-
-        for (Field field : TestObjectAllTypes.class.getDeclaredFields())
-            mutPo.setField(field.getName(), field.get(exp));
-
-        TestObjectAllTypes res = mutPo.build().deserialize();
-
-        GridTestUtils.deepEquals(exp, res);
-    }
-
-    /**
-     * @throws Exception If any error occurs.
-     */
-    public void testSimpleTypeFieldSetNull() throws Exception {
-        TestObjectAllTypes exp = new TestObjectAllTypes();
-
-        exp.setDefaultData();
-
-        BinaryObjectBuilderImpl mutPo = wrap(exp);
-
-        for (Field field : TestObjectAllTypes.class.getDeclaredFields()) {
-            if (!field.getType().isPrimitive())
-                mutPo.setField(field.getName(), null);
-        }
-
-        TestObjectAllTypes res = mutPo.build().deserialize();
-
-        for (Field field : TestObjectAllTypes.class.getDeclaredFields()) {
-            if (!field.getType().isPrimitive())
-                assertNull(field.getName(), field.get(res));
-        }
-    }
-
-    /**
-     * @throws IgniteCheckedException If any error occurs.
-     */
-    public void testMakeCyclicDependency() throws IgniteCheckedException {
-        TestObjectOuter outer = new TestObjectOuter();
-        outer.inner = new TestObjectInner();
-
-        BinaryObjectBuilderImpl mutOuter = wrap(outer);
-
-        BinaryObjectBuilderImpl mutInner = mutOuter.getField("inner");
-
-        mutInner.setField("outer", mutOuter);
-        mutInner.setField("foo", mutInner);
-
-        TestObjectOuter res = mutOuter.build().deserialize();
-
-        assertEquals(res, res.inner.outer);
-        assertEquals(res.inner, res.inner.foo);
-    }
-
-    /**
-     *
-     */
-    public void testDateArrayModification() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-
-        obj.dateArr =  new Date[] {new Date(11111), new Date(11111), new Date(11111)};
-
-        BinaryObjectBuilderImpl mutObj = wrap(obj);
-
-        Date[] arr = mutObj.getField("dateArr");
-        arr[0] = new Date(22222);
-
-        TestObjectAllTypes res = mutObj.build().deserialize();
-
-        Assert.assertArrayEquals(new Date[] {new Date(22222), new Date(11111), new Date(11111)}, res.dateArr);
-    }
-
-    /**
-     *
-     */
-    public void testTimestampArrayModification() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-
-        obj.tsArr = new Timestamp[] {new Timestamp(111222333), new Timestamp(222333444)};
-
-        BinaryObjectBuilderImpl mutObj = wrap(obj);
-
-        Timestamp[] arr = mutObj.getField("tsArr");
-        arr[0] = new Timestamp(333444555);
-
-        TestObjectAllTypes res = mutObj.build().deserialize();
-
-        Assert.assertArrayEquals(new Timestamp[] {new Timestamp(333444555), new Timestamp(222333444)}, res.tsArr);
-    }
-
-    /**
-     *
-     */
-    public void testUUIDArrayModification() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-
-        obj.uuidArr = new UUID[] {new UUID(1, 1), new UUID(1, 1), new UUID(1, 1)};
-
-        BinaryObjectBuilderImpl mutObj = wrap(obj);
-
-        UUID[] arr = mutObj.getField("uuidArr");
-        arr[0] = new UUID(2, 2);
-
-        TestObjectAllTypes res = mutObj.build().deserialize();
-
-        Assert.assertArrayEquals(new UUID[] {new UUID(2, 2), new UUID(1, 1), new UUID(1, 1)}, res.uuidArr);
-    }
-
-    /**
-     *
-     */
-    public void testDecimalArrayModification() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-
-        obj.bdArr = new BigDecimal[] {new BigDecimal(1000), new BigDecimal(1000), new BigDecimal(1000)};
-
-        BinaryObjectBuilderImpl mutObj = wrap(obj);
-
-        BigDecimal[] arr = mutObj.getField("bdArr");
-        arr[0] = new BigDecimal(2000);
-
-        TestObjectAllTypes res = mutObj.build().deserialize();
-
-        Assert.assertArrayEquals(new BigDecimal[] {new BigDecimal(1000), new BigDecimal(1000), new BigDecimal(1000)},
-            res.bdArr);
-    }
-
-    /**
-     *
-     */
-    public void testBooleanArrayModification() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-
-        obj.zArr = new boolean[] {false, false, false};
-
-        BinaryObjectBuilderImpl mutObj = wrap(obj);
-
-        boolean[] arr = mutObj.getField("zArr");
-        arr[0] = true;
-
-        TestObjectAllTypes res = mutObj.build().deserialize();
-
-        boolean[] expected = new boolean[] {true, false, false};
-
-        assertEquals(expected.length, res.zArr.length);
-
-        for (int i = 0; i < expected.length; i++)
-            assertEquals(expected[i], res.zArr[i]);
-    }
-
-    /**
-     *
-     */
-    public void testCharArrayModification() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-
-        obj.cArr = new char[] {'a', 'a', 'a'};
-
-        BinaryObjectBuilderImpl mutObj = wrap(obj);
-
-        char[] arr = mutObj.getField("cArr");
-        arr[0] = 'b';
-
-        TestObjectAllTypes res = mutObj.build().deserialize();
-
-        Assert.assertArrayEquals(new char[] {'b', 'a', 'a'}, res.cArr);
-    }
-
-    /**
-     *
-     */
-    public void testDoubleArrayModification() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-
-        obj.dArr = new double[] {1.0, 1.0, 1.0};
-
-        BinaryObjectBuilderImpl mutObj = wrap(obj);
-
-        double[] arr = mutObj.getField("dArr");
-        arr[0] = 2.0;
-
-        TestObjectAllTypes res = mutObj.build().deserialize();
-
-        Assert.assertArrayEquals(new double[] {2.0, 1.0, 1.0}, res.dArr, 0);
-    }
-
-    /**
-     *
-     */
-    public void testFloatArrayModification() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-
-        obj.fArr = new float[] {1.0f, 1.0f, 1.0f};
-
-        BinaryObjectBuilderImpl mutObj = wrap(obj);
-
-        float[] arr = mutObj.getField("fArr");
-        arr[0] = 2.0f;
-
-        TestObjectAllTypes res = mutObj.build().deserialize();
-
-        Assert.assertArrayEquals(new float[] {2.0f, 1.0f, 1.0f}, res.fArr, 0);
-    }
-
-    /**
-     *
-     */
-    public void testLongArrayModification() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-
-        obj.lArr = new long[] {1, 1, 1};
-
-        BinaryObjectBuilderImpl mutObj = wrap(obj);
-
-        long[] arr = mutObj.getField("lArr");
-        arr[0] = 2;
-
-        TestObjectAllTypes res = mutObj.build().deserialize();
-
-        Assert.assertArrayEquals(new long[] {2, 1, 1}, res.lArr);
-    }
-
-    /**
-     *
-     */
-    public void testIntArrayModification() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-
-        obj.iArr = new int[] {1, 1, 1};
-
-        BinaryObjectBuilderImpl mutObj = wrap(obj);
-
-        int[] arr = mutObj.getField("iArr");
-        arr[0] = 2;
-
-        TestObjectAllTypes res = mutObj.build().deserialize();
-
-        Assert.assertArrayEquals(new int[] {2, 1, 1}, res.iArr);
-    }
-
-    /**
-     *
-     */
-    public void testShortArrayModification() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-
-        obj.sArr = new short[] {1, 1, 1};
-
-        BinaryObjectBuilderImpl mutObj = wrap(obj);
-
-        short[] arr = mutObj.getField("sArr");
-        arr[0] = 2;
-
-        TestObjectAllTypes res = mutObj.build().deserialize();
-
-        Assert.assertArrayEquals(new short[] {2, 1, 1}, res.sArr);
-    }
-
-    /**
-     *
-     */
-    public void testByteArrayModification() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-
-        obj.bArr = new byte[] {1, 1, 1};
-
-        BinaryObjectBuilderImpl mutObj = wrap(obj);
-
-        byte[] arr = mutObj.getField("bArr");
-        arr[0] = 2;
-
-        TestObjectAllTypes res = mutObj.build().deserialize();
-
-        Assert.assertArrayEquals(new byte[] {2, 1, 1}, res.bArr);
-    }
-
-    /**
-     *
-     */
-    public void testStringArrayModification() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-
-        obj.strArr = new String[] {"a", "a", "a"};
-
-        BinaryObjectBuilderImpl mutObj = wrap(obj);
-
-        String[] arr = mutObj.getField("strArr");
-        arr[0] = "b";
-
-        TestObjectAllTypes res = mutObj.build().deserialize();
-
-        Assert.assertArrayEquals(new String[] {"b", "a", "a"}, res.strArr);
-    }
-
-    /**
-     *
-     */
-    public void testModifyObjectArray() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = new Object[] {"a"};
-
-        BinaryObjectBuilderImpl mutObj = wrap(obj);
-
-        Object[] arr = mutObj.getField("foo");
-
-        Assert.assertArrayEquals(new Object[] {"a"}, arr);
-
-        arr[0] = "b";
-
-        TestObjectContainer res = mutObj.build().deserialize();
-
-        Assert.assertArrayEquals(new Object[] {"b"}, (Object[])res.foo);
-    }
-
-    /**
-     *
-     */
-    public void testOverrideObjectArrayField() {
-        BinaryObjectBuilderImpl mutObj = wrap(new TestObjectContainer());
-
-        Object[] createdArr = {mutObj, "a", 1, new String[] {"s", "s"}, new byte[] {1, 2}, new UUID(3, 0)};
-
-        mutObj.setField("foo", createdArr.clone());
-
-        TestObjectContainer res = mutObj.build().deserialize();
-
-        createdArr[0] = res;
-
-        assertTrue(Objects.deepEquals(createdArr, res.foo));
-    }
-
-    /**
-     *
-     */
-    public void testDeepArray() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = new Object[] {new Object[] {"a", obj}};
-
-        BinaryObjectBuilderImpl mutObj = wrap(obj);
-
-        Object[] arr = (Object[])mutObj.<Object[]>getField("foo")[0];
-
-        assertEquals("a", arr[0]);
-        assertSame(mutObj, arr[1]);
-
-        arr[0] = mutObj;
-
-        TestObjectContainer res = mutObj.build().deserialize();
-
-        arr = (Object[])((Object[])res.foo)[0];
-
-        assertSame(arr[0], res);
-        assertSame(arr[0], arr[1]);
-    }
-
-    /**
-     *
-     */
-    public void testArrayListRead() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = Lists.newArrayList(obj, "a");
-
-        BinaryObjectBuilderImpl mutObj = wrap(obj);
-
-        List<Object> list = mutObj.getField("foo");
-
-        assert list.equals(Lists.newArrayList(mutObj, "a"));
-    }
-
-    /**
-     *
-     */
-    public void testArrayListOverride() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        BinaryObjectBuilderImpl mutObj = wrap(obj);
-
-        ArrayList<Object> list = Lists.newArrayList(mutObj, "a", Lists.newArrayList(1, 2));
-
-        mutObj.setField("foo", list);
-
-        TestObjectContainer res = mutObj.build().deserialize();
-
-        list.set(0, res);
-
-        assertNotSame(list, res.foo);
-        assertEquals(list, res.foo);
-    }
-
-    /**
-     *
-     */
-    public void testArrayListModification() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = Lists.newArrayList("a", "b", "c");
-
-        BinaryObjectBuilderImpl mutObj = wrap(obj);
-
-        List<String> list = mutObj.getField("foo");
-
-        list.add("!"); // "a", "b", "c", "!"
-        list.add(0, "_"); // "_", "a", "b", "c", "!"
-
-        String s = list.remove(1); // "_", "b", "c", "!"
-        assertEquals("a", s);
-
-        assertEquals(Arrays.asList("c", "!"), list.subList(2, 4));
-        assertEquals(1, list.indexOf("b"));
-        assertEquals(1, list.lastIndexOf("b"));
-
-        TestObjectContainer res = mutObj.build().deserialize();
-
-        assertTrue(res.foo instanceof ArrayList);
-        assertEquals(Arrays.asList("_", "b", "c", "!"), res.foo);
-    }
-
-    /**
-     *
-     */
-    public void testArrayListClear() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = Lists.newArrayList("a", "b", "c");
-
-        BinaryObjectBuilderImpl mutObj = wrap(obj);
-
-        List<String> list = mutObj.getField("foo");
-
-        list.clear();
-
-        assertEquals(Collections.emptyList(), mutObj.build().<TestObjectContainer>deserialize().foo);
-    }
-
-    /**
-     *
-     */
-    public void testArrayListWriteUnmodifiable() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        ArrayList<Object> src = Lists.newArrayList(obj, "a", "b", "c");
-
-        obj.foo = src;
-
-        BinaryObjectBuilderImpl mutObj = wrap(obj);
-
-        TestObjectContainer deserialized = mutObj.build().deserialize();
-
-        List<Object> res = (List<Object>)deserialized.foo;
-
-        src.set(0, deserialized);
-
-        assertEquals(src, res);
-    }
-
-    /**
-     *
-     */
-    public void testLinkedListRead() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = Lists.newLinkedList(Arrays.asList(obj, "a"));
-
-        BinaryObjectBuilderImpl mutObj = wrap(obj);
-
-        List<Object> list = mutObj.getField("foo");
-
-        assert list.equals(Lists.newLinkedList(Arrays.asList(mutObj, "a")));
-    }
-
-    /**
-     *
-     */
-    public void testLinkedListOverride() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        BinaryObjectBuilderImpl mutObj = wrap(obj);
-
-        List<Object> list = Lists.newLinkedList(Arrays.asList(mutObj, "a", Lists.newLinkedList(Arrays.asList(1, 2))));
-
-        mutObj.setField("foo", list);
-
-        TestObjectContainer res = mutObj.build().deserialize();
-
-        list.set(0, res);
-
-        assertNotSame(list, res.foo);
-        assertEquals(list, res.foo);
-    }
-
-    /**
-     *
-     */
-    public void testLinkedListModification() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        obj.foo = Lists.newLinkedList(Arrays.asList("a", "b", "c"));
-
-        BinaryObjectBuilderImpl mutObj = wrap(obj);
-
-        List<String> list = mutObj.getField("foo");
-
-        list.add("!"); // "a", "b", "c", "!"
-        list.add(0, "_"); // "_", "a", "b", "c", "!"
-
-        String s = list.remove(1); // "_", "b", "c", "!"
-        assertEquals("a", s);
-
-        assertEquals(Arrays.asList("c", "!"), list.subList(2, 4));
-        assertEquals(1, list.indexOf("b"));
-        assertEquals(1, list.lastIndexOf("b"));
-
-        TestObjectContainer res = mutObj.build().deserialize();
-
-        assertTrue(res.foo instanceof LinkedList);
-        assertEquals(Arrays.asList("_", "b", "c", "!"), res.foo);
-    }
-
-    /**
-     *
-     */
-    public void testLinkedListWriteUnmodifiable() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        LinkedList<Object> src = Lists.newLinkedList(Arrays.asList(obj, "a", "b", "c"));
-
-        obj.foo = src;
-
-        BinaryObjectBuilderImpl mutObj = wrap(obj);
-
-        TestObjectContainer deserialized = mutObj.build().deserialize();
-
-        List<Object> res = (List<Object>)deserialized.foo;
-
-        src.set(0, deserialized);
-
-        assertEquals(src, res);
-    }
-
-    /**
-     *
-     */
-    public void testHashSetRead() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = Sets.newHashSet(obj, "a");
-
-        BinaryObjectBuilderImpl mutObj = wrap(obj);
-
-        Set<Object> set = mutObj.getField("foo");
-
-        assert set.equals(Sets.newHashSet(mutObj, "a"));
-    }
-
-    /**
-     *
-     */
-    public void testHashSetOverride() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        BinaryObjectBuilderImpl mutObj = wrap(obj);
-
-        Set<Object> c = Sets.newHashSet(mutObj, "a", Sets.newHashSet(1, 2));
-
-        mutObj.setField("foo", c);
-
-        TestObjectContainer res = mutObj.build().deserialize();
-
-        c.remove(mutObj);
-        c.add(res);
-
-        assertNotSame(c, res.foo);
-        assertEquals(c, res.foo);
-    }
-
-    /**
-     *
-     */
-    public void testHashSetModification() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = Sets.newHashSet("a", "b", "c");
-
-        BinaryObjectBuilderImpl mutObj = wrap(obj);
-
-        Set<String> set = mutObj.getField("foo");
-
-        set.remove("b");
-        set.add("!");
-
-        assertEquals(Sets.newHashSet("a", "!", "c"), set);
-        assertTrue(set.contains("a"));
-        assertTrue(set.contains("!"));
-
-        TestObjectContainer res = mutObj.build().deserialize();
-
-        assertTrue(res.foo instanceof HashSet);
-        assertEquals(Sets.newHashSet("a", "!", "c"), res.foo);
-    }
-
-    /**
-     *
-     */
-    public void testHashSetWriteUnmodifiable() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        Set<Object> src = Sets.newHashSet(obj, "a", "b", "c");
-
-        obj.foo = src;
-
-        TestObjectContainer deserialized = wrap(obj).build().deserialize();
-
-        Set<Object> res = (Set<Object>)deserialized.foo;
-
-        src.remove(obj);
-        src.add(deserialized);
-
-        assertEquals(src, res);
-    }
-
-    /**
-     *
-     */
-    public void testMapRead() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = Maps.newHashMap(ImmutableMap.of(obj, "a", "b", obj));
-
-        BinaryObjectBuilderImpl mutObj = wrap(obj);
-
-        Map<Object, Object> map = mutObj.getField("foo");
-
-        assert map.equals(ImmutableMap.of(mutObj, "a", "b", mutObj));
-    }
-
-    /**
-     *
-     */
-    public void testMapOverride() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        BinaryObjectBuilderImpl mutObj = wrap(obj);
-
-        Map<Object, Object> map = Maps.newHashMap(ImmutableMap.of(mutObj, "a", "b", mutObj));
-
-        mutObj.setField("foo", map);
-
-        TestObjectContainer res = mutObj.build().deserialize();
-
-        assertEquals(ImmutableMap.of(res, "a", "b", res), res.foo);
-    }
-
-    /**
-     *
-     */
-    public void testMapModification() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = Maps.newHashMap(ImmutableMap.of(1, "a", 2, "b"));
-
-        BinaryObjectBuilderImpl mutObj = wrap(obj);
-
-        Map<Object, Object> map = mutObj.getField("foo");
-
-        map.put(3, mutObj);
-        Object rmv = map.remove(1);
-
-        assertEquals("a", rmv);
-
-        TestObjectContainer res = mutObj.build().deserialize();
-
-        assertEquals(ImmutableMap.of(2, "b", 3, res), res.foo);
-    }
-
-    /**
-     *
-     */
-    public void testEnumArrayModification() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-
-        obj.enumArr = new TestObjectEnum[] {TestObjectEnum.A, TestObjectEnum.B};
-
-        BinaryObjectBuilderImpl mutObj = wrap(obj);
-
-        PortableBuilderEnum[] arr = mutObj.getField("enumArr");
-        arr[0] = new PortableBuilderEnum(mutObj.typeId(), TestObjectEnum.B);
-
-        TestObjectAllTypes res = mutObj.build().deserialize();
-
-        Assert.assertArrayEquals(new TestObjectEnum[] {TestObjectEnum.A, TestObjectEnum.B}, res.enumArr);
-    }
-
-    /**
-     *
-     */
-    public void testEditObjectWithRawData() {
-        GridBinaryMarshalerAwareTestClass obj = new GridBinaryMarshalerAwareTestClass();
-
-        obj.s = "a";
-        obj.sRaw = "aa";
-
-        BinaryObjectBuilderImpl mutableObj = wrap(obj);
-
-        mutableObj.setField("s", "z");
-
-        GridBinaryMarshalerAwareTestClass res = mutableObj.build().deserialize();
-        assertEquals("z", res.s);
-        assertEquals("aa", res.sRaw);
-    }
-
-    /**
-     *
-     */
-    public void testHashCode() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        BinaryObjectBuilderImpl mutableObj = wrap(obj);
-
-        assertEquals(obj.hashCode(), mutableObj.build().hashCode());
-
-        mutableObj.hashCode(25);
-
-        assertEquals(25, mutableObj.build().hashCode());
-    }
-
-    /**
-     *
-     */
-    public void testCollectionsInCollection() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = Lists.newArrayList(
-            Lists.newArrayList(1, 2),
-            Lists.newLinkedList(Arrays.asList(1, 2)),
-            Sets.newHashSet("a", "b"),
-            Sets.newLinkedHashSet(Arrays.asList("a", "b")),
-            Maps.newHashMap(ImmutableMap.of(1, "a", 2, "b")));
-
-        TestObjectContainer deserialized = wrap(obj).build().deserialize();
-
-        assertEquals(obj.foo, deserialized.foo);
-    }
-
-    /**
-     *
-     */
-    public void testMapEntryModification() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = ImmutableMap.of(1, "a").entrySet().iterator().next();
-
-        BinaryObjectBuilderImpl mutableObj = wrap(obj);
-
-        Map.Entry<Object, Object> entry = mutableObj.getField("foo");
-
-        assertEquals(1, entry.getKey());
-        assertEquals("a", entry.getValue());
-
-        entry.setValue("b");
-
-        TestObjectContainer res = mutableObj.build().deserialize();
-
-        assertEquals(new GridMapEntry<>(1, "b"), res.foo);
-    }
-
-    /**
-     *
-     */
-    public void testMapEntryOverride() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        BinaryObjectBuilderImpl mutableObj = wrap(obj);
-
-        mutableObj.setField("foo", new GridMapEntry<>(1, "a"));
-
-        TestObjectContainer res = mutableObj.build().deserialize();
-
-        assertEquals(new GridMapEntry<>(1, "a"), res.foo);
-    }
-
-    /**
-     *
-     */
-    public void testMetadataChangingDoublePut() {
-        BinaryObjectBuilderImpl mutableObj = wrap(new TestObjectContainer());
-
-        mutableObj.setField("xx567", "a");
-        mutableObj.setField("xx567", "b");
-
-        mutableObj.build();
-
-        BinaryType metadata = portables().metadata(TestObjectContainer.class);
-
-        assertEquals("String", metadata.fieldTypeName("xx567"));
-    }
-
-    /**
-     *
-     */
-    public void testMetadataChangingDoublePut2() {
-        BinaryObjectBuilderImpl mutableObj = wrap(new TestObjectContainer());
-
-        mutableObj.setField("xx567", "a");
-        mutableObj.setField("xx567", "b");
-
-        mutableObj.build();
-
-        BinaryType metadata = portables().metadata(TestObjectContainer.class);
-
-        assertEquals("String", metadata.fieldTypeName("xx567"));
-    }
-
-    /**
-     *
-     */
-    public void testMetadataChanging() {
-        TestObjectContainer c = new TestObjectContainer();
-
-        BinaryObjectBuilderImpl mutableObj = wrap(c);
-
-        mutableObj.setField("intField", 1);
-        mutableObj.setField("intArrField", new int[] {1});
-        mutableObj.setField("arrField", new String[] {"1"});
-        mutableObj.setField("strField", "1");
-        mutableObj.setField("colField", Lists.newArrayList("1"));
-        mutableObj.setField("mapField", Maps.newHashMap(ImmutableMap.of(1, "1")));
-        mutableObj.setField("enumField", TestObjectEnum.A);
-        mutableObj.setField("enumArrField", new Enum[] {TestObjectEnum.A});
-
-        mutableObj.build();
-
-        BinaryType metadata = portables().metadata(c.getClass());
-
-        assertTrue(metadata.fieldNames().containsAll(Arrays.asList("intField", "intArrField", "arrField", "strField",
-            "colField", "mapField", "enumField", "enumArrField")));
-
-        assertEquals("int", metadata.fieldTypeName("intField"));
-        assertEquals("int[]", metadata.fieldTypeName("intArrField"));
-        assertEquals("String[]", metadata.fieldTypeName("arrField"));
-        assertEquals("String", metadata.fieldTypeName("strField"));
-        assertEquals("Collection", metadata.fieldTypeName("colField"));
-        assertEquals("Map", metadata.fieldTypeName("mapField"));
-        assertEquals("Enum", metadata.fieldTypeName("enumField"));
-        assertEquals("Enum[]", metadata.fieldTypeName("enumArrField"));
-    }
-
-    /**
-     *
-     */
-    public void testDateInObjectField() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        obj.foo = new Date();
-
-        BinaryObjectBuilderImpl mutableObj = wrap(obj);
-
-        assertEquals(Date.class, mutableObj.getField("foo").getClass());
-    }
-
-    /**
-     *
-     */
-    public void testTimestampInObjectField() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        obj.foo = new Timestamp(100020003);
-
-        BinaryObjectBuilderImpl mutableObj = wrap(obj);
-
-        assertEquals(Timestamp.class, mutableObj.getField("foo").getClass());
-    }
-
-    /**
-     *
-     */
-    public void testDateInCollection() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        obj.foo = Lists.newArrayList(new Date());
-
-        BinaryObjectBuilderImpl mutableObj = wrap(obj);
-
-        assertEquals(Date.class, ((List<?>)mutableObj.getField("foo")).get(0).getClass());
-    }
-
-    /**
-     *
-     */
-    public void testTimestampInCollection() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        obj.foo = Lists.newArrayList(new Timestamp(100020003));
-
-        BinaryObjectBuilderImpl mutableObj = wrap(obj);
-
-        assertEquals(Timestamp.class, ((List<?>)mutableObj.getField("foo")).get(0).getClass());
-    }
-
-    /**
-     *
-     */
-    @SuppressWarnings("AssertEqualsBetweenInconvertibleTypes")
-    public void testDateArrayOverride() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        BinaryObjectBuilderImpl mutableObj = wrap(obj);
-
-        Date[] arr = { new Date() };
-
-        mutableObj.setField("foo", arr);
-
-        TestObjectContainer res = mutableObj.build().deserialize();
-
-        assertEquals(Date[].class, res.foo.getClass());
-        assertTrue(Objects.deepEquals(arr, res.foo));
-    }
-
-    /**
-     *
-     */
-    @SuppressWarnings("AssertEqualsBetweenInconvertibleTypes")
-    public void testTimestampArrayOverride() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        BinaryObjectBuilderImpl mutableObj = wrap(obj);
-
-        Timestamp[] arr = { new Timestamp(100020003) };
-
-        mutableObj.setField("foo", arr);
-
-        TestObjectContainer res = mutableObj.build().deserialize();
-
-        assertEquals(Timestamp[].class, res.foo.getClass());
-        assertTrue(Objects.deepEquals(arr, res.foo));
-    }
-
-    /**
-     *
-     */
-    public void testChangeMap() {
-        AddressBook addrBook = new AddressBook();
-
-        addrBook.addCompany(new Company(1, "Google inc", 100, new Address("Saint-Petersburg", "Torzhkovskya", 1, 53), "occupation"));
-        addrBook.addCompany(new Company(2, "Apple inc", 100, new Address("Saint-Petersburg", "Torzhkovskya", 1, 54), "occupation"));
-        addrBook.addCompany(new Company(3, "Microsoft", 100, new Address("Saint-Petersburg", "Torzhkovskya", 1, 55), "occupation"));
-        addrBook.addCompany(new Company(4, "Oracle", 100, new Address("Saint-Petersburg", "Nevskiy", 1, 1), "occupation"));
-
-        BinaryObjectBuilderImpl mutableObj = wrap(addrBook);
-
-        Map<String, List<BinaryObjectBuilderImpl>> map = mutableObj.getField("companyByStreet");
-
-        List<BinaryObjectBuilderImpl> list = map.get("Torzhkovskya");
-
-        BinaryObjectBuilderImpl company = list.get(0);
-
-        assert "Google inc".equals(company.<String>getField("name"));
-
-        list.remove(0);
-
-        AddressBook res = mutableObj.build().deserialize();
-
-        assertEquals(Arrays.asList("Nevskiy", "Torzhkovskya"), new ArrayList<>(res.getCompanyByStreet().keySet()));
-
-        List<Company> torzhkovskyaCompanies = res.getCompanyByStreet().get("Torzhkovskya");
-
-        assertEquals(2, torzhkovskyaCompanies.size());
-        assertEquals("Apple inc", torzhkovskyaCompanies.get(0).name);
-    }
-
-    /**
-     *
-     */
-    public void testSavingObjectWithNotZeroStart() {
-        TestObjectOuter out = new TestObjectOuter();
-        TestObjectInner inner = new TestObjectInner();
-
-        out.inner = inner;
-        inner.outer = out;
-
-        BinaryObjectBuilderImpl builder = wrap(out);
-
-        BinaryObjectBuilderImpl innerBuilder = builder.getField("inner");
-
-        TestObjectInner res = innerBuilder.build().deserialize();
-
-        assertSame(res, res.outer.inner);
-    }
-
-    /**
-     *
-     */
-    public void testPortableObjectField() {
-        TestObjectContainer container = new TestObjectContainer(toPortable(new TestObjectArrayList()));
-
-        BinaryObjectBuilderImpl wrapper = wrap(container);
-
-        assertTrue(wrapper.getField("foo") instanceof BinaryObject);
-
-        TestObjectContainer deserialized = wrapper.build().deserialize();
-        assertTrue(deserialized.foo instanceof BinaryObject);
-    }
-
-    /**
-     *
-     */
-    public void testAssignPortableObject() {
-        TestObjectContainer container = new TestObjectContainer();
-
-        BinaryObjectBuilderImpl wrapper = wrap(container);
-
-        wrapper.setField("foo", toPortable(new TestObjectArrayList()));
-
-        TestObjectContainer deserialized = wrapper.build().deserialize();
-        assertTrue(deserialized.foo instanceof TestObjectArrayList);
-    }
-
-    /**
-     *
-     */
-    public void testRemoveFromNewObject() {
-        BinaryObjectBuilderImpl wrapper = newWrapper(TestObjectAllTypes.class);
-
-        wrapper.setField("str", "a");
-
-        wrapper.removeField("str");
-
-        assertNull(wrapper.build().<TestObjectAllTypes>deserialize().str);
-    }
-
-    /**
-     *
-     */
-    public void testRemoveFromExistingObject() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-        obj.setDefaultData();
-
-        BinaryObjectBuilderImpl wrapper = wrap(toPortable(obj));
-
-        wrapper.removeField("str");
-
-        assertNull(wrapper.build().<TestObjectAllTypes>deserialize().str);
-    }
-
-    /**
-     *
-     */
-    public void testCyclicArrays() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        Object[] arr1 = new Object[1];
-        Object[] arr2 = new Object[] {arr1};
-
-        arr1[0] = arr2;
-
-        obj.foo = arr1;
-
-        TestObjectContainer res = toPortable(obj).deserialize();
-
-        Object[] resArr = (Object[])res.foo;
-
-        assertSame(((Object[])resArr[0])[0], resArr);
-    }
-
-    /**
-     *
-     */
-    @SuppressWarnings("TypeMayBeWeakened")
-    public void testCyclicArrayList() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        List<Object> arr1 = new ArrayList<>();
-        List<Object> arr2 = new ArrayList<>();
-
-        arr1.add(arr2);
-        arr2.add(arr1);
-
-        obj.foo = arr1;
-
-        TestObjectContainer res = toPortable(obj).deserialize();
-
-        List<?> resArr = (List<?>)res.foo;
-
-        assertSame(((List<Object>)resArr.get(0)).get(0), resArr);
-    }
-
-    /**
-     * @param obj Object.
-     * @return Object in portable format.
-     */
-    private BinaryObject toPortable(Object obj) {
-        return portables().toBinary(obj);
-    }
-
-    /**
-     * @param obj Object.
-     * @return GridMutablePortableObject.
-     */
-    private BinaryObjectBuilderImpl wrap(Object obj) {
-        return BinaryObjectBuilderImpl.wrap(toPortable(obj));
-    }
-
-    /**
-     * @param aCls Class.
-     * @return Wrapper.
-     */
-    private BinaryObjectBuilderImpl newWrapper(Class<?> aCls) {
-        CacheObjectBinaryProcessorImpl processor = (CacheObjectBinaryProcessorImpl)(
-            (IgniteBinaryImpl)portables()).processor();
-
-        return new BinaryObjectBuilderImpl(processor.portableContext(), processor.typeId(aCls.getName()),
-            aCls.getSimpleName());
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/test/java/org/apache/ignite/internal/portable/GridBinaryObjectBuilderSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridBinaryObjectBuilderSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridBinaryObjectBuilderSelfTest.java
deleted file mode 100644
index a74315b..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridBinaryObjectBuilderSelfTest.java
+++ /dev/null
@@ -1,1053 +0,0 @@
-/*
- * 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.sql.Timestamp;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Date;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.IgniteBinary;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.portable.builder.BinaryObjectBuilderImpl;
-import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectAllTypes;
-import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectContainer;
-import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectInner;
-import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectOuter;
-import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectPlainPortable;
-import org.apache.ignite.internal.processors.cache.portable.CacheObjectBinaryProcessorImpl;
-import org.apache.ignite.internal.util.GridUnsafe;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.binary.BinaryObjectBuilder;
-import org.apache.ignite.binary.BinaryIdMapper;
-import org.apache.ignite.binary.BinaryType;
-import org.apache.ignite.binary.BinaryObject;
-import org.apache.ignite.binary.BinaryTypeConfiguration;
-import org.apache.ignite.testframework.GridTestUtils;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import sun.misc.Unsafe;
-
-/**
- * Portable builder test.
- */
-public class GridBinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
-    /** */
-    private static final Unsafe UNSAFE = GridUnsafe.unsafe();
-
-    /** */
-    protected static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setClassNames(Arrays.asList(Key.class.getName(), Value.class.getName(),
-            "org.gridgain.grid.internal.util.portable.mutabletest.*"));
-
-        BinaryTypeConfiguration customIdMapper = new BinaryTypeConfiguration();
-
-        customIdMapper.setClassName(CustomIdMapper.class.getName());
-        customIdMapper.setIdMapper(new BinaryIdMapper() {
-            @Override public int typeId(String clsName) {
-                return ~PortableContext.DFLT_ID_MAPPER.typeId(clsName);
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                return typeId + ~PortableContext.DFLT_ID_MAPPER.fieldId(typeId, fieldName);
-            }
-        });
-
-        marsh.setTypeConfigurations(Collections.singleton(customIdMapper));
-
-        cfg.setMarshaller(marsh);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        startGrids(1);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopAllGrids();
-    }
-
-    /**
-     *
-     */
-    public void testAllFieldsSerialization() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-        obj.setDefaultData();
-        obj.enumArr = null;
-
-        TestObjectAllTypes deserialized = builder(toPortable(obj)).build().deserialize();
-
-        GridTestUtils.deepEquals(obj, deserialized);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testByteField() throws Exception {
-        BinaryObjectBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("byteField", (byte)1);
-
-        BinaryObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals((byte) 1, po.<Byte>field("byteField").byteValue());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testShortField() throws Exception {
-        BinaryObjectBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("shortField", (short)1);
-
-        BinaryObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals((short)1, po.<Short>field("shortField").shortValue());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testIntField() throws Exception {
-        BinaryObjectBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("intField", 1);
-
-        BinaryObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals(1, po.<Integer>field("intField").intValue());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testLongField() throws Exception {
-        BinaryObjectBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("longField", 1L);
-
-        BinaryObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals(1L, po.<Long>field("longField").longValue());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testFloatField() throws Exception {
-        BinaryObjectBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("floatField", 1.0f);
-
-        BinaryObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals(1.0f, po.<Float>field("floatField").floatValue(), 0);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDoubleField() throws Exception {
-        BinaryObjectBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("doubleField", 1.0d);
-
-        BinaryObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals(1.0d, po.<Double>field("doubleField").doubleValue(), 0);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCharField() throws Exception {
-        BinaryObjectBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("charField", (char)1);
-
-        BinaryObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals((char)1, po.<Character>field("charField").charValue());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testBooleanField() throws Exception {
-        BinaryObjectBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("booleanField", true);
-
-        BinaryObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertTrue(po.<Boolean>field("booleanField"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDecimalField() throws Exception {
-        BinaryObjectBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("decimalField", BigDecimal.TEN);
-
-        BinaryObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals(BigDecimal.TEN, po.<String>field("decimalField"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testStringField() throws Exception {
-        BinaryObjectBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("stringField", "str");
-
-        BinaryObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals("str", po.<String>field("stringField"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDateField() throws Exception {
-        Date date = new Date();
-
-        assertEquals(date, builder("C").setField("d", date).build().<Date>field("d"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testTimestampField() throws Exception {
-        Timestamp ts = new Timestamp(new Date().getTime());
-        ts.setNanos(1000);
-
-        assertEquals(ts, builder("C").setField("t", ts).build().<Timestamp>field("t"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testUuidField() throws Exception {
-        BinaryObjectBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        UUID uuid = UUID.randomUUID();
-
-        builder.setField("uuidField", uuid);
-
-        BinaryObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals(uuid, po.<UUID>field("uuidField"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testByteArrayField() throws Exception {
-        BinaryObjectBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("byteArrayField", new byte[] {1, 2, 3});
-
-        BinaryObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertTrue(Arrays.equals(new byte[] {1, 2, 3}, po.<byte[]>field("byteArrayField")));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testShortArrayField() throws Exception {
-        BinaryObjectBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("shortArrayField", new short[] {1, 2, 3});
-
-        BinaryObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertTrue(Arrays.equals(new short[] {1, 2, 3}, po.<short[]>field("shortArrayField")));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testIntArrayField() throws Exception {
-        BinaryObjectBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("intArrayField", new int[] {1, 2, 3});
-
-        BinaryObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertTrue(Arrays.equals(new int[] {1, 2, 3}, po.<int[]>field("intArrayField")));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testLongArrayField() throws Exception {
-        BinaryObjectBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("longArrayField", new long[] {1, 2, 3});
-
-        BinaryObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertTrue(Arrays.equals(new long[] {1, 2, 3}, po.<long[]>field("longArrayField")));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testFloatArrayField() throws Exception {
-        BinaryObjectBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("floatArrayField", new float[] {1, 2, 3});
-
-        BinaryObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertTrue(Arrays.equals(new float[] {1, 2, 3}, po.<float[]>field("floatArrayField")));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDoubleArrayField() throws Exception {
-        BinaryObjectBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("doubleArrayField", new double[] {1, 2, 3});
-
-        BinaryObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertTrue(Arrays.equals(new double[] {1, 2, 3}, po.<double[]>field("doubleArrayField")));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCharArrayField() throws Exception {
-        BinaryObjectBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("charArrayField", new char[] {1, 2, 3});
-
-        BinaryObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertTrue(Arrays.equals(new char[] {1, 2, 3}, po.<char[]>field("charArrayField")));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testBooleanArrayField() throws Exception {
-        BinaryObjectBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("booleanArrayField", new boolean[] {true, false});
-
-        BinaryObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        boolean[] arr = po.field("booleanArrayField");
-
-        assertEquals(2, arr.length);
-
-        assertTrue(arr[0]);
-        assertFalse(arr[1]);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDecimalArrayField() throws Exception {
-        BinaryObjectBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("decimalArrayField", new BigDecimal[] {BigDecimal.ONE, BigDecimal.TEN});
-
-        BinaryObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertTrue(Arrays.equals(new BigDecimal[] {BigDecimal.ONE, BigDecimal.TEN}, po.<String[]>field("decimalArrayField")));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testStringArrayField() throws Exception {
-        BinaryObjectBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("stringArrayField", new String[] {"str1", "str2", "str3"});
-
-        BinaryObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertTrue(Arrays.equals(new String[] {"str1", "str2", "str3"}, po.<String[]>field("stringArrayField")));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDateArrayField() throws Exception {
-        Date date1 = new Date();
-        Date date2 = new Date(date1.getTime() + 1000);
-
-        Date[] dateArr = new Date[] { date1, date2 };
-
-        assertTrue(Arrays.equals(dateArr, builder("C").setField("da", dateArr).build().<Date[]>field("da")));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testTimestampArrayField() throws Exception {
-        Timestamp ts1 = new Timestamp(new Date().getTime());
-        Timestamp ts2 = new Timestamp(new Date().getTime() + 1000);
-
-        ts1.setNanos(1000);
-        ts2.setNanos(2000);
-
-        Timestamp[] tsArr = new Timestamp[] { ts1, ts2 };
-
-        assertTrue(Arrays.equals(tsArr, builder("C").setField("ta", tsArr).build().<Timestamp[]>field("ta")));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testUuidArrayField() throws Exception {
-        BinaryObjectBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        UUID[] arr = new UUID[] {UUID.randomUUID(), UUID.randomUUID()};
-
-        builder.setField("uuidArrayField", arr);
-
-        BinaryObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertTrue(Arrays.equals(arr, po.<UUID[]>field("uuidArrayField")));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testObjectField() throws Exception {
-        BinaryObjectBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("objectField", new Value(1));
-
-        BinaryObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals(1, po.<BinaryObject>field("objectField").<Value>deserialize().i);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testObjectArrayField() throws Exception {
-        BinaryObjectBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("objectArrayField", new Value[] {new Value(1), new Value(2)});
-
-        BinaryObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        Object[] arr = po.field("objectArrayField");
-
-        assertEquals(2, arr.length);
-
-        assertEquals(1, ((BinaryObject)arr[0]).<Value>deserialize().i);
-        assertEquals(2, ((BinaryObject)arr[1]).<Value>deserialize().i);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCollectionField() throws Exception {
-        BinaryObjectBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("collectionField", Arrays.asList(new Value(1), new Value(2)));
-
-        BinaryObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        List<BinaryObject> list = po.field("collectionField");
-
-        assertEquals(2, list.size());
-
-        assertEquals(1, list.get(0).<Value>deserialize().i);
-        assertEquals(2, list.get(1).<Value>deserialize().i);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMapField() throws Exception {
-        BinaryObjectBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("mapField", F.asMap(new Key(1), new Value(1), new Key(2), new Value(2)));
-
-        BinaryObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        Map<BinaryObject, BinaryObject> map = po.field("mapField");
-
-        assertEquals(2, map.size());
-
-        for (Map.Entry<BinaryObject, BinaryObject> e : map.entrySet())
-            assertEquals(e.getKey().<Key>deserialize().i, e.getValue().<Value>deserialize().i);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testSeveralFields() throws Exception {
-        BinaryObjectBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("i", 111);
-        builder.setField("f", 111.111f);
-        builder.setField("iArr", new int[] {1, 2, 3});
-        builder.setField("obj", new Key(1));
-        builder.setField("col", Arrays.asList(new Value(1), new Value(2)));
-
-        BinaryObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals(111, po.<Integer>field("i").intValue());
-        assertEquals(111.111f, po.<Float>field("f").floatValue(), 0);
-        assertTrue(Arrays.equals(new int[] {1, 2, 3}, po.<int[]>field("iArr")));
-        assertEquals(1, po.<BinaryObject>field("obj").<Key>deserialize().i);
-
-        List<BinaryObject> list = po.field("col");
-
-        assertEquals(2, list.size());
-
-        assertEquals(1, list.get(0).<Value>deserialize().i);
-        assertEquals(2, list.get(1).<Value>deserialize().i);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testOffheapPortable() throws Exception {
-        BinaryObjectBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("i", 111);
-        builder.setField("f", 111.111f);
-        builder.setField("iArr", new int[] {1, 2, 3});
-        builder.setField("obj", new Key(1));
-        builder.setField("col", Arrays.asList(new Value(1), new Value(2)));
-
-        BinaryObject po = builder.build();
-
-        byte[] arr = ((CacheObjectBinaryProcessorImpl)(grid(0)).context().cacheObjects()).marshal(po);
-
-        long ptr = UNSAFE.allocateMemory(arr.length + 5);
-
-        try {
-            long ptr0 = ptr;
-
-            UNSAFE.putBoolean(null, ptr0++, false);
-
-            UNSAFE.putInt(ptr0, arr.length);
-
-            UNSAFE.copyMemory(arr, BYTE_ARR_OFF, null, ptr0 + 4, arr.length);
-
-            BinaryObject offheapObj = (BinaryObject)
-                ((CacheObjectBinaryProcessorImpl)(grid(0)).context().cacheObjects()).unmarshal(ptr, false);
-
-            assertEquals(BinaryObjectOffheapImpl.class, offheapObj.getClass());
-
-            assertEquals("class".hashCode(), offheapObj.typeId());
-            assertEquals(100, offheapObj.hashCode());
-
-            assertEquals(111, offheapObj.<Integer>field("i").intValue());
-            assertEquals(111.111f, offheapObj.<Float>field("f").floatValue(), 0);
-            assertTrue(Arrays.equals(new int[] {1, 2, 3}, offheapObj.<int[]>field("iArr")));
-            assertEquals(1, offheapObj.<BinaryObject>field("obj").<Key>deserialize().i);
-
-            List<BinaryObject> list = offheapObj.field("col");
-
-            assertEquals(2, list.size());
-
-            assertEquals(1, list.get(0).<Value>deserialize().i);
-            assertEquals(2, list.get(1).<Value>deserialize().i);
-
-            assertEquals(po, offheapObj);
-            assertEquals(offheapObj, po);
-        }
-        finally {
-            UNSAFE.freeMemory(ptr);
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testBuildAndDeserialize() throws Exception {
-        BinaryObjectBuilder builder = builder(Value.class.getName());
-
-        builder.hashCode(100);
-
-        builder.setField("i", 1);
-
-        BinaryObject po = builder.build();
-
-        assertEquals("value".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals(1, po.<Value>deserialize().i);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMetaData2() throws Exception {
-        BinaryObjectBuilder builder = builder("org.test.MetaTest2");
-
-        builder.setField("objectField", "a", Object.class);
-
-        BinaryObject po = builder.build();
-
-        BinaryType meta = po.type();
-
-        assertEquals("MetaTest2", meta.typeName());
-        assertEquals("Object", meta.fieldTypeName("objectField"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMetaData() throws Exception {
-        BinaryObjectBuilder builder = builder("org.test.MetaTest");
-
-        builder.hashCode(100);
-
-        builder.setField("intField", 1);
-        builder.setField("byteArrayField", new byte[] {1, 2, 3});
-
-        BinaryObject po = builder.build();
-
-        BinaryType meta = po.type();
-
-        assertEquals("MetaTest", meta.typeName());
-
-        Collection<String> fields = meta.fieldNames();
-
-        assertEquals(2, fields.size());
-
-        assertTrue(fields.contains("intField"));
-        assertTrue(fields.contains("byteArrayField"));
-
-        assertEquals("int", meta.fieldTypeName("intField"));
-        assertEquals("byte[]", meta.fieldTypeName("byteArrayField"));
-
-        builder = builder("org.test.MetaTest");
-
-        builder.hashCode(100);
-
-        builder.setField("intField", 2);
-        builder.setField("uuidField", UUID.randomUUID());
-
-        po = builder.build();
-
-        meta = po.type();
-
-        assertEquals("MetaTest", meta.typeName());
-
-        fields = meta.fieldNames();
-
-        assertEquals(3, fields.size());
-
-        assertTrue(fields.contains("intField"));
-        assertTrue(fields.contains("byteArrayField"));
-        assertTrue(fields.contains("uuidField"));
-
-        assertEquals("int", meta.fieldTypeName("intField"));
-        assertEquals("byte[]", meta.fieldTypeName("byteArrayField"));
-        assertEquals("UUID", meta.fieldTypeName("uuidField"));
-    }
-
-    /**
-     *
-     */
-    public void testGetFromCopiedObj() {
-        BinaryObject objStr = builder(TestObjectAllTypes.class.getName()).setField("str", "aaa").build();
-
-        BinaryObjectBuilderImpl builder = builder(objStr);
-        assertEquals("aaa", builder.getField("str"));
-
-        builder.setField("str", "bbb");
-        assertEquals("bbb", builder.getField("str"));
-
-        assertNull(builder.getField("i_"));
-        assertEquals("bbb", builder.build().<TestObjectAllTypes>deserialize().str);
-    }
-
-    /**
-     *
-     */
-    public void testCopyFromInnerObjects() {
-        ArrayList<Object> list = new ArrayList<>();
-        list.add(new TestObjectAllTypes());
-        list.add(list.get(0));
-
-        TestObjectContainer c = new TestObjectContainer(list);
-
-        BinaryObjectBuilderImpl builder = builder(toPortable(c));
-        builder.<List>getField("foo").add("!!!");
-
-        BinaryObject res = builder.build();
-
-        TestObjectContainer deserialized = res.deserialize();
-
-        List deserializedList = (List)deserialized.foo;
-
-        assertSame(deserializedList.get(0), deserializedList.get(1));
-        assertEquals("!!!", deserializedList.get(2));
-        assertTrue(deserializedList.get(0) instanceof TestObjectAllTypes);
-    }
-
-    /**
-     *
-     */
-    public void testSetPortableObject() {
-        BinaryObject portableObj = builder(TestObjectContainer.class.getName())
-            .setField("foo", toPortable(new TestObjectAllTypes()))
-            .build();
-
-        assertTrue(portableObj.<TestObjectContainer>deserialize().foo instanceof TestObjectAllTypes);
-    }
-
-    /**
-     *
-     */
-    public void testPlainPortableObjectCopyFrom() {
-        TestObjectPlainPortable obj = new TestObjectPlainPortable(toPortable(new TestObjectAllTypes()));
-
-        BinaryObjectBuilderImpl builder = builder(toPortable(obj));
-        assertTrue(builder.getField("plainPortable") instanceof BinaryObject);
-
-        TestObjectPlainPortable deserialized = builder.build().deserialize();
-        assertTrue(deserialized.plainPortable instanceof BinaryObject);
-    }
-
-    /**
-     *
-     */
-    public void testRemoveFromNewObject() {
-        BinaryObjectBuilder builder = builder(TestObjectAllTypes.class.getName());
-
-        builder.setField("str", "a");
-
-        builder.removeField("str");
-
-        assertNull(builder.build().<TestObjectAllTypes>deserialize().str);
-    }
-
-    /**
-     *
-     */
-    public void testRemoveFromExistingObject() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-        obj.setDefaultData();
-        obj.enumArr = null;
-
-        BinaryObjectBuilder builder = builder(toPortable(obj));
-
-        builder.removeField("str");
-
-        assertNull(builder.build().<TestObjectAllTypes>deserialize().str);
-    }
-
-    /**
-     *
-     */
-    public void testRemoveFromExistingObjectAfterGet() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-        obj.setDefaultData();
-        obj.enumArr = null;
-
-        BinaryObjectBuilderImpl builder = builder(toPortable(obj));
-
-        builder.getField("i_");
-
-        builder.removeField("str");
-
-        assertNull(builder.build().<TestObjectAllTypes>deserialize().str);
-    }
-
-    /**
-     * @throws IgniteCheckedException If any error occurs.
-     */
-    public void testDontBrokeCyclicDependency() throws IgniteCheckedException {
-        TestObjectOuter outer = new TestObjectOuter();
-        outer.inner = new TestObjectInner();
-        outer.inner.outer = outer;
-        outer.foo = "a";
-
-        BinaryObjectBuilder builder = builder(toPortable(outer));
-
-        builder.setField("foo", "b");
-
-        TestObjectOuter res = builder.build().deserialize();
-
-        assertEquals("b", res.foo);
-        assertSame(res, res.inner.outer);
-    }
-
-    /**
-     * @return Portables.
-     */
-    private IgniteBinary portables() {
-        return grid(0).binary();
-    }
-
-    /**
-     * @param obj Object.
-     * @return Portable object.
-     */
-    private BinaryObject toPortable(Object obj) {
-        return portables().toBinary(obj);
-    }
-
-    /**
-     * @return Builder.
-     */
-    private <T> BinaryObjectBuilder builder(String clsName) {
-        return portables().builder(clsName);
-    }
-
-    /**
-     * @return Builder.
-     */
-    private <T> BinaryObjectBuilderImpl builder(BinaryObject obj) {
-        return (BinaryObjectBuilderImpl)portables().builder(obj);
-    }
-
-    /**
-     *
-     */
-    private static class CustomIdMapper {
-        /** */
-        private String str = "a";
-
-        /** */
-        private int i = 10;
-    }
-
-    /**
-     */
-    private static class Key {
-        /** */
-        private int i;
-
-        /**
-         */
-        private Key() {
-            // No-op.
-        }
-
-        /**
-         * @param i Index.
-         */
-        private Key(int i) {
-            this.i = i;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object o) {
-            if (this == o)
-                return true;
-
-            if (o == null || getClass() != o.getClass())
-                return false;
-
-            Key key = (Key)o;
-
-            return i == key.i;
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            return i;
-        }
-    }
-
-    /**
-     */
-    private static class Value {
-        /** */
-        private int i;
-
-        /**
-         */
-        private Value() {
-            // No-op.
-        }
-
-        /**
-         * @param i Index.
-         */
-        private Value(int i) {
-            this.i = i;
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
index 747f8ea..9c0824e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
@@ -45,7 +45,7 @@ public class GridPortableMarshallerCtxDisabledSelfTest extends GridCommonAbstrac
         PortableMarshaller marsh = new PortableMarshaller();
         marsh.setContext(new MarshallerContextWithNoStorage());
 
-        PortableContext context = new PortableContext(BinaryNoopMetadataHandler.instance(), new IgniteConfiguration());
+        PortableContext context = new PortableContext(BinaryCachingMetadataHandler.create(), new IgniteConfiguration());
 
         IgniteUtils.invoke(PortableMarshaller.class, marsh, "setPortableContext", context);
 


[08/13] ignite git commit: ignite-1282 - test added

Posted by ag...@apache.org.
ignite-1282 - test added


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

Branch: refs/heads/ignite-1945
Commit: a7b22f8387c95931b0aa407eb951b9d2bc54c33c
Parents: 0b4a8f8
Author: S.Vladykin <sv...@gridgain.com>
Authored: Wed Nov 18 20:08:02 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Wed Nov 18 20:08:02 2015 +0300

----------------------------------------------------------------------
 .../query/IgniteSqlSplitterSelfTest.java        | 54 ++++++++++++++++++++
 1 file changed, 54 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a7b22f83/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
index 75112fd..0868fe6 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
@@ -227,6 +227,39 @@ public class IgniteSqlSplitterSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     *
+     */
+    public void testFunctionNpe() {
+        // TODO IGNITE-1886
+        IgniteCache<Integer, User> userCache = ignite(0).createCache(
+            cacheConfig("UserCache", true, Integer.class, User.class));
+        IgniteCache<Integer, UserOrder> userOrderCache = ignite(0).createCache(
+            cacheConfig("UserOrderCache", true, Integer.class, UserOrder.class));
+        IgniteCache<Integer, OrderGood> orderGoodCache = ignite(0).createCache(
+            cacheConfig("OrderGoodCache", true, Integer.class, OrderGood.class));
+
+        try {
+            String sql =
+                "SELECT a.* FROM (" +
+                    "SELECT CASE WHEN u.id < 100 THEN u.id ELSE ug.id END id " +
+                    "FROM \"UserCache\".User u, UserOrder ug " +
+                    "WHERE u.id = ug.userId" +
+                    ") a, (" +
+                    "SELECT CASE WHEN og.goodId < 5 THEN 100 ELSE og.goodId END id " +
+                    "FROM UserOrder ug, \"OrderGoodCache\".OrderGood og " +
+                    "WHERE ug.id = og.orderId) b " +
+                    "WHERE a.id = b.id";
+
+            userOrderCache.query(new SqlFieldsQuery(sql)).getAll();
+        }
+        finally {
+            userCache.destroy();
+            userOrderCache.destroy();
+            orderGoodCache.destroy();
+        }
+    }
+
+    /**
      * Test value.
      */
     private static class GroupIndexTestValue implements Serializable {
@@ -245,4 +278,25 @@ public class IgniteSqlSplitterSelfTest extends GridCommonAbstractTest {
             this.b = b;
         }
     }
+
+    private static class User implements Serializable {
+        @QuerySqlField
+        private int id;
+    }
+
+    private static class UserOrder implements Serializable {
+        @QuerySqlField
+        private int id;
+
+        @QuerySqlField
+        private int userId;
+    }
+
+    private static class OrderGood implements Serializable {
+        @QuerySqlField
+        private int orderId;
+
+        @QuerySqlField
+        private int goodId;
+    }
 }
\ No newline at end of file


[11/13] ignite git commit: Merge branch ignite-1282 into ignite-1945

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a511fa17/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderAdditionalSelfTest.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderAdditionalSelfTest.java
index 0000000,0e31451..356a25b
mode 000000,100644..100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderAdditionalSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderAdditionalSelfTest.java
@@@ -1,0 -1,1291 +1,1292 @@@
+ /*
+  * 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 com.google.common.collect.ImmutableMap;
+ import com.google.common.collect.Lists;
+ import com.google.common.collect.Maps;
+ import com.google.common.collect.Sets;
+ import java.lang.reflect.Field;
+ import java.math.BigDecimal;
+ import java.sql.Timestamp;
+ import java.util.ArrayList;
+ import java.util.Arrays;
+ import java.util.Collections;
+ import java.util.Date;
+ import java.util.HashSet;
+ import java.util.LinkedList;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Objects;
+ import java.util.Set;
+ import java.util.UUID;
+ import org.apache.ignite.IgniteCheckedException;
+ import org.apache.ignite.IgniteBinary;
++import org.apache.ignite.configuration.BinaryConfiguration;
+ import org.apache.ignite.configuration.CacheConfiguration;
+ import org.apache.ignite.configuration.IgniteConfiguration;
+ import org.apache.ignite.internal.portable.builder.PortableBuilderEnum;
+ import org.apache.ignite.internal.portable.builder.BinaryObjectBuilderImpl;
+ import org.apache.ignite.internal.portable.mutabletest.GridBinaryMarshalerAwareTestClass;
+ import org.apache.ignite.internal.processors.cache.portable.CacheObjectBinaryProcessorImpl;
+ import org.apache.ignite.internal.processors.cache.portable.IgniteBinaryImpl;
+ import org.apache.ignite.internal.util.lang.GridMapEntry;
 -import org.apache.ignite.marshaller.portable.PortableMarshaller;
++import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+ import org.apache.ignite.binary.BinaryObjectBuilder;
+ import org.apache.ignite.binary.BinaryType;
+ import org.apache.ignite.binary.BinaryObject;
+ import org.apache.ignite.testframework.GridTestUtils;
+ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+ import org.junit.Assert;
+ 
+ import static org.apache.ignite.cache.CacheMode.REPLICATED;
+ import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.Address;
+ import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.AddressBook;
+ import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.Company;
+ import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectAllTypes;
+ import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectArrayList;
+ import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectContainer;
+ import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectEnum;
+ import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectInner;
+ import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectOuter;
+ 
+ /**
+  *
+  */
+ public class BinaryObjectBuilderAdditionalSelfTest extends GridCommonAbstractTest {
+     /** {@inheritDoc} */
+     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+         IgniteConfiguration cfg = super.getConfiguration(gridName);
+ 
+         CacheConfiguration cacheCfg = new CacheConfiguration();
+ 
+         cacheCfg.setCacheMode(REPLICATED);
+ 
+         cfg.setCacheConfiguration(cacheCfg);
+ 
 -        PortableMarshaller marsh = new PortableMarshaller();
++        BinaryConfiguration bCfg = new BinaryConfiguration();
+ 
 -        marsh.setCompactFooter(compactFooter());
++        bCfg.setCompactFooter(compactFooter());
++        
++        bCfg.setClassNames(Arrays.asList("org.apache.ignite.internal.portable.mutabletest.*"));
+ 
 -        marsh.setClassNames(Arrays.asList("org.apache.ignite.internal.portable.mutabletest.*"));
 -
 -        cfg.setMarshaller(marsh);
++        cfg.setMarshaller(new BinaryMarshaller());
+ 
+         return cfg;
+     }
+ 
+     /** {@inheritDoc} */
+     @Override protected void beforeTestsStarted() throws Exception {
+         startGrids(1);
+     }
+ 
+     /** {@inheritDoc} */
+     @Override protected void afterTestsStopped() throws Exception {
+         stopAllGrids();
+     }
+ 
+     /** {@inheritDoc} */
+     @Override protected void afterTest() throws Exception {
+         jcache(0).clear();
+     }
+ 
+     /**
+      * @return Compact footer.
+      */
+     protected boolean compactFooter() {
+         return true;
+     }
+ 
+     /**
+      * @return Portables API.
+      */
+     protected IgniteBinary portables() {
+         return grid(0).binary();
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testSimpleTypeFieldRead() throws Exception {
+         TestObjectAllTypes exp = new TestObjectAllTypes();
+ 
+         exp.setDefaultData();
+ 
+         BinaryObjectBuilder mutPo = wrap(exp);
+ 
+         for (Field field : TestObjectAllTypes.class.getDeclaredFields()) {
+             Object expVal = field.get(exp);
+             Object actVal = mutPo.getField(field.getName());
+ 
+             switch (field.getName()) {
+                 case "anEnum":
+                     assertEquals(((PortableBuilderEnum)actVal).getOrdinal(), ((Enum)expVal).ordinal());
+                     break;
+ 
+                 case "enumArr": {
+                     PortableBuilderEnum[] actArr = (PortableBuilderEnum[])actVal;
+                     Enum[] expArr = (Enum[])expVal;
+ 
+                     assertEquals(expArr.length, actArr.length);
+ 
+                     for (int i = 0; i < actArr.length; i++)
+                         assertEquals(expArr[i].ordinal(), actArr[i].getOrdinal());
+ 
+                     break;
+                 }
+ 
+                 case "entry":
+                     assertEquals(((Map.Entry)expVal).getKey(), ((Map.Entry)actVal).getKey());
+                     assertEquals(((Map.Entry)expVal).getValue(), ((Map.Entry)actVal).getValue());
+                     break;
+ 
+                 default:
+                     assertTrue(field.getName(), Objects.deepEquals(expVal, actVal));
+                     break;
+             }
+         }
+     }
+ 
+     /**
+      *
+      */
+     public void testSimpleTypeFieldSerialize() {
+         TestObjectAllTypes exp = new TestObjectAllTypes();
+ 
+         exp.setDefaultData();
+ 
+         BinaryObjectBuilderImpl mutPo = wrap(exp);
+ 
+         TestObjectAllTypes res = mutPo.build().deserialize();
+ 
+         GridTestUtils.deepEquals(exp, res);
+     }
+ 
+     /**
+      * @throws Exception If any error occurs.
+      */
+     public void testSimpleTypeFieldOverride() throws Exception {
+         TestObjectAllTypes exp = new TestObjectAllTypes();
+ 
+         exp.setDefaultData();
+ 
+         BinaryObjectBuilderImpl mutPo = wrap(new TestObjectAllTypes());
+ 
+         for (Field field : TestObjectAllTypes.class.getDeclaredFields())
+             mutPo.setField(field.getName(), field.get(exp));
+ 
+         TestObjectAllTypes res = mutPo.build().deserialize();
+ 
+         GridTestUtils.deepEquals(exp, res);
+     }
+ 
+     /**
+      * @throws Exception If any error occurs.
+      */
+     public void testSimpleTypeFieldSetNull() throws Exception {
+         TestObjectAllTypes exp = new TestObjectAllTypes();
+ 
+         exp.setDefaultData();
+ 
+         BinaryObjectBuilderImpl mutPo = wrap(exp);
+ 
+         for (Field field : TestObjectAllTypes.class.getDeclaredFields()) {
+             if (!field.getType().isPrimitive())
+                 mutPo.setField(field.getName(), null);
+         }
+ 
+         TestObjectAllTypes res = mutPo.build().deserialize();
+ 
+         for (Field field : TestObjectAllTypes.class.getDeclaredFields()) {
+             if (!field.getType().isPrimitive())
+                 assertNull(field.getName(), field.get(res));
+         }
+     }
+ 
+     /**
+      * @throws IgniteCheckedException If any error occurs.
+      */
+     public void testMakeCyclicDependency() throws IgniteCheckedException {
+         TestObjectOuter outer = new TestObjectOuter();
+         outer.inner = new TestObjectInner();
+ 
+         BinaryObjectBuilderImpl mutOuter = wrap(outer);
+ 
+         BinaryObjectBuilderImpl mutInner = mutOuter.getField("inner");
+ 
+         mutInner.setField("outer", mutOuter);
+         mutInner.setField("foo", mutInner);
+ 
+         TestObjectOuter res = mutOuter.build().deserialize();
+ 
+         assertEquals(res, res.inner.outer);
+         assertEquals(res.inner, res.inner.foo);
+     }
+ 
+     /**
+      *
+      */
+     public void testDateArrayModification() {
+         TestObjectAllTypes obj = new TestObjectAllTypes();
+ 
+         obj.dateArr =  new Date[] {new Date(11111), new Date(11111), new Date(11111)};
+ 
+         BinaryObjectBuilderImpl mutObj = wrap(obj);
+ 
+         Date[] arr = mutObj.getField("dateArr");
+         arr[0] = new Date(22222);
+ 
+         TestObjectAllTypes res = mutObj.build().deserialize();
+ 
+         Assert.assertArrayEquals(new Date[] {new Date(22222), new Date(11111), new Date(11111)}, res.dateArr);
+     }
+ 
+     /**
+      *
+      */
+     public void testTimestampArrayModification() {
+         TestObjectAllTypes obj = new TestObjectAllTypes();
+ 
+         obj.tsArr = new Timestamp[] {new Timestamp(111222333), new Timestamp(222333444)};
+ 
+         BinaryObjectBuilderImpl mutObj = wrap(obj);
+ 
+         Timestamp[] arr = mutObj.getField("tsArr");
+         arr[0] = new Timestamp(333444555);
+ 
+         TestObjectAllTypes res = mutObj.build().deserialize();
+ 
+         Assert.assertArrayEquals(new Timestamp[] {new Timestamp(333444555), new Timestamp(222333444)}, res.tsArr);
+     }
+ 
+     /**
+      *
+      */
+     public void testUUIDArrayModification() {
+         TestObjectAllTypes obj = new TestObjectAllTypes();
+ 
+         obj.uuidArr = new UUID[] {new UUID(1, 1), new UUID(1, 1), new UUID(1, 1)};
+ 
+         BinaryObjectBuilderImpl mutObj = wrap(obj);
+ 
+         UUID[] arr = mutObj.getField("uuidArr");
+         arr[0] = new UUID(2, 2);
+ 
+         TestObjectAllTypes res = mutObj.build().deserialize();
+ 
+         Assert.assertArrayEquals(new UUID[] {new UUID(2, 2), new UUID(1, 1), new UUID(1, 1)}, res.uuidArr);
+     }
+ 
+     /**
+      *
+      */
+     public void testDecimalArrayModification() {
+         TestObjectAllTypes obj = new TestObjectAllTypes();
+ 
+         obj.bdArr = new BigDecimal[] {new BigDecimal(1000), new BigDecimal(1000), new BigDecimal(1000)};
+ 
+         BinaryObjectBuilderImpl mutObj = wrap(obj);
+ 
+         BigDecimal[] arr = mutObj.getField("bdArr");
+         arr[0] = new BigDecimal(2000);
+ 
+         TestObjectAllTypes res = mutObj.build().deserialize();
+ 
+         Assert.assertArrayEquals(new BigDecimal[] {new BigDecimal(1000), new BigDecimal(1000), new BigDecimal(1000)},
+             res.bdArr);
+     }
+ 
+     /**
+      *
+      */
+     public void testBooleanArrayModification() {
+         TestObjectAllTypes obj = new TestObjectAllTypes();
+ 
+         obj.zArr = new boolean[] {false, false, false};
+ 
+         BinaryObjectBuilderImpl mutObj = wrap(obj);
+ 
+         boolean[] arr = mutObj.getField("zArr");
+         arr[0] = true;
+ 
+         TestObjectAllTypes res = mutObj.build().deserialize();
+ 
+         boolean[] expected = new boolean[] {true, false, false};
+ 
+         assertEquals(expected.length, res.zArr.length);
+ 
+         for (int i = 0; i < expected.length; i++)
+             assertEquals(expected[i], res.zArr[i]);
+     }
+ 
+     /**
+      *
+      */
+     public void testCharArrayModification() {
+         TestObjectAllTypes obj = new TestObjectAllTypes();
+ 
+         obj.cArr = new char[] {'a', 'a', 'a'};
+ 
+         BinaryObjectBuilderImpl mutObj = wrap(obj);
+ 
+         char[] arr = mutObj.getField("cArr");
+         arr[0] = 'b';
+ 
+         TestObjectAllTypes res = mutObj.build().deserialize();
+ 
+         Assert.assertArrayEquals(new char[] {'b', 'a', 'a'}, res.cArr);
+     }
+ 
+     /**
+      *
+      */
+     public void testDoubleArrayModification() {
+         TestObjectAllTypes obj = new TestObjectAllTypes();
+ 
+         obj.dArr = new double[] {1.0, 1.0, 1.0};
+ 
+         BinaryObjectBuilderImpl mutObj = wrap(obj);
+ 
+         double[] arr = mutObj.getField("dArr");
+         arr[0] = 2.0;
+ 
+         TestObjectAllTypes res = mutObj.build().deserialize();
+ 
+         Assert.assertArrayEquals(new double[] {2.0, 1.0, 1.0}, res.dArr, 0);
+     }
+ 
+     /**
+      *
+      */
+     public void testFloatArrayModification() {
+         TestObjectAllTypes obj = new TestObjectAllTypes();
+ 
+         obj.fArr = new float[] {1.0f, 1.0f, 1.0f};
+ 
+         BinaryObjectBuilderImpl mutObj = wrap(obj);
+ 
+         float[] arr = mutObj.getField("fArr");
+         arr[0] = 2.0f;
+ 
+         BinaryObject resBinary = mutObj.build();
+ 
+         TestObjectAllTypes res = resBinary.deserialize();
+ 
+         Assert.assertArrayEquals(new float[] {2.0f, 1.0f, 1.0f}, res.fArr, 0);
+     }
+ 
+     /**
+      *
+      */
+     public void testLongArrayModification() {
+         TestObjectAllTypes obj = new TestObjectAllTypes();
+ 
+         obj.lArr = new long[] {1, 1, 1};
+ 
+         BinaryObjectBuilderImpl mutObj = wrap(obj);
+ 
+         long[] arr = mutObj.getField("lArr");
+         arr[0] = 2;
+ 
+         TestObjectAllTypes res = mutObj.build().deserialize();
+ 
+         Assert.assertArrayEquals(new long[] {2, 1, 1}, res.lArr);
+     }
+ 
+     /**
+      *
+      */
+     public void testIntArrayModification() {
+         TestObjectAllTypes obj = new TestObjectAllTypes();
+ 
+         obj.iArr = new int[] {1, 1, 1};
+ 
+         BinaryObjectBuilderImpl mutObj = wrap(obj);
+ 
+         int[] arr = mutObj.getField("iArr");
+         arr[0] = 2;
+ 
+         TestObjectAllTypes res = mutObj.build().deserialize();
+ 
+         Assert.assertArrayEquals(new int[] {2, 1, 1}, res.iArr);
+     }
+ 
+     /**
+      *
+      */
+     public void testShortArrayModification() {
+         TestObjectAllTypes obj = new TestObjectAllTypes();
+ 
+         obj.sArr = new short[] {1, 1, 1};
+ 
+         BinaryObjectBuilderImpl mutObj = wrap(obj);
+ 
+         short[] arr = mutObj.getField("sArr");
+         arr[0] = 2;
+ 
+         TestObjectAllTypes res = mutObj.build().deserialize();
+ 
+         Assert.assertArrayEquals(new short[] {2, 1, 1}, res.sArr);
+     }
+ 
+     /**
+      *
+      */
+     public void testByteArrayModification() {
+         TestObjectAllTypes obj = new TestObjectAllTypes();
+ 
+         obj.bArr = new byte[] {1, 1, 1};
+ 
+         BinaryObjectBuilderImpl mutObj = wrap(obj);
+ 
+         byte[] arr = mutObj.getField("bArr");
+         arr[0] = 2;
+ 
+         TestObjectAllTypes res = mutObj.build().deserialize();
+ 
+         Assert.assertArrayEquals(new byte[] {2, 1, 1}, res.bArr);
+     }
+ 
+     /**
+      *
+      */
+     public void testStringArrayModification() {
+         TestObjectAllTypes obj = new TestObjectAllTypes();
+ 
+         obj.strArr = new String[] {"a", "a", "a"};
+ 
+         BinaryObjectBuilderImpl mutObj = wrap(obj);
+ 
+         String[] arr = mutObj.getField("strArr");
+         arr[0] = "b";
+ 
+         TestObjectAllTypes res = mutObj.build().deserialize();
+ 
+         Assert.assertArrayEquals(new String[] {"b", "a", "a"}, res.strArr);
+     }
+ 
+     /**
+      *
+      */
+     public void testModifyObjectArray() {
+         TestObjectContainer obj = new TestObjectContainer();
+         obj.foo = new Object[] {"a"};
+ 
+         BinaryObjectBuilderImpl mutObj = wrap(obj);
+ 
+         Object[] arr = mutObj.getField("foo");
+ 
+         Assert.assertArrayEquals(new Object[] {"a"}, arr);
+ 
+         arr[0] = "b";
+ 
+         TestObjectContainer res = mutObj.build().deserialize();
+ 
+         Assert.assertArrayEquals(new Object[] {"b"}, (Object[])res.foo);
+     }
+ 
+     /**
+      *
+      */
+     public void testOverrideObjectArrayField() {
+         BinaryObjectBuilderImpl mutObj = wrap(new TestObjectContainer());
+ 
+         Object[] createdArr = {mutObj, "a", 1, new String[] {"s", "s"}, new byte[] {1, 2}, new UUID(3, 0)};
+ 
+         mutObj.setField("foo", createdArr.clone());
+ 
+         TestObjectContainer res = mutObj.build().deserialize();
+ 
+         createdArr[0] = res;
+ 
+         assertTrue(Objects.deepEquals(createdArr, res.foo));
+     }
+ 
+     /**
+      *
+      */
+     public void testDeepArray() {
+         TestObjectContainer obj = new TestObjectContainer();
+         obj.foo = new Object[] {new Object[] {"a", obj}};
+ 
+         BinaryObjectBuilderImpl mutObj = wrap(obj);
+ 
+         Object[] arr = (Object[])mutObj.<Object[]>getField("foo")[0];
+ 
+         assertEquals("a", arr[0]);
+         assertSame(mutObj, arr[1]);
+ 
+         arr[0] = mutObj;
+ 
+         TestObjectContainer res = mutObj.build().deserialize();
+ 
+         arr = (Object[])((Object[])res.foo)[0];
+ 
+         assertSame(arr[0], res);
+         assertSame(arr[0], arr[1]);
+     }
+ 
+     /**
+      *
+      */
+     public void testArrayListRead() {
+         TestObjectContainer obj = new TestObjectContainer();
+         obj.foo = Lists.newArrayList(obj, "a");
+ 
+         BinaryObjectBuilderImpl mutObj = wrap(obj);
+ 
+         List<Object> list = mutObj.getField("foo");
+ 
+         assert list.equals(Lists.newArrayList(mutObj, "a"));
+     }
+ 
+     /**
+      *
+      */
+     public void testArrayListOverride() {
+         TestObjectContainer obj = new TestObjectContainer();
+ 
+         BinaryObjectBuilderImpl mutObj = wrap(obj);
+ 
+         ArrayList<Object> list = Lists.newArrayList(mutObj, "a", Lists.newArrayList(1, 2));
+ 
+         mutObj.setField("foo", list);
+ 
+         TestObjectContainer res = mutObj.build().deserialize();
+ 
+         list.set(0, res);
+ 
+         assertNotSame(list, res.foo);
+         assertEquals(list, res.foo);
+     }
+ 
+     /**
+      *
+      */
+     public void testArrayListModification() {
+         TestObjectContainer obj = new TestObjectContainer();
+         obj.foo = Lists.newArrayList("a", "b", "c");
+ 
+         BinaryObjectBuilderImpl mutObj = wrap(obj);
+ 
+         List<String> list = mutObj.getField("foo");
+ 
+         list.add("!"); // "a", "b", "c", "!"
+         list.add(0, "_"); // "_", "a", "b", "c", "!"
+ 
+         String s = list.remove(1); // "_", "b", "c", "!"
+         assertEquals("a", s);
+ 
+         assertEquals(Arrays.asList("c", "!"), list.subList(2, 4));
+         assertEquals(1, list.indexOf("b"));
+         assertEquals(1, list.lastIndexOf("b"));
+ 
+         TestObjectContainer res = mutObj.build().deserialize();
+ 
+         assertTrue(res.foo instanceof ArrayList);
+         assertEquals(Arrays.asList("_", "b", "c", "!"), res.foo);
+     }
+ 
+     /**
+      *
+      */
+     public void testArrayListClear() {
+         TestObjectContainer obj = new TestObjectContainer();
+         obj.foo = Lists.newArrayList("a", "b", "c");
+ 
+         BinaryObjectBuilderImpl mutObj = wrap(obj);
+ 
+         List<String> list = mutObj.getField("foo");
+ 
+         list.clear();
+ 
+         assertEquals(Collections.emptyList(), mutObj.build().<TestObjectContainer>deserialize().foo);
+     }
+ 
+     /**
+      *
+      */
+     public void testArrayListWriteUnmodifiable() {
+         TestObjectContainer obj = new TestObjectContainer();
+ 
+         ArrayList<Object> src = Lists.newArrayList(obj, "a", "b", "c");
+ 
+         obj.foo = src;
+ 
+         BinaryObjectBuilderImpl mutObj = wrap(obj);
+ 
+         TestObjectContainer deserialized = mutObj.build().deserialize();
+ 
+         List<Object> res = (List<Object>)deserialized.foo;
+ 
+         src.set(0, deserialized);
+ 
+         assertEquals(src, res);
+     }
+ 
+     /**
+      *
+      */
+     public void testLinkedListRead() {
+         TestObjectContainer obj = new TestObjectContainer();
+         obj.foo = Lists.newLinkedList(Arrays.asList(obj, "a"));
+ 
+         BinaryObjectBuilderImpl mutObj = wrap(obj);
+ 
+         List<Object> list = mutObj.getField("foo");
+ 
+         assert list.equals(Lists.newLinkedList(Arrays.asList(mutObj, "a")));
+     }
+ 
+     /**
+      *
+      */
+     public void testLinkedListOverride() {
+         TestObjectContainer obj = new TestObjectContainer();
+ 
+         BinaryObjectBuilderImpl mutObj = wrap(obj);
+ 
+         List<Object> list = Lists.newLinkedList(Arrays.asList(mutObj, "a", Lists.newLinkedList(Arrays.asList(1, 2))));
+ 
+         mutObj.setField("foo", list);
+ 
+         TestObjectContainer res = mutObj.build().deserialize();
+ 
+         list.set(0, res);
+ 
+         assertNotSame(list, res.foo);
+         assertEquals(list, res.foo);
+     }
+ 
+     /**
+      *
+      */
+     public void testLinkedListModification() {
+         TestObjectContainer obj = new TestObjectContainer();
+ 
+         obj.foo = Lists.newLinkedList(Arrays.asList("a", "b", "c"));
+ 
+         BinaryObjectBuilderImpl mutObj = wrap(obj);
+ 
+         List<String> list = mutObj.getField("foo");
+ 
+         list.add("!"); // "a", "b", "c", "!"
+         list.add(0, "_"); // "_", "a", "b", "c", "!"
+ 
+         String s = list.remove(1); // "_", "b", "c", "!"
+         assertEquals("a", s);
+ 
+         assertEquals(Arrays.asList("c", "!"), list.subList(2, 4));
+         assertEquals(1, list.indexOf("b"));
+         assertEquals(1, list.lastIndexOf("b"));
+ 
+         TestObjectContainer res = mutObj.build().deserialize();
+ 
+         assertTrue(res.foo instanceof LinkedList);
+         assertEquals(Arrays.asList("_", "b", "c", "!"), res.foo);
+     }
+ 
+     /**
+      *
+      */
+     public void testLinkedListWriteUnmodifiable() {
+         TestObjectContainer obj = new TestObjectContainer();
+ 
+         LinkedList<Object> src = Lists.newLinkedList(Arrays.asList(obj, "a", "b", "c"));
+ 
+         obj.foo = src;
+ 
+         BinaryObjectBuilderImpl mutObj = wrap(obj);
+ 
+         TestObjectContainer deserialized = mutObj.build().deserialize();
+ 
+         List<Object> res = (List<Object>)deserialized.foo;
+ 
+         src.set(0, deserialized);
+ 
+         assertEquals(src, res);
+     }
+ 
+     /**
+      *
+      */
+     public void testHashSetRead() {
+         TestObjectContainer obj = new TestObjectContainer();
+         obj.foo = Sets.newHashSet(obj, "a");
+ 
+         BinaryObjectBuilderImpl mutObj = wrap(obj);
+ 
+         Set<Object> set = mutObj.getField("foo");
+ 
+         assert set.equals(Sets.newHashSet(mutObj, "a"));
+     }
+ 
+     /**
+      *
+      */
+     public void testHashSetOverride() {
+         TestObjectContainer obj = new TestObjectContainer();
+ 
+         BinaryObjectBuilderImpl mutObj = wrap(obj);
+ 
+         Set<Object> c = Sets.newHashSet(mutObj, "a", Sets.newHashSet(1, 2));
+ 
+         mutObj.setField("foo", c);
+ 
+         TestObjectContainer res = mutObj.build().deserialize();
+ 
+         c.remove(mutObj);
+         c.add(res);
+ 
+         assertNotSame(c, res.foo);
+         assertEquals(c, res.foo);
+     }
+ 
+     /**
+      *
+      */
+     public void testHashSetModification() {
+         TestObjectContainer obj = new TestObjectContainer();
+         obj.foo = Sets.newHashSet("a", "b", "c");
+ 
+         BinaryObjectBuilderImpl mutObj = wrap(obj);
+ 
+         Set<String> set = mutObj.getField("foo");
+ 
+         set.remove("b");
+         set.add("!");
+ 
+         assertEquals(Sets.newHashSet("a", "!", "c"), set);
+         assertTrue(set.contains("a"));
+         assertTrue(set.contains("!"));
+ 
+         TestObjectContainer res = mutObj.build().deserialize();
+ 
+         assertTrue(res.foo instanceof HashSet);
+         assertEquals(Sets.newHashSet("a", "!", "c"), res.foo);
+     }
+ 
+     /**
+      *
+      */
+     public void testHashSetWriteUnmodifiable() {
+         TestObjectContainer obj = new TestObjectContainer();
+ 
+         Set<Object> src = Sets.newHashSet(obj, "a", "b", "c");
+ 
+         obj.foo = src;
+ 
+         TestObjectContainer deserialized = wrap(obj).build().deserialize();
+ 
+         Set<Object> res = (Set<Object>)deserialized.foo;
+ 
+         src.remove(obj);
+         src.add(deserialized);
+ 
+         assertEquals(src, res);
+     }
+ 
+     /**
+      *
+      */
+     public void testMapRead() {
+         TestObjectContainer obj = new TestObjectContainer();
+         obj.foo = Maps.newHashMap(ImmutableMap.of(obj, "a", "b", obj));
+ 
+         BinaryObjectBuilderImpl mutObj = wrap(obj);
+ 
+         Map<Object, Object> map = mutObj.getField("foo");
+ 
+         assert map.equals(ImmutableMap.of(mutObj, "a", "b", mutObj));
+     }
+ 
+     /**
+      *
+      */
+     public void testMapOverride() {
+         TestObjectContainer obj = new TestObjectContainer();
+ 
+         BinaryObjectBuilderImpl mutObj = wrap(obj);
+ 
+         Map<Object, Object> map = Maps.newHashMap(ImmutableMap.of(mutObj, "a", "b", mutObj));
+ 
+         mutObj.setField("foo", map);
+ 
+         TestObjectContainer res = mutObj.build().deserialize();
+ 
+         assertEquals(ImmutableMap.of(res, "a", "b", res), res.foo);
+     }
+ 
+     /**
+      *
+      */
+     public void testMapModification() {
+         TestObjectContainer obj = new TestObjectContainer();
+         obj.foo = Maps.newHashMap(ImmutableMap.of(1, "a", 2, "b"));
+ 
+         BinaryObjectBuilderImpl mutObj = wrap(obj);
+ 
+         Map<Object, Object> map = mutObj.getField("foo");
+ 
+         map.put(3, mutObj);
+         Object rmv = map.remove(1);
+ 
+         assertEquals("a", rmv);
+ 
+         TestObjectContainer res = mutObj.build().deserialize();
+ 
+         assertEquals(ImmutableMap.of(2, "b", 3, res), res.foo);
+     }
+ 
+     /**
+      *
+      */
+     public void testEnumArrayModification() {
+         TestObjectAllTypes obj = new TestObjectAllTypes();
+ 
+         obj.enumArr = new TestObjectEnum[] {TestObjectEnum.A, TestObjectEnum.B};
+ 
+         BinaryObjectBuilderImpl mutObj = wrap(obj);
+ 
+         PortableBuilderEnum[] arr = mutObj.getField("enumArr");
+         arr[0] = new PortableBuilderEnum(mutObj.typeId(), TestObjectEnum.B);
+ 
+         TestObjectAllTypes res = mutObj.build().deserialize();
+ 
+         Assert.assertArrayEquals(new TestObjectEnum[] {TestObjectEnum.A, TestObjectEnum.B}, res.enumArr);
+     }
+ 
+     /**
+      *
+      */
+     public void testEditObjectWithRawData() {
+         GridBinaryMarshalerAwareTestClass obj = new GridBinaryMarshalerAwareTestClass();
+ 
+         obj.s = "a";
+         obj.sRaw = "aa";
+ 
+         BinaryObjectBuilderImpl mutableObj = wrap(obj);
+ 
+         mutableObj.setField("s", "z");
+ 
+         GridBinaryMarshalerAwareTestClass res = mutableObj.build().deserialize();
+         assertEquals("z", res.s);
+         assertEquals("aa", res.sRaw);
+     }
+ 
+     /**
+      *
+      */
+     public void testHashCode() {
+         TestObjectContainer obj = new TestObjectContainer();
+ 
+         BinaryObjectBuilderImpl mutableObj = wrap(obj);
+ 
+         assertEquals(obj.hashCode(), mutableObj.build().hashCode());
+ 
+         mutableObj.hashCode(25);
+ 
+         assertEquals(25, mutableObj.build().hashCode());
+     }
+ 
+     /**
+      *
+      */
+     public void testCollectionsInCollection() {
+         TestObjectContainer obj = new TestObjectContainer();
+         obj.foo = Lists.newArrayList(
+             Lists.newArrayList(1, 2),
+             Lists.newLinkedList(Arrays.asList(1, 2)),
+             Sets.newHashSet("a", "b"),
+             Sets.newLinkedHashSet(Arrays.asList("a", "b")),
+             Maps.newHashMap(ImmutableMap.of(1, "a", 2, "b")));
+ 
+         TestObjectContainer deserialized = wrap(obj).build().deserialize();
+ 
+         assertEquals(obj.foo, deserialized.foo);
+     }
+ 
+     /**
+      *
+      */
+     public void testMapEntryModification() {
+         TestObjectContainer obj = new TestObjectContainer();
+         obj.foo = ImmutableMap.of(1, "a").entrySet().iterator().next();
+ 
+         BinaryObjectBuilderImpl mutableObj = wrap(obj);
+ 
+         Map.Entry<Object, Object> entry = mutableObj.getField("foo");
+ 
+         assertEquals(1, entry.getKey());
+         assertEquals("a", entry.getValue());
+ 
+         entry.setValue("b");
+ 
+         TestObjectContainer res = mutableObj.build().deserialize();
+ 
+         assertEquals(new GridMapEntry<>(1, "b"), res.foo);
+     }
+ 
+     /**
+      *
+      */
+     public void testMapEntryOverride() {
+         TestObjectContainer obj = new TestObjectContainer();
+ 
+         BinaryObjectBuilderImpl mutableObj = wrap(obj);
+ 
+         mutableObj.setField("foo", new GridMapEntry<>(1, "a"));
+ 
+         TestObjectContainer res = mutableObj.build().deserialize();
+ 
+         assertEquals(new GridMapEntry<>(1, "a"), res.foo);
+     }
+ 
+     /**
+      *
+      */
+     public void testMetadataChangingDoublePut() {
+         BinaryObjectBuilderImpl mutableObj = wrap(new TestObjectContainer());
+ 
+         mutableObj.setField("xx567", "a");
+         mutableObj.setField("xx567", "b");
+ 
+         mutableObj.build();
+ 
+         BinaryType metadata = portables().metadata(TestObjectContainer.class);
+ 
+         assertEquals("String", metadata.fieldTypeName("xx567"));
+     }
+ 
+     /**
+      *
+      */
+     public void testMetadataChangingDoublePut2() {
+         BinaryObjectBuilderImpl mutableObj = wrap(new TestObjectContainer());
+ 
+         mutableObj.setField("xx567", "a");
+         mutableObj.setField("xx567", "b");
+ 
+         mutableObj.build();
+ 
+         BinaryType metadata = portables().metadata(TestObjectContainer.class);
+ 
+         assertEquals("String", metadata.fieldTypeName("xx567"));
+     }
+ 
+     /**
+      *
+      */
+     public void testMetadataChanging() {
+         TestObjectContainer c = new TestObjectContainer();
+ 
+         BinaryObjectBuilderImpl mutableObj = wrap(c);
+ 
+         mutableObj.setField("intField", 1);
+         mutableObj.setField("intArrField", new int[] {1});
+         mutableObj.setField("arrField", new String[] {"1"});
+         mutableObj.setField("strField", "1");
+         mutableObj.setField("colField", Lists.newArrayList("1"));
+         mutableObj.setField("mapField", Maps.newHashMap(ImmutableMap.of(1, "1")));
+         mutableObj.setField("enumField", TestObjectEnum.A);
+         mutableObj.setField("enumArrField", new Enum[] {TestObjectEnum.A});
+ 
+         mutableObj.build();
+ 
+         BinaryType metadata = portables().metadata(c.getClass());
+ 
+         assertTrue(metadata.fieldNames().containsAll(Arrays.asList("intField", "intArrField", "arrField", "strField",
+             "colField", "mapField", "enumField", "enumArrField")));
+ 
+         assertEquals("int", metadata.fieldTypeName("intField"));
+         assertEquals("int[]", metadata.fieldTypeName("intArrField"));
+         assertEquals("String[]", metadata.fieldTypeName("arrField"));
+         assertEquals("String", metadata.fieldTypeName("strField"));
+         assertEquals("Collection", metadata.fieldTypeName("colField"));
+         assertEquals("Map", metadata.fieldTypeName("mapField"));
+         assertEquals("Enum", metadata.fieldTypeName("enumField"));
+         assertEquals("Enum[]", metadata.fieldTypeName("enumArrField"));
+     }
+ 
+     /**
+      *
+      */
+     public void testDateInObjectField() {
+         TestObjectContainer obj = new TestObjectContainer();
+ 
+         obj.foo = new Date();
+ 
+         BinaryObjectBuilderImpl mutableObj = wrap(obj);
+ 
+         assertEquals(Date.class, mutableObj.getField("foo").getClass());
+     }
+ 
+     /**
+      *
+      */
+     public void testTimestampInObjectField() {
+         TestObjectContainer obj = new TestObjectContainer();
+ 
+         obj.foo = new Timestamp(100020003);
+ 
+         BinaryObjectBuilderImpl mutableObj = wrap(obj);
+ 
+         assertEquals(Timestamp.class, mutableObj.getField("foo").getClass());
+     }
+ 
+     /**
+      *
+      */
+     public void testDateInCollection() {
+         TestObjectContainer obj = new TestObjectContainer();
+ 
+         obj.foo = Lists.newArrayList(new Date());
+ 
+         BinaryObjectBuilderImpl mutableObj = wrap(obj);
+ 
+         assertEquals(Date.class, ((List<?>)mutableObj.getField("foo")).get(0).getClass());
+     }
+ 
+     /**
+      *
+      */
+     public void testTimestampInCollection() {
+         TestObjectContainer obj = new TestObjectContainer();
+ 
+         obj.foo = Lists.newArrayList(new Timestamp(100020003));
+ 
+         BinaryObjectBuilderImpl mutableObj = wrap(obj);
+ 
+         assertEquals(Timestamp.class, ((List<?>)mutableObj.getField("foo")).get(0).getClass());
+     }
+ 
+     /**
+      *
+      */
+     @SuppressWarnings("AssertEqualsBetweenInconvertibleTypes")
+     public void testDateArrayOverride() {
+         TestObjectContainer obj = new TestObjectContainer();
+ 
+         BinaryObjectBuilderImpl mutableObj = wrap(obj);
+ 
+         Date[] arr = { new Date() };
+ 
+         mutableObj.setField("foo", arr);
+ 
+         TestObjectContainer res = mutableObj.build().deserialize();
+ 
+         assertEquals(Date[].class, res.foo.getClass());
+         assertTrue(Objects.deepEquals(arr, res.foo));
+     }
+ 
+     /**
+      *
+      */
+     @SuppressWarnings("AssertEqualsBetweenInconvertibleTypes")
+     public void testTimestampArrayOverride() {
+         TestObjectContainer obj = new TestObjectContainer();
+ 
+         BinaryObjectBuilderImpl mutableObj = wrap(obj);
+ 
+         Timestamp[] arr = { new Timestamp(100020003) };
+ 
+         mutableObj.setField("foo", arr);
+ 
+         TestObjectContainer res = mutableObj.build().deserialize();
+ 
+         assertEquals(Timestamp[].class, res.foo.getClass());
+         assertTrue(Objects.deepEquals(arr, res.foo));
+     }
+ 
+     /**
+      *
+      */
+     public void testChangeMap() {
+         AddressBook addrBook = new AddressBook();
+ 
+         addrBook.addCompany(new Company(1, "Google inc", 100, new Address("Saint-Petersburg", "Torzhkovskya", 1, 53), "occupation"));
+         addrBook.addCompany(new Company(2, "Apple inc", 100, new Address("Saint-Petersburg", "Torzhkovskya", 1, 54), "occupation"));
+         addrBook.addCompany(new Company(3, "Microsoft", 100, new Address("Saint-Petersburg", "Torzhkovskya", 1, 55), "occupation"));
+         addrBook.addCompany(new Company(4, "Oracle", 100, new Address("Saint-Petersburg", "Nevskiy", 1, 1), "occupation"));
+ 
+         BinaryObjectBuilderImpl mutableObj = wrap(addrBook);
+ 
+         Map<String, List<BinaryObjectBuilderImpl>> map = mutableObj.getField("companyByStreet");
+ 
+         List<BinaryObjectBuilderImpl> list = map.get("Torzhkovskya");
+ 
+         BinaryObjectBuilderImpl company = list.get(0);
+ 
+         assert "Google inc".equals(company.<String>getField("name"));
+ 
+         list.remove(0);
+ 
+         AddressBook res = mutableObj.build().deserialize();
+ 
+         assertEquals(Arrays.asList("Nevskiy", "Torzhkovskya"), new ArrayList<>(res.getCompanyByStreet().keySet()));
+ 
+         List<Company> torzhkovskyaCompanies = res.getCompanyByStreet().get("Torzhkovskya");
+ 
+         assertEquals(2, torzhkovskyaCompanies.size());
+         assertEquals("Apple inc", torzhkovskyaCompanies.get(0).name);
+     }
+ 
+     /**
+      *
+      */
+     public void testSavingObjectWithNotZeroStart() {
+         TestObjectOuter out = new TestObjectOuter();
+         TestObjectInner inner = new TestObjectInner();
+ 
+         out.inner = inner;
+         inner.outer = out;
+ 
+         BinaryObjectBuilderImpl builder = wrap(out);
+ 
+         BinaryObjectBuilderImpl innerBuilder = builder.getField("inner");
+ 
+         TestObjectInner res = innerBuilder.build().deserialize();
+ 
+         assertSame(res, res.outer.inner);
+     }
+ 
+     /**
+      *
+      */
+     public void testPortableObjectField() {
+         TestObjectContainer container = new TestObjectContainer(toPortable(new TestObjectArrayList()));
+ 
+         BinaryObjectBuilderImpl wrapper = wrap(container);
+ 
+         assertTrue(wrapper.getField("foo") instanceof BinaryObject);
+ 
+         TestObjectContainer deserialized = wrapper.build().deserialize();
+         assertTrue(deserialized.foo instanceof BinaryObject);
+     }
+ 
+     /**
+      *
+      */
+     public void testAssignPortableObject() {
+         TestObjectContainer container = new TestObjectContainer();
+ 
+         BinaryObjectBuilderImpl wrapper = wrap(container);
+ 
+         wrapper.setField("foo", toPortable(new TestObjectArrayList()));
+ 
+         TestObjectContainer deserialized = wrapper.build().deserialize();
+         assertTrue(deserialized.foo instanceof TestObjectArrayList);
+     }
+ 
+     /**
+      *
+      */
+     public void testRemoveFromNewObject() {
+         BinaryObjectBuilderImpl wrapper = newWrapper(TestObjectAllTypes.class);
+ 
+         wrapper.setField("str", "a");
+ 
+         wrapper.removeField("str");
+ 
+         assertNull(wrapper.build().<TestObjectAllTypes>deserialize().str);
+     }
+ 
+     /**
+      *
+      */
+     public void testRemoveFromExistingObject() {
+         TestObjectAllTypes obj = new TestObjectAllTypes();
+         obj.setDefaultData();
+ 
+         BinaryObjectBuilderImpl wrapper = wrap(toPortable(obj));
+ 
+         wrapper.removeField("str");
+ 
+         assertNull(wrapper.build().<TestObjectAllTypes>deserialize().str);
+     }
+ 
+     /**
+      *
+      */
+     public void testCyclicArrays() {
+         TestObjectContainer obj = new TestObjectContainer();
+ 
+         Object[] arr1 = new Object[1];
+         Object[] arr2 = new Object[] {arr1};
+ 
+         arr1[0] = arr2;
+ 
+         obj.foo = arr1;
+ 
+         TestObjectContainer res = toPortable(obj).deserialize();
+ 
+         Object[] resArr = (Object[])res.foo;
+ 
+         assertSame(((Object[])resArr[0])[0], resArr);
+     }
+ 
+     /**
+      *
+      */
+     @SuppressWarnings("TypeMayBeWeakened")
+     public void testCyclicArrayList() {
+         TestObjectContainer obj = new TestObjectContainer();
+ 
+         List<Object> arr1 = new ArrayList<>();
+         List<Object> arr2 = new ArrayList<>();
+ 
+         arr1.add(arr2);
+         arr2.add(arr1);
+ 
+         obj.foo = arr1;
+ 
+         TestObjectContainer res = toPortable(obj).deserialize();
+ 
+         List<?> resArr = (List<?>)res.foo;
+ 
+         assertSame(((List<Object>)resArr.get(0)).get(0), resArr);
+     }
+ 
+     /**
+      * @param obj Object.
+      * @return Object in portable format.
+      */
+     private BinaryObject toPortable(Object obj) {
+         return portables().toBinary(obj);
+     }
+ 
+     /**
+      * @param obj Object.
+      * @return GridMutablePortableObject.
+      */
+     private BinaryObjectBuilderImpl wrap(Object obj) {
+         return BinaryObjectBuilderImpl.wrap(toPortable(obj));
+     }
+ 
+     /**
+      * @param aCls Class.
+      * @return Wrapper.
+      */
+     private BinaryObjectBuilderImpl newWrapper(Class<?> aCls) {
+         CacheObjectBinaryProcessorImpl processor = (CacheObjectBinaryProcessorImpl)(
+             (IgniteBinaryImpl)portables()).processor();
+ 
+         return new BinaryObjectBuilderImpl(processor.portableContext(), processor.typeId(aCls.getName()),
+             aCls.getSimpleName());
+     }
+ }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a511fa17/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderSelfTest.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderSelfTest.java
index 0000000,e88db99..f177119
mode 000000,100644..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
@@@ -1,0 -1,1066 +1,1069 @@@
+ /*
+  * 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.sql.Timestamp;
+ import java.util.ArrayList;
+ import java.util.Arrays;
+ import java.util.Collection;
 -import java.util.Collections;
+ import java.util.Date;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.UUID;
 -import org.apache.ignite.IgniteCheckedException;
+ import org.apache.ignite.IgniteBinary;
++import org.apache.ignite.IgniteCheckedException;
++import org.apache.ignite.binary.BinaryIdMapper;
++import org.apache.ignite.binary.BinaryObject;
++import org.apache.ignite.binary.BinaryObjectBuilder;
++import org.apache.ignite.binary.BinaryType;
++import org.apache.ignite.binary.BinaryTypeConfiguration;
++import org.apache.ignite.configuration.BinaryConfiguration;
+ import org.apache.ignite.configuration.IgniteConfiguration;
+ import org.apache.ignite.internal.portable.builder.BinaryObjectBuilderImpl;
+ import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectAllTypes;
+ import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectContainer;
+ import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectInner;
+ import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectOuter;
+ import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectPlainPortable;
+ import org.apache.ignite.internal.processors.cache.portable.CacheObjectBinaryProcessorImpl;
+ import org.apache.ignite.internal.util.GridUnsafe;
+ import org.apache.ignite.internal.util.typedef.F;
 -import org.apache.ignite.marshaller.portable.PortableMarshaller;
 -import org.apache.ignite.binary.BinaryObjectBuilder;
 -import org.apache.ignite.binary.BinaryIdMapper;
 -import org.apache.ignite.binary.BinaryType;
 -import org.apache.ignite.binary.BinaryObject;
 -import org.apache.ignite.binary.BinaryTypeConfiguration;
++import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+ import org.apache.ignite.testframework.GridTestUtils;
+ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+ import sun.misc.Unsafe;
+ 
+ /**
+  * Portable builder test.
+  */
+ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
+     /** */
+     private static final Unsafe UNSAFE = GridUnsafe.unsafe();
+ 
+     /** */
+     protected static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+ 
+     /** {@inheritDoc} */
+     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+         IgniteConfiguration cfg = super.getConfiguration(gridName);
+ 
 -        PortableMarshaller marsh = new PortableMarshaller();
 -
 -        marsh.setCompactFooter(compactFooter());
++        BinaryTypeConfiguration customTypeCfg = new BinaryTypeConfiguration();
+ 
 -        marsh.setClassNames(Arrays.asList(Key.class.getName(), Value.class.getName(),
 -            "org.gridgain.grid.internal.util.portable.mutabletest.*"));
 -
 -        BinaryTypeConfiguration customIdMapper = new BinaryTypeConfiguration();
 -
 -        customIdMapper.setClassName(CustomIdMapper.class.getName());
 -        customIdMapper.setIdMapper(new BinaryIdMapper() {
++        customTypeCfg.setTypeName(CustomIdMapper.class.getName());
++        customTypeCfg.setIdMapper(new BinaryIdMapper() {
+             @Override public int typeId(String clsName) {
+                 return ~PortableContext.DFLT_ID_MAPPER.typeId(clsName);
+             }
+ 
+             @Override public int fieldId(int typeId, String fieldName) {
+                 return typeId + ~PortableContext.DFLT_ID_MAPPER.fieldId(typeId, fieldName);
+             }
+         });
+ 
 -        marsh.setTypeConfigurations(Collections.singleton(customIdMapper));
++        BinaryConfiguration bCfg = new BinaryConfiguration();
++        
++        bCfg.setCompactFooter(compactFooter());
++
++        bCfg.setTypeConfigurations(Arrays.asList(
++            new BinaryTypeConfiguration(Key.class.getName()),
++            new BinaryTypeConfiguration(Value.class.getName()),
++            new BinaryTypeConfiguration("org.gridgain.grid.internal.util.portable.mutabletest.*"),
++            customTypeCfg));
++
++        cfg.setBinaryConfiguration(bCfg);
+ 
 -        cfg.setMarshaller(marsh);
++        cfg.setMarshaller(new BinaryMarshaller());
+ 
+         return cfg;
+     }
+ 
+     /** {@inheritDoc} */
+     @Override protected void beforeTestsStarted() throws Exception {
+         startGrids(1);
+     }
+ 
+     /** {@inheritDoc} */
+     @Override protected void afterTestsStopped() throws Exception {
+         stopAllGrids();
+     }
+ 
+     /**
+      * @return Whether to use compact footer.
+      */
+     protected boolean compactFooter() {
+         return true;
+     }
+ 
+     /**
+      *
+      */
+     public void testAllFieldsSerialization() {
+         TestObjectAllTypes obj = new TestObjectAllTypes();
+         obj.setDefaultData();
+         obj.enumArr = null;
+ 
+         TestObjectAllTypes deserialized = builder(toPortable(obj)).build().deserialize();
+ 
+         GridTestUtils.deepEquals(obj, deserialized);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testByteField() throws Exception {
+         BinaryObjectBuilder builder = builder("Class");
+ 
+         builder.hashCode(100);
+ 
+         builder.setField("byteField", (byte)1);
+ 
+         BinaryObject po = builder.build();
+ 
+         assertEquals("class".hashCode(), po.typeId());
+         assertEquals(100, po.hashCode());
+ 
+         assertEquals((byte) 1, po.<Byte>field("byteField").byteValue());
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testShortField() throws Exception {
+         BinaryObjectBuilder builder = builder("Class");
+ 
+         builder.hashCode(100);
+ 
+         builder.setField("shortField", (short)1);
+ 
+         BinaryObject po = builder.build();
+ 
+         assertEquals("class".hashCode(), po.typeId());
+         assertEquals(100, po.hashCode());
+ 
+         assertEquals((short)1, po.<Short>field("shortField").shortValue());
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testIntField() throws Exception {
+         BinaryObjectBuilder builder = builder("Class");
+ 
+         builder.hashCode(100);
+ 
+         builder.setField("intField", 1);
+ 
+         BinaryObject po = builder.build();
+ 
+         assertEquals("class".hashCode(), po.typeId());
+         assertEquals(100, po.hashCode());
+ 
+         assertEquals(1, po.<Integer>field("intField").intValue());
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testLongField() throws Exception {
+         BinaryObjectBuilder builder = builder("Class");
+ 
+         builder.hashCode(100);
+ 
+         builder.setField("longField", 1L);
+ 
+         BinaryObject po = builder.build();
+ 
+         assertEquals("class".hashCode(), po.typeId());
+         assertEquals(100, po.hashCode());
+ 
+         assertEquals(1L, po.<Long>field("longField").longValue());
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testFloatField() throws Exception {
+         BinaryObjectBuilder builder = builder("Class");
+ 
+         builder.hashCode(100);
+ 
+         builder.setField("floatField", 1.0f);
+ 
+         BinaryObject po = builder.build();
+ 
+         assertEquals("class".hashCode(), po.typeId());
+         assertEquals(100, po.hashCode());
+ 
+         assertEquals(1.0f, po.<Float>field("floatField").floatValue(), 0);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testDoubleField() throws Exception {
+         BinaryObjectBuilder builder = builder("Class");
+ 
+         builder.hashCode(100);
+ 
+         builder.setField("doubleField", 1.0d);
+ 
+         BinaryObject po = builder.build();
+ 
+         assertEquals("class".hashCode(), po.typeId());
+         assertEquals(100, po.hashCode());
+ 
+         assertEquals(1.0d, po.<Double>field("doubleField").doubleValue(), 0);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testCharField() throws Exception {
+         BinaryObjectBuilder builder = builder("Class");
+ 
+         builder.hashCode(100);
+ 
+         builder.setField("charField", (char)1);
+ 
+         BinaryObject po = builder.build();
+ 
+         assertEquals("class".hashCode(), po.typeId());
+         assertEquals(100, po.hashCode());
+ 
+         assertEquals((char)1, po.<Character>field("charField").charValue());
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testBooleanField() throws Exception {
+         BinaryObjectBuilder builder = builder("Class");
+ 
+         builder.hashCode(100);
+ 
+         builder.setField("booleanField", true);
+ 
+         BinaryObject po = builder.build();
+ 
+         assertEquals("class".hashCode(), po.typeId());
+         assertEquals(100, po.hashCode());
+ 
+         assertTrue(po.<Boolean>field("booleanField"));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testDecimalField() throws Exception {
+         BinaryObjectBuilder builder = builder("Class");
+ 
+         builder.hashCode(100);
+ 
+         builder.setField("decimalField", BigDecimal.TEN);
+ 
+         BinaryObject po = builder.build();
+ 
+         assertEquals("class".hashCode(), po.typeId());
+         assertEquals(100, po.hashCode());
+ 
+         assertEquals(BigDecimal.TEN, po.<String>field("decimalField"));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testStringField() throws Exception {
+         BinaryObjectBuilder builder = builder("Class");
+ 
+         builder.hashCode(100);
+ 
+         builder.setField("stringField", "str");
+ 
+         BinaryObject po = builder.build();
+ 
+         assertEquals("class".hashCode(), po.typeId());
+         assertEquals(100, po.hashCode());
+ 
+         assertEquals("str", po.<String>field("stringField"));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testDateField() throws Exception {
+         Date date = new Date();
+ 
+         assertEquals(date, builder("C").setField("d", date).build().<Date>field("d"));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testTimestampField() throws Exception {
+         Timestamp ts = new Timestamp(new Date().getTime());
+         ts.setNanos(1000);
+ 
+         assertEquals(ts, builder("C").setField("t", ts).build().<Timestamp>field("t"));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testUuidField() throws Exception {
+         BinaryObjectBuilder builder = builder("Class");
+ 
+         builder.hashCode(100);
+ 
+         UUID uuid = UUID.randomUUID();
+ 
+         builder.setField("uuidField", uuid);
+ 
+         BinaryObject po = builder.build();
+ 
+         assertEquals("class".hashCode(), po.typeId());
+         assertEquals(100, po.hashCode());
+ 
+         assertEquals(uuid, po.<UUID>field("uuidField"));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testByteArrayField() throws Exception {
+         BinaryObjectBuilder builder = builder("Class");
+ 
+         builder.hashCode(100);
+ 
+         builder.setField("byteArrayField", new byte[] {1, 2, 3});
+ 
+         BinaryObject po = builder.build();
+ 
+         assertEquals("class".hashCode(), po.typeId());
+         assertEquals(100, po.hashCode());
+ 
+         assertTrue(Arrays.equals(new byte[] {1, 2, 3}, po.<byte[]>field("byteArrayField")));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testShortArrayField() throws Exception {
+         BinaryObjectBuilder builder = builder("Class");
+ 
+         builder.hashCode(100);
+ 
+         builder.setField("shortArrayField", new short[] {1, 2, 3});
+ 
+         BinaryObject po = builder.build();
+ 
+         assertEquals("class".hashCode(), po.typeId());
+         assertEquals(100, po.hashCode());
+ 
+         assertTrue(Arrays.equals(new short[] {1, 2, 3}, po.<short[]>field("shortArrayField")));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testIntArrayField() throws Exception {
+         BinaryObjectBuilder builder = builder("Class");
+ 
+         builder.hashCode(100);
+ 
+         builder.setField("intArrayField", new int[] {1, 2, 3});
+ 
+         BinaryObject po = builder.build();
+ 
+         assertEquals("class".hashCode(), po.typeId());
+         assertEquals(100, po.hashCode());
+ 
+         assertTrue(Arrays.equals(new int[] {1, 2, 3}, po.<int[]>field("intArrayField")));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testLongArrayField() throws Exception {
+         BinaryObjectBuilder builder = builder("Class");
+ 
+         builder.hashCode(100);
+ 
+         builder.setField("longArrayField", new long[] {1, 2, 3});
+ 
+         BinaryObject po = builder.build();
+ 
+         assertEquals("class".hashCode(), po.typeId());
+         assertEquals(100, po.hashCode());
+ 
+         assertTrue(Arrays.equals(new long[] {1, 2, 3}, po.<long[]>field("longArrayField")));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testFloatArrayField() throws Exception {
+         BinaryObjectBuilder builder = builder("Class");
+ 
+         builder.hashCode(100);
+ 
+         builder.setField("floatArrayField", new float[] {1, 2, 3});
+ 
+         BinaryObject po = builder.build();
+ 
+         assertEquals("class".hashCode(), po.typeId());
+         assertEquals(100, po.hashCode());
+ 
+         assertTrue(Arrays.equals(new float[] {1, 2, 3}, po.<float[]>field("floatArrayField")));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testDoubleArrayField() throws Exception {
+         BinaryObjectBuilder builder = builder("Class");
+ 
+         builder.hashCode(100);
+ 
+         builder.setField("doubleArrayField", new double[] {1, 2, 3});
+ 
+         BinaryObject po = builder.build();
+ 
+         assertEquals("class".hashCode(), po.typeId());
+         assertEquals(100, po.hashCode());
+ 
+         assertTrue(Arrays.equals(new double[] {1, 2, 3}, po.<double[]>field("doubleArrayField")));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testCharArrayField() throws Exception {
+         BinaryObjectBuilder builder = builder("Class");
+ 
+         builder.hashCode(100);
+ 
+         builder.setField("charArrayField", new char[] {1, 2, 3});
+ 
+         BinaryObject po = builder.build();
+ 
+         assertEquals("class".hashCode(), po.typeId());
+         assertEquals(100, po.hashCode());
+ 
+         assertTrue(Arrays.equals(new char[] {1, 2, 3}, po.<char[]>field("charArrayField")));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testBooleanArrayField() throws Exception {
+         BinaryObjectBuilder builder = builder("Class");
+ 
+         builder.hashCode(100);
+ 
+         builder.setField("booleanArrayField", new boolean[] {true, false});
+ 
+         BinaryObject po = builder.build();
+ 
+         assertEquals("class".hashCode(), po.typeId());
+         assertEquals(100, po.hashCode());
+ 
+         boolean[] arr = po.field("booleanArrayField");
+ 
+         assertEquals(2, arr.length);
+ 
+         assertTrue(arr[0]);
+         assertFalse(arr[1]);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testDecimalArrayField() throws Exception {
+         BinaryObjectBuilder builder = builder("Class");
+ 
+         builder.hashCode(100);
+ 
+         builder.setField("decimalArrayField", new BigDecimal[] {BigDecimal.ONE, BigDecimal.TEN});
+ 
+         BinaryObject po = builder.build();
+ 
+         assertEquals("class".hashCode(), po.typeId());
+         assertEquals(100, po.hashCode());
+ 
+         assertTrue(Arrays.equals(new BigDecimal[] {BigDecimal.ONE, BigDecimal.TEN}, po.<String[]>field("decimalArrayField")));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testStringArrayField() throws Exception {
+         BinaryObjectBuilder builder = builder("Class");
+ 
+         builder.hashCode(100);
+ 
+         builder.setField("stringArrayField", new String[] {"str1", "str2", "str3"});
+ 
+         BinaryObject po = builder.build();
+ 
+         assertEquals("class".hashCode(), po.typeId());
+         assertEquals(100, po.hashCode());
+ 
+         assertTrue(Arrays.equals(new String[] {"str1", "str2", "str3"}, po.<String[]>field("stringArrayField")));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testDateArrayField() throws Exception {
+         Date date1 = new Date();
+         Date date2 = new Date(date1.getTime() + 1000);
+ 
+         Date[] dateArr = new Date[] { date1, date2 };
+ 
+         assertTrue(Arrays.equals(dateArr, builder("C").setField("da", dateArr).build().<Date[]>field("da")));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testTimestampArrayField() throws Exception {
+         Timestamp ts1 = new Timestamp(new Date().getTime());
+         Timestamp ts2 = new Timestamp(new Date().getTime() + 1000);
+ 
+         ts1.setNanos(1000);
+         ts2.setNanos(2000);
+ 
+         Timestamp[] tsArr = new Timestamp[] { ts1, ts2 };
+ 
+         assertTrue(Arrays.equals(tsArr, builder("C").setField("ta", tsArr).build().<Timestamp[]>field("ta")));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testUuidArrayField() throws Exception {
+         BinaryObjectBuilder builder = builder("Class");
+ 
+         builder.hashCode(100);
+ 
+         UUID[] arr = new UUID[] {UUID.randomUUID(), UUID.randomUUID()};
+ 
+         builder.setField("uuidArrayField", arr);
+ 
+         BinaryObject po = builder.build();
+ 
+         assertEquals("class".hashCode(), po.typeId());
+         assertEquals(100, po.hashCode());
+ 
+         assertTrue(Arrays.equals(arr, po.<UUID[]>field("uuidArrayField")));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testObjectField() throws Exception {
+         BinaryObjectBuilder builder = builder("Class");
+ 
+         builder.hashCode(100);
+ 
+         builder.setField("objectField", new Value(1));
+ 
+         BinaryObject po = builder.build();
+ 
+         assertEquals("class".hashCode(), po.typeId());
+         assertEquals(100, po.hashCode());
+ 
+         assertEquals(1, po.<BinaryObject>field("objectField").<Value>deserialize().i);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testObjectArrayField() throws Exception {
+         BinaryObjectBuilder builder = builder("Class");
+ 
+         builder.hashCode(100);
+ 
+         builder.setField("objectArrayField", new Value[] {new Value(1), new Value(2)});
+ 
+         BinaryObject po = builder.build();
+ 
+         assertEquals("class".hashCode(), po.typeId());
+         assertEquals(100, po.hashCode());
+ 
+         Object[] arr = po.field("objectArrayField");
+ 
+         assertEquals(2, arr.length);
+ 
+         assertEquals(1, ((BinaryObject)arr[0]).<Value>deserialize().i);
+         assertEquals(2, ((BinaryObject)arr[1]).<Value>deserialize().i);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testCollectionField() throws Exception {
+         BinaryObjectBuilder builder = builder("Class");
+ 
+         builder.hashCode(100);
+ 
+         builder.setField("collectionField", Arrays.asList(new Value(1), new Value(2)));
+ 
+         BinaryObject po = builder.build();
+ 
+         assertEquals("class".hashCode(), po.typeId());
+         assertEquals(100, po.hashCode());
+ 
+         List<BinaryObject> list = po.field("collectionField");
+ 
+         assertEquals(2, list.size());
+ 
+         assertEquals(1, list.get(0).<Value>deserialize().i);
+         assertEquals(2, list.get(1).<Value>deserialize().i);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testMapField() throws Exception {
+         BinaryObjectBuilder builder = builder("Class");
+ 
+         builder.hashCode(100);
+ 
+         builder.setField("mapField", F.asMap(new Key(1), new Value(1), new Key(2), new Value(2)));
+ 
+         BinaryObject po = builder.build();
+ 
+         assertEquals("class".hashCode(), po.typeId());
+         assertEquals(100, po.hashCode());
+ 
+         Map<BinaryObject, BinaryObject> map = po.field("mapField");
+ 
+         assertEquals(2, map.size());
+ 
+         for (Map.Entry<BinaryObject, BinaryObject> e : map.entrySet())
+             assertEquals(e.getKey().<Key>deserialize().i, e.getValue().<Value>deserialize().i);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testSeveralFields() throws Exception {
+         BinaryObjectBuilder builder = builder("Class");
+ 
+         builder.hashCode(100);
+ 
+         builder.setField("i", 111);
+         builder.setField("f", 111.111f);
+         builder.setField("iArr", new int[] {1, 2, 3});
+         builder.setField("obj", new Key(1));
+         builder.setField("col", Arrays.asList(new Value(1), new Value(2)));
+ 
+         BinaryObject po = builder.build();
+ 
+         assertEquals("class".hashCode(), po.typeId());
+         assertEquals(100, po.hashCode());
+ 
+         assertEquals(111, po.<Integer>field("i").intValue());
+         assertEquals(111.111f, po.<Float>field("f").floatValue(), 0);
+         assertTrue(Arrays.equals(new int[] {1, 2, 3}, po.<int[]>field("iArr")));
+         assertEquals(1, po.<BinaryObject>field("obj").<Key>deserialize().i);
+ 
+         List<BinaryObject> list = po.field("col");
+ 
+         assertEquals(2, list.size());
+ 
+         assertEquals(1, list.get(0).<Value>deserialize().i);
+         assertEquals(2, list.get(1).<Value>deserialize().i);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testOffheapPortable() throws Exception {
+         BinaryObjectBuilder builder = builder("Class");
+ 
+         builder.hashCode(100);
+ 
+         builder.setField("i", 111);
+         builder.setField("f", 111.111f);
+         builder.setField("iArr", new int[] {1, 2, 3});
+         builder.setField("obj", new Key(1));
+         builder.setField("col", Arrays.asList(new Value(1), new Value(2)));
+ 
+         BinaryObject po = builder.build();
+ 
+         byte[] arr = ((CacheObjectBinaryProcessorImpl)(grid(0)).context().cacheObjects()).marshal(po);
+ 
+         long ptr = UNSAFE.allocateMemory(arr.length + 5);
+ 
+         try {
+             long ptr0 = ptr;
+ 
+             UNSAFE.putBoolean(null, ptr0++, false);
+ 
+             UNSAFE.putInt(ptr0, arr.length);
+ 
+             UNSAFE.copyMemory(arr, BYTE_ARR_OFF, null, ptr0 + 4, arr.length);
+ 
+             BinaryObject offheapObj = (BinaryObject)
+                 ((CacheObjectBinaryProcessorImpl)(grid(0)).context().cacheObjects()).unmarshal(ptr, false);
+ 
+             assertEquals(BinaryObjectOffheapImpl.class, offheapObj.getClass());
+ 
+             assertEquals("class".hashCode(), offheapObj.typeId());
+             assertEquals(100, offheapObj.hashCode());
+ 
+             assertEquals(111, offheapObj.<Integer>field("i").intValue());
+             assertEquals(111.111f, offheapObj.<Float>field("f").floatValue(), 0);
+             assertTrue(Arrays.equals(new int[] {1, 2, 3}, offheapObj.<int[]>field("iArr")));
+             assertEquals(1, offheapObj.<BinaryObject>field("obj").<Key>deserialize().i);
+ 
+             List<BinaryObject> list = offheapObj.field("col");
+ 
+             assertEquals(2, list.size());
+ 
+             assertEquals(1, list.get(0).<Value>deserialize().i);
+             assertEquals(2, list.get(1).<Value>deserialize().i);
+ 
+             assertEquals(po, offheapObj);
+             assertEquals(offheapObj, po);
+         }
+         finally {
+             UNSAFE.freeMemory(ptr);
+         }
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testBuildAndDeserialize() throws Exception {
+         BinaryObjectBuilder builder = builder(Value.class.getName());
+ 
+         builder.hashCode(100);
+ 
+         builder.setField("i", 1);
+ 
+         BinaryObject po = builder.build();
+ 
+         assertEquals("value".hashCode(), po.typeId());
+         assertEquals(100, po.hashCode());
+ 
+         assertEquals(1, po.<Value>deserialize().i);
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testMetaData2() throws Exception {
+         BinaryObjectBuilder builder = builder("org.test.MetaTest2");
+ 
+         builder.setField("objectField", "a", Object.class);
+ 
+         BinaryObject po = builder.build();
+ 
+         BinaryType meta = po.type();
+ 
+         assertEquals("MetaTest2", meta.typeName());
+         assertEquals("Object", meta.fieldTypeName("objectField"));
+     }
+ 
+     /**
+      * @throws Exception If failed.
+      */
+     public void testMetaData() throws Exception {
+         BinaryObjectBuilder builder = builder("org.test.MetaTest");
+ 
+         builder.hashCode(100);
+ 
+         builder.setField("intField", 1);
+         builder.setField("byteArrayField", new byte[] {1, 2, 3});
+ 
+         BinaryObject po = builder.build();
+ 
+         BinaryType meta = po.type();
+ 
+         assertEquals("MetaTest", meta.typeName());
+ 
+         Collection<String> fields = meta.fieldNames();
+ 
+         assertEquals(2, fields.size());
+ 
+         assertTrue(fields.contains("intField"));
+         assertTrue(fields.contains("byteArrayField"));
+ 
+         assertEquals("int", meta.fieldTypeName("intField"));
+         assertEquals("byte[]", meta.fieldTypeName("byteArrayField"));
+ 
+         builder = builder("org.test.MetaTest");
+ 
+         builder.hashCode(100);
+ 
+         builder.setField("intField", 2);
+         builder.setField("uuidField", UUID.randomUUID());
+ 
+         po = builder.build();
+ 
+         meta = po.type();
+ 
+         assertEquals("MetaTest", meta.typeName());
+ 
+         fields = meta.fieldNames();
+ 
+         assertEquals(3, fields.size());
+ 
+         assertTrue(fields.contains("intField"));
+         assertTrue(fields.contains("byteArrayField"));
+         assertTrue(fields.contains("uuidField"));
+ 
+         assertEquals("int", meta.fieldTypeName("intField"));
+         assertEquals("byte[]", meta.fieldTypeName("byteArrayField"));
+         assertEquals("UUID", meta.fieldTypeName("uuidField"));
+     }
+ 
+     /**
+      *
+      */
+     public void testGetFromCopiedObj() {
+         BinaryObject objStr = builder(TestObjectAllTypes.class.getName()).setField("str", "aaa").build();
+ 
+         BinaryObjectBuilderImpl builder = builder(objStr);
+         assertEquals("aaa", builder.getField("str"));
+ 
+         builder.setField("str", "bbb");
+         assertEquals("bbb", builder.getField("str"));
+ 
+         assertNull(builder.getField("i_"));
+         assertEquals("bbb", builder.build().<TestObjectAllTypes>deserialize().str);
+     }
+ 
+     /**
+      *
+      */
+     public void testCopyFromInnerObjects() {
+         ArrayList<Object> list = new ArrayList<>();
+         list.add(new TestObjectAllTypes());
+         list.add(list.get(0));
+ 
+         TestObjectContainer c = new TestObjectContainer(list);
+ 
+         BinaryObjectBuilderImpl builder = builder(toPortable(c));
+         builder.<List>getField("foo").add("!!!");
+ 
+         BinaryObject res = builder.build();
+ 
+         TestObjectContainer deserialized = res.deserialize();
+ 
+         List deserializedList = (List)deserialized.foo;
+ 
+         assertSame(deserializedList.get(0), deserializedList.get(1));
+         assertEquals("!!!", deserializedList.get(2));
+         assertTrue(deserializedList.get(0) instanceof TestObjectAllTypes);
+     }
+ 
+     /**
+      *
+      */
+     public void testSetPortableObject() {
+         BinaryObject portableObj = builder(TestObjectContainer.class.getName())
+             .setField("foo", toPortable(new TestObjectAllTypes()))
+             .build();
+ 
+         assertTrue(portableObj.<TestObjectContainer>deserialize().foo instanceof TestObjectAllTypes);
+     }
+ 
+     /**
+      *
+      */
+     public void testPlainPortableObjectCopyFrom() {
+         TestObjectPlainPortable obj = new TestObjectPlainPortable(toPortable(new TestObjectAllTypes()));
+ 
+         BinaryObjectBuilderImpl builder = builder(toPortable(obj));
+         assertTrue(builder.getField("plainPortable") instanceof BinaryObject);
+ 
+         TestObjectPlainPortable deserialized = builder.build().deserialize();
+         assertTrue(deserialized.plainPortable instanceof BinaryObject);
+     }
+ 
+     /**
+      *
+      */
+     public void testRemoveFromNewObject() {
+         BinaryObjectBuilder builder = builder(TestObjectAllTypes.class.getName());
+ 
+         builder.setField("str", "a");
+ 
+         builder.removeField("str");
+ 
+         assertNull(builder.build().<TestObjectAllTypes>deserialize().str);
+     }
+ 
+     /**
+      *
+      */
+     public void testRemoveFromExistingObject() {
+         TestObjectAllTypes obj = new TestObjectAllTypes();
+         obj.setDefaultData();
+         obj.enumArr = null;
+ 
+         BinaryObjectBuilder builder = builder(toPortable(obj));
+ 
+         builder.removeField("str");
+ 
+         BinaryObject binary = builder.build();
+ 
+         TestObjectAllTypes deserialzied = binary.deserialize();
+ 
+         assertNull(deserialzied.str);
+     }
+ 
+     /**
+      *
+      */
+     public void testRemoveFromExistingObjectAfterGet() {
+         TestObjectAllTypes obj = new TestObjectAllTypes();
+         obj.setDefaultData();
+         obj.enumArr = null;
+ 
+         BinaryObjectBuilderImpl builder = builder(toPortable(obj));
+ 
+         builder.getField("i_");
+ 
+         builder.removeField("str");
+ 
+         assertNull(builder.build().<TestObjectAllTypes>deserialize().str);
+     }
+ 
+     /**
+      * @throws IgniteCheckedException If any error occurs.
+      */
+     public void testDontBrokeCyclicDependency() throws IgniteCheckedException {
+         TestObjectOuter outer = new TestObjectOuter();
+         outer.inner = new TestObjectInner();
+         outer.inner.outer = outer;
+         outer.foo = "a";
+ 
+         BinaryObjectBuilder builder = builder(toPortable(outer));
+ 
+         builder.setField("foo", "b");
+ 
+         TestObjectOuter res = builder.build().deserialize();
+ 
+         assertEquals("b", res.foo);
+         assertSame(res, res.inner.outer);
+     }
+ 
+     /**
+      * @return Portables.
+      */
+     private IgniteBinary portables() {
+         return grid(0).binary();
+     }
+ 
+     /**
+      * @param obj Object.
+      * @return Portable object.
+      */
+     private BinaryObject toPortable(Object obj) {
+         return portables().toBinary(obj);
+     }
+ 
+     /**
+      * @return Builder.
+      */
+     private <T> BinaryObjectBuilder builder(String clsName) {
+         return portables().builder(clsName);
+     }
+ 
+     /**
+      * @return Builder.
+      */
+     private <T> BinaryObjectBuilderImpl builder(BinaryObject obj) {
+         return (BinaryObjectBuilderImpl)portables().builder(obj);
+     }
+ 
+     /**
+      *
+      */
+     private static class CustomIdMapper {
+         /** */
+         private String str = "a";
+ 
+         /** */
+         private int i = 10;
+     }
+ 
+     /**
+      */
+     private static class Key {
+         /** */
+         private int i;
+ 
+         /**
+          */
+         private Key() {
+             // No-op.
+         }
+ 
+         /**
+          * @param i Index.
+          */
+         private Key(int i) {
+             this.i = i;
+         }
+ 
+         /** {@inheritDoc} */
+         @Override public boolean equals(Object o) {
+             if (this == o)
+                 return true;
+ 
+             if (o == null || getClass() != o.getClass())
+                 return false;
+ 
+             Key key = (Key)o;
+ 
+             return i == key.i;
+         }
+ 
+         /** {@inheritDoc} */
+         @Override public int hashCode() {
+             return i;
+         }
+     }
+ 
+     /**
+      */
+     private static class Value {
+         /** */
+         private int i;
+ 
+         /**
+          */
+         private Value() {
+             // No-op.
+         }
+ 
+         /**
+          * @param i Index.
+          */
+         private Value(int i) {
+             this.i = i;
+         }
+     }
+ }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a511fa17/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
index d2291c3,9c0824e..917298a
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
@@@ -42,14 -42,12 +42,14 @@@ public class GridPortableMarshallerCtxD
       * @throws Exception If failed.
       */
      public void testObjectExchange() throws Exception {
 -        PortableMarshaller marsh = new PortableMarshaller();
 +        BinaryMarshaller marsh = new BinaryMarshaller();
          marsh.setContext(new MarshallerContextWithNoStorage());
  
 -        PortableContext context = new PortableContext(BinaryCachingMetadataHandler.create(), new IgniteConfiguration());
 +        IgniteConfiguration cfg = new IgniteConfiguration();
  
-         PortableContext context = new PortableContext(BinaryNoopMetadataHandler.instance(), cfg);
 -        IgniteUtils.invoke(PortableMarshaller.class, marsh, "setPortableContext", context);
++        PortableContext context = new PortableContext(BinaryCachingMetadataHandler.create(), cfg);
 +
 +        IgniteUtils.invoke(BinaryMarshaller.class, marsh, "setPortableContext", context, cfg);
  
          SimpleObject simpleObj = new SimpleObject();
  

http://git-wip-us.apache.org/repos/asf/ignite/blob/a511fa17/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java
index 8a44b3b,1128d67..16eeb2b
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java
@@@ -55,16 -62,16 +62,16 @@@ public class IgnitePortableObjectsTestS
       * @throws Exception If failed.
       */
      public static TestSuite suite() throws Exception {
 -        TestSuite suite = new TestSuite("GridGain Portable Objects Test Suite");
 +        TestSuite suite = new TestSuite("Ignite Binary Objects Test Suite");
  
-         suite.addTestSuite(GridPortableMarshallerSelfTest.class);
+         suite.addTestSuite(BinaryMarshallerSelfTest.class);
          suite.addTestSuite(GridPortableMarshallerCtxDisabledSelfTest.class);
-         suite.addTestSuite(GridBinaryObjectBuilderSelfTest.class);
-         suite.addTestSuite(GridBinaryObjectBuilderAdditionalSelfTest.class);
+         suite.addTestSuite(BinaryObjectBuilderSelfTest.class);
+         suite.addTestSuite(BinaryObjectBuilderAdditionalSelfTest.class);
          suite.addTestSuite(BinaryFieldsHeapSelfTest.class);
          suite.addTestSuite(BinaryFieldsOffheapSelfTest.class);
-         suite.addTestSuite(PortableCompactOffsetsHeapSelfTest.class);
-         suite.addTestSuite(PortableCompactOffsetsOffheapSelfTest.class);
+         suite.addTestSuite(BinaryFooterOffsetsHeapSelfTest.class);
+         suite.addTestSuite(BinaryFooterOffsetsOffheapSelfTest.class);
          suite.addTestSuite(GridPortableMetaDataSelfTest.class);
          suite.addTestSuite(GridPortableAffinityKeySelfTest.class);
          suite.addTestSuite(GridPortableWildcardsSelfTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a511fa17/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml
----------------------------------------------------------------------
diff --cc modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml
index 492d4c7,9755033..79909c3
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml
@@@ -52,25 -52,6 +52,26 @@@
              </list>
          </property>
  
 +        <property name="binaryConfiguration">
 +            <bean class="org.apache.ignite.configuration.BinaryConfiguration">
++                <property name="compactFooter" value="false"/>
 +                <property name="typeConfigurations">
 +                    <list>
 +                        <bean class="org.apache.ignite.binary.BinaryTypeConfiguration">
 +                            <property name="typeName" value="org.apache.ignite.platform.PlatformComputeBinarizable"/>
 +                        </bean>
 +                        <bean class="org.apache.ignite.binary.BinaryTypeConfiguration">
 +                            <property name="typeName" value="org.apache.ignite.platform.PlatformComputeJavaBinarizable"/>
 +                        </bean>
 +                        <bean class="org.apache.ignite.binary.BinaryTypeConfiguration">
 +                            <property name="typeName" value="org.apache.ignite.platform.PlatformComputeEnum"/>
 +                        </bean>
 +                    </list>
 +                </property>
 +
 +            </bean>
 +        </property>
 +
          <property name="discoverySpi">
              <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
                  <property name="ipFinder">


[05/13] ignite git commit: IGNITE-1816: Implemented compact footers.

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/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
new file mode 100644
index 0000000..9809a7e
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryMarshallerSelfTest.java
@@ -0,0 +1,3795 @@
+/*
+ * 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 junit.framework.Assert;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.binary.BinaryIdMapper;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.binary.BinaryObjectBuilder;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryRawReader;
+import org.apache.ignite.binary.BinaryRawWriter;
+import org.apache.ignite.binary.BinaryReader;
+import org.apache.ignite.binary.BinarySerializer;
+import org.apache.ignite.binary.BinaryTypeConfiguration;
+import org.apache.ignite.binary.BinaryWriter;
+import org.apache.ignite.binary.Binarylizable;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.portable.builder.BinaryObjectBuilderImpl;
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.internal.util.lang.GridMapEntry;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.marshaller.MarshallerContextTestImpl;
+import org.apache.ignite.marshaller.portable.PortableMarshaller;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.jsr166.ConcurrentHashMap8;
+import sun.misc.Unsafe;
+
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.net.InetSocketAddress;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListSet;
+
+import static org.apache.ignite.internal.portable.PortableThreadLocalMemoryAllocator.THREAD_LOCAL_ALLOC;
+import static org.junit.Assert.assertArrayEquals;
+
+/**
+ * Portable marshaller tests.
+ */
+@SuppressWarnings({"OverlyStrongTypeCast", "ArrayHashCode", "ConstantConditions"})
+public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final Unsafe UNSAFE = GridUnsafe.unsafe();
+
+    /** */
+    protected static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNull() throws Exception {
+        assertNull(marshalUnmarshal(null));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testByte() throws Exception {
+        assertEquals((byte)100, marshalUnmarshal((byte)100).byteValue());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testShort() throws Exception {
+        assertEquals((short)100, marshalUnmarshal((short)100).shortValue());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInt() throws Exception {
+        assertEquals(100, marshalUnmarshal(100).intValue());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLong() throws Exception {
+        assertEquals(100L, marshalUnmarshal(100L).longValue());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testFloat() throws Exception {
+        assertEquals(100.001f, marshalUnmarshal(100.001f).floatValue(), 0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDouble() throws Exception {
+        assertEquals(100.001d, marshalUnmarshal(100.001d).doubleValue(), 0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testChar() throws Exception {
+        assertEquals((char)100, marshalUnmarshal((char)100).charValue());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBoolean() throws Exception {
+        assertEquals(true, marshalUnmarshal(true).booleanValue());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDecimal() throws Exception {
+        BigDecimal val;
+
+        assertEquals((val = BigDecimal.ZERO), marshalUnmarshal(val));
+        assertEquals((val = BigDecimal.valueOf(Long.MAX_VALUE, 0)), marshalUnmarshal(val));
+        assertEquals((val = BigDecimal.valueOf(Long.MIN_VALUE, 0)), marshalUnmarshal(val));
+        assertEquals((val = BigDecimal.valueOf(Long.MAX_VALUE, 8)), marshalUnmarshal(val));
+        assertEquals((val = BigDecimal.valueOf(Long.MIN_VALUE, 8)), marshalUnmarshal(val));
+
+        assertEquals((val = new BigDecimal(new BigInteger("-79228162514264337593543950336"))), marshalUnmarshal(val));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testString() throws Exception {
+        assertEquals("str", marshalUnmarshal("str"));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testUuid() throws Exception {
+        UUID uuid = UUID.randomUUID();
+
+        assertEquals(uuid, marshalUnmarshal(uuid));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDate() throws Exception {
+        Date date = new Date();
+
+        Date val = marshalUnmarshal(date);
+
+        assertEquals(date, val);
+        assertEquals(Date.class, val.getClass());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTimestamp() throws Exception {
+        Timestamp ts = new Timestamp(System.currentTimeMillis());
+
+        ts.setNanos(999999999);
+
+        assertEquals(ts, marshalUnmarshal(ts));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testByteArray() throws Exception {
+        byte[] arr = new byte[] {10, 20, 30};
+
+        assertArrayEquals(arr, marshalUnmarshal(arr));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testShortArray() throws Exception {
+        short[] arr = new short[] {10, 20, 30};
+
+        assertArrayEquals(arr, marshalUnmarshal(arr));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIntArray() throws Exception {
+        int[] arr = new int[] {10, 20, 30};
+
+        assertArrayEquals(arr, marshalUnmarshal(arr));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLongArray() throws Exception {
+        long[] arr = new long[] {10, 20, 30};
+
+        assertArrayEquals(arr, marshalUnmarshal(arr));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testFloatArray() throws Exception {
+        float[] arr = new float[] {10.1f, 20.1f, 30.1f};
+
+        assertArrayEquals(arr, marshalUnmarshal(arr), 0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDoubleArray() throws Exception {
+        double[] arr = new double[] {10.1d, 20.1d, 30.1d};
+
+        assertArrayEquals(arr, marshalUnmarshal(arr), 0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCharArray() throws Exception {
+        char[] arr = new char[] {10, 20, 30};
+
+        assertArrayEquals(arr, marshalUnmarshal(arr));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBooleanArray() throws Exception {
+        boolean[] arr = new boolean[] {true, false, true};
+
+        assertBooleanArrayEquals(arr, marshalUnmarshal(arr));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDecimalArray() throws Exception {
+        BigDecimal[] arr = new BigDecimal[] { BigDecimal.ZERO, BigDecimal.ONE, BigDecimal.TEN } ;
+
+        assertArrayEquals(arr, marshalUnmarshal(arr));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStringArray() throws Exception {
+        String[] arr = new String[] {"str1", "str2", "str3"};
+
+        assertArrayEquals(arr, marshalUnmarshal(arr));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testUuidArray() throws Exception {
+        UUID[] arr = new UUID[] {UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID()};
+
+        assertArrayEquals(arr, marshalUnmarshal(arr));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDateArray() throws Exception {
+        Date[] arr = new Date[] {new Date(11111), new Date(22222), new Date(33333)};
+
+        assertArrayEquals(arr, marshalUnmarshal(arr));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testObjectArray() throws Exception {
+        Object[] arr = new Object[] {1, 2, 3};
+
+        assertArrayEquals(arr, marshalUnmarshal(arr));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCollection() throws Exception {
+        testCollection(new ArrayList<Integer>(3));
+        testCollection(new LinkedHashSet<Integer>());
+        testCollection(new HashSet<Integer>());
+        testCollection(new TreeSet<Integer>());
+        testCollection(new ConcurrentSkipListSet<Integer>());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void testCollection(Collection<Integer> col) throws Exception {
+        col.add(1);
+        col.add(2);
+        col.add(3);
+
+        assertEquals(col, marshalUnmarshal(col));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMap() throws Exception {
+        testMap(new HashMap<Integer, String>());
+        testMap(new LinkedHashMap());
+        testMap(new TreeMap<Integer, String>());
+        testMap(new ConcurrentHashMap8<Integer, String>());
+        testMap(new ConcurrentHashMap<Integer, String>());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void testMap(Map<Integer, String> map) throws Exception {
+        map.put(1, "str1");
+        map.put(2, "str2");
+        map.put(3, "str3");
+
+        assertEquals(map, marshalUnmarshal(map));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMapEntry() throws Exception {
+        Map.Entry<Integer, String> e = new GridMapEntry<>(1, "str1");
+
+        assertEquals(e, marshalUnmarshal(e));
+
+        Map<Integer, String> map = new HashMap<>(1);
+
+        map.put(2, "str2");
+
+        e = F.firstEntry(map);
+
+        Map.Entry<Integer, String> e0 = marshalUnmarshal(e);
+
+        assertEquals(2, e0.getKey().intValue());
+        assertEquals("str2", e0.getValue());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBinaryObject() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        marsh.setTypeConfigurations(Arrays.asList(new BinaryTypeConfiguration(SimpleObject.class.getName())));
+
+        SimpleObject obj = simpleObject();
+
+        BinaryObject po = marshal(obj, marsh);
+
+        BinaryObject po0 = marshalUnmarshal(po, marsh);
+
+        assertTrue(po.hasField("b"));
+        assertTrue(po.hasField("s"));
+        assertTrue(po.hasField("i"));
+        assertTrue(po.hasField("l"));
+        assertTrue(po.hasField("f"));
+        assertTrue(po.hasField("d"));
+        assertTrue(po.hasField("c"));
+        assertTrue(po.hasField("bool"));
+
+        assertFalse(po.hasField("no_such_field"));
+
+        assertEquals(obj, po.deserialize());
+        assertEquals(obj, po0.deserialize());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testEnum() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        marsh.setClassNames(Arrays.asList(TestEnum.class.getName()));
+
+        assertEquals(TestEnum.B, marshalUnmarshal(TestEnum.B, marsh));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDateAndTimestampInSingleObject() throws Exception {
+        BinaryTypeConfiguration cfg1 = new BinaryTypeConfiguration(DateClass1.class.getName());
+
+        PortableMarshaller marsh = createMarshaller();
+
+        marsh.setTypeConfigurations(Arrays.asList(cfg1));
+
+        Date date = new Date();
+        Timestamp ts = new Timestamp(System.currentTimeMillis());
+
+        DateClass1 obj1 = new DateClass1();
+        obj1.date = date;
+        obj1.ts = ts;
+
+        BinaryObject po1 = marshal(obj1, marsh);
+
+        assertEquals(date, po1.field("date"));
+        assertEquals(Date.class, po1.field("date").getClass());
+        assertEquals(ts, po1.field("ts"));
+        assertEquals(Timestamp.class, po1.field("ts").getClass());
+
+        obj1 = po1.deserialize();
+        assertEquals(date, obj1.date);
+        assertEquals(ts, obj1.ts);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSimpleObject() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        marsh.setTypeConfigurations(Arrays.asList(
+            new BinaryTypeConfiguration(SimpleObject.class.getName())
+        ));
+
+        SimpleObject obj = simpleObject();
+
+        BinaryObject po = marshal(obj, marsh);
+
+        assertEquals(obj.hashCode(), po.hashCode());
+
+        assertEquals(obj, po.deserialize());
+
+        assertEquals(obj.b, (byte)po.field("b"));
+        assertEquals(obj.s, (short)po.field("s"));
+        assertEquals(obj.i, (int)po.field("i"));
+        assertEquals(obj.l, (long)po.field("l"));
+        assertEquals(obj.f, (float)po.field("f"), 0);
+        assertEquals(obj.d, (double)po.field("d"), 0);
+        assertEquals(obj.c, (char)po.field("c"));
+        assertEquals(obj.bool, (boolean)po.field("bool"));
+        assertEquals(obj.str, po.field("str"));
+        assertEquals(obj.uuid, po.field("uuid"));
+        assertEquals(obj.date, po.field("date"));
+        assertEquals(Date.class, obj.date.getClass());
+        assertEquals(obj.ts, po.field("ts"));
+        assertArrayEquals(obj.bArr, (byte[])po.field("bArr"));
+        assertArrayEquals(obj.sArr, (short[])po.field("sArr"));
+        assertArrayEquals(obj.iArr, (int[])po.field("iArr"));
+        assertArrayEquals(obj.lArr, (long[])po.field("lArr"));
+        assertArrayEquals(obj.fArr, (float[])po.field("fArr"), 0);
+        assertArrayEquals(obj.dArr, (double[])po.field("dArr"), 0);
+        assertArrayEquals(obj.cArr, (char[])po.field("cArr"));
+        assertBooleanArrayEquals(obj.boolArr, (boolean[])po.field("boolArr"));
+        assertArrayEquals(obj.strArr, (String[])po.field("strArr"));
+        assertArrayEquals(obj.uuidArr, (UUID[])po.field("uuidArr"));
+        assertArrayEquals(obj.dateArr, (Date[])po.field("dateArr"));
+        assertArrayEquals(obj.objArr, (Object[])po.field("objArr"));
+        assertEquals(obj.col, po.field("col"));
+        assertEquals(obj.map, po.field("map"));
+        assertEquals(new Integer(obj.enumVal.ordinal()), new Integer(((Enum<?>)po.field("enumVal")).ordinal()));
+        assertArrayEquals(ordinals(obj.enumArr), ordinals((Enum<?>[])po.field("enumArr")));
+        assertNull(po.field("unknown"));
+
+        BinaryObject innerPo = po.field("inner");
+
+        assertEquals(obj.inner, innerPo.deserialize());
+
+        assertEquals(obj.inner.b, (byte)innerPo.field("b"));
+        assertEquals(obj.inner.s, (short)innerPo.field("s"));
+        assertEquals(obj.inner.i, (int)innerPo.field("i"));
+        assertEquals(obj.inner.l, (long)innerPo.field("l"));
+        assertEquals(obj.inner.f, (float)innerPo.field("f"), 0);
+        assertEquals(obj.inner.d, (double)innerPo.field("d"), 0);
+        assertEquals(obj.inner.c, (char)innerPo.field("c"));
+        assertEquals(obj.inner.bool, (boolean)innerPo.field("bool"));
+        assertEquals(obj.inner.str, innerPo.field("str"));
+        assertEquals(obj.inner.uuid, innerPo.field("uuid"));
+        assertEquals(obj.inner.date, innerPo.field("date"));
+        assertEquals(Date.class, obj.inner.date.getClass());
+        assertEquals(obj.inner.ts, innerPo.field("ts"));
+        assertArrayEquals(obj.inner.bArr, (byte[])innerPo.field("bArr"));
+        assertArrayEquals(obj.inner.sArr, (short[])innerPo.field("sArr"));
+        assertArrayEquals(obj.inner.iArr, (int[])innerPo.field("iArr"));
+        assertArrayEquals(obj.inner.lArr, (long[])innerPo.field("lArr"));
+        assertArrayEquals(obj.inner.fArr, (float[])innerPo.field("fArr"), 0);
+        assertArrayEquals(obj.inner.dArr, (double[])innerPo.field("dArr"), 0);
+        assertArrayEquals(obj.inner.cArr, (char[])innerPo.field("cArr"));
+        assertBooleanArrayEquals(obj.inner.boolArr, (boolean[])innerPo.field("boolArr"));
+        assertArrayEquals(obj.inner.strArr, (String[])innerPo.field("strArr"));
+        assertArrayEquals(obj.inner.uuidArr, (UUID[])innerPo.field("uuidArr"));
+        assertArrayEquals(obj.inner.dateArr, (Date[])innerPo.field("dateArr"));
+        assertArrayEquals(obj.inner.objArr, (Object[])innerPo.field("objArr"));
+        assertEquals(obj.inner.col, innerPo.field("col"));
+        assertEquals(obj.inner.map, innerPo.field("map"));
+        assertEquals(new Integer(obj.inner.enumVal.ordinal()),
+            new Integer(((Enum<?>)innerPo.field("enumVal")).ordinal()));
+        assertArrayEquals(ordinals(obj.inner.enumArr), ordinals((Enum<?>[])innerPo.field("enumArr")));
+        assertNull(innerPo.field("inner"));
+        assertNull(innerPo.field("unknown"));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPortable() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        marsh.setTypeConfigurations(Arrays.asList(
+            new BinaryTypeConfiguration(SimpleObject.class.getName()),
+            new BinaryTypeConfiguration(TestBinary.class.getName())
+        ));
+
+        TestBinary obj = binaryObject();
+
+        BinaryObject po = marshal(obj, marsh);
+
+        assertEquals(obj.hashCode(), po.hashCode());
+
+        assertEquals(obj, po.deserialize());
+
+        assertEquals(obj.b, (byte)po.field("_b"));
+        assertEquals(obj.s, (short)po.field("_s"));
+        assertEquals(obj.i, (int)po.field("_i"));
+        assertEquals(obj.l, (long)po.field("_l"));
+        assertEquals(obj.f, (float)po.field("_f"), 0);
+        assertEquals(obj.d, (double)po.field("_d"), 0);
+        assertEquals(obj.c, (char)po.field("_c"));
+        assertEquals(obj.bool, (boolean)po.field("_bool"));
+        assertEquals(obj.str, po.field("_str"));
+        assertEquals(obj.uuid, po.field("_uuid"));
+        assertEquals(obj.date, po.field("_date"));
+        assertEquals(obj.ts, po.field("_ts"));
+        assertArrayEquals(obj.bArr, (byte[])po.field("_bArr"));
+        assertArrayEquals(obj.sArr, (short[])po.field("_sArr"));
+        assertArrayEquals(obj.iArr, (int[])po.field("_iArr"));
+        assertArrayEquals(obj.lArr, (long[])po.field("_lArr"));
+        assertArrayEquals(obj.fArr, (float[])po.field("_fArr"), 0);
+        assertArrayEquals(obj.dArr, (double[])po.field("_dArr"), 0);
+        assertArrayEquals(obj.cArr, (char[])po.field("_cArr"));
+        assertBooleanArrayEquals(obj.boolArr, (boolean[])po.field("_boolArr"));
+        assertArrayEquals(obj.strArr, (String[])po.field("_strArr"));
+        assertArrayEquals(obj.uuidArr, (UUID[])po.field("_uuidArr"));
+        assertArrayEquals(obj.dateArr, (Date[])po.field("_dateArr"));
+        assertArrayEquals(obj.objArr, (Object[])po.field("_objArr"));
+        assertEquals(obj.col, po.field("_col"));
+        assertEquals(obj.map, po.field("_map"));
+        assertEquals(new Integer(obj.enumVal.ordinal()), new Integer(((Enum<?>)po.field("_enumVal")).ordinal()));
+        assertArrayEquals(ordinals(obj.enumArr), ordinals((Enum<?>[])po.field("_enumArr")));
+        assertNull(po.field("unknown"));
+
+        BinaryObject simplePo = po.field("_simple");
+
+        assertEquals(obj.simple, simplePo.deserialize());
+
+        assertEquals(obj.simple.b, (byte)simplePo.field("b"));
+        assertEquals(obj.simple.s, (short)simplePo.field("s"));
+        assertEquals(obj.simple.i, (int)simplePo.field("i"));
+        assertEquals(obj.simple.l, (long)simplePo.field("l"));
+        assertEquals(obj.simple.f, (float)simplePo.field("f"), 0);
+        assertEquals(obj.simple.d, (double)simplePo.field("d"), 0);
+        assertEquals(obj.simple.c, (char)simplePo.field("c"));
+        assertEquals(obj.simple.bool, (boolean)simplePo.field("bool"));
+        assertEquals(obj.simple.str, simplePo.field("str"));
+        assertEquals(obj.simple.uuid, simplePo.field("uuid"));
+        assertEquals(obj.simple.date, simplePo.field("date"));
+        assertEquals(Date.class, obj.simple.date.getClass());
+        assertEquals(obj.simple.ts, simplePo.field("ts"));
+        assertArrayEquals(obj.simple.bArr, (byte[])simplePo.field("bArr"));
+        assertArrayEquals(obj.simple.sArr, (short[])simplePo.field("sArr"));
+        assertArrayEquals(obj.simple.iArr, (int[])simplePo.field("iArr"));
+        assertArrayEquals(obj.simple.lArr, (long[])simplePo.field("lArr"));
+        assertArrayEquals(obj.simple.fArr, (float[])simplePo.field("fArr"), 0);
+        assertArrayEquals(obj.simple.dArr, (double[])simplePo.field("dArr"), 0);
+        assertArrayEquals(obj.simple.cArr, (char[])simplePo.field("cArr"));
+        assertBooleanArrayEquals(obj.simple.boolArr, (boolean[])simplePo.field("boolArr"));
+        assertArrayEquals(obj.simple.strArr, (String[])simplePo.field("strArr"));
+        assertArrayEquals(obj.simple.uuidArr, (UUID[])simplePo.field("uuidArr"));
+        assertArrayEquals(obj.simple.dateArr, (Date[])simplePo.field("dateArr"));
+        assertArrayEquals(obj.simple.objArr, (Object[])simplePo.field("objArr"));
+        assertEquals(obj.simple.col, simplePo.field("col"));
+        assertEquals(obj.simple.map, simplePo.field("map"));
+        assertEquals(new Integer(obj.simple.enumVal.ordinal()),
+            new Integer(((Enum<?>)simplePo.field("enumVal")).ordinal()));
+        assertArrayEquals(ordinals(obj.simple.enumArr), ordinals((Enum<?>[])simplePo.field("enumArr")));
+        assertNull(simplePo.field("simple"));
+        assertNull(simplePo.field("portable"));
+        assertNull(simplePo.field("unknown"));
+
+        BinaryObject portablePo = po.field("_portable");
+
+        assertEquals(obj.portable, portablePo.deserialize());
+
+        assertEquals(obj.portable.b, (byte)portablePo.field("_b"));
+        assertEquals(obj.portable.s, (short)portablePo.field("_s"));
+        assertEquals(obj.portable.i, (int)portablePo.field("_i"));
+        assertEquals(obj.portable.l, (long)portablePo.field("_l"));
+        assertEquals(obj.portable.f, (float)portablePo.field("_f"), 0);
+        assertEquals(obj.portable.d, (double)portablePo.field("_d"), 0);
+        assertEquals(obj.portable.c, (char)portablePo.field("_c"));
+        assertEquals(obj.portable.bool, (boolean)portablePo.field("_bool"));
+        assertEquals(obj.portable.str, portablePo.field("_str"));
+        assertEquals(obj.portable.uuid, portablePo.field("_uuid"));
+        assertEquals(obj.portable.date, portablePo.field("_date"));
+        assertEquals(obj.portable.ts, portablePo.field("_ts"));
+        assertArrayEquals(obj.portable.bArr, (byte[])portablePo.field("_bArr"));
+        assertArrayEquals(obj.portable.sArr, (short[])portablePo.field("_sArr"));
+        assertArrayEquals(obj.portable.iArr, (int[])portablePo.field("_iArr"));
+        assertArrayEquals(obj.portable.lArr, (long[])portablePo.field("_lArr"));
+        assertArrayEquals(obj.portable.fArr, (float[])portablePo.field("_fArr"), 0);
+        assertArrayEquals(obj.portable.dArr, (double[])portablePo.field("_dArr"), 0);
+        assertArrayEquals(obj.portable.cArr, (char[])portablePo.field("_cArr"));
+        assertBooleanArrayEquals(obj.portable.boolArr, (boolean[])portablePo.field("_boolArr"));
+        assertArrayEquals(obj.portable.strArr, (String[])portablePo.field("_strArr"));
+        assertArrayEquals(obj.portable.uuidArr, (UUID[])portablePo.field("_uuidArr"));
+        assertArrayEquals(obj.portable.dateArr, (Date[])portablePo.field("_dateArr"));
+        assertArrayEquals(obj.portable.objArr, (Object[])portablePo.field("_objArr"));
+        assertEquals(obj.portable.col, portablePo.field("_col"));
+        assertEquals(obj.portable.map, portablePo.field("_map"));
+        assertEquals(new Integer(obj.portable.enumVal.ordinal()),
+            new Integer(((Enum<?>)portablePo.field("_enumVal")).ordinal()));
+        assertArrayEquals(ordinals(obj.portable.enumArr), ordinals((Enum<?>[])portablePo.field("_enumArr")));
+        assertNull(portablePo.field("_simple"));
+        assertNull(portablePo.field("_portable"));
+        assertNull(portablePo.field("unknown"));
+    }
+
+    /**
+     * @param obj Simple object.
+     * @param po Portable object.
+     */
+    private void checkSimpleObjectData(SimpleObject obj, BinaryObject po) {
+        assertEquals(obj.b, (byte)po.field("b"));
+        assertEquals(obj.s, (short)po.field("s"));
+        assertEquals(obj.i, (int)po.field("i"));
+        assertEquals(obj.l, (long)po.field("l"));
+        assertEquals(obj.f, (float)po.field("f"), 0);
+        assertEquals(obj.d, (double)po.field("d"), 0);
+        assertEquals(obj.c, (char)po.field("c"));
+        assertEquals(obj.bool, (boolean)po.field("bool"));
+        assertEquals(obj.str, po.field("str"));
+        assertEquals(obj.uuid, po.field("uuid"));
+        assertEquals(obj.date, po.field("date"));
+        assertEquals(Date.class, obj.date.getClass());
+        assertEquals(obj.ts, po.field("ts"));
+        assertArrayEquals(obj.bArr, (byte[])po.field("bArr"));
+        assertArrayEquals(obj.sArr, (short[])po.field("sArr"));
+        assertArrayEquals(obj.iArr, (int[])po.field("iArr"));
+        assertArrayEquals(obj.lArr, (long[])po.field("lArr"));
+        assertArrayEquals(obj.fArr, (float[])po.field("fArr"), 0);
+        assertArrayEquals(obj.dArr, (double[])po.field("dArr"), 0);
+        assertArrayEquals(obj.cArr, (char[])po.field("cArr"));
+        assertBooleanArrayEquals(obj.boolArr, (boolean[])po.field("boolArr"));
+        assertArrayEquals(obj.strArr, (String[])po.field("strArr"));
+        assertArrayEquals(obj.uuidArr, (UUID[])po.field("uuidArr"));
+        assertArrayEquals(obj.dateArr, (Date[])po.field("dateArr"));
+        assertArrayEquals(obj.objArr, (Object[])po.field("objArr"));
+        assertEquals(obj.col, po.field("col"));
+        assertEquals(obj.map, po.field("map"));
+        assertEquals(new Integer(obj.enumVal.ordinal()), new Integer(((Enum<?>)po.field("enumVal")).ordinal()));
+        assertArrayEquals(ordinals(obj.enumArr), ordinals((Enum<?>[])po.field("enumArr")));
+        assertNull(po.field("unknown"));
+
+        assertEquals(obj, po.deserialize());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClassWithoutPublicConstructor() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        marsh.setTypeConfigurations(Arrays.asList(
+                new BinaryTypeConfiguration(NoPublicConstructor.class.getName()),
+                new BinaryTypeConfiguration(NoPublicDefaultConstructor.class.getName()),
+                new BinaryTypeConfiguration(ProtectedConstructor.class.getName()))
+        );
+
+        NoPublicConstructor npc = new NoPublicConstructor();
+        BinaryObject npc2 = marshal(npc, marsh);
+
+        assertEquals("test", npc2.<NoPublicConstructor>deserialize().val);
+
+        NoPublicDefaultConstructor npdc = new NoPublicDefaultConstructor(239);
+        BinaryObject npdc2 = marshal(npdc, marsh);
+
+        assertEquals(239, npdc2.<NoPublicDefaultConstructor>deserialize().val);
+
+        ProtectedConstructor pc = new ProtectedConstructor();
+        BinaryObject pc2 = marshal(pc, marsh);
+
+        assertEquals(ProtectedConstructor.class, pc2.<ProtectedConstructor>deserialize().getClass());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCustomSerializer() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        BinaryTypeConfiguration type =
+            new BinaryTypeConfiguration(CustomSerializedObject1.class.getName());
+
+        type.setSerializer(new CustomSerializer1());
+
+        marsh.setTypeConfigurations(Arrays.asList(type));
+
+        CustomSerializedObject1 obj1 = new CustomSerializedObject1(10);
+
+        BinaryObject po1 = marshal(obj1, marsh);
+
+        assertEquals(20, po1.<CustomSerializedObject1>deserialize().val);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCustomSerializerWithGlobal() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        marsh.setSerializer(new CustomSerializer1());
+
+        BinaryTypeConfiguration type1 =
+            new BinaryTypeConfiguration(CustomSerializedObject1.class.getName());
+        BinaryTypeConfiguration type2 =
+            new BinaryTypeConfiguration(CustomSerializedObject2.class.getName());
+
+        type2.setSerializer(new CustomSerializer2());
+
+        marsh.setTypeConfigurations(Arrays.asList(type1, type2));
+
+        CustomSerializedObject1 obj1 = new CustomSerializedObject1(10);
+
+        BinaryObject po1 = marshal(obj1, marsh);
+
+        assertEquals(20, po1.<CustomSerializedObject1>deserialize().val);
+
+        CustomSerializedObject2 obj2 = new CustomSerializedObject2(10);
+
+        BinaryObject po2 = marshal(obj2, marsh);
+
+        assertEquals(30, po2.<CustomSerializedObject2>deserialize().val);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCustomIdMapper() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        BinaryTypeConfiguration type =
+            new BinaryTypeConfiguration(CustomMappedObject1.class.getName());
+
+        type.setIdMapper(new BinaryIdMapper() {
+            @Override public int typeId(String clsName) {
+                return 11111;
+            }
+
+            @Override public int fieldId(int typeId, String fieldName) {
+                assert typeId == 11111;
+
+                if ("val1".equals(fieldName))
+                    return 22222;
+                else if ("val2".equals(fieldName))
+                    return 33333;
+
+                assert false : "Unknown field: " + fieldName;
+
+                return 0;
+            }
+        });
+
+        marsh.setTypeConfigurations(Arrays.asList(type));
+
+        CustomMappedObject1 obj1 = new CustomMappedObject1(10, "str");
+
+        BinaryObjectEx po1 = marshal(obj1, marsh);
+
+        assertEquals(11111, po1.typeId());
+        assertEquals(10, po1.field(22222));
+        assertEquals("str", po1.field(33333));
+
+        assertEquals(10, po1.<CustomMappedObject1>deserialize().val1);
+        assertEquals("str", po1.<CustomMappedObject1>deserialize().val2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCustomIdMapperWithGlobal() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        marsh.setIdMapper(new BinaryIdMapper() {
+            @Override public int typeId(String clsName) {
+                return 11111;
+            }
+
+            @Override public int fieldId(int typeId, String fieldName) {
+                assert typeId == 11111;
+
+                if ("val1".equals(fieldName)) return 22222;
+                else if ("val2".equals(fieldName)) return 33333;
+
+                assert false : "Unknown field: " + fieldName;
+
+                return 0;
+            }
+        });
+
+        BinaryTypeConfiguration type1 =
+            new BinaryTypeConfiguration(CustomMappedObject1.class.getName());
+        BinaryTypeConfiguration type2 =
+            new BinaryTypeConfiguration(CustomMappedObject2.class.getName());
+
+        type2.setIdMapper(new BinaryIdMapper() {
+            @Override public int typeId(String clsName) {
+                return 44444;
+            }
+
+            @Override public int fieldId(int typeId, String fieldName) {
+                assert typeId == 44444;
+
+                if ("val1".equals(fieldName)) return 55555;
+                else if ("val2".equals(fieldName)) return 66666;
+
+                assert false : "Unknown field: " + fieldName;
+
+                return 0;
+            }
+        });
+
+        marsh.setTypeConfigurations(Arrays.asList(type1, type2));
+
+        CustomMappedObject1 obj1 = new CustomMappedObject1(10, "str1");
+
+        BinaryObjectEx po1 = marshal(obj1, marsh);
+
+        assertEquals(11111, po1.typeId());
+        assertEquals(10, po1.field(22222));
+        assertEquals("str1", po1.field(33333));
+
+        assertEquals(10, po1.<CustomMappedObject1>deserialize().val1);
+        assertEquals("str1", po1.<CustomMappedObject1>deserialize().val2);
+
+        CustomMappedObject2 obj2 = new CustomMappedObject2(20, "str2");
+
+        BinaryObjectEx po2 = marshal(obj2, marsh);
+
+        assertEquals(44444, po2.typeId());
+        assertEquals(20, po2.field(55555));
+        assertEquals("str2", po2.field(66666));
+
+        assertEquals(20, po2.<CustomMappedObject2>deserialize().val1);
+        assertEquals("str2", po2.<CustomMappedObject2>deserialize().val2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDynamicObject() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        marsh.setTypeConfigurations(Arrays.asList(
+            new BinaryTypeConfiguration(DynamicObject.class.getName())
+        ));
+
+        initializePortableContext(marsh);
+
+        BinaryObject po1 = marshal(new DynamicObject(0, 10, 20, 30), marsh);
+
+        assertEquals(new Integer(10), po1.field("val1"));
+        assertEquals(null, po1.field("val2"));
+        assertEquals(null, po1.field("val3"));
+
+        DynamicObject do1 = po1.deserialize();
+
+        assertEquals(10, do1.val1);
+        assertEquals(0, do1.val2);
+        assertEquals(0, do1.val3);
+
+        BinaryObject po2 = marshal(new DynamicObject(1, 10, 20, 30), marsh);
+
+        assertEquals(new Integer(10), po2.field("val1"));
+        assertEquals(new Integer(20), po2.field("val2"));
+        assertEquals(null, po2.field("val3"));
+
+        DynamicObject do2 = po2.deserialize();
+
+        assertEquals(10, do2.val1);
+        assertEquals(20, do2.val2);
+        assertEquals(0, do2.val3);
+
+        BinaryObject po3 = marshal(new DynamicObject(2, 10, 20, 30), marsh);
+
+        assertEquals(new Integer(10), po3.field("val1"));
+        assertEquals(new Integer(20), po3.field("val2"));
+        assertEquals(new Integer(30), po3.field("val3"));
+
+        DynamicObject do3 = po3.deserialize();
+
+        assertEquals(10, do3.val1);
+        assertEquals(20, do3.val2);
+        assertEquals(30, do3.val3);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCycleLink() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        marsh.setTypeConfigurations(Arrays.asList(
+            new BinaryTypeConfiguration(CycleLinkObject.class.getName())
+        ));
+
+        CycleLinkObject obj = new CycleLinkObject();
+
+        obj.self = obj;
+
+        BinaryObject po = marshal(obj, marsh);
+
+        CycleLinkObject obj0 = po.deserialize();
+
+        assert obj0.self == obj0;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDetached() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        marsh.setTypeConfigurations(Arrays.asList(
+            new BinaryTypeConfiguration(DetachedTestObject.class.getName()),
+            new BinaryTypeConfiguration(DetachedInnerTestObject.class.getName())
+        ));
+
+        UUID id = UUID.randomUUID();
+
+        DetachedTestObject obj = marshal(new DetachedTestObject(
+            new DetachedInnerTestObject(null, id)), marsh).deserialize();
+
+        assertEquals(id, obj.inner1.id);
+        assertEquals(id, obj.inner4.id);
+
+        assert obj.inner1 == obj.inner4;
+
+        BinaryObjectImpl innerPo = (BinaryObjectImpl)obj.inner2;
+
+        assert innerPo.detached();
+
+        DetachedInnerTestObject inner = innerPo.deserialize();
+
+        assertEquals(id, inner.id);
+
+        BinaryObjectImpl detachedPo = (BinaryObjectImpl)innerPo.detach();
+
+        assert detachedPo.detached();
+
+        inner = detachedPo.deserialize();
+
+        assertEquals(id, inner.id);
+
+        innerPo = (BinaryObjectImpl)obj.inner3;
+
+        assert innerPo.detached();
+
+        inner = innerPo.deserialize();
+
+        assertEquals(id, inner.id);
+        assertNotNull(inner.inner);
+
+        detachedPo = (BinaryObjectImpl)innerPo.detach();
+
+        assert detachedPo.detached();
+
+        inner = innerPo.deserialize();
+
+        assertEquals(id, inner.id);
+        assertNotNull(inner.inner);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCollectionFields() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        marsh.setTypeConfigurations(Arrays.asList(
+            new BinaryTypeConfiguration(CollectionFieldsObject.class.getName()),
+            new BinaryTypeConfiguration(Key.class.getName()),
+            new BinaryTypeConfiguration(Value.class.getName())
+        ));
+
+        Object[] arr = new Object[] {new Value(1), new Value(2), new Value(3)};
+        Collection<Value> col = Arrays.asList(new Value(4), new Value(5), new Value(6));
+        Map<Key, Value> map = F.asMap(new Key(10), new Value(10), new Key(20), new Value(20), new Key(30), new Value(30));
+
+        CollectionFieldsObject obj = new CollectionFieldsObject(arr, col, map);
+
+        BinaryObject po = marshal(obj, marsh);
+
+        Object[] arr0 = po.field("arr");
+
+        assertEquals(3, arr0.length);
+
+        int i = 1;
+
+        for (Object valPo : arr0)
+            assertEquals(i++, ((BinaryObject)valPo).<Value>deserialize().val);
+
+        Collection<BinaryObject> col0 = po.field("col");
+
+        i = 4;
+
+        for (BinaryObject valPo : col0)
+            assertEquals(i++, valPo.<Value>deserialize().val);
+
+        Map<BinaryObject, BinaryObject> map0 = po.field("map");
+
+        for (Map.Entry<BinaryObject, BinaryObject> e : map0.entrySet())
+            assertEquals(e.getKey().<Key>deserialize().key, e.getValue().<Value>deserialize().val);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    // TODO: Only with full headers.
+    public void _testDefaultMapping() throws Exception {
+        PortableMarshaller marsh1 = createMarshaller();
+
+        BinaryTypeConfiguration customMappingType =
+            new BinaryTypeConfiguration(TestBinary.class.getName());
+
+        customMappingType.setIdMapper(new BinaryIdMapper() {
+            @Override public int typeId(String clsName) {
+                String typeName;
+
+                try {
+                    Method mtd = PortableContext.class.getDeclaredMethod("typeName", String.class);
+
+                    mtd.setAccessible(true);
+
+                    typeName = (String)mtd.invoke(null, clsName);
+                }
+                catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) {
+                    throw new RuntimeException(e);
+                }
+
+                return typeName.toLowerCase().hashCode();
+            }
+
+            @Override public int fieldId(int typeId, String fieldName) {
+                return fieldName.toLowerCase().hashCode();
+            }
+        });
+
+        marsh1.setTypeConfigurations(Arrays.asList(
+            new BinaryTypeConfiguration(SimpleObject.class.getName()),
+            customMappingType
+        ));
+
+        TestBinary obj = binaryObject();
+
+        BinaryObjectImpl po = marshal(obj, marsh1);
+
+        PortableMarshaller marsh2 = createMarshaller();
+
+        marsh2.setTypeConfigurations(Arrays.asList(
+            new BinaryTypeConfiguration(SimpleObject.class.getName()),
+            new BinaryTypeConfiguration(TestBinary.class.getName())
+        ));
+
+        PortableContext ctx = initializePortableContext(marsh2);
+
+        po.context(ctx);
+
+        assertEquals(obj, po.deserialize());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTypeNames() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        BinaryTypeConfiguration customType1 = new BinaryTypeConfiguration(Value.class.getName());
+
+        customType1.setIdMapper(new BinaryIdMapper() {
+            @Override public int typeId(String clsName) {
+                return 300;
+            }
+
+            @Override public int fieldId(int typeId, String fieldName) {
+                return 0;
+            }
+        });
+
+        BinaryTypeConfiguration customType2 = new BinaryTypeConfiguration("org.gridgain.NonExistentClass1");
+
+        customType2.setIdMapper(new BinaryIdMapper() {
+            @Override public int typeId(String clsName) {
+                return 400;
+            }
+
+            @Override public int fieldId(int typeId, String fieldName) {
+                return 0;
+            }
+        });
+
+        BinaryTypeConfiguration customType3 = new BinaryTypeConfiguration("NonExistentClass2");
+
+        customType3.setIdMapper(new BinaryIdMapper() {
+            @Override public int typeId(String clsName) {
+                return 500;
+            }
+
+            @Override public int fieldId(int typeId, String fieldName) {
+                return 0;
+            }
+        });
+
+        BinaryTypeConfiguration customType4 = new BinaryTypeConfiguration("NonExistentClass5");
+
+        customType4.setIdMapper(new BinaryIdMapper() {
+            @Override public int typeId(String clsName) {
+                return 0;
+            }
+
+            @Override public int fieldId(int typeId, String fieldName) {
+                return 0;
+            }
+        });
+
+        marsh.setTypeConfigurations(Arrays.asList(
+            new BinaryTypeConfiguration(Key.class.getName()),
+            new BinaryTypeConfiguration("org.gridgain.NonExistentClass3"),
+            new BinaryTypeConfiguration("NonExistentClass4"),
+            customType1,
+            customType2,
+            customType3,
+            customType4
+        ));
+
+        PortableContext ctx = initializePortableContext(marsh);
+
+        assertEquals("notconfiguredclass".hashCode(), ctx.typeId("NotConfiguredClass"));
+        assertEquals("key".hashCode(), ctx.typeId("Key"));
+        assertEquals("nonexistentclass3".hashCode(), ctx.typeId("NonExistentClass3"));
+        assertEquals("nonexistentclass4".hashCode(), ctx.typeId("NonExistentClass4"));
+        assertEquals(300, ctx.typeId(getClass().getSimpleName() + "$Value"));
+        assertEquals(400, ctx.typeId("NonExistentClass1"));
+        assertEquals(500, ctx.typeId("NonExistentClass2"));
+        assertEquals("nonexistentclass5".hashCode(), ctx.typeId("NonExistentClass5"));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testFieldIdMapping() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        BinaryTypeConfiguration customType1 = new BinaryTypeConfiguration(Value.class.getName());
+
+        customType1.setIdMapper(new BinaryIdMapper() {
+            @Override public int typeId(String clsName) {
+                return 300;
+            }
+
+            @Override public int fieldId(int typeId, String fieldName) {
+                switch (fieldName) {
+                    case "val1":
+                        return 301;
+
+                    case "val2":
+                        return 302;
+
+                    default:
+                        return 0;
+                }
+            }
+        });
+
+        BinaryTypeConfiguration customType2 = new BinaryTypeConfiguration("NonExistentClass1");
+
+        customType2.setIdMapper(new BinaryIdMapper() {
+            @Override public int typeId(String clsName) {
+                return 400;
+            }
+
+            @Override public int fieldId(int typeId, String fieldName) {
+                switch (fieldName) {
+                    case "val1":
+                        return 401;
+
+                    case "val2":
+                        return 402;
+
+                    default:
+                        return 0;
+                }
+            }
+        });
+
+        marsh.setTypeConfigurations(Arrays.asList(new BinaryTypeConfiguration(Key.class.getName()),
+            new BinaryTypeConfiguration("NonExistentClass2"),
+            customType1,
+            customType2));
+
+        PortableContext ctx = initializePortableContext(marsh);
+
+        assertEquals("val".hashCode(), ctx.fieldId("key".hashCode(), "val"));
+        assertEquals("val".hashCode(), ctx.fieldId("nonexistentclass2".hashCode(), "val"));
+        assertEquals("val".hashCode(), ctx.fieldId("notconfiguredclass".hashCode(), "val"));
+        assertEquals(301, ctx.fieldId(300, "val1"));
+        assertEquals(302, ctx.fieldId(300, "val2"));
+        assertEquals("val3".hashCode(), ctx.fieldId(300, "val3"));
+        assertEquals(401, ctx.fieldId(400, "val1"));
+        assertEquals(402, ctx.fieldId(400, "val2"));
+        assertEquals("val3".hashCode(), ctx.fieldId(400, "val3"));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDuplicateTypeId() throws Exception {
+        final PortableMarshaller marsh = createMarshaller();
+
+        BinaryTypeConfiguration customType1 = new BinaryTypeConfiguration("org.gridgain.Class1");
+
+        customType1.setIdMapper(new BinaryIdMapper() {
+            @Override public int typeId(String clsName) {
+                return 100;
+            }
+
+            @Override public int fieldId(int typeId, String fieldName) {
+                return 0;
+            }
+        });
+
+        BinaryTypeConfiguration customType2 = new BinaryTypeConfiguration("org.gridgain.Class2");
+
+        customType2.setIdMapper(new BinaryIdMapper() {
+            @Override public int typeId(String clsName) {
+                return 100;
+            }
+
+            @Override public int fieldId(int typeId, String fieldName) {
+                return 0;
+            }
+        });
+
+        marsh.setTypeConfigurations(Arrays.asList(customType1, customType2));
+
+        try {
+            initializePortableContext(marsh);
+        }
+        catch (IgniteCheckedException e) {
+            assertEquals("Duplicate type ID [clsName=org.gridgain.Class1, id=100]",
+                e.getCause().getCause().getMessage());
+
+            return;
+        }
+
+        assert false;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPortableCopy() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        marsh.setTypeConfigurations(Arrays.asList(
+            new BinaryTypeConfiguration(SimpleObject.class.getName())
+        ));
+
+        initializePortableContext(marsh);
+
+        SimpleObject obj = simpleObject();
+
+        final BinaryObject po = marshal(obj, marsh);
+
+        assertEquals(obj, po.deserialize());
+
+        BinaryObject copy = copy(po, null);
+
+        assertEquals(obj, copy.deserialize());
+
+        copy = copy(po, new HashMap<String, Object>());
+
+        assertEquals(obj, copy.deserialize());
+
+        Map<String, Object> map = new HashMap<>(1, 1.0f);
+
+        map.put("i", 3);
+
+        copy = copy(po, map);
+
+        assertEquals((byte)2, copy.<Byte>field("b").byteValue());
+        assertEquals((short)2, copy.<Short>field("s").shortValue());
+        assertEquals(3, copy.<Integer>field("i").intValue());
+        assertEquals(2L, copy.<Long>field("l").longValue());
+        assertEquals(2.2f, copy.<Float>field("f").floatValue(), 0);
+        assertEquals(2.2d, copy.<Double>field("d").doubleValue(), 0);
+        assertEquals((char)2, copy.<Character>field("c").charValue());
+        assertEquals(false, copy.<Boolean>field("bool").booleanValue());
+
+        SimpleObject obj0 = copy.deserialize();
+
+        assertEquals((byte)2, obj0.b);
+        assertEquals((short)2, obj0.s);
+        assertEquals(3, obj0.i);
+        assertEquals(2L, obj0.l);
+        assertEquals(2.2f, obj0.f, 0);
+        assertEquals(2.2d, obj0.d, 0);
+        assertEquals((char)2, obj0.c);
+        assertEquals(false, obj0.bool);
+
+        map = new HashMap<>(3, 1.0f);
+
+        map.put("b", (byte)3);
+        map.put("l", 3L);
+        map.put("bool", true);
+
+        copy = copy(po, map);
+
+        assertEquals((byte)3, copy.<Byte>field("b").byteValue());
+        assertEquals((short)2, copy.<Short>field("s").shortValue());
+        assertEquals(2, copy.<Integer>field("i").intValue());
+        assertEquals(3L, copy.<Long>field("l").longValue());
+        assertEquals(2.2f, copy.<Float>field("f").floatValue(), 0);
+        assertEquals(2.2d, copy.<Double>field("d").doubleValue(), 0);
+        assertEquals((char)2, copy.<Character>field("c").charValue());
+        assertEquals(true, copy.<Boolean>field("bool").booleanValue());
+
+        obj0 = copy.deserialize();
+
+        assertEquals((byte)3, obj0.b);
+        assertEquals((short)2, obj0.s);
+        assertEquals(2, obj0.i);
+        assertEquals(3L, obj0.l);
+        assertEquals(2.2f, obj0.f, 0);
+        assertEquals(2.2d, obj0.d, 0);
+        assertEquals((char)2, obj0.c);
+        assertEquals(true, obj0.bool);
+
+        map = new HashMap<>(8, 1.0f);
+
+        map.put("b", (byte)3);
+        map.put("s", (short)3);
+        map.put("i", 3);
+        map.put("l", 3L);
+        map.put("f", 3.3f);
+        map.put("d", 3.3d);
+        map.put("c", (char)3);
+        map.put("bool", true);
+
+        copy = copy(po, map);
+
+        assertEquals((byte)3, copy.<Byte>field("b").byteValue());
+        assertEquals((short)3, copy.<Short>field("s").shortValue());
+        assertEquals(3, copy.<Integer>field("i").intValue());
+        assertEquals(3L, copy.<Long>field("l").longValue());
+        assertEquals(3.3f, copy.<Float>field("f").floatValue(), 0);
+        assertEquals(3.3d, copy.<Double>field("d").doubleValue(), 0);
+        assertEquals((char)3, copy.<Character>field("c").charValue());
+        assertEquals(true, copy.<Boolean>field("bool").booleanValue());
+
+        obj0 = copy.deserialize();
+
+        assertEquals((byte)3, obj0.b);
+        assertEquals((short)3, obj0.s);
+        assertEquals(3, obj0.i);
+        assertEquals(3L, obj0.l);
+        assertEquals(3.3f, obj0.f, 0);
+        assertEquals(3.3d, obj0.d, 0);
+        assertEquals((char)3, obj0.c);
+        assertEquals(true, obj0.bool);
+
+//        GridTestUtils.assertThrows(
+//            log,
+//            new Callable<Object>() {
+//                @Override public Object call() throws Exception {
+//                    po.copy(F.<String, Object>asMap("i", false));
+//
+//                    return null;
+//                }
+//            },
+//            PortableException.class,
+//            "Invalid value type for field: i"
+//        );
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPortableCopyString() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        marsh.setTypeConfigurations(Arrays.asList(
+            new BinaryTypeConfiguration(SimpleObject.class.getName())
+        ));
+
+        SimpleObject obj = simpleObject();
+
+        BinaryObject po = marshal(obj, marsh);
+
+        BinaryObject copy = copy(po, F.<String, Object>asMap("str", "str3"));
+
+        assertEquals("str3", copy.<String>field("str"));
+
+        SimpleObject obj0 = copy.deserialize();
+
+        assertEquals("str3", obj0.str);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPortableCopyUuid() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        marsh.setTypeConfigurations(Arrays.asList(
+            new BinaryTypeConfiguration(SimpleObject.class.getName())
+        ));
+
+        SimpleObject obj = simpleObject();
+
+        BinaryObject po = marshal(obj, marsh);
+
+        UUID uuid = UUID.randomUUID();
+
+        BinaryObject copy = copy(po, F.<String, Object>asMap("uuid", uuid));
+
+        assertEquals(uuid, copy.<UUID>field("uuid"));
+
+        SimpleObject obj0 = copy.deserialize();
+
+        assertEquals(uuid, obj0.uuid);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPortableCopyByteArray() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        marsh.setTypeConfigurations(Arrays.asList(
+            new BinaryTypeConfiguration(SimpleObject.class.getName())
+        ));
+
+        SimpleObject obj = simpleObject();
+
+        BinaryObject po = marshal(obj, marsh);
+
+        BinaryObject copy = copy(po, F.<String, Object>asMap("bArr", new byte[]{1, 2, 3}));
+
+        assertArrayEquals(new byte[] {1, 2, 3}, copy.<byte[]>field("bArr"));
+
+        SimpleObject obj0 = copy.deserialize();
+
+        assertArrayEquals(new byte[] {1, 2, 3}, obj0.bArr);
+    }
+
+    /**
+     * @param po Portable object.
+     * @param fields Fields.
+     * @return Copy.
+     */
+    private BinaryObject copy(BinaryObject po, Map<String, Object> fields) {
+        BinaryObjectBuilder builder = BinaryObjectBuilderImpl.wrap(po);
+
+        if (fields != null) {
+            for (Map.Entry<String, Object> e : fields.entrySet())
+                builder.setField(e.getKey(), e.getValue());
+        }
+
+        return builder.build();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPortableCopyShortArray() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        marsh.setTypeConfigurations(Arrays.asList(
+            new BinaryTypeConfiguration(SimpleObject.class.getName())
+        ));
+
+        SimpleObject obj = simpleObject();
+
+        BinaryObject po = marshal(obj, marsh);
+
+        BinaryObject copy = copy(po, F.<String, Object>asMap("sArr", new short[]{1, 2, 3}));
+
+        assertArrayEquals(new short[] {1, 2, 3}, copy.<short[]>field("sArr"));
+
+        SimpleObject obj0 = copy.deserialize();
+
+        assertArrayEquals(new short[] {1, 2, 3}, obj0.sArr);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPortableCopyIntArray() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        marsh.setTypeConfigurations(Arrays.asList(
+            new BinaryTypeConfiguration(SimpleObject.class.getName())
+        ));
+
+        SimpleObject obj = simpleObject();
+
+        BinaryObject po = marshal(obj, marsh);
+
+        BinaryObject copy = copy(po, F.<String, Object>asMap("iArr", new int[]{1, 2, 3}));
+
+        assertArrayEquals(new int[] {1, 2, 3}, copy.<int[]>field("iArr"));
+
+        SimpleObject obj0 = copy.deserialize();
+
+        assertArrayEquals(new int[] {1, 2, 3}, obj0.iArr);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPortableCopyLongArray() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        marsh.setTypeConfigurations(Arrays.asList(
+            new BinaryTypeConfiguration(SimpleObject.class.getName())
+        ));
+
+        SimpleObject obj = simpleObject();
+
+        BinaryObject po = marshal(obj, marsh);
+
+        BinaryObject copy = copy(po, F.<String, Object>asMap("lArr", new long[]{1, 2, 3}));
+
+        assertArrayEquals(new long[] {1, 2, 3}, copy.<long[]>field("lArr"));
+
+        SimpleObject obj0 = copy.deserialize();
+
+        assertArrayEquals(new long[] {1, 2, 3}, obj0.lArr);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPortableCopyFloatArray() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        marsh.setTypeConfigurations(Arrays.asList(
+            new BinaryTypeConfiguration(SimpleObject.class.getName())
+        ));
+
+        SimpleObject obj = simpleObject();
+
+        BinaryObject po = marshal(obj, marsh);
+
+        BinaryObject copy = copy(po, F.<String, Object>asMap("fArr", new float[]{1, 2, 3}));
+
+        assertArrayEquals(new float[] {1, 2, 3}, copy.<float[]>field("fArr"), 0);
+
+        SimpleObject obj0 = copy.deserialize();
+
+        assertArrayEquals(new float[] {1, 2, 3}, obj0.fArr, 0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPortableCopyDoubleArray() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        marsh.setTypeConfigurations(Arrays.asList(
+            new BinaryTypeConfiguration(SimpleObject.class.getName())
+        ));
+
+        SimpleObject obj = simpleObject();
+
+        BinaryObject po = marshal(obj, marsh);
+
+        BinaryObject copy = copy(po, F.<String, Object>asMap("dArr", new double[]{1, 2, 3}));
+
+        assertArrayEquals(new double[] {1, 2, 3}, copy.<double[]>field("dArr"), 0);
+
+        SimpleObject obj0 = copy.deserialize();
+
+        assertArrayEquals(new double[] {1, 2, 3}, obj0.dArr, 0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPortableCopyCharArray() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        marsh.setTypeConfigurations(Arrays.asList(
+            new BinaryTypeConfiguration(SimpleObject.class.getName())
+        ));
+
+        SimpleObject obj = simpleObject();
+
+        BinaryObject po = marshal(obj, marsh);
+
+        BinaryObject copy = copy(po, F.<String, Object>asMap("cArr", new char[]{1, 2, 3}));
+
+        assertArrayEquals(new char[]{1, 2, 3}, copy.<char[]>field("cArr"));
+
+        SimpleObject obj0 = copy.deserialize();
+
+        assertArrayEquals(new char[]{1, 2, 3}, obj0.cArr);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPortableCopyStringArray() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        marsh.setTypeConfigurations(Arrays.asList(
+            new BinaryTypeConfiguration(SimpleObject.class.getName())
+        ));
+
+        SimpleObject obj = simpleObject();
+
+        BinaryObject po = marshal(obj, marsh);
+
+        BinaryObject copy = copy(po, F.<String, Object>asMap("strArr", new String[]{"str1", "str2"}));
+
+        assertArrayEquals(new String[]{"str1", "str2"}, copy.<String[]>field("strArr"));
+
+        SimpleObject obj0 = copy.deserialize();
+
+        assertArrayEquals(new String[]{"str1", "str2"}, obj0.strArr);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPortableCopyObject() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        marsh.setTypeConfigurations(Arrays.asList(
+            new BinaryTypeConfiguration(SimpleObject.class.getName())
+        ));
+
+        SimpleObject obj = simpleObject();
+
+        BinaryObject po = marshal(obj, marsh);
+
+        SimpleObject newObj = new SimpleObject();
+
+        newObj.i = 12345;
+        newObj.fArr = new float[] {5, 8, 0};
+        newObj.str = "newStr";
+
+        BinaryObject copy = copy(po, F.<String, Object>asMap("inner", newObj));
+
+        assertEquals(newObj, copy.<BinaryObject>field("inner").deserialize());
+
+        SimpleObject obj0 = copy.deserialize();
+
+        assertEquals(newObj, obj0.inner);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPortableCopyNonPrimitives() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        marsh.setTypeConfigurations(Arrays.asList(
+            new BinaryTypeConfiguration(SimpleObject.class.getName())
+        ));
+
+        SimpleObject obj = simpleObject();
+
+        BinaryObject po = marshal(obj, marsh);
+
+        Map<String, Object> map = new HashMap<>(3, 1.0f);
+
+        SimpleObject newObj = new SimpleObject();
+
+        newObj.i = 12345;
+        newObj.fArr = new float[] {5, 8, 0};
+        newObj.str = "newStr";
+
+        map.put("str", "str555");
+        map.put("inner", newObj);
+        map.put("bArr", new byte[]{6, 7, 9});
+
+        BinaryObject copy = copy(po, map);
+
+        assertEquals("str555", copy.<String>field("str"));
+        assertEquals(newObj, copy.<BinaryObject>field("inner").deserialize());
+        assertArrayEquals(new byte[]{6, 7, 9}, copy.<byte[]>field("bArr"));
+
+        SimpleObject obj0 = copy.deserialize();
+
+        assertEquals("str555", obj0.str);
+        assertEquals(newObj, obj0.inner);
+        assertArrayEquals(new byte[] {6, 7, 9}, obj0.bArr);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPortableCopyMixed() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        marsh.setTypeConfigurations(Arrays.asList(new BinaryTypeConfiguration(SimpleObject.class.getName())));
+
+        SimpleObject obj = simpleObject();
+
+        BinaryObject po = marshal(obj, marsh);
+
+        Map<String, Object> map = new HashMap<>(3, 1.0f);
+
+        SimpleObject newObj = new SimpleObject();
+
+        newObj.i = 12345;
+        newObj.fArr = new float[] {5, 8, 0};
+        newObj.str = "newStr";
+
+        map.put("i", 1234);
+        map.put("str", "str555");
+        map.put("inner", newObj);
+        map.put("s", (short)2323);
+        map.put("bArr", new byte[]{6, 7, 9});
+        map.put("b", (byte)111);
+
+        BinaryObject copy = copy(po, map);
+
+        assertEquals(1234, copy.<Integer>field("i").intValue());
+        assertEquals("str555", copy.<String>field("str"));
+        assertEquals(newObj, copy.<BinaryObject>field("inner").deserialize());
+        assertEquals((short)2323, copy.<Short>field("s").shortValue());
+        assertArrayEquals(new byte[] {6, 7, 9}, copy.<byte[]>field("bArr"));
+        assertEquals((byte)111, copy.<Byte>field("b").byteValue());
+
+        SimpleObject obj0 = copy.deserialize();
+
+        assertEquals(1234, obj0.i);
+        assertEquals("str555", obj0.str);
+        assertEquals(newObj, obj0.inner);
+        assertEquals((short)2323, obj0.s);
+        assertArrayEquals(new byte[] {6, 7, 9}, obj0.bArr);
+        assertEquals((byte)111, obj0.b);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testKeepDeserialized() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        marsh.setClassNames(Arrays.asList(SimpleObject.class.getName()));
+        marsh.setKeepDeserialized(true);
+
+        BinaryObject po = marshal(simpleObject(), marsh);
+
+        assert po.deserialize() == po.deserialize();
+
+        marsh = createMarshaller();
+
+        marsh.setClassNames(Arrays.asList(SimpleObject.class.getName()));
+        marsh.setKeepDeserialized(false);
+
+        po = marshal(simpleObject(), marsh);
+
+        assert po.deserialize() != po.deserialize();
+
+        marsh = createMarshaller();
+
+        marsh.setKeepDeserialized(true);
+        marsh.setTypeConfigurations(Arrays.asList(
+            new BinaryTypeConfiguration(SimpleObject.class.getName())));
+
+        po = marshal(simpleObject(), marsh);
+
+        assert po.deserialize() == po.deserialize();
+
+        marsh = createMarshaller();
+
+        marsh.setKeepDeserialized(false);
+        marsh.setTypeConfigurations(Arrays.asList(
+            new BinaryTypeConfiguration(SimpleObject.class.getName())));
+
+        po = marshal(simpleObject(), marsh);
+
+        assert po.deserialize() != po.deserialize();
+
+        marsh = createMarshaller();
+
+        marsh.setKeepDeserialized(true);
+
+        BinaryTypeConfiguration typeCfg = new BinaryTypeConfiguration(SimpleObject.class.getName());
+
+        typeCfg.setKeepDeserialized(false);
+
+        marsh.setTypeConfigurations(Arrays.asList(typeCfg));
+
+        po = marshal(simpleObject(), marsh);
+
+        assert po.deserialize() != po.deserialize();
+
+        marsh = createMarshaller();
+
+        marsh.setKeepDeserialized(false);
+
+        typeCfg = new BinaryTypeConfiguration(SimpleObject.class.getName());
+
+        typeCfg.setKeepDeserialized(true);
+
+        marsh.setTypeConfigurations(Arrays.asList(typeCfg));
+
+        po = marshal(simpleObject(), marsh);
+
+        assert po.deserialize() == po.deserialize();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testOffheapPortable() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        marsh.setTypeConfigurations(Arrays.asList(new BinaryTypeConfiguration(SimpleObject.class.getName())));
+
+        PortableContext ctx = initializePortableContext(marsh);
+
+        SimpleObject simpleObj = simpleObject();
+
+        BinaryObjectImpl obj = marshal(simpleObj, marsh);
+
+        long ptr = 0;
+
+        long ptr1 = 0;
+
+        long ptr2 = 0;
+
+        try {
+            ptr = copyOffheap(obj);
+
+            BinaryObjectOffheapImpl offheapObj = new BinaryObjectOffheapImpl(ctx,
+                ptr,
+                0,
+                obj.array().length);
+
+            assertTrue(offheapObj.equals(offheapObj));
+            assertFalse(offheapObj.equals(null));
+            assertFalse(offheapObj.equals("str"));
+            assertTrue(offheapObj.equals(obj));
+            assertTrue(obj.equals(offheapObj));
+
+            ptr1 = copyOffheap(obj);
+
+            BinaryObjectOffheapImpl offheapObj1 = new BinaryObjectOffheapImpl(ctx,
+                ptr1,
+                0,
+                obj.array().length);
+
+            assertTrue(offheapObj.equals(offheapObj1));
+            assertTrue(offheapObj1.equals(offheapObj));
+
+            assertEquals(obj.typeId(), offheapObj.typeId());
+            assertEquals(obj.hashCode(), offheapObj.hashCode());
+
+            checkSimpleObjectData(simpleObj, offheapObj);
+
+            BinaryObjectOffheapImpl innerOffheapObj = offheapObj.field("inner");
+
+            assertNotNull(innerOffheapObj);
+
+            checkSimpleObjectData(simpleObj.inner, innerOffheapObj);
+
+            obj = (BinaryObjectImpl)offheapObj.heapCopy();
+
+            assertEquals(obj.typeId(), offheapObj.typeId());
+            assertEquals(obj.hashCode(), offheapObj.hashCode());
+
+            checkSimpleObjectData(simpleObj, obj);
+
+            BinaryObjectImpl innerObj = obj.field("inner");
+
+            assertNotNull(innerObj);
+
+            checkSimpleObjectData(simpleObj.inner, innerObj);
+
+            simpleObj.d = 0;
+
+            obj = marshal(simpleObj, marsh);
+
+            assertFalse(offheapObj.equals(obj));
+            assertFalse(obj.equals(offheapObj));
+
+            ptr2 = copyOffheap(obj);
+
+            BinaryObjectOffheapImpl offheapObj2 = new BinaryObjectOffheapImpl(ctx,
+                ptr2,
+                0,
+                obj.array().length);
+
+            assertFalse(offheapObj.equals(offheapObj2));
+            assertFalse(offheapObj2.equals(offheapObj));
+        }
+        finally {
+            UNSAFE.freeMemory(ptr);
+
+            if (ptr1 > 0)
+                UNSAFE.freeMemory(ptr1);
+
+            if (ptr2 > 0)
+                UNSAFE.freeMemory(ptr2);
+        }
+    }
+
+    /**
+     *
+     */
+    public void testReadResolve() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        marsh.setClassNames(
+            Arrays.asList(MySingleton.class.getName(), SingletonMarker.class.getName()));
+
+        BinaryObjectImpl portableObj = marshal(MySingleton.INSTANCE, marsh);
+
+        assertTrue(portableObj.array().length <= 1024); // Check that big string was not serialized.
+
+        MySingleton singleton = portableObj.deserialize();
+
+        assertSame(MySingleton.INSTANCE, singleton);
+    }
+
+    /**
+     *
+     */
+    public void testReadResolveOnPortableAware() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        marsh.setClassNames(Collections.singletonList(MyTestClass.class.getName()));
+
+        BinaryObjectImpl portableObj = marshal(new MyTestClass(), marsh);
+
+        MyTestClass obj = portableObj.deserialize();
+
+        assertEquals("readResolve", obj.s);
+    }
+
+    /**
+     * @throws Exception If ecxeption thrown.
+     */
+    public void testDeclareReadResolveInParent() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        marsh.setClassNames(Arrays.asList(ChildPortable.class.getName()));
+
+        BinaryObjectImpl portableObj = marshal(new ChildPortable(), marsh);
+
+        ChildPortable singleton = portableObj.deserialize();
+
+        assertNotNull(singleton.s);
+    }
+
+    /**
+     *
+     */
+    public void testDecimalFields() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        Collection<String> clsNames = new ArrayList<>();
+
+        clsNames.add(DecimalReflective.class.getName());
+        clsNames.add(DecimalMarshalAware.class.getName());
+
+        marsh.setClassNames(clsNames);
+
+        // 1. Test reflective stuff.
+        DecimalReflective obj1 = new DecimalReflective();
+
+        obj1.val = BigDecimal.ZERO;
+        obj1.valArr = new BigDecimal[] { BigDecimal.ONE, BigDecimal.TEN };
+
+        BinaryObjectImpl portObj = marshal(obj1, marsh);
+
+        assertEquals(obj1.val, portObj.field("val"));
+        assertArrayEquals(obj1.valArr, portObj.<BigDecimal[]>field("valArr"));
+
+        assertEquals(obj1.val, portObj.<DecimalReflective>deserialize().val);
+        assertArrayEquals(obj1.valArr, portObj.<DecimalReflective>deserialize().valArr);
+
+        // 2. Test marshal aware stuff.
+        DecimalMarshalAware obj2 = new DecimalMarshalAware();
+
+        obj2.val = BigDecimal.ZERO;
+        obj2.valArr = new BigDecimal[] { BigDecimal.ONE, BigDecimal.TEN.negate() };
+        obj2.rawVal = BigDecimal.TEN;
+        obj2.rawValArr = new BigDecimal[] { BigDecimal.ZERO, BigDecimal.ONE };
+
+        portObj = marshal(obj2, marsh);
+
+        assertEquals(obj2.val, portObj.field("val"));
+        assertArrayEquals(obj2.valArr, portObj.<BigDecimal[]>field("valArr"));
+
+        assertEquals(obj2.val, portObj.<DecimalMarshalAware>deserialize().val);
+        assertArrayEquals(obj2.valArr, portObj.<DecimalMarshalAware>deserialize().valArr);
+        assertEquals(obj2.rawVal, portObj.<DecimalMarshalAware>deserialize().rawVal);
+        assertArrayEquals(obj2.rawValArr, portObj.<DecimalMarshalAware>deserialize().rawValArr);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testFinalField() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        SimpleObjectWithFinal obj = new SimpleObjectWithFinal();
+
+        SimpleObjectWithFinal po0 = marshalUnmarshal(obj, marsh);
+
+        assertEquals(obj.time, po0.time);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testThreadLocalArrayReleased() throws Exception {
+        // Checking the writer directly.
+        assertEquals(false, THREAD_LOCAL_ALLOC.isThreadLocalArrayAcquired());
+
+        PortableMarshaller marsh0 = createMarshaller();
+
+        try (BinaryWriterExImpl writer = new BinaryWriterExImpl(portableContext(marsh0))) {
+            assertEquals(true, THREAD_LOCAL_ALLOC.isThreadLocalArrayAcquired());
+
+            writer.writeString("Thread local test");
+
+            writer.array();
+
+            assertEquals(true, THREAD_LOCAL_ALLOC.isThreadLocalArrayAcquired());
+        }
+
+        // Checking the portable marshaller.
+        assertEquals(false, THREAD_LOCAL_ALLOC.isThreadLocalArrayAcquired());
+
+        PortableMarshaller marsh = createMarshaller();
+
+        marsh.marshal(new SimpleObject());
+
+        assertEquals(false, THREAD_LOCAL_ALLOC.isThreadLocalArrayAcquired());
+
+        // Checking the builder.
+        PortableMarshaller marsh2 = createMarshaller();
+
+        BinaryObjectBuilder builder = new BinaryObjectBuilderImpl(portableContext(marsh2),
+            "org.gridgain.foo.bar.TestClass");
+
+        builder.setField("a", "1");
+
+        BinaryObject portableObj = builder.build();
+
+        assertEquals(false, THREAD_LOCAL_ALLOC.isThreadLocalArrayAcquired());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDuplicateName() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        Test1.Job job1 = new Test1().new Job();
+        Test2.Job job2 = new Test2().new Job();
+
+        marsh.marshal(job1);
+
+        try {
+            marsh.marshal(job2);
+        }
+        catch (BinaryObjectException e) {
+            assertEquals(true, e.getMessage().contains("Failed to register class"));
+
+            return;
+        }
+
+        assert false;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClass() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        Class cls = BinaryMarshallerSelfTest.class;
+
+        Class unmarshalledCls = marshalUnmarshal(cls, marsh);
+
+        Assert.assertEquals(cls, unmarshalledCls);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClassFieldsMarshalling() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        ObjectWithClassFields obj = new ObjectWithClassFields();
+        obj.cls1 = BinaryMarshallerSelfTest.class;
+
+        byte[] marshal = marsh.marshal(obj);
+
+        ObjectWithClassFields obj2 = marsh.unmarshal(marshal, null);
+
+        assertEquals(obj.cls1, obj2.cls1);
+        assertNull(obj2.cls2);
+
+        BinaryObject portObj = marshal(obj, marsh);
+
+        Class cls1 = portObj.field("cls1");
+
+        assertEquals(obj.cls1, cls1);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMarshallingThroughJdk() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        InetSocketAddress addr = new InetSocketAddress("192.168.0.2", 4545);
+
+        byte[] arr = marsh.marshal(addr);
+
+        InetSocketAddress addr2 = marsh.unmarshal(arr, null);
+
+        assertEquals(addr.getHostString(), addr2.getHostString());
+        assertEquals(addr.getPort(), addr2.getPort());
+
+        TestAddress testAddr = new TestAddress();
+        testAddr.addr = addr;
+        testAddr.str1 = "Hello World";
+
+        SimpleObject simpleObj = new SimpleObject();
+        simpleObj.c = 'g';
+        simpleObj.date = new Date();
+
+        testAddr.obj = simpleObj;
+
+        arr = marsh.marshal(testAddr);
+
+        TestAddress testAddr2 = marsh.unmarshal(arr, null);
+
+        assertEquals(testAddr.addr.getHostString(), testAddr2.addr.getHostString());
+        assertEquals(testAddr.addr.getPort(), testAddr2.addr.getPort());
+        assertEquals(testAddr.str1, testAddr2.str1);
+        assertEquals(testAddr.obj.c, testAddr2.obj.c);
+        assertEquals(testAddr.obj.date, testAddr2.obj.date);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPredefinedTypeIds() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        PortableContext pCtx = initializePortableContext(marsh);
+
+        Field field = pCtx.getClass().getDeclaredField("predefinedTypeNames");
+
+        field.setAccessible(true);
+
+        Map<String, Integer> map = (Map<String, Integer>)field.get(pCtx);
+
+        assertTrue(map.size() > 0);
+
+        for (Map.Entry<String, Integer> entry : map.entrySet()) {
+            int id = entry.getValue();
+
+            if (id == GridPortableMarshaller.UNREGISTERED_TYPE_ID)
+                continue;
+
+            PortableClassDescriptor desc = pCtx.descriptorForTypeId(false, entry.getValue(), null);
+
+            assertEquals(desc.typeId(), pCtx.typeId(desc.describedClass().getName()));
+            assertEquals(desc.typeId(), pCtx.typeId(pCtx.typeName(desc.describedClass().getName())));
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCyclicReferencesMarshalling() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
+
+        SimpleObject obj = simpleObject();
+
+        obj.bArr = obj.inner.bArr;
+        obj.cArr = obj.inner.cArr;
+        obj.boolArr = obj.inner.boolArr;
+        obj.sArr = obj.inner.sArr;
+        obj.strArr = obj.inner.strArr;
+        obj.iArr = obj.inner.iArr;
+        obj.lArr = obj.inner.lArr;
+        obj.fArr = obj.inner.fArr;
+        obj.dArr = obj.inner.dArr;
+        obj.dateArr = obj.inner.dateArr;
+        obj.uuidArr = obj.inner.uuidArr;
+        obj.objArr = obj.inner.objArr;
+        obj.bdArr = obj.inner.bdArr;
+        obj.map = obj.inner.map;
+        obj.col = obj.inner.col;
+        obj.mEntry = obj.inner.mEntry;
+
+        SimpleObject res = (SimpleObject)marshalUnmarshal(obj, marsh);
+
+        assertEquals(obj, res);
+
+        assertTrue(res.bArr == res.inner.bArr);
+        assertTrue(res.cArr == res.inner.cArr);
+        assertTrue(res.boolArr == res.inner.boolArr);
+        assertTrue(res.sArr == res.inner.sArr);
+        assertTrue(res.strArr == res.inner.strArr);
+        assertTrue(res.iArr == res.inner.iArr);
+        assertTrue(res.lArr == res.inner.lArr);
+        assertTrue(res.fArr == res.inner.fArr);
+        assertTrue(res.dArr == res.inner.dArr);
+        assertTrue(res.dateArr == res.inner.dateArr);
+        assertTrue(res.uuidArr == res.inner.uuidArr);
+        assertTrue(res.objArr == res.inner.objArr);
+        assertTrue(res.bdArr == res.inner.bdArr);
+        assertTrue(res.map == res.inner.map);
+        assertTrue(res.col == res.inner.col);
+        assertTrue(res.mEntry == res.inner.mEntry);
+    }
+
+    /**
+     * Object with class fields.
+     */
+    private static class ObjectWithClassFields {
+        /** */
+        private Class<?> cls1;
+
+        /** */
+        private Class<?> cls2;
+    }
+
+    /**
+     *
+     */
+    private static class TestAddress {
+        /** */
+        private SimpleObject obj;
+
+        /** */
+        private InetSocketAddress addr;
+
+        /** */
+        private String str1;
+    }
+
+    /**
+     *
+     */
+    private static class Test1 {
+        /**
+         *
+         */
+        private class Job {
+
+        }
+    }
+
+    /**
+     *
+     */
+    private static class Test2 {
+        /**
+         *
+         */
+        private class Job {
+
+        }
+    }
+
+    /**
+     * @param obj Object.
+     * @return Offheap address.
+     */
+    private long copyOffheap(BinaryObjectImpl obj) {
+        byte[] arr = obj.array();
+
+        long ptr = UNSAFE.allocateMemory(arr.length);
+
+        UNSAFE.copyMemory(arr, BYTE_ARR_OFF, null, ptr, arr.length);
+
+        return ptr;
+    }
+
+    /**
+     * @param enumArr Enum array.
+     * @return Ordinals.
+     */
+    private <T extends Enum<?>> Integer[] ordinals(T[] enumArr) {
+        Integer[] ords = new Integer[enumArr.length];
+
+        for (int i = 0; i < enumArr.length; i++)
+            ords[i] = enumArr[i].ordinal();
+
+        return ords;
+    }
+
+    /**
+     * @param po Portable object.
+     * @param off Offset.
+     * @return Value.
+     */
+    private int intFromPortable(BinaryObject po, int off) {
+        byte[] arr = U.field(po, "arr");
+
+        return Integer.reverseBytes(U.bytesToInt(arr, off));
+    }
+
+    /**
+     * @param obj Original object.
+     * @return Result object.
+     */
+    private <T> T marshalUnmarshal(T obj) throws Exception {
+        return marshalUnmarshal(obj, createMarshaller());
+    }
+
+    /**
+     * @param obj Original object.
+     * @param marsh Marshaller.
+     * @return Result object.
+     */
+    private <T> T marshalUnmarshal(Object obj, PortableMarshaller marsh) throws IgniteCheckedException {
+        initializePortableContext(marsh);
+
+        byte[] bytes = marsh.marshal(obj);
+
+        return marsh.unmarshal(bytes, null);
+    }
+
+    /**
+     * @param obj Object.
+     * @param marsh Marshaller.
+     * @return Portable object.
+     */
+    private <T> BinaryObjectImpl marshal(T obj, PortableMarshaller marsh) throws IgniteCheckedException {
+        initializePortableContext(marsh);
+
+        byte[] bytes = marsh.marshal(obj);
+
+        return new BinaryObjectImpl(U.<GridPortableMarshaller>field(marsh, "impl").context(),
+            bytes, 0);
+    }
+
+    /**
+     * Create portable marshaller.
+     *
+     * @return Portable marshaller.
+     * @throws Exception If failed.
+     */
+    private PortableMarshaller createMarshaller() throws Exception {
+        PortableMarshaller marsh = new PortableMarshaller();
+
+        marsh.setCompactFooter(compactFooter());
+
+        initializePortableContext(marsh);
+
+        return marsh;
+    }
+
+    /**
+     * @return Whether to use compact footers or not.
+     */
+    protected boolean compactFooter() {
+        return true;
+    }
+
+    /**
+     * Get portable context of the given marshaller.
+     *
+     * @param marsh Marshaller.
+     * @return Context.
+     * @throws Exception If failed.
+     */
+    private PortableContext portableContext(PortableMarshaller marsh) throws Exception {
+        GridPortableMarshaller marsh0 = IgniteUtils.field(marsh, "impl");
+
+        return marsh0.context();
+    }
+
+    /**
+     * @return Portable context.
+     */
+    private PortableContext initializePortableContext(PortableMarshaller marsh) throws IgniteCheckedException {
+        IgniteConfiguration iCfg = new IgniteConfiguration();
+
+        PortableContext ctx = new PortableContext(BinaryCachingMetadataHandler.create(), iCfg);
+
+        marsh.setContext(new MarshallerContextTestImpl(null));
+
+        IgniteUtils.invoke(PortableMarshaller.class, marsh, "setPortableContext", ctx);
+
+        return ctx;
+    }
+
+    /**
+     * @param exp Expected.
+     * @param act Actual.
+     */
+    private void assertBooleanArrayEquals(boolean[] exp, boolean[] act) {
+        assertEquals(exp.length, act.length);
+
+        for (int i = 0; i < act.length; i++)
+            assertEquals(exp[i], act[i]);
+    }
+
+    /**
+     *
+     */
+    private static class SimpleObjectWithFinal {
+        /** */
+        private final long time = System.currentTimeMillis();
+    }
+
+    /**
+     * @return Simple object.
+     */
+    private SimpleObject simpleObject() {
+        SimpleObject inner = new SimpleObject();
+
+        inner.b = 1;
+        inner.s = 1;
+        inner.i = 1;
+        inner.l = 1;
+        inner.f = 1.1f;
+        inner.d = 1.1d;
+        inner.c = 1;
+        inner.bool = true;
+        inner.str = "str1";
+        inner.uuid = UUID.randomUUID();
+        inner.date = new Date();
+        inner.ts = new Timestamp(System.currentTimeMillis());
+        inner.bArr = new byte[] {1, 2, 3};
+        inner.sArr = new short[] {1, 2, 3};
+        inner.iArr = new int[] {1, 2, 3};
+        inner.lArr = new long[] {1, 2, 3};
+        inner.fArr = new float[] {1.1f, 2.2f, 3.3f};
+        inner.dArr = new double[] {1.1d, 2.2d, 3.3d};
+        inner.cArr = new char[] {1, 2, 3};
+        inner.boolArr = new boolean[] {true, false, true};
+        inner.strArr = new String[] {"str1", "str2", "str3"};
+        inner.uuidArr = new UUID[] {UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID()};
+        inner.dateArr = new Date[] {new Date(11111), new Date(22222), new Date(33333)};
+        inner.objArr = new Object[] {UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID()};
+        inner.col = new ArrayList<>();
+        inner.map = new HashMap<>();
+        inner.enumVal = TestEnum.A;
+        inner.enumArr = new TestEnum[] {TestEnum.A, TestEnum.B};
+        inner.bdArr = new BigDecimal[] {new BigDecimal(1000), BigDecimal.ONE};
+
+        inner.col.add("str1");
+        inner.col.add("str2");
+        inner.col.add("str3");
+
+        inner.map.put(1, "str1");
+        inner.map.put(2, "str2");
+        inner.map.put(3, "str3");
+
+        inner.mEntry = inner.map.entrySet().iterator().next();
+
+        SimpleObject outer = new SimpleObject();
+
+        outer.b = 2;
+        outer.s = 2;
+        outer.i = 2;
+        outer.l = 2;
+        outer.f = 2.2f;
+        outer.d = 2.2d;
+        outer.c = 2;
+        outer.bool = false;
+        outer.str = "str2";
+        outer.uuid = UUID.randomUUID();
+        outer.date = new Date();
+        outer.ts = new Timestamp(System.currentTimeMillis());
+        outer.bArr = new byte[] {10, 20, 30};
+        outer.sArr = new short[] {10, 20, 30};
+        outer.iArr = new int[] {10, 20, 30};
+        outer.lArr = new long[] {10, 20, 30};
+        outer.fArr = new float[] {10.01f, 20.02f, 30.03f};
+        outer.dArr = new double[] {10.01d, 20.02d, 30.03d};
+        outer.cArr = new char[] {10, 20, 30};
+        outer.boolArr = new boolean[] {false, true, false};
+        outer.strArr = new String[] {"str10", "str20", "str30"};
+        outer.uuidArr = new UUID[] {UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID()};
+        outer.dateArr = new Date[] {new Date(44444), new Date(55555), new Date(66666)};
+        outer.objArr = new Object[] {UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID()};
+        outer.col = new ArrayList<>();
+        outer.map = new HashMap<>();
+        outer.enumVal = TestEnum.B;
+        outer.enumArr = new TestEnum[] {TestEnum.B, TestEnum.C};
+        outer.inner = inner;
+        outer.bdArr = new BigDecimal[] {new BigDecimal(5000), BigDecimal.TEN};
+
+
+        outer.col.add("str4");
+        outer.col.add("str5");
+        outer.col.add("str6");
+
+        outer.map.put(4, "str4");
+        outer.map.put(5, "str5");
+        outer.map.put(6, "str6");
+
+        outer.mEntry = outer.map.entrySet().iterator().next();
+
+        return outer;
+    }
+
+    /**
+     * @return Portable object.
+     */
+    private TestBinary binaryObject() {
+        SimpleObject innerSimple = new SimpleObject();
+
+        innerSimple.b = 1;
+        innerSimple.s = 1;
+        innerSimple.i = 1;
+        innerSimple.l = 1;
+        innerSimple.f = 1.1f;
+        innerSimple.d = 1.1d;
+        innerSimple.c = 1;
+        innerSimple.bool = true;
+        innerSimple.str = "str1";
+        innerSimple.uuid = UUID.randomUUID();
+        innerSimple.date = new Date();
+        innerSimple.ts = new Timestamp(System.currentTimeMillis());
+        innerSimple.bArr = new byte[] {1, 2, 3};
+        innerSimple.sArr = new short[] {1, 2, 3};
+        innerSimple.iArr = new int[] {1, 2, 3};
+        innerSimple.lArr = new long[] {1, 2, 3};
+        innerSimple.fArr = new float[] {1.1f, 2.2f, 3.3f};
+        innerSimple.dArr = new double[] {1.1d, 2.2d, 3.3d};
+        innerSimple.cArr = new char[] {1, 2, 3};
+        innerSimple.boolArr = new boolean[] {true, false, true};
+        innerSimple.strArr = new String[] {"str1", "str2", "str3"};
+        innerSimple.uuidArr = new UUID[] {UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID()};
+        innerSimple.dateArr = new Date[] {new Date(11111), new Date(22222), new Date(33333)};
+        innerSimple.objArr = new UUID[] {UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID()};
+    

<TRUNCATED>

[10/13] ignite git commit: IGNITE-1941: Removed TODOs. Let's keep these methods because they are convenient.

Posted by ag...@apache.org.
IGNITE-1941: Removed TODOs. Let's keep these methods because they are convenient.


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

Branch: refs/heads/ignite-1945
Commit: 5ea0625b47962a12da0aaa5357df514a7aad83c5
Parents: 94afe3e
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Nov 19 13:49:40 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Nov 19 13:49:40 2015 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/binary/BinaryObject.java   | 6 ++----
 1 file changed, 2 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5ea0625b/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 9481618..f098c16 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
@@ -123,16 +123,14 @@ public interface BinaryObject extends Serializable, Cloneable {
      * @param fieldName Field name.
      * @return Field value.
      * @throws BinaryObjectException In case of any other error.
-     * TODO ignite-1282 remove.
      */
     public <F> F field(String fieldName) throws BinaryObjectException;
 
     /**
-     * Checks whether field is set.
-     ** TODO ignite-1282 remove.
+     * Checks whether field exists in the object.
      *
      * @param fieldName Field name.
-     * @return {@code true} if field is set.
+     * @return {@code True} if field exists.
      */
     public boolean hasField(String fieldName);
 


[13/13] ignite git commit: Merge branch ignite-1282 into ignite-1945

Posted by ag...@apache.org.
Merge branch ignite-1282 into ignite-1945


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

Branch: refs/heads/ignite-1945
Commit: a511fa178d0e1c0cbc6587141a87daa134e36c4b
Parents: 29e586f 5ea0625
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Thu Nov 19 17:07:35 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Nov 19 17:07:35 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/binary/BinaryObject.java  |    6 +-
 .../apache/ignite/internal/IgniteKernal.java    |    6 +
 .../ignite/internal/IgniteNodeAttributes.java   |    3 +
 .../portable/BinaryCachingMetadataHandler.java  |   70 +
 .../internal/portable/BinaryMetadata.java       |   16 +-
 .../portable/BinaryMetadataCollector.java       |   49 +-
 .../internal/portable/BinaryObjectEx.java       |    2 +-
 .../internal/portable/BinaryObjectImpl.java     |   14 +-
 .../portable/BinaryObjectOffheapImpl.java       |   14 +-
 .../internal/portable/BinaryReaderExImpl.java   |  132 +-
 .../internal/portable/BinaryTypeImpl.java       |    7 +-
 .../internal/portable/BinaryWriterExImpl.java   |  161 +-
 .../portable/PortableClassDescriptor.java       |  102 +-
 .../internal/portable/PortableContext.java      |   66 +-
 .../internal/portable/PortableSchema.java       |  296 +-
 .../ignite/internal/portable/PortableUtils.java |  457 ++-
 .../builder/BinaryObjectBuilderImpl.java        |  130 +-
 .../portable/builder/PortableBuilderReader.java |   21 +-
 .../portable/CacheObjectBinaryProcessor.java    |    2 +-
 .../CacheObjectBinaryProcessorImpl.java         |  162 +-
 .../platform/PlatformContextImpl.java           |   10 +-
 .../cpp/PlatformCppConfigurationClosure.java    |    7 +
 .../PlatformDotNetConfigurationClosure.java     |    9 +-
 .../ignite/internal/util/IgniteUtils.java       |   25 +
 .../ignite/spi/discovery/tcp/ServerImpl.java    |   51 +-
 .../portable/BinaryFieldsAbstractSelfTest.java  |   13 +-
 .../BinaryFooterOffsetsAbstractSelfTest.java    |  206 +
 .../BinaryFooterOffsetsHeapSelfTest.java        |   32 +
 .../BinaryFooterOffsetsOffheapSelfTest.java     |   61 +
 .../portable/BinaryMarshallerSelfTest.java      | 3687 ++++++++++++++++++
 .../BinaryObjectBuilderAdditionalSelfTest.java  | 1292 ++++++
 .../portable/BinaryObjectBuilderSelfTest.java   | 1069 +++++
 ...idBinaryObjectBuilderAdditionalSelfTest.java | 1281 ------
 .../GridBinaryObjectBuilderSelfTest.java        | 1056 -----
 ...idPortableMarshallerCtxDisabledSelfTest.java |    2 +-
 .../GridPortableMarshallerSelfTest.java         | 3672 -----------------
 .../PortableCompactOffsetsAbstractSelfTest.java |  197 -
 .../PortableCompactOffsetsHeapSelfTest.java     |   32 -
 .../PortableCompactOffsetsOffheapSelfTest.java  |   61 -
 .../BinaryFieldsHeapNonCompactSelfTest.java     |   34 +
 .../BinaryFieldsOffheapNonCompactSelfTest.java  |   30 +
 ...naryFooterOffsetsHeapNonCompactSelfTest.java |   30 +
 ...yFooterOffsetsOffheapNonCompactSelfTest.java |   30 +
 .../BinaryMarshallerNonCompactSelfTest.java     |   30 +
 ...jectBuilderAdditionalNonCompactSelfTest.java |   30 +
 .../BinaryObjectBuilderNonCompactSelfTest.java  |   30 +
 .../IgnitePortableObjectsTestSuite.java         |   36 +-
 .../query/IgniteSqlSplitterSelfTest.java        |   54 +
 .../platforms/cpp/common/include/Makefile.am    |    3 +
 .../cpp/common/os/linux/include/Makefile.am     |    3 +
 .../core-test/src/binary_reader_writer_test.cpp |   64 +-
 modules/platforms/cpp/core/include/Makefile.am  |    3 +
 .../include/ignite/impl/binary/binary_common.h  |   22 +-
 .../ignite/impl/binary/binary_reader_impl.h     |   58 +-
 .../include/ignite/impl/binary/binary_schema.h  |    6 +-
 .../ignite/impl/binary/binary_writer_impl.h     |    2 +-
 .../cpp/core/os/linux/include/Makefile.am       |    3 +
 .../core/src/impl/binary/binary_reader_impl.cpp |    8 +-
 .../cpp/core/src/impl/binary/binary_schema.cpp  |   12 +-
 .../core/src/impl/binary/binary_writer_impl.cpp |   29 +-
 .../Config/Compute/compute-grid1.xml            |    1 +
 .../Config/marshaller-explicit.xml              |    4 +-
 .../Impl/Binary/BinaryObjectBuilder.cs          |   25 +-
 .../Impl/Binary/BinaryObjectHeader.cs           |  131 +-
 .../Impl/Binary/BinaryObjectSchemaHolder.cs     |    9 +-
 .../Impl/Binary/BinaryReader.cs                 |    2 +-
 .../Impl/Binary/BinaryWriter.cs                 |   27 +-
 .../Impl/Binary/IgniteBinary.cs                 |    3 +-
 68 files changed, 8087 insertions(+), 7111 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a511fa17/modules/core/src/main/java/org/apache/ignite/binary/BinaryObject.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/a511fa17/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/a511fa17/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
index e37275d,8543ce6..01bfcaa
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
@@@ -91,9 -96,18 +95,15 @@@ public class PortableClassDescriptor 
      private final Method readResolveMtd;
  
      /** */
-     private final Map<String, Integer> fieldsMeta;
+     private final Map<String, Integer> stableFieldsMeta;
+ 
+     /** Object schemas. Initialized only for serializable classes and contains only 1 entry. */
+     private final Collection<PortableSchema> stableSchemas;
+ 
+     /** Schema registry. */
+     private final PortableSchemaRegistry schemaReg;
  
      /** */
 -    private final boolean keepDeserialized;
 -
 -    /** */
      private final boolean registered;
  
      /** */
@@@ -111,9 -126,10 +122,9 @@@
       * @param idMapper ID mapper.
       * @param serializer Serializer.
       * @param metaDataEnabled Metadata enabled flag.
 -     * @param keepDeserialized Keep deserialized flag.
       * @param registered Whether typeId has been successfully registered by MarshallerContext or not.
       * @param predefined Whether the class is predefined or not.
-      * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
+      * @throws BinaryObjectException In case of error.
       */
      PortableClassDescriptor(
          PortableContext ctx,
@@@ -132,13 -151,17 +145,16 @@@
  
          this.ctx = ctx;
          this.cls = cls;
-         this.userType = userType;
          this.typeId = typeId;
+         this.userType = userType;
          this.typeName = typeName;
+         this.affKeyFieldName = affKeyFieldName;
          this.serializer = serializer;
          this.idMapper = idMapper;
 -        this.keepDeserialized = keepDeserialized;
          this.registered = registered;
  
+         schemaReg = ctx.schemaRegistry(typeId);
+ 
          excluded = MarshallerExclusions.isExcluded(cls);
  
          useOptMarshaller = !predefined && initUseOptimizedMarshallerFlag();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a511fa17/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
index 7c61f50,afc23e1..62917b6
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
@@@ -149,6 -147,12 +148,9 @@@ public class PortableContext implement
      /** */
      private final OptimizedMarshaller optmMarsh = new OptimizedMarshaller();
  
 -    /** */
 -    private boolean keepDeserialized;
 -
+     /** Compact footer flag. */
+     private boolean compactFooter;
+ 
      /** Object schemas. */
      private volatile Map<Integer, PortableSchemaRegistry> schemas;
  
@@@ -257,10 -258,14 +259,12 @@@
          optmMarsh.setContext(marshCtx);
  
          configure(
 -            marsh.getIdMapper(),
 -            marsh.getSerializer(),
 -            marsh.isKeepDeserialized(),
 -            marsh.getClassNames(),
 -            marsh.getTypeConfigurations()
 +            binaryCfg.getIdMapper(),
 +            binaryCfg.getSerializer(),
 +            binaryCfg.getTypeConfigurations()
          );
+ 
+         compactFooter = marsh.isCompactFooter();
      }
  
      /**
@@@ -532,8 -564,6 +538,9 @@@
              false /* predefined */
          );
  
-         metaHnd.addMeta(typeId, new BinaryMetadata(typeId, typeName, desc.fieldsMeta(), null).wrap(this));
++        metaHnd.addMeta(typeId,
++            new BinaryMetadata(typeId, typeName, desc.fieldsMeta(), null, desc.schemas()).wrap(this));
 +
          // perform put() instead of putIfAbsent() because "registered" flag might have been changed or class loader
          // might have reloaded described class.
          if (IgniteUtils.detectClassLoader(cls).equals(dfltLdr))

http://git-wip-us.apache.org/repos/asf/ignite/blob/a511fa17/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/a511fa17/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationClosure.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationClosure.java
index 12c9d36,e9cd1e3..6cdf53b
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationClosure.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationClosure.java
@@@ -71,14 -71,21 +71,21 @@@ public class PlatformCppConfigurationCl
          Marshaller marsh = igniteCfg.getMarshaller();
  
          if (marsh == null) {
 -            PortableMarshaller marsh0 = new PortableMarshaller();
++            BinaryMarshaller marsh0 = new BinaryMarshaller();
+ 
+             marsh0.setCompactFooter(false);
 -
 -            igniteCfg.setMarshaller(marsh0);
++            
 +            igniteCfg.setMarshaller(new BinaryMarshaller());
  
              cppCfg0.warnings(Collections.singleton("Marshaller is automatically set to " +
 -                PortableMarshaller.class.getName() + " (other nodes must have the same marshaller type)."));
 +                BinaryMarshaller.class.getName() + " (other nodes must have the same marshaller type)."));
          }
 -        else if (!(marsh instanceof PortableMarshaller))
 -            throw new IgniteException("Unsupported marshaller (only " + PortableMarshaller.class.getName() +
 +        else if (!(marsh instanceof BinaryMarshaller))
 +            throw new IgniteException("Unsupported marshaller (only " + BinaryMarshaller.class.getName() +
                  " can be used when running Apache Ignite C++): " + marsh.getClass().getName());
 -        else if (((PortableMarshaller)marsh).isCompactFooter())
 -            throw new IgniteException("Unsupported " + PortableMarshaller.class.getName() +
++        else if (((BinaryMarshaller)marsh).isCompactFooter())
++            throw new IgniteException("Unsupported " + BinaryMarshaller.class.getName() +
+                 " \"compactFooter\" flag: must be false when running Apache Ignite C++.");
  
          // Set Ignite home so that marshaller context works.
          String ggHome = igniteCfg.getIgniteHome();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a511fa17/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
index f68c08e,a59fd22..4289d4c
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
@@@ -92,14 -92,21 +92,21 @@@ public class PlatformDotNetConfiguratio
          Marshaller marsh = igniteCfg.getMarshaller();
  
          if (marsh == null) {
-             igniteCfg.setMarshaller(new BinaryMarshaller());
 -            PortableMarshaller marsh0 = new PortableMarshaller();
++            BinaryMarshaller marsh0 = new BinaryMarshaller();
+ 
+             marsh0.setCompactFooter(false);
+ 
+             igniteCfg.setMarshaller(marsh0);
  
              dotNetCfg0.warnings(Collections.singleton("Marshaller is automatically set to " +
 -                PortableMarshaller.class.getName() + " (other nodes must have the same marshaller type)."));
 +                BinaryMarshaller.class.getName() + " (other nodes must have the same marshaller type)."));
          }
 -        else if (!(marsh instanceof PortableMarshaller))
 -            throw new IgniteException("Unsupported marshaller (only " + PortableMarshaller.class.getName() +
 +        else if (!(marsh instanceof BinaryMarshaller))
 +            throw new IgniteException("Unsupported marshaller (only " + BinaryMarshaller.class.getName() +
                  " can be used when running Apache Ignite.NET): " + marsh.getClass().getName());
 -        else if (((PortableMarshaller)marsh).isCompactFooter())
 -            throw new IgniteException("Unsupported " + PortableMarshaller.class.getName() +
++        else if (((BinaryMarshaller)marsh).isCompactFooter())
++            throw new IgniteException("Unsupported " + BinaryMarshaller.class.getName() +
+                 " \"compactFooter\" flag: must be false when running Apache Ignite.NET.");
  
          // Set Ignite home so that marshaller context works.
          String ggHome = igniteCfg.getIgniteHome();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a511fa17/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java
index 21fab62,8f79db1..dd08390
--- 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
@@@ -47,14 -46,14 +47,16 @@@ public abstract class BinaryFieldsAbstr
       * @return Portable marshaller.
       * @throws Exception If failed.
       */
-     protected static BinaryMarshaller createMarshaller() throws Exception {
-         PortableContext ctx = new PortableContext(new TestCachingMetadataHandler(), new IgniteConfiguration());
 -    protected PortableMarshaller createMarshaller() throws Exception {
++    protected BinaryMarshaller createMarshaller() throws Exception {
+         PortableContext ctx = new PortableContext(BinaryCachingMetadataHandler.create(), new IgniteConfiguration());
  
 -        PortableMarshaller marsh = new PortableMarshaller();
 +        BinaryMarshaller marsh = new BinaryMarshaller();
  
 -        marsh.setCompactFooter(compactFooter());
 +        BinaryConfiguration bCfg = new BinaryConfiguration();
++        
++        bCfg.setCompactFooter(compactFooter());
  
 -        marsh.setTypeConfigurations(Arrays.asList(
 +        bCfg.setTypeConfigurations(Arrays.asList(
              new BinaryTypeConfiguration(TestObject.class.getName()),
              new BinaryTypeConfiguration(TestOuterObject.class.getName()),
              new BinaryTypeConfiguration(TestInnerObject.class.getName())

http://git-wip-us.apache.org/repos/asf/ignite/blob/a511fa17/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsAbstractSelfTest.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsAbstractSelfTest.java
index 0000000,3ec0b83..7267b16
mode 000000,100644..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
@@@ -1,0 -1,199 +1,206 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one or more
+  * contributor license agreements.  See the NOTICE file distributed with
+  * this work for additional information regarding copyright ownership.
+  * The ASF licenses this file to You under the Apache License, Version 2.0
+  * (the "License"); you may not use this file except in compliance with
+  * the License.  You may obtain a copy of the License at
+  *
+  *      http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ 
+ package org.apache.ignite.internal.portable;
+ 
++import java.util.Arrays;
+ import org.apache.ignite.binary.BinaryField;
+ import org.apache.ignite.binary.BinaryTypeConfiguration;
++import org.apache.ignite.configuration.BinaryConfiguration;
+ import org.apache.ignite.configuration.IgniteConfiguration;
+ import org.apache.ignite.internal.util.IgniteUtils;
+ import org.apache.ignite.marshaller.MarshallerContextTestImpl;
 -import org.apache.ignite.marshaller.portable.PortableMarshaller;
++import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+ 
 -import java.util.Arrays;
 -
+ /**
+  * Contains tests for compact offsets.
+  */
+ public abstract class BinaryFooterOffsetsAbstractSelfTest extends GridCommonAbstractTest {
+     /** 2 pow 8. */
+     private static int POW_8 = 1 << 8;
+ 
+     /** 2 pow 16. */
+     private static int POW_16 = 1 << 16;
+ 
+     /** Marshaller. */
 -    protected PortableMarshaller marsh;
++    protected BinaryMarshaller marsh;
+ 
+     /** Portable context. */
+     protected PortableContext ctx;
+ 
+     /** {@inheritDoc} */
+     @Override protected void beforeTest() throws Exception {
+         super.beforeTest();
+ 
+         ctx = new PortableContext(BinaryCachingMetadataHandler.create(), new IgniteConfiguration());
+ 
 -        marsh = new PortableMarshaller();
++        marsh = new BinaryMarshaller();
++
++        IgniteConfiguration iCfg = new IgniteConfiguration();
+ 
 -        marsh.setCompactFooter(compactFooter());
++        BinaryConfiguration bCfg = new BinaryConfiguration();
++
++        bCfg.setTypeConfigurations(Arrays.asList(new BinaryTypeConfiguration(TestObject.class.getName())));
++        
++        bCfg.setCompactFooter(compactFooter());
++
++        iCfg.setBinaryConfiguration(bCfg);
+ 
 -        marsh.setTypeConfigurations(Arrays.asList(new BinaryTypeConfiguration(TestObject.class.getName())));
+         marsh.setContext(new MarshallerContextTestImpl(null));
+ 
 -        IgniteUtils.invoke(PortableMarshaller.class, marsh, "setPortableContext", ctx);
++        IgniteUtils.invoke(BinaryMarshaller.class, marsh, "setPortableContext", ctx, iCfg);
+     }
+ 
+     /**
+      * @return Whether to use compact footers.
+      */
+     protected boolean compactFooter() {
+         return true;
+     }
 -
++    
+     /**
+      * Test 1 byte.
+      *
+      * @throws Exception If failed.
+      */
+     public void test1Byte() throws Exception {
+         check(POW_8 >> 2);
+     }
+ 
+     /**
+      * Test 1 byte with sign altering.
+      *
+      * @throws Exception If failed.
+      */
+     public void test1ByteSign() throws Exception {
+         check(POW_8 >> 1);
+     }
+ 
+     /**
+      * Test 2 bytes.
+      *
+      * @throws Exception If failed.
+      */
+     public void test2Bytes() throws Exception {
+         check(POW_16 >> 2);
+     }
+ 
+     /**
+      * Test 2 bytes with sign altering.
+      *
+      * @throws Exception If failed.
+      */
+     public void test2BytesSign() throws Exception {
+         check(POW_16 >> 1);
+     }
+ 
+     /**
+      * Test 4 bytes.
+      *
+      * @throws Exception If failed.
+      */
+     public void test4Bytes() throws Exception {
+         check(POW_16 << 2);
+     }
+ 
+     /**
+      * Main check routine.
+      *
+      * @param len Length of the first field.
+      *
+      * @throws Exception If failed.
+      */
+     private void check(int len) throws Exception {
+         TestObject obj = new TestObject(len);
+ 
+         BinaryObjectEx portObj = toPortable(marsh, obj);
+ 
+         // 1. Test portable object content.
+         assert portObj.hasField("field1");
+         assert portObj.hasField("field2");
+ 
+         byte[] field1 = portObj.field("field1");
+         Integer field2 = portObj.field("field2");
+ 
+         assert field1 != null;
+         assert field2 != null;
+ 
+         assert Arrays.equals(obj.field1, field1);
+         assert obj.field2 == field2;
+ 
+         // 2. Test fields API.
+         BinaryField field1Desc = portObj.type().field("field1");
+         BinaryField field2Desc = portObj.type().field("field2");
+ 
+         assert field1Desc.exists(portObj);
+         assert field2Desc.exists(portObj);
+ 
+         assert Arrays.equals(obj.field1, (byte[])field1Desc.value(portObj));
+         assert obj.field2 == (Integer)field2Desc.value(portObj);
+ 
+         // 3. Test deserialize.
+         TestObject objRestored = portObj.deserialize();
+ 
+         assert objRestored != null;
+ 
+         assert Arrays.equals(obj.field1, objRestored.field1);
+         assert obj.field2 == objRestored.field2;
+     }
+ 
+     /**
+      * Convert object to portable object.
+      *
+      * @param marsh Marshaller.
+      * @param obj Object.
+      * @return Portable object.
+      * @throws Exception If failed.
+      */
 -    protected abstract BinaryObjectEx toPortable(PortableMarshaller marsh, Object obj) throws Exception;
++    protected abstract BinaryObjectEx toPortable(BinaryMarshaller marsh, Object obj) throws Exception;
+ 
+     /**
+      * Test object.
+      */
+     public static class TestObject {
+         /** First field with variable length. */
+         public byte[] field1;
+ 
+         /** Second field. */
+         public int field2;
+ 
+         /**
+          * Default constructor.
+          */
+         public TestObject() {
+             // No-op.
+         }
+ 
+         /**
+          * Constructor.
+          *
+          * @param len Array length.
+          */
+         public TestObject(int len) {
+             field1 = new byte[len];
+ 
+             field1[0] = 1;
+             field1[len - 1] = 2;
+ 
+             field2 = len;
+         }
+     }
+ }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a511fa17/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsHeapSelfTest.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsHeapSelfTest.java
index 0000000,b23f012..471bd44
mode 000000,100644..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
@@@ -1,0 -1,32 +1,32 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one or more
+  * contributor license agreements.  See the NOTICE file distributed with
+  * this work for additional information regarding copyright ownership.
+  * The ASF licenses this file to You under the Apache License, Version 2.0
+  * (the "License"); you may not use this file except in compliance with
+  * the License.  You may obtain a copy of the License at
+  *
+  *      http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ 
+ package org.apache.ignite.internal.portable;
+ 
 -import org.apache.ignite.marshaller.portable.PortableMarshaller;
++import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+ 
+ /**
+  * Compact offsets tests for heap portable objects.
+  */
+ public class BinaryFooterOffsetsHeapSelfTest extends BinaryFooterOffsetsAbstractSelfTest {
+     /** {@inheritDoc} */
 -    @Override protected BinaryObjectEx toPortable(PortableMarshaller marsh, Object obj) throws Exception {
++    @Override protected BinaryObjectEx 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/a511fa17/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsOffheapSelfTest.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsOffheapSelfTest.java
index 0000000,e52ebe7..7b44b80
mode 000000,100644..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
@@@ -1,0 -1,61 +1,61 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one or more
+  * contributor license agreements.  See the NOTICE file distributed with
+  * this work for additional information regarding copyright ownership.
+  * The ASF licenses this file to You under the Apache License, Version 2.0
+  * (the "License"); you may not use this file except in compliance with
+  * the License.  You may obtain a copy of the License at
+  *
+  *      http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ 
+ package org.apache.ignite.internal.portable;
+ 
+ import org.apache.ignite.internal.util.GridUnsafe;
 -import org.apache.ignite.marshaller.portable.PortableMarshaller;
++import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+ import org.eclipse.jetty.util.ConcurrentHashSet;
+ import sun.misc.Unsafe;
+ 
+ /**
+  * Compact offsets tests for offheap portable objects.
+  */
+ public class BinaryFooterOffsetsOffheapSelfTest extends BinaryFooterOffsetsAbstractSelfTest {
+     /** Unsafe instance. */
+     private static final Unsafe UNSAFE = GridUnsafe.unsafe();
+ 
+     /** Byte array offset for unsafe mechanics. */
+     protected static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+ 
+     /** Allocated unsafe pointer. */
+     private final ConcurrentHashSet<Long> ptrs = new ConcurrentHashSet<>();
+ 
+     /** {@inheritDoc} */
+     @Override protected void afterTest() throws Exception {
+         super.afterTest();
+ 
+         // Cleanup allocated objects.
+         for (Long ptr : ptrs)
+             UNSAFE.freeMemory(ptr);
+ 
+         ptrs.clear();
+     }
+ 
+     /** {@inheritDoc} */
 -    @Override protected BinaryObjectEx toPortable(PortableMarshaller marsh, Object obj) throws Exception {
++    @Override protected BinaryObjectEx toPortable(BinaryMarshaller marsh, Object obj) throws Exception {
+         byte[] arr = marsh.marshal(obj);
+ 
+         long ptr = UNSAFE.allocateMemory(arr.length);
+ 
+         ptrs.add(ptr);
+ 
+         UNSAFE.copyMemory(arr, BYTE_ARR_OFF, null, ptr, arr.length);
+ 
+         return new BinaryObjectOffheapImpl(ctx, ptr, 0, arr.length);
+     }
+ }