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 2016/02/04 11:09:17 UTC

[14/50] [abbrv] ignite git commit: IGNITE-2191 - Support classes with the same simple name for Binary marshaller - Fixes #398.

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/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 3cfbcaa..2e60025 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
@@ -19,6 +19,10 @@ package org.apache.ignite.internal.processors.platform.cpp;
 
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.binary.BinaryIdMapper;
+import org.apache.ignite.binary.BinaryBasicIdMapper;
+import org.apache.ignite.binary.BinaryNameMapper;
+import org.apache.ignite.binary.BinaryBasicNameMapper;
 import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.PlatformConfiguration;
@@ -87,8 +91,36 @@ public class PlatformCppConfigurationClosure extends PlatformAbstractConfigurati
             bCfg = new BinaryConfiguration();
 
             bCfg.setCompactFooter(false);
+            bCfg.setNameMapper(new BinaryBasicNameMapper(true));
+            bCfg.setIdMapper(new BinaryBasicIdMapper(true));
 
             igniteCfg.setBinaryConfiguration(bCfg);
+
+            cppCfg0.warnings(Collections.singleton("Binary configuration is automatically initiated, " +
+                "note that binary name mapper is set to " + bCfg.getNameMapper()
+                + " and binary ID mapper is set to " + bCfg.getIdMapper()
+                + " (other nodes must have the same binary name and ID mapper types)."));
+        }
+        else {
+            BinaryNameMapper nameMapper = bCfg.getNameMapper();
+
+            if (nameMapper == null) {
+                bCfg.setNameMapper(new BinaryBasicNameMapper(true));
+
+                cppCfg0.warnings(Collections.singleton("Binary name mapper is automatically set to " +
+                    bCfg.getNameMapper()
+                    + " (other nodes must have the same binary name mapper type)."));
+            }
+
+            BinaryIdMapper idMapper = bCfg.getIdMapper();
+
+            if (idMapper == null) {
+                bCfg.setIdMapper(new BinaryBasicIdMapper(true));
+
+                cppCfg0.warnings(Collections.singleton("Binary ID mapper is automatically set to " +
+                    bCfg.getIdMapper()
+                    + " (other nodes must have the same binary ID mapper type)."));
+            }
         }
 
         if (bCfg.isCompactFooter())

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/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 c068d59..6b9b441 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
@@ -19,6 +19,10 @@ package org.apache.ignite.internal.processors.platform.dotnet;
 
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.binary.BinaryIdMapper;
+import org.apache.ignite.binary.BinaryBasicIdMapper;
+import org.apache.ignite.binary.BinaryNameMapper;
+import org.apache.ignite.binary.BinaryBasicNameMapper;
 import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.PlatformConfiguration;
@@ -90,7 +94,7 @@ public class PlatformDotNetConfigurationClosure extends PlatformAbstractConfigur
 
         igniteCfg.setPlatformConfiguration(dotNetCfg0);
 
-        // Check marshaller
+        // Check marshaller.
         Marshaller marsh = igniteCfg.getMarshaller();
 
         if (marsh == null) {
@@ -109,8 +113,36 @@ public class PlatformDotNetConfigurationClosure extends PlatformAbstractConfigur
             bCfg = new BinaryConfiguration();
 
             bCfg.setCompactFooter(false);
+            bCfg.setNameMapper(new BinaryBasicNameMapper(true));
+            bCfg.setIdMapper(new BinaryBasicIdMapper(true));
 
             igniteCfg.setBinaryConfiguration(bCfg);
+
+            dotNetCfg0.warnings(Collections.singleton("Binary configuration is automatically initiated, " +
+                "note that binary name mapper is set to " + bCfg.getNameMapper()
+                + " and binary ID mapper is set to " + bCfg.getIdMapper()
+                + " (other nodes must have the same binary name and ID mapper types)."));
+        }
+        else {
+            BinaryNameMapper nameMapper = bCfg.getNameMapper();
+
+            if (nameMapper == null) {
+                bCfg.setNameMapper(new BinaryBasicNameMapper(true));
+
+                dotNetCfg0.warnings(Collections.singleton("Binary name mapper is automatically set to " +
+                    bCfg.getNameMapper()
+                    + " (other nodes must have the same binary name mapper type)."));
+            }
+
+            BinaryIdMapper idMapper = bCfg.getIdMapper();
+
+            if (idMapper == null) {
+                bCfg.setIdMapper(new BinaryBasicIdMapper(true));
+
+                dotNetCfg0.warnings(Collections.singleton("Binary ID mapper is automatically set to " +
+                    bCfg.getIdMapper()
+                    + " (other nodes must have the same binary ID mapper type)."));
+            }
         }
 
         if (bCfg.isCompactFooter())

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryBasicIdMapperSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryBasicIdMapperSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryBasicIdMapperSelfTest.java
new file mode 100644
index 0000000..1d6da2c
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryBasicIdMapperSelfTest.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.binary;
+
+import org.apache.ignite.binary.BinaryBasicIdMapper;
+import org.apache.ignite.internal.binary.test.GridBinaryTestClass1;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ *
+ */
+public class BinaryBasicIdMapperSelfTest extends GridCommonAbstractTest {
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLowerCase() throws Exception {
+        BinaryBasicIdMapper mapper = new BinaryBasicIdMapper(true);
+
+        assertEquals(GridBinaryTestClass1.class.getName().toLowerCase().hashCode(),
+            mapper.typeId(GridBinaryTestClass1.class.getName()));
+        assertEquals((GridBinaryTestClass1.class.getName() + "$InnerClass").toLowerCase().hashCode(),
+            mapper.typeId(GridBinaryTestClass1.class.getName() + "$InnerClass"));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDefaultCase() throws Exception {
+        BinaryBasicIdMapper mapper = new BinaryBasicIdMapper(false);
+
+        assertEquals(GridBinaryTestClass1.class.getName().hashCode(),
+            mapper.typeId(GridBinaryTestClass1.class.getName()));
+        assertEquals((GridBinaryTestClass1.class.getName() + "$InnerClass").hashCode(),
+            mapper.typeId(GridBinaryTestClass1.class.getName() + "$InnerClass"));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryBasicNameMapperSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryBasicNameMapperSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryBasicNameMapperSelfTest.java
new file mode 100644
index 0000000..70fb8e7
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryBasicNameMapperSelfTest.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.binary;
+
+import org.apache.ignite.binary.BinaryBasicNameMapper;
+import org.apache.ignite.internal.binary.test.GridBinaryTestClass1;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ *
+ */
+public class BinaryBasicNameMapperSelfTest extends GridCommonAbstractTest {
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSimpleName() throws Exception {
+        BinaryBasicNameMapper mapper = new BinaryBasicNameMapper(true);
+
+        assertEquals("GridBinaryTestClass1", mapper.typeName(GridBinaryTestClass1.class.getName()));
+
+        assertEquals("InnerClass", mapper.typeName(GridBinaryTestClass1.class.getName() + "$InnerClass"));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testFullName() throws Exception {
+        BinaryBasicNameMapper mapper = new BinaryBasicNameMapper(false);
+
+        assertEquals(GridBinaryTestClass1.class.getName(), mapper.typeName(GridBinaryTestClass1.class.getName()));
+
+        assertEquals(GridBinaryTestClass1.class.getName() + "$InnerClass",
+            mapper.typeName(GridBinaryTestClass1.class.getName() + "$InnerClass"));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryConfigurationConsistencySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryConfigurationConsistencySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryConfigurationConsistencySelfTest.java
new file mode 100644
index 0000000..7a56d9c
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryConfigurationConsistencySelfTest.java
@@ -0,0 +1,231 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.binary;
+
+import java.util.Arrays;
+import java.util.concurrent.Callable;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryReader;
+import org.apache.ignite.binary.BinarySerializer;
+import org.apache.ignite.binary.BinaryBasicIdMapper;
+import org.apache.ignite.binary.BinaryTypeConfiguration;
+import org.apache.ignite.binary.BinaryWriter;
+import org.apache.ignite.configuration.BinaryConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK;
+
+/**
+ * Tests a check of binary configuration consistency.
+ */
+public class BinaryConfigurationConsistencySelfTest extends GridCommonAbstractTest {
+    /** */
+    private BinaryConfiguration binaryCfg;
+
+    /** */
+    private boolean isClient;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setClientMode(isClient);
+
+        cfg.setMarshaller(new BinaryMarshaller());
+
+        cfg.setBinaryConfiguration(binaryCfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        isClient = false;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSkipCheckConsistencyFlagEnabled() throws Exception {
+        String backup = System.setProperty(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK, "true");
+
+        try {
+            // Wrong usage of Ignite (done only in test purposes).
+            binaryCfg = null;
+
+            startGrid(0);
+
+            binaryCfg = new BinaryConfiguration();
+
+            startGrid(1);
+
+            isClient = true;
+            binaryCfg = customConfig(true);
+
+            startGrid(2);
+        }
+        finally {
+            if (backup != null)
+                System.setProperty(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK, backup);
+            else
+                System.clearProperty(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPositiveNullConfig() throws Exception {
+        binaryCfg = null;
+
+        startGrids(2);
+
+        isClient = true;
+
+        startGrid(2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPositiveEmptyConfig() throws Exception {
+        binaryCfg = new BinaryConfiguration();
+
+        startGrids(2);
+
+        isClient = true;
+
+        startGrid(2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPositiveCustomConfig() throws Exception {
+        binaryCfg = customConfig(false);
+
+        startGrids(2);
+
+        isClient = true;
+
+        startGrid(2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNegativeNullEmptyConfigs() throws Exception {
+        checkNegative(null, new BinaryConfiguration());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNegativeEmptyNullConfigs() throws Exception {
+        checkNegative(new BinaryConfiguration(), null);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNegativeEmptyCustomConfigs() throws Exception {
+        checkNegative(new BinaryConfiguration(), customConfig(false));
+    }
+
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNegativeCustomNullConfigs() throws Exception {
+        checkNegative(customConfig(false), null);
+    }
+
+    /**
+     * @param bCfg1 BinaryConfiguration 1.
+     * @param bCfg2 BinaryConfiguration 2.
+     * @throws Exception If failed.
+     */
+    private void checkNegative(final BinaryConfiguration bCfg1, BinaryConfiguration bCfg2) throws Exception {
+        binaryCfg = bCfg1;
+
+        startGrid(0);
+
+        binaryCfg = bCfg2;
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                startGrid(1);
+
+                return null;
+            }
+        }, IgniteCheckedException.class, "");
+
+        isClient = true;
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                startGrid(2);
+
+                return null;
+            }
+        }, IgniteCheckedException.class, "");
+    }
+
+    /**
+     * @return Custom BinaryConfiguration.
+     * @param compactFooter Compact footer.
+     */
+    private BinaryConfiguration customConfig(boolean compactFooter) {
+        BinaryConfiguration c = new BinaryConfiguration();
+
+        c.setIdMapper(new BinaryBasicIdMapper(true));
+        c.setSerializer(new BinarySerializer() {
+            @Override public void writeBinary(Object obj, BinaryWriter writer) throws BinaryObjectException {
+                // No-op.
+            }
+
+            @Override public void readBinary(Object obj, BinaryReader reader) throws BinaryObjectException {
+                // No-op.
+            }
+        });
+        c.setCompactFooter(compactFooter);
+
+        BinaryTypeConfiguration btc = new BinaryTypeConfiguration("org.MyClass");
+
+        btc.setIdMapper(BinaryContext.defaultIdMapper());
+        btc.setEnum(false);
+        btc.setSerializer(new BinarySerializer() {
+            @Override public void writeBinary(Object obj, BinaryWriter writer) throws BinaryObjectException {
+                // No-op.
+            }
+
+            @Override public void readBinary(Object obj, BinaryReader reader) throws BinaryObjectException {
+                // No-op.
+            }
+        });
+
+        c.setTypeConfigurations(Arrays.asList(btc));
+
+        return c;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryEnumsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryEnumsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryEnumsSelfTest.java
index e503e07..6bd9b65 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryEnumsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryEnumsSelfTest.java
@@ -221,10 +221,10 @@ public class BinaryEnumsSelfTest extends GridCommonAbstractTest {
     private void checkNestedBuilder(boolean registered) throws Exception {
         startUp(registered);
 
-        BinaryObject obj = node1.binary().builder("EnumHolder").setField("val", EnumType.ONE).build();
+        BinaryObject obj = node1.binary().builder(EnumHolder.class.getName()).setField("val", EnumType.ONE).build();
 
-        assert node1.binary().type("EnumHolder") != null;
-        assert node1.binary().type("EnumType") != null;
+        assert node1.binary().type(EnumHolder.class.getName()) != null;
+        assert node1.binary().type(EnumType.class.getName()) != null;
 
         cacheBinary1.put(1, obj);
 
@@ -271,7 +271,7 @@ public class BinaryEnumsSelfTest extends GridCommonAbstractTest {
     public void checkSimpleBuilder(boolean registered) throws Exception {
         startUp(registered);
 
-        BinaryObject binary = node1.binary().buildEnum(EnumType.class.getSimpleName(), EnumType.ONE.ordinal());
+        BinaryObject binary = node1.binary().buildEnum(EnumType.class.getName(), EnumType.ONE.ordinal());
 
         cacheBinary1.put(1, binary);
 
@@ -337,8 +337,8 @@ public class BinaryEnumsSelfTest extends GridCommonAbstractTest {
     public void checkSimpleBuilderArray(boolean registered) throws Exception {
         startUp(registered);
 
-        BinaryObject binaryOne = node1.binary().buildEnum(EnumType.class.getSimpleName(), EnumType.ONE.ordinal());
-        BinaryObject binaryTwo = node1.binary().buildEnum(EnumType.class.getSimpleName(), EnumType.TWO.ordinal());
+        BinaryObject binaryOne = node1.binary().buildEnum(EnumType.class.getName(), EnumType.ONE.ordinal());
+        BinaryObject binaryTwo = node1.binary().buildEnum(EnumType.class.getName(), EnumType.TWO.ordinal());
 
         cacheBinary1.put(1, new BinaryObject[] { binaryOne, binaryTwo });
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java
index c347b9f..fa0f9a7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java
@@ -17,12 +17,49 @@
 
 package org.apache.ignite.internal.binary;
 
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.io.Serializable;
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.net.InetSocketAddress;
+import java.sql.Timestamp;
+import java.util.AbstractQueue;
+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.Iterator;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Queue;
+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.BinaryBasicIdMapper;
+import org.apache.ignite.binary.BinaryBasicNameMapper;
 import org.apache.ignite.binary.BinaryCollectionFactory;
 import org.apache.ignite.binary.BinaryField;
 import org.apache.ignite.binary.BinaryIdMapper;
 import org.apache.ignite.binary.BinaryMapFactory;
+import org.apache.ignite.binary.BinaryNameMapper;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryObjectBuilder;
 import org.apache.ignite.binary.BinaryObjectException;
@@ -53,43 +90,9 @@ import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 import sun.misc.Unsafe;
 
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.io.Serializable;
-import java.lang.reflect.Field;
-import java.lang.reflect.InvocationHandler;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.lang.reflect.Proxy;
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.net.InetSocketAddress;
-import java.sql.Timestamp;
-import java.util.AbstractQueue;
-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.Iterator;
-import java.util.LinkedHashMap;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Queue;
-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.binary.streams.BinaryMemoryAllocator.*;
-import static org.junit.Assert.*;
+import static org.apache.ignite.internal.binary.streams.BinaryMemoryAllocator.INSTANCE;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertNotEquals;
 
 /**
  * Binary marshaller tests.
@@ -1047,7 +1050,7 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
             }
         });
 
-        BinaryMarshaller marsh = binaryMarshaller(new BinaryIdMapper() {
+        BinaryMarshaller marsh = binaryMarshaller(null, new BinaryIdMapper() {
             @Override public int typeId(String clsName) {
                 return 11111;
             }
@@ -1092,6 +1095,63 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
+    public void testSimpleNameLowerCaseMappers() throws Exception {
+        BinaryTypeConfiguration innerClassType = new BinaryTypeConfiguration(InnerMappedObject.class.getName());
+        BinaryTypeConfiguration publicClassType = new BinaryTypeConfiguration(TestMappedObject.class.getName());
+        BinaryTypeConfiguration typeWithCustomMapper = new BinaryTypeConfiguration(CustomMappedObject2.class.getName());
+
+        typeWithCustomMapper.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;
+            }
+        });
+
+        BinaryMarshaller marsh = binaryMarshaller(new BinaryBasicNameMapper(true), new BinaryBasicIdMapper(true),
+            Arrays.asList(innerClassType, publicClassType, typeWithCustomMapper));
+
+        InnerMappedObject innerObj = new InnerMappedObject(10, "str1");
+
+        BinaryObjectExImpl innerBo = marshal(innerObj, marsh);
+
+        assertEquals("InnerMappedObject".toLowerCase().hashCode(), innerBo.type().typeId());
+
+        assertEquals(10, innerBo.<CustomMappedObject1>deserialize().val1);
+        assertEquals("str1", innerBo.<CustomMappedObject1>deserialize().val2);
+
+        TestMappedObject publicObj = new TestMappedObject();
+
+        BinaryObjectExImpl publicBo = marshal(publicObj, marsh);
+
+        assertEquals("TestMappedObject".toLowerCase().hashCode(), publicBo.type().typeId());
+
+        CustomMappedObject2 obj2 = new CustomMappedObject2(20, "str2");
+
+        BinaryObjectExImpl po2 = marshal(obj2, marsh);
+
+        assertEquals(44444, po2.type().typeId());
+        assertEquals((Integer)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 {
         BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(
             new BinaryTypeConfiguration(DynamicObject.class.getName())
@@ -1299,7 +1359,7 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
-    public void testTypeNames() throws Exception {
+    public void testTypeNamesSimpleNameMapper() throws Exception {
         BinaryTypeConfiguration customType1 = new BinaryTypeConfiguration(Value.class.getName());
 
         customType1.setIdMapper(new BinaryIdMapper() {
@@ -1336,7 +1396,7 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
             }
         });
 
-        BinaryTypeConfiguration customType4 = new BinaryTypeConfiguration("NonExistentClass5");
+        BinaryTypeConfiguration customType4 = new BinaryTypeConfiguration("NonExistentClass0");
 
         customType4.setIdMapper(new BinaryIdMapper() {
             @Override public int typeId(String clsName) {
@@ -1348,7 +1408,8 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
             }
         });
 
-        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(
+        BinaryMarshaller marsh = binaryMarshaller(new BinaryBasicNameMapper(true), new BinaryBasicIdMapper(true),
+            Arrays.asList(
             new BinaryTypeConfiguration(Key.class.getName()),
             new BinaryTypeConfiguration("org.gridgain.NonExistentClass3"),
             new BinaryTypeConfiguration("NonExistentClass4"),
@@ -1360,14 +1421,296 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
 
         BinaryContext ctx = binaryContext(marsh);
 
+        // Full name hashCode.
         assertEquals("notconfiguredclass".hashCode(), ctx.typeId("NotConfiguredClass"));
-        assertEquals("key".hashCode(), ctx.typeId("Key"));
-        assertEquals("nonexistentclass3".hashCode(), ctx.typeId("NonExistentClass3"));
+        assertEquals("key".hashCode(), ctx.typeId(Key.class.getName()));
+        assertEquals("nonexistentclass3".hashCode(), ctx.typeId("org.gridgain.NonExistentClass3"));
         assertEquals("nonexistentclass4".hashCode(), ctx.typeId("NonExistentClass4"));
-        assertEquals(300, ctx.typeId(getClass().getSimpleName() + "$Value"));
-        assertEquals(400, ctx.typeId("NonExistentClass1"));
+        assertEquals(300, ctx.typeId(Value.class.getName()));
+        assertEquals(400, ctx.typeId("org.gridgain.NonExistentClass1"));
         assertEquals(500, ctx.typeId("NonExistentClass2"));
-        assertEquals("nonexistentclass5".hashCode(), ctx.typeId("NonExistentClass5"));
+
+        // BinaryIdMapper.typeId() contract.
+        assertEquals("nonexistentclass0".hashCode(), ctx.typeId("NonExistentClass0"));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTypeNamesFullNameMappers() throws Exception {
+        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("NonExistentClass0");
+
+        customType4.setIdMapper(new BinaryIdMapper() {
+            @Override public int typeId(String clsName) {
+                return 0;
+            }
+
+            @Override public int fieldId(int typeId, String fieldName) {
+                return 0;
+            }
+        });
+
+        BinaryMarshaller marsh = binaryMarshaller(new BinaryBasicNameMapper(false), new BinaryBasicIdMapper(false),
+            Arrays.asList(
+                new BinaryTypeConfiguration(Key.class.getName()),
+                new BinaryTypeConfiguration("org.gridgain.NonExistentClass3"),
+                new BinaryTypeConfiguration("NonExistentClass4"),
+                customType1,
+                customType2,
+                customType3,
+                customType4
+            ));
+
+        BinaryContext ctx = binaryContext(marsh);
+
+        // Full name hashCode.
+        assertEquals("NotConfiguredClass".hashCode(), ctx.typeId("NotConfiguredClass"));
+        assertEquals(Key.class.getName().hashCode(), ctx.typeId(Key.class.getName()));
+        assertEquals("org.gridgain.NonExistentClass3".hashCode(), ctx.typeId("org.gridgain.NonExistentClass3"));
+        assertEquals("NonExistentClass4".hashCode(), ctx.typeId("NonExistentClass4"));
+        assertEquals(300, ctx.typeId(Value.class.getName()));
+        assertEquals(400, ctx.typeId("org.gridgain.NonExistentClass1"));
+        assertEquals(500, ctx.typeId("NonExistentClass2"));
+
+        // BinaryIdMapper.typeId() contract.
+        assertEquals("nonexistentclass0".hashCode(), ctx.typeId("NonExistentClass0"));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTypeNamesSimpleNameMappers() throws Exception {
+        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("NonExistentClass0");
+
+        customType4.setIdMapper(new BinaryIdMapper() {
+            @Override public int typeId(String clsName) {
+                return 0;
+            }
+
+            @Override public int fieldId(int typeId, String fieldName) {
+                return 0;
+            }
+        });
+
+        BinaryTypeConfiguration customType5 = new BinaryTypeConfiguration(DateClass1.class.getName());
+
+        customType5.setNameMapper(new BinaryBasicNameMapper(false));
+        customType5.setIdMapper(new BinaryBasicIdMapper(false));
+
+        BinaryMarshaller marsh = binaryMarshaller(new BinaryBasicNameMapper(true), new BinaryBasicIdMapper(true),
+            Arrays.asList(
+                new BinaryTypeConfiguration(Key.class.getName()),
+                new BinaryTypeConfiguration("org.gridgain.NonExistentClass3"),
+                new BinaryTypeConfiguration("NonExistentClass4"),
+                customType1,
+                customType2,
+                customType3,
+                customType4,
+                customType5
+            ));
+
+        BinaryContext ctx = binaryContext(marsh);
+
+        assertEquals("notconfiguredclass".hashCode(), ctx.typeId("NotConfiguredClass"));
+        assertEquals("notconfiguredclass".hashCode(), ctx.typeId("org.blabla.NotConfiguredClass"));
+        assertEquals("key".hashCode(), ctx.typeId(Key.class.getName()));
+        assertEquals("nonexistentclass3".hashCode(), ctx.typeId("org.gridgain.NonExistentClass3"));
+        assertEquals("nonexistentclass4".hashCode(), ctx.typeId("NonExistentClass4"));
+
+        assertEquals(300, ctx.typeId(Value.class.getName()));
+        assertEquals(400, ctx.typeId("org.gridgain.NonExistentClass1"));
+        assertEquals(500, ctx.typeId("NonExistentClass2"));
+
+        assertEquals(DateClass1.class.getName().hashCode(), ctx.typeId(DateClass1.class.getName()));
+
+        // BinaryIdMapper.typeId() contract.
+        assertEquals("nonexistentclass0".hashCode(), ctx.typeId("NonExistentClass0"));
+    }
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTypeNamesCustomIdMapper() throws Exception {
+        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("NonExistentClass0");
+
+        customType4.setIdMapper(new BinaryIdMapper() {
+            @Override public int typeId(String clsName) {
+                return 0;
+            }
+
+            @Override public int fieldId(int typeId, String fieldName) {
+                return 0;
+            }
+        });
+
+        BinaryTypeConfiguration customType5 = new BinaryTypeConfiguration(DateClass1.class.getName());
+
+        customType5.setIdMapper(new BinaryBasicIdMapper(false));
+
+        BinaryTypeConfiguration customType6 = new BinaryTypeConfiguration(MyTestClass.class.getName());
+
+        customType6.setIdMapper(new BinaryBasicIdMapper(true));
+        customType6.setNameMapper(new BinaryBasicNameMapper(true));
+
+        BinaryMarshaller marsh = binaryMarshaller(new BinaryBasicNameMapper(false), new BinaryIdMapper() {
+            @Override public int typeId(String clsName) {
+                if ("org.blabla.NotConfiguredSpecialClass".equals(clsName))
+                    return 0;
+                else if (Key.class.getName().equals(clsName))
+                    return 991;
+                else if ("org.gridgain.NonExistentClass3".equals(clsName))
+                    return 992;
+                else if ("NonExistentClass4".equals(clsName))
+                    return 993;
+
+                return 999;
+            }
+
+            @Override public int fieldId(int typeId, String fieldName) {
+                return 0;
+            }
+        }, Arrays.asList(
+            new BinaryTypeConfiguration(Key.class.getName()),
+            new BinaryTypeConfiguration("org.gridgain.NonExistentClass3"),
+            new BinaryTypeConfiguration("NonExistentClass4"),
+            customType1,
+            customType2,
+            customType3,
+            customType4,
+            customType5,
+            customType6
+        ));
+
+        BinaryContext ctx = binaryContext(marsh);
+
+        assertEquals(999, ctx.typeId("NotConfiguredClass"));
+        assertEquals(999, ctx.typeId("org.blabla.NotConfiguredClass"));
+
+        // BinaryIdMapper.typeId() contract.
+        assertEquals("notconfiguredspecialclass".hashCode(), ctx.typeId("org.blabla.NotConfiguredSpecialClass"));
+
+        assertEquals(991, ctx.typeId(Key.class.getName()));
+        assertEquals(992, ctx.typeId("org.gridgain.NonExistentClass3"));
+        assertEquals(993, ctx.typeId("NonExistentClass4"));
+
+
+        // Custom types.
+        assertEquals(300, ctx.typeId(Value.class.getName()));
+        assertEquals(400, ctx.typeId("org.gridgain.NonExistentClass1"));
+        assertEquals(500, ctx.typeId("NonExistentClass2"));
+
+        // BinaryIdMapper.typeId() contract.
+        assertEquals("nonexistentclass0".hashCode(), ctx.typeId("NonExistentClass0"));
+
+        assertEquals(DateClass1.class.getName().hashCode(), ctx.typeId(DateClass1.class.getName()));
+        assertEquals("mytestclass".hashCode(), ctx.typeId(MyTestClass.class.getName()));
     }
 
     /**
@@ -1656,11 +1999,11 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
 
         BinaryObject copy = copy(po, F.<String, Object>asMap("bArr", new byte[]{1, 2, 3}));
 
-        assertArrayEquals(new byte[]{1, 2, 3}, copy.<byte[]>field("bArr"));
+        assertArrayEquals(new byte[] {1, 2, 3}, copy.<byte[]>field("bArr"));
 
         SimpleObject obj0 = copy.deserialize();
 
-        assertArrayEquals(new byte[]{1, 2, 3}, obj0.bArr);
+        assertArrayEquals(new byte[] {1, 2, 3}, obj0.bArr);
     }
 
     /**
@@ -1819,11 +2162,11 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
 
         BinaryObject copy = copy(po, F.<String, Object>asMap("strArr", new String[]{"str1", "str2"}));
 
-        assertArrayEquals(new String[]{"str1", "str2"}, copy.<String[]>field("strArr"));
+        assertArrayEquals(new String[] {"str1", "str2"}, copy.<String[]>field("strArr"));
 
         SimpleObject obj0 = copy.deserialize();
 
-        assertArrayEquals(new String[]{"str1", "str2"}, obj0.strArr);
+        assertArrayEquals(new String[] {"str1", "str2"}, obj0.strArr);
     }
 
     /**
@@ -1887,7 +2230,7 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
 
         assertEquals("str555", obj0.str);
         assertEquals(newObj, obj0.inner);
-        assertArrayEquals(new byte[]{6, 7, 9}, obj0.bArr);
+        assertArrayEquals(new byte[] {6, 7, 9}, obj0.bArr);
     }
 
     /**
@@ -1912,8 +2255,8 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
         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);
+        map.put("bArr", new byte[] {6, 7, 9});
+        map.put("b", (byte)111);
 
         BinaryObject copy = copy(po, map);
 
@@ -1930,8 +2273,8 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
         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);
+        assertArrayEquals(new byte[] {6, 7, 9}, obj0.bArr);
+        assertEquals((byte)111, obj0.b);
     }
 
     /**
@@ -2190,8 +2533,9 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
-    public void testDuplicateName() throws Exception {
-        BinaryMarshaller marsh = binaryMarshaller();
+    public void testDuplicateNameSimpleNameMapper() throws Exception {
+        BinaryMarshaller marsh = binaryMarshaller(new BinaryBasicNameMapper(true),
+            new BinaryBasicIdMapper(true), null, null);
 
         Test1.Job job1 = new Test1().new Job();
         Test2.Job job2 = new Test2().new Job();
@@ -2213,6 +2557,21 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
+    public void testDuplicateNameFullNameMapper() throws Exception {
+        BinaryMarshaller marsh = binaryMarshaller(new BinaryBasicNameMapper(false),
+            new BinaryBasicIdMapper(false), null, null);
+
+        Test1.Job job1 = new Test1().new Job();
+        Test2.Job job2 = new Test2().new Job();
+
+        marsh.marshal(job1);
+
+        marsh.marshal(job2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testClass() throws Exception {
         BinaryMarshaller marsh = binaryMarshaller();
 
@@ -2288,13 +2647,13 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
     public void testPredefinedTypeIds() throws Exception {
         BinaryMarshaller marsh = binaryMarshaller();
 
-        BinaryContext pCtx = binaryContext(marsh);
+        BinaryContext bCtx = binaryContext(marsh);
 
-        Field field = pCtx.getClass().getDeclaredField("predefinedTypeNames");
+        Field field = bCtx.getClass().getDeclaredField("predefinedTypeNames");
 
         field.setAccessible(true);
 
-        Map<String, Integer> map = (Map<String, Integer>)field.get(pCtx);
+        Map<String, Integer> map = (Map<String, Integer>)field.get(bCtx);
 
         assertTrue(map.size() > 0);
 
@@ -2304,10 +2663,9 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
             if (id == GridBinaryMarshaller.UNREGISTERED_TYPE_ID)
                 continue;
 
-            BinaryClassDescriptor desc = pCtx.descriptorForTypeId(false, entry.getValue(), null, false);
+            BinaryClassDescriptor desc = bCtx.descriptorForTypeId(false, entry.getValue(), null, false);
 
-            assertEquals(desc.typeId(), pCtx.typeId(desc.describedClass().getName()));
-            assertEquals(desc.typeId(), pCtx.typeId(pCtx.typeName(desc.describedClass().getName())));
+            assertEquals(desc.typeId(), bCtx.typeId(desc.describedClass().getName()));
         }
     }
 
@@ -2714,7 +3072,7 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
      */
     protected BinaryMarshaller binaryMarshaller()
         throws IgniteCheckedException {
-        return binaryMarshaller(null, null, null);
+        return binaryMarshaller(null, null, null, null);
     }
 
     /**
@@ -2722,15 +3080,15 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
      */
     protected BinaryMarshaller binaryMarshaller(Collection<BinaryTypeConfiguration> cfgs)
         throws IgniteCheckedException {
-        return binaryMarshaller(null, null, cfgs);
+        return binaryMarshaller(null, null, null, cfgs);
     }
 
     /**
      *
      */
-    protected BinaryMarshaller binaryMarshaller(BinaryIdMapper mapper, Collection<BinaryTypeConfiguration> cfgs)
+    protected BinaryMarshaller binaryMarshaller(BinaryNameMapper nameMapper, BinaryIdMapper mapper, Collection<BinaryTypeConfiguration> cfgs)
         throws IgniteCheckedException {
-        return binaryMarshaller(mapper, null, cfgs);
+        return binaryMarshaller(nameMapper, mapper, null, cfgs);
     }
 
     /**
@@ -2738,13 +3096,14 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
      */
     protected BinaryMarshaller binaryMarshaller(BinarySerializer serializer, Collection<BinaryTypeConfiguration> cfgs)
         throws IgniteCheckedException {
-        return binaryMarshaller(null, serializer, cfgs);
+        return binaryMarshaller(null, null, serializer, cfgs);
     }
 
     /**
      * @return Binary marshaller.
      */
     protected BinaryMarshaller binaryMarshaller(
+        BinaryNameMapper nameMapper,
         BinaryIdMapper mapper,
         BinarySerializer serializer,
         Collection<BinaryTypeConfiguration> cfgs
@@ -2753,6 +3112,7 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
 
         BinaryConfiguration bCfg = new BinaryConfiguration();
 
+        bCfg.setNameMapper(nameMapper);
         bCfg.setIdMapper(mapper);
         bCfg.setSerializer(serializer);
         bCfg.setCompactFooter(compactFooter());
@@ -3657,6 +4017,18 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
 
     /**
      */
+    private static class InnerMappedObject extends CustomMappedObject1 {
+        /**
+         * @param val1 Val1
+         * @param val2 Val2
+         */
+        InnerMappedObject(int val1, String val2) {
+            super(val1, val2);
+        }
+    }
+
+    /**
+     */
     private static class CustomMappedObject2 {
         /** */
         private int val1;

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderAdditionalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderAdditionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderAdditionalSelfTest.java
index 572df88..06e1e93 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderAdditionalSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderAdditionalSelfTest.java
@@ -1258,7 +1258,7 @@ public class BinaryObjectBuilderAdditionalSelfTest extends GridCommonAbstractTes
         CacheObjectBinaryProcessorImpl processor = (CacheObjectBinaryProcessorImpl)(
             (IgniteBinaryImpl)binaries()).processor();
 
-        return new BinaryObjectBuilderImpl(processor.binaryContext(), processor.typeId(aCls.getName()),
-            aCls.getSimpleName());
+        return new BinaryObjectBuilderImpl(processor.binaryContext(), processor.typeId(aCls.getName()), 
+            processor.binaryContext().userTypeName(aCls.getName()));
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderDefaultMappersSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderDefaultMappersSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderDefaultMappersSelfTest.java
new file mode 100644
index 0000000..059703e
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderDefaultMappersSelfTest.java
@@ -0,0 +1,1149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.binary;
+
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import junit.framework.TestCase;
+import org.apache.ignite.IgniteBinary;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.binary.BinaryIdMapper;
+import org.apache.ignite.binary.BinaryNameMapper;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.binary.BinaryObjectBuilder;
+import org.apache.ignite.binary.BinaryBasicNameMapper;
+import org.apache.ignite.binary.BinaryBasicIdMapper;
+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.binary.builder.BinaryObjectBuilderImpl;
+import org.apache.ignite.internal.binary.mutabletest.GridBinaryTestClasses;
+import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import sun.misc.Unsafe;
+
+/**
+ * Binary builder test.
+ */
+@SuppressWarnings("ResultOfMethodCallIgnored")
+public class BinaryObjectBuilderDefaultMappersSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final Unsafe UNSAFE = GridUnsafe.unsafe();
+
+    /** */
+    protected static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** */
+    private static IgniteConfiguration cfg;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        BinaryTypeConfiguration customTypeCfg = new BinaryTypeConfiguration();
+
+        customTypeCfg.setTypeName(CustomIdMapper.class.getName());
+        customTypeCfg.setIdMapper(new BinaryIdMapper() {
+            @Override public int typeId(String clsName) {
+                return ~BinaryContext.defaultIdMapper().typeId(clsName);
+            }
+
+            @Override public int fieldId(int typeId, String fieldName) {
+                return typeId + ~BinaryContext.defaultIdMapper().fieldId(typeId, fieldName);
+            }
+        });
+
+        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.binary.mutabletest.*"),
+            customTypeCfg));
+
+        bCfg.setIdMapper(new BinaryBasicIdMapper(false));
+        bCfg.setNameMapper(new BinaryBasicNameMapper(false));
+
+        cfg.setBinaryConfiguration(bCfg);
+
+        cfg.setMarshaller(new BinaryMarshaller());
+
+        this.cfg = cfg;
+
+        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() {
+        GridBinaryTestClasses.TestObjectAllTypes obj = new GridBinaryTestClasses.TestObjectAllTypes();
+        obj.setDefaultData();
+        obj.enumArr = null;
+
+        GridBinaryTestClasses.TestObjectAllTypes deserialized = builder(toBinary(obj)).build().deserialize();
+
+        GridTestUtils.deepEquals(obj, deserialized);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNullField() throws Exception {
+        BinaryObjectBuilder builder = builder("Class");
+
+        builder.hashCode(42);
+
+        builder.setField("objField", (Object)null);
+
+        builder.setField("otherField", "value");
+
+        BinaryObject obj = builder.build();
+
+        assertNull(obj.field("objField"));
+        assertEquals("value", obj.field("otherField"));
+        assertEquals(42, obj.hashCode());
+
+        builder = builder(obj);
+
+        builder.setField("objField", "value");
+        builder.setField("otherField", (Object)null);
+
+        obj = builder.build();
+
+        assertNull(obj.field("otherField"));
+        assertEquals("value", obj.field("objField"));
+        assertEquals(42, obj.hashCode());
+    }
+
+    /**
+     * @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(expectedHashCode("Class"), po.type().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(expectedHashCode("Class"), po.type().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(expectedHashCode("Class"), po.type().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(expectedHashCode("Class"), po.type().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(expectedHashCode("Class"), po.type().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(expectedHashCode("Class"), po.type().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(expectedHashCode("Class"), po.type().typeId());
+        assertEquals(100, po.hashCode());
+
+        assertEquals((char)1, po.<Character>field("charField").charValue());
+    }
+
+    /**
+     * @return Expected hash code.
+     * @param fullName Full name of type.
+     */
+    private int expectedHashCode(String fullName) {
+        BinaryIdMapper idMapper = cfg.getBinaryConfiguration().getIdMapper();
+        BinaryNameMapper nameMapper = cfg.getBinaryConfiguration().getNameMapper();
+
+        if (idMapper == null)
+            idMapper = BinaryContext.defaultIdMapper();
+
+        if (nameMapper == null)
+            nameMapper = BinaryContext.defaultNameMapper();
+
+        return idMapper.typeId(nameMapper.typeName(fullName));
+    }
+
+    /**
+     * @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(expectedHashCode("Class"), po.type().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(expectedHashCode("Class"), po.type().typeId());
+        assertEquals(100, po.hashCode());
+
+        assertEquals(BigDecimal.TEN, po.<BigDecimal>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(expectedHashCode("Class"), po.type().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(expectedHashCode("Class"), po.type().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(expectedHashCode("Class"), po.type().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(expectedHashCode("Class"), po.type().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(expectedHashCode("Class"), po.type().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(expectedHashCode("Class"), po.type().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(expectedHashCode("Class"), po.type().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(expectedHashCode("Class"), po.type().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(expectedHashCode("Class"), po.type().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(expectedHashCode("Class"), po.type().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(expectedHashCode("Class"), po.type().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(expectedHashCode("Class"), po.type().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(expectedHashCode("Class"), po.type().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(expectedHashCode("Class"), po.type().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(expectedHashCode("Class"), po.type().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(expectedHashCode("Class"), po.type().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(expectedHashCode("Class"), po.type().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(expectedHashCode("Class"), po.type().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 testOffheapBinary() 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(expectedHashCode("Class"), offheapObj.type().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 bo = builder.build();
+
+        assertEquals(expectedHashCode(Value.class.getName()), bo.type().typeId());
+        assertEquals(100, bo.hashCode());
+
+        assertEquals(1, bo.<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 bo = builder.build();
+
+        BinaryType meta = bo.type();
+
+        assertEquals(expectedTypeName("org.test.MetaTest2"), meta.typeName());
+        assertEquals("Object", meta.fieldTypeName("objectField"));
+    }
+
+    /**
+     * @param fullClsName Class name.
+     * @return Expected type name according to configuration.
+     */
+    private String expectedTypeName(String fullClsName) {
+        BinaryNameMapper mapper = cfg.getBinaryConfiguration().getNameMapper();
+
+        if (mapper == null)
+            mapper = BinaryContext.defaultNameMapper();
+
+        return mapper.typeName(fullClsName);
+    }
+
+    /**
+     * @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(expectedTypeName("org.test.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(expectedTypeName("org.test.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(GridBinaryTestClasses.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_"));
+        TestCase.assertEquals("bbb", builder.build().<GridBinaryTestClasses.TestObjectAllTypes>deserialize().str);
+    }
+
+    /**
+     *
+     */
+    @SuppressWarnings("unchecked")
+    public void testCopyFromInnerObjects() {
+        ArrayList<Object> list = new ArrayList<>();
+        list.add(new GridBinaryTestClasses.TestObjectAllTypes());
+        list.add(list.get(0));
+
+        GridBinaryTestClasses.TestObjectContainer c = new GridBinaryTestClasses.TestObjectContainer(list);
+
+        BinaryObjectBuilderImpl builder = builder(toBinary(c));
+        builder.<List>getField("foo").add("!!!");
+
+        BinaryObject res = builder.build();
+
+        GridBinaryTestClasses.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 GridBinaryTestClasses.TestObjectAllTypes);
+    }
+
+    /**
+     *
+     */
+    public void testSetBinaryObject() {
+        // Prepare marshaller context.
+        CacheObjectBinaryProcessorImpl proc = ((CacheObjectBinaryProcessorImpl)(grid(0)).context().cacheObjects());
+
+        proc.marshal(new GridBinaryTestClasses.TestObjectContainer());
+        proc.marshal(new GridBinaryTestClasses.TestObjectAllTypes());
+
+        // Actual test.
+        BinaryObject binaryObj = builder(GridBinaryTestClasses.TestObjectContainer.class.getName())
+            .setField("foo", toBinary(new GridBinaryTestClasses.TestObjectAllTypes()))
+            .build();
+
+        assertTrue(binaryObj.<GridBinaryTestClasses.TestObjectContainer>deserialize().foo instanceof
+            GridBinaryTestClasses.TestObjectAllTypes);
+    }
+
+    /**
+     *
+     */
+    public void testPlainBinaryObjectCopyFrom() {
+        GridBinaryTestClasses.TestObjectPlainBinary obj = new GridBinaryTestClasses.TestObjectPlainBinary(toBinary(new GridBinaryTestClasses.TestObjectAllTypes()));
+
+        BinaryObjectBuilderImpl builder = builder(toBinary(obj));
+        assertTrue(builder.getField("plainBinary") instanceof BinaryObject);
+
+        GridBinaryTestClasses.TestObjectPlainBinary deserialized = builder.build().deserialize();
+        assertTrue(deserialized.plainBinary != null);
+    }
+
+    /**
+     *
+     */
+    public void testRemoveFromNewObject() {
+        BinaryObjectBuilder builder = builder(GridBinaryTestClasses.TestObjectAllTypes.class.getName());
+
+        builder.setField("str", "a");
+
+        builder.removeField("str");
+
+        TestCase.assertNull(builder.build().<GridBinaryTestClasses.TestObjectAllTypes>deserialize().str);
+    }
+
+    /**
+     *
+     */
+    public void testRemoveFromExistingObject() {
+        GridBinaryTestClasses.TestObjectAllTypes obj = new GridBinaryTestClasses.TestObjectAllTypes();
+        obj.setDefaultData();
+        obj.enumArr = null;
+
+        BinaryObjectBuilder builder = builder(toBinary(obj));
+
+        builder.removeField("str");
+
+        BinaryObject binary = builder.build();
+
+        GridBinaryTestClasses.TestObjectAllTypes deserialzied = binary.deserialize();
+
+        assertNull(deserialzied.str);
+    }
+
+    /**
+     *
+     */
+    public void testRemoveFromExistingObjectAfterGet() {
+        GridBinaryTestClasses.TestObjectAllTypes obj = new GridBinaryTestClasses.TestObjectAllTypes();
+        obj.setDefaultData();
+        obj.enumArr = null;
+
+        BinaryObjectBuilderImpl builder = builder(toBinary(obj));
+
+        builder.getField("i_");
+
+        builder.removeField("str");
+
+        TestCase.assertNull(builder.build().<GridBinaryTestClasses.TestObjectAllTypes>deserialize().str);
+    }
+
+    /**
+     * @throws IgniteCheckedException If any error occurs.
+     */
+    public void testDontBrokeCyclicDependency() throws IgniteCheckedException {
+        GridBinaryTestClasses.TestObjectOuter outer = new GridBinaryTestClasses.TestObjectOuter();
+        outer.inner = new GridBinaryTestClasses.TestObjectInner();
+        outer.inner.outer = outer;
+        outer.foo = "a";
+
+        BinaryObjectBuilder builder = builder(toBinary(outer));
+
+        builder.setField("foo", "b");
+
+        GridBinaryTestClasses.TestObjectOuter res = builder.build().deserialize();
+
+        assertEquals("b", res.foo);
+        assertSame(res, res.inner.outer);
+    }
+
+    /**
+     * @return Binaries.
+     */
+    private IgniteBinary binaries() {
+        return grid(0).binary();
+    }
+
+    /**
+     * @param obj Object.
+     * @return Binary object.
+     */
+    private BinaryObject toBinary(Object obj) {
+        return binaries().toBinary(obj);
+    }
+
+    /**
+     * @return Builder.
+     */
+    private BinaryObjectBuilder builder(String clsName) {
+        return binaries().builder(clsName);
+    }
+
+    /**
+     * @return Builder.
+     */
+    private BinaryObjectBuilderImpl builder(BinaryObject obj) {
+        return (BinaryObjectBuilderImpl)binaries().builder(obj);
+    }
+
+    /**
+     *
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    private static class CustomIdMapper {
+        /** */
+        private String str = "a";
+
+        /** */
+        private int i = 10;
+    }
+
+    /**
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    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;
+        }
+    }
+
+    /**
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    private static class Value {
+        /** */
+        private int i;
+
+        /**
+         */
+        private Value() {
+            // No-op.
+        }
+
+        /**
+         * @param i Index.
+         */
+        private Value(int i) {
+            this.i = i;
+        }
+    }
+}