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/01 16:17:07 UTC

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

Repository: ignite
Updated Branches:
  refs/heads/master 58dce88b8 -> 62502b2c4


http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
index 93c80d3..33656fb 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
@@ -43,6 +43,7 @@ import org.yardstickframework.BenchmarkServer;
 import org.yardstickframework.BenchmarkUtils;
 
 import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_VALUES;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER;
 
 /**
  * Standalone Ignite node.
@@ -150,6 +151,8 @@ public class IgniteNode implements BenchmarkServer {
         c.setCommunicationSpi(commSpi);
 
         ignite = IgniteSpring.start(c, appCtx);
+
+        BenchmarkUtils.println("Configured marshaller: " + ignite.cluster().localNode().attribute(ATTR_MARSHALLER));
     }
 
     /**


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

Posted by ag...@apache.org.
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;
+        }
+    }
+}


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

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/test/java/org/apache/ignite/internal/binary/GridDefaultBinaryMappersBinaryMetaDataSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/GridDefaultBinaryMappersBinaryMetaDataSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/GridDefaultBinaryMappersBinaryMetaDataSelfTest.java
new file mode 100644
index 0000000..041238f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/GridDefaultBinaryMappersBinaryMetaDataSelfTest.java
@@ -0,0 +1,389 @@
+/*
+ * 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.util.Arrays;
+import java.util.Collection;
+import java.util.Date;
+import java.util.HashMap;
+import org.apache.ignite.IgniteBinary;
+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.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.Binarylizable;
+import org.apache.ignite.binary.BinaryType;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.binary.BinaryRawWriter;
+import org.apache.ignite.binary.BinaryReader;
+import org.apache.ignite.binary.BinaryWriter;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * Binary meta data test.
+ */
+public class GridDefaultBinaryMappersBinaryMetaDataSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static IgniteConfiguration cfg;
+
+    /** */
+    private static int idx;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        BinaryConfiguration bCfg = new BinaryConfiguration();
+
+        bCfg.setNameMapper(new BinaryBasicNameMapper(false));
+        bCfg.setIdMapper(new BinaryBasicIdMapper(false));
+
+        bCfg.setClassNames(Arrays.asList(TestObject1.class.getName(), TestObject2.class.getName()));
+
+        cfg.setBinaryConfiguration(bCfg);
+
+        cfg.setMarshaller(new BinaryMarshaller());
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        cfg.setCacheConfiguration(ccfg);
+
+        GridDefaultBinaryMappersBinaryMetaDataSelfTest.cfg = cfg;
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        idx = 0;
+
+        startGrid();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopGrid();
+    }
+
+    /**
+     * @return Binaries API.
+     */
+    protected IgniteBinary binaries() {
+        return grid().binary();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetAll() throws Exception {
+        binaries().toBinary(new TestObject2());
+
+        Collection<BinaryType> metas = binaries().types();
+
+        assertEquals(2, metas.size());
+
+        for (BinaryType meta : metas) {
+            Collection<String> fields;
+
+            if (expectedTypeName(TestObject1.class.getName()).equals(meta.typeName())) {
+                fields = meta.fieldNames();
+
+                assertEquals(7, fields.size());
+
+                assertTrue(fields.contains("intVal"));
+                assertTrue(fields.contains("strVal"));
+                assertTrue(fields.contains("arrVal"));
+                assertTrue(fields.contains("obj1Val"));
+                assertTrue(fields.contains("obj2Val"));
+                assertTrue(fields.contains("decVal"));
+                assertTrue(fields.contains("decArrVal"));
+
+                assertEquals("int", meta.fieldTypeName("intVal"));
+                assertEquals("String", meta.fieldTypeName("strVal"));
+                assertEquals("byte[]", meta.fieldTypeName("arrVal"));
+                assertEquals("Object", meta.fieldTypeName("obj1Val"));
+                assertEquals("Object", meta.fieldTypeName("obj2Val"));
+                assertEquals("decimal", meta.fieldTypeName("decVal"));
+                assertEquals("decimal[]", meta.fieldTypeName("decArrVal"));
+            }
+            else if (expectedTypeName(TestObject2.class.getName()).equals(meta.typeName())) {
+                fields = meta.fieldNames();
+
+                assertEquals(7, fields.size());
+
+                assertTrue(fields.contains("boolVal"));
+                assertTrue(fields.contains("dateVal"));
+                assertTrue(fields.contains("uuidArrVal"));
+                assertTrue(fields.contains("objVal"));
+                assertTrue(fields.contains("mapVal"));
+                assertTrue(fields.contains("decVal"));
+                assertTrue(fields.contains("decArrVal"));
+
+                assertEquals("boolean", meta.fieldTypeName("boolVal"));
+                assertEquals("Date", meta.fieldTypeName("dateVal"));
+                assertEquals("UUID[]", meta.fieldTypeName("uuidArrVal"));
+                assertEquals("Object", meta.fieldTypeName("objVal"));
+                assertEquals("Map", meta.fieldTypeName("mapVal"));
+                assertEquals("decimal", meta.fieldTypeName("decVal"));
+                assertEquals("decimal[]", meta.fieldTypeName("decArrVal"));
+            }
+            else
+                assert false : meta.typeName();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNoConfiguration() throws Exception {
+        binaries().toBinary(new TestObject3());
+
+        assertNotNull(binaries().type(TestObject3.class));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReflection() throws Exception {
+        BinaryType meta = binaries().type(TestObject1.class);
+
+        assertNotNull(meta);
+
+        assertEquals(expectedTypeName(TestObject1.class.getName()), meta.typeName());
+
+        Collection<String> fields = meta.fieldNames();
+
+        assertEquals(7, fields.size());
+
+        assertTrue(fields.contains("intVal"));
+        assertTrue(fields.contains("strVal"));
+        assertTrue(fields.contains("arrVal"));
+        assertTrue(fields.contains("obj1Val"));
+        assertTrue(fields.contains("obj2Val"));
+        assertTrue(fields.contains("decVal"));
+        assertTrue(fields.contains("decArrVal"));
+
+        assertEquals("int", meta.fieldTypeName("intVal"));
+        assertEquals("String", meta.fieldTypeName("strVal"));
+        assertEquals("byte[]", meta.fieldTypeName("arrVal"));
+        assertEquals("Object", meta.fieldTypeName("obj1Val"));
+        assertEquals("Object", meta.fieldTypeName("obj2Val"));
+        assertEquals("decimal", meta.fieldTypeName("decVal"));
+        assertEquals("decimal[]", meta.fieldTypeName("decArrVal"));
+    }
+
+    /**
+     * @param clsName Class name.
+     * @return Type name.
+     */
+    private String expectedTypeName(String clsName) {
+        BinaryNameMapper mapper = cfg.getBinaryConfiguration().getNameMapper();
+
+        if (mapper == null)
+            mapper = BinaryContext.defaultNameMapper();
+
+        return mapper.typeName(clsName);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBinaryMarshalAware() throws Exception {
+        binaries().toBinary(new TestObject2());
+
+        BinaryType meta = binaries().type(TestObject2.class);
+
+        assertNotNull(meta);
+
+        assertEquals(expectedTypeName(TestObject2.class.getName()), meta.typeName());
+
+        Collection<String> fields = meta.fieldNames();
+
+        assertEquals(7, fields.size());
+
+        assertTrue(fields.contains("boolVal"));
+        assertTrue(fields.contains("dateVal"));
+        assertTrue(fields.contains("uuidArrVal"));
+        assertTrue(fields.contains("objVal"));
+        assertTrue(fields.contains("mapVal"));
+        assertTrue(fields.contains("decVal"));
+        assertTrue(fields.contains("decArrVal"));
+
+        assertEquals("boolean", meta.fieldTypeName("boolVal"));
+        assertEquals("Date", meta.fieldTypeName("dateVal"));
+        assertEquals("UUID[]", meta.fieldTypeName("uuidArrVal"));
+        assertEquals("Object", meta.fieldTypeName("objVal"));
+        assertEquals("Map", meta.fieldTypeName("mapVal"));
+        assertEquals("decimal", meta.fieldTypeName("decVal"));
+        assertEquals("decimal[]", meta.fieldTypeName("decArrVal"));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMerge() throws Exception {
+        binaries().toBinary(new TestObject2());
+
+        idx = 1;
+
+        binaries().toBinary(new TestObject2());
+
+        BinaryType meta = binaries().type(TestObject2.class);
+
+        assertNotNull(meta);
+
+        assertEquals(expectedTypeName(TestObject2.class.getName()), meta.typeName());
+
+        Collection<String> fields = meta.fieldNames();
+
+        assertEquals(9, fields.size());
+
+        assertTrue(fields.contains("boolVal"));
+        assertTrue(fields.contains("dateVal"));
+        assertTrue(fields.contains("uuidArrVal"));
+        assertTrue(fields.contains("objVal"));
+        assertTrue(fields.contains("mapVal"));
+        assertTrue(fields.contains("charVal"));
+        assertTrue(fields.contains("colVal"));
+        assertTrue(fields.contains("decVal"));
+        assertTrue(fields.contains("decArrVal"));
+
+        assertEquals("boolean", meta.fieldTypeName("boolVal"));
+        assertEquals("Date", meta.fieldTypeName("dateVal"));
+        assertEquals("UUID[]", meta.fieldTypeName("uuidArrVal"));
+        assertEquals("Object", meta.fieldTypeName("objVal"));
+        assertEquals("Map", meta.fieldTypeName("mapVal"));
+        assertEquals("char", meta.fieldTypeName("charVal"));
+        assertEquals("Collection", meta.fieldTypeName("colVal"));
+        assertEquals("decimal", meta.fieldTypeName("decVal"));
+        assertEquals("decimal[]", meta.fieldTypeName("decArrVal"));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSerializedObject() throws Exception {
+        TestObject1 obj = new TestObject1();
+
+        obj.intVal = 10;
+        obj.strVal = "str";
+        obj.arrVal = new byte[] {2, 4, 6};
+        obj.obj1Val = null;
+        obj.obj2Val = new TestObject2();
+        obj.decVal = BigDecimal.ZERO;
+        obj.decArrVal = new BigDecimal[] { BigDecimal.ONE };
+
+        BinaryObject po = binaries().toBinary(obj);
+
+        info(po.toString());
+
+        BinaryType meta = po.type();
+
+        assertNotNull(meta);
+
+        assertEquals(expectedTypeName(TestObject1.class.getName()), meta.typeName());
+
+        Collection<String> fields = meta.fieldNames();
+
+        assertEquals(7, fields.size());
+
+        assertTrue(fields.contains("intVal"));
+        assertTrue(fields.contains("strVal"));
+        assertTrue(fields.contains("arrVal"));
+        assertTrue(fields.contains("obj1Val"));
+        assertTrue(fields.contains("obj2Val"));
+        assertTrue(fields.contains("decVal"));
+        assertTrue(fields.contains("decArrVal"));
+
+        assertEquals("int", meta.fieldTypeName("intVal"));
+        assertEquals("String", meta.fieldTypeName("strVal"));
+        assertEquals("byte[]", meta.fieldTypeName("arrVal"));
+        assertEquals("Object", meta.fieldTypeName("obj1Val"));
+        assertEquals("Object", meta.fieldTypeName("obj2Val"));
+        assertEquals("decimal", meta.fieldTypeName("decVal"));
+        assertEquals("decimal[]", meta.fieldTypeName("decArrVal"));
+    }
+
+    /**
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    private static class TestObject1 {
+        /** */
+        private int intVal;
+
+        /** */
+        private String strVal;
+
+        /** */
+        private byte[] arrVal;
+
+        /** */
+        private TestObject1 obj1Val;
+
+        /** */
+        private TestObject2 obj2Val;
+
+        /** */
+        private BigDecimal decVal;
+
+        /** */
+        private BigDecimal[] decArrVal;
+    }
+
+    /**
+     */
+    private static class TestObject2 implements Binarylizable {
+        /** {@inheritDoc} */
+        @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+            writer.writeBoolean("boolVal", false);
+            writer.writeDate("dateVal", new Date());
+            writer.writeUuidArray("uuidArrVal", null);
+            writer.writeObject("objVal", null);
+            writer.writeMap("mapVal", new HashMap<>());
+            writer.writeDecimal("decVal", BigDecimal.ZERO);
+            writer.writeDecimalArray("decArrVal", new BigDecimal[] { BigDecimal.ONE });
+
+            if (idx == 1) {
+                writer.writeChar("charVal", (char)0);
+                writer.writeCollection("colVal", null);
+            }
+
+            BinaryRawWriter raw = writer.rawWriter();
+
+            raw.writeChar((char)0);
+            raw.writeCollection(null);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+            // No-op.
+        }
+    }
+
+    /**
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    private static class TestObject3 {
+        /** */
+        private int intVal;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/test/java/org/apache/ignite/internal/binary/GridSimpleLowerCaseBinaryMappersBinaryMetaDataSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/GridSimpleLowerCaseBinaryMappersBinaryMetaDataSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/GridSimpleLowerCaseBinaryMappersBinaryMetaDataSelfTest.java
new file mode 100644
index 0000000..74c13ce
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/GridSimpleLowerCaseBinaryMappersBinaryMetaDataSelfTest.java
@@ -0,0 +1,41 @@
+/*
+ * 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.binary.BinaryBasicNameMapper;
+import org.apache.ignite.configuration.BinaryConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+
+/**
+ * Binary meta data test.
+ */
+public class GridSimpleLowerCaseBinaryMappersBinaryMetaDataSelfTest
+    extends GridDefaultBinaryMappersBinaryMetaDataSelfTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        BinaryConfiguration bCfg = cfg.getBinaryConfiguration();
+
+        bCfg.setNameMapper(new BinaryBasicNameMapper());
+        bCfg.setIdMapper(new BinaryBasicIdMapper(true));
+
+        return cfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/test/java/org/apache/ignite/internal/binary/TestMappedObject.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/TestMappedObject.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/TestMappedObject.java
new file mode 100644
index 0000000..8705f6a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/TestMappedObject.java
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+/**
+ *
+ */
+public class TestMappedObject {
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/test/java/org/apache/ignite/internal/binary/noncompact/BinaryObjectBuilderNonCompactDefaultMappersSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/noncompact/BinaryObjectBuilderNonCompactDefaultMappersSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/noncompact/BinaryObjectBuilderNonCompactDefaultMappersSelfTest.java
new file mode 100644
index 0000000..88a1296
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/noncompact/BinaryObjectBuilderNonCompactDefaultMappersSelfTest.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.binary.noncompact;
+
+import org.apache.ignite.internal.binary.BinaryObjectBuilderDefaultMappersSelfTest;
+
+/**
+ * Binary builder test for objects with non-compact footer.
+ */
+public class BinaryObjectBuilderNonCompactDefaultMappersSelfTest extends BinaryObjectBuilderDefaultMappersSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean compactFooter() {
+        return false;
+    }
+}

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConditionalDeploymentSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConditionalDeploymentSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConditionalDeploymentSelfTest.java
index ea16c1f..3e96cf7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConditionalDeploymentSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConditionalDeploymentSelfTest.java
@@ -21,8 +21,8 @@ import org.apache.ignite.Ignite;
 import org.apache.ignite.Ignition;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.managers.communication.GridIoMessageFactory;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.internal.managers.communication.GridIoMessageFactory;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.typedef.CO;
 import org.apache.ignite.plugin.extensions.communication.Message;
@@ -91,6 +91,8 @@ public class GridCacheConditionalDeploymentSelfTest extends GridCommonAbstractTe
 
         startGrid(1);
 
+        awaitPartitionMapExchange();
+
         ignite0.cache(null).put(1, new TestValue());
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java
index 2da781b..271e3b5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java
@@ -30,6 +30,7 @@ import javax.cache.processor.MutableEntry;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteBinary;
+import org.apache.ignite.binary.BinaryNameMapper;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CachePeekMode;
@@ -38,6 +39,7 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.binary.BinaryContext;
 import org.apache.ignite.internal.binary.BinaryObjectImpl;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
@@ -77,6 +79,9 @@ public abstract class GridCacheBinaryObjectsAbstractSelfTest extends GridCommonA
     /** */
     private static final int ENTRY_CNT = 100;
 
+    /** */
+    private static IgniteConfiguration cfg;
+
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
@@ -109,6 +114,8 @@ public abstract class GridCacheBinaryObjectsAbstractSelfTest extends GridCommonA
 
         cfg.setMarshaller(new BinaryMarshaller());
 
+        this.cfg = cfg;
+
         return cfg;
     }
 
@@ -200,8 +207,14 @@ public abstract class GridCacheBinaryObjectsAbstractSelfTest extends GridCommonA
 
         assertNotNull(str);
 
-        assertTrue("Unexpected toString: " + str,
-            str.startsWith("TestReferenceObject") && str.contains("obj=TestReferenceObject ["));
+        BinaryNameMapper nameMapper = BinaryContext.defaultNameMapper();
+
+        if (cfg.getBinaryConfiguration() != null && cfg.getBinaryConfiguration().getNameMapper() != null)
+            nameMapper = cfg.getBinaryConfiguration().getNameMapper();
+
+        String typeName = nameMapper.typeName(TestReferenceObject.class.getName());
+
+        assertTrue("Unexpected toString: " + str, str.startsWith(typeName) && str.contains("obj=" + typeName + " ["));
 
         TestReferenceObject obj1_r = po.deserialize();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryStoreAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryStoreAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryStoreAbstractSelfTest.java
index 0cfa88b..63ed30e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryStoreAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryStoreAbstractSelfTest.java
@@ -24,6 +24,8 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 import javax.cache.Cache;
+import org.apache.ignite.binary.BinaryBasicIdMapper;
+import org.apache.ignite.binary.BinaryBasicNameMapper;
 import org.apache.ignite.cache.store.CacheStoreAdapter;
 import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -46,6 +48,9 @@ public abstract class GridCacheBinaryStoreAbstractSelfTest extends GridCommonAbs
     /** */
     private static final TestStore STORE = new TestStore();
 
+    /** */
+    protected static IgniteConfiguration cfg;
+
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
@@ -53,6 +58,9 @@ public abstract class GridCacheBinaryStoreAbstractSelfTest extends GridCommonAbs
 
         BinaryConfiguration bCfg = new BinaryConfiguration();
 
+        bCfg.setNameMapper(new BinaryBasicNameMapper(false));
+        bCfg.setIdMapper(new BinaryBasicIdMapper(false));
+
         bCfg.setClassNames(Arrays.asList(Key.class.getName(), Value.class.getName()));
 
         cfg.setBinaryConfiguration(bCfg);
@@ -75,6 +83,8 @@ public abstract class GridCacheBinaryStoreAbstractSelfTest extends GridCommonAbs
 
         cfg.setDiscoverySpi(disco);
 
+        GridCacheBinaryStoreAbstractSelfTest.cfg = cfg;
+
         return cfg;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryStoreBinariesDefaultMappersSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryStoreBinariesDefaultMappersSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryStoreBinariesDefaultMappersSelfTest.java
new file mode 100644
index 0000000..47fc53b
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryStoreBinariesDefaultMappersSelfTest.java
@@ -0,0 +1,81 @@
+/*
+ * 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.processors.cache.binary;
+
+import java.util.Map;
+import org.apache.ignite.binary.BinaryNameMapper;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.internal.binary.BinaryContext;
+
+/**
+ * Tests for cache store with binary.
+ */
+public class GridCacheBinaryStoreBinariesDefaultMappersSelfTest extends GridCacheBinaryStoreAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean keepBinaryInStore() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void populateMap(Map<Object, Object> map, int... idxs) {
+        assert map != null;
+        assert idxs != null;
+
+        for (int idx : idxs)
+            map.put(binary(new Key(idx)), binary(new Value(idx)));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void checkMap(Map<Object, Object> map, int... idxs) {
+        assert map != null;
+        assert idxs != null;
+
+        assertEquals(idxs.length, map.size());
+
+        for (int idx : idxs) {
+            Object val = map.get(binary(new Key(idx)));
+
+            assertTrue(String.valueOf(val), val instanceof BinaryObject);
+
+            BinaryObject po = (BinaryObject)val;
+
+            assertEquals(expectedTypeName(Value.class.getName()), po.type().typeName());
+            assertEquals(new Integer(idx), po.field("idx"));
+        }
+    }
+
+    /**
+     * @param clsName Class name.
+     * @return Type name.
+     */
+    private String expectedTypeName(String clsName) {
+        BinaryNameMapper nameMapper = cfg.getBinaryConfiguration().getNameMapper();
+
+        if (nameMapper == null)
+            nameMapper = BinaryContext.defaultNameMapper();
+
+        return nameMapper.typeName(clsName);
+    }
+
+    /**
+     * @param obj Object.
+     * @return Binary object.
+     */
+    private Object binary(Object obj) {
+        return grid().binary().toBinary(obj);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryStoreBinariesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryStoreBinariesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryStoreBinariesSelfTest.java
deleted file mode 100644
index 4d163b3..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryStoreBinariesSelfTest.java
+++ /dev/null
@@ -1,66 +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.processors.cache.binary;
-
-import java.util.Map;
-import org.apache.ignite.binary.BinaryObject;
-
-/**
- * Tests for cache store with binary.
- */
-public class GridCacheBinaryStoreBinariesSelfTest extends GridCacheBinaryStoreAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected boolean keepBinaryInStore() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void populateMap(Map<Object, Object> map, int... idxs) {
-        assert map != null;
-        assert idxs != null;
-
-        for (int idx : idxs)
-            map.put(binary(new Key(idx)), binary(new Value(idx)));
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void checkMap(Map<Object, Object> map, int... idxs) {
-        assert map != null;
-        assert idxs != null;
-
-        assertEquals(idxs.length, map.size());
-
-        for (int idx : idxs) {
-            Object val = map.get(binary(new Key(idx)));
-
-            assertTrue(String.valueOf(val), val instanceof BinaryObject);
-
-            BinaryObject po = (BinaryObject)val;
-
-            assertEquals("Value", po.type().typeName());
-            assertEquals(new Integer(idx), po.field("idx"));
-        }
-    }
-
-    /**
-     * @param obj Object.
-     * @return Binary object.
-     */
-    private Object binary(Object obj) {
-        return grid().binary().toBinary(obj);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryStoreBinariesSimpleNameMappersSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryStoreBinariesSimpleNameMappersSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryStoreBinariesSimpleNameMappersSelfTest.java
new file mode 100644
index 0000000..0aee2f2
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryStoreBinariesSimpleNameMappersSelfTest.java
@@ -0,0 +1,40 @@
+/*
+ * 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.processors.cache.binary;
+
+import org.apache.ignite.binary.BinaryBasicIdMapper;
+import org.apache.ignite.binary.BinaryBasicNameMapper;
+import org.apache.ignite.configuration.BinaryConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+
+/**
+ * Tests for cache store with binary.
+ */
+public class GridCacheBinaryStoreBinariesSimpleNameMappersSelfTest
+    extends GridCacheBinaryStoreBinariesDefaultMappersSelfTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        BinaryConfiguration bCfg = cfg.getBinaryConfiguration();
+
+        bCfg.setNameMapper(new BinaryBasicNameMapper());
+        bCfg.setIdMapper(new BinaryBasicIdMapper(true));
+
+        return cfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/test/java/org/apache/ignite/testframework/config/GridTestProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/config/GridTestProperties.java b/modules/core/src/test/java/org/apache/ignite/testframework/config/GridTestProperties.java
index 491f38f..d13f117 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/config/GridTestProperties.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/config/GridTestProperties.java
@@ -79,6 +79,9 @@ public final class GridTestProperties {
     public static final String BINARY_COMPACT_FOOTERS = "binary.marshaller.compact.footers";
 
     /** */
+    public static final String BINARY_MARSHALLER_USE_SIMPLE_NAME_MAPPER = "binary.marshaller.use.simple.name.mapper";
+
+    /** */
     static {
         // Initialize IGNITE_HOME system property.
         String igniteHome = System.getProperty("IGNITE_HOME");
@@ -321,4 +324,4 @@ public final class GridTestProperties {
 
         return props;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
index 8bf877a..27511ff 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
@@ -50,6 +50,7 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.Ignition;
+import org.apache.ignite.binary.BinaryBasicNameMapper;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -104,6 +105,7 @@ import org.springframework.context.support.FileSystemXmlApplicationContext;
 import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.PRIMARY;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.apache.ignite.testframework.config.GridTestProperties.BINARY_MARSHALLER_USE_SIMPLE_NAME_MAPPER;
 
 /**
  * Common abstract test for Ignite tests.
@@ -732,7 +734,14 @@ public abstract class GridAbstractTest extends TestCase {
             startingGrid.set(gridName);
 
             try {
-                return IgnitionEx.start(optimize(getConfiguration(gridName)), ctx);
+                Ignite node = IgnitionEx.start(optimize(getConfiguration(gridName)), ctx);
+
+                IgniteConfiguration cfg = node.configuration();
+
+                log.info("Node started with the following configuration [id=" + node.cluster().localNode().id()
+                    + ", marshaller=" + cfg.getMarshaller() + ", binaryCfg=" + cfg.getBinaryConfiguration() + "]");
+
+                return node;
             }
             finally {
                 startingGrid.set(null);
@@ -1143,6 +1152,18 @@ public abstract class GridAbstractTest extends TestCase {
             }
         }
 
+        if (Boolean.valueOf(GridTestProperties.getProperty(BINARY_MARSHALLER_USE_SIMPLE_NAME_MAPPER))) {
+            BinaryConfiguration bCfg = cfg.getBinaryConfiguration();
+
+            if (bCfg == null) {
+                bCfg = new BinaryConfiguration();
+
+                cfg.setBinaryConfiguration(bCfg);
+            }
+
+            bCfg.setNameMapper(new BinaryBasicNameMapper(true));
+        }
+
         if (gridName != null && gridName.matches(".*\\d")) {
             String idStr = UUID.randomUUID().toString();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
index bcd1ede..dece258 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
@@ -111,7 +111,7 @@ public class IgniteBasicTestSuite extends TestSuite {
         suite.addTestSuite(GridFailFastNodeFailureDetectionSelfTest.class);
         suite.addTestSuite(OffHeapTieredTransactionSelfTest.class);
         suite.addTestSuite(IgniteSlowClientDetectionSelfTest.class);
-        suite.addTestSuite(IgniteDaemonNodeMarshallerCacheTest.class);
+        GridTestUtils.addTestIfNeeded(suite, IgniteDaemonNodeMarshallerCacheTest.class, ignoredTests);
         suite.addTestSuite(IgniteMarshallerCacheConcurrentReadWriteTest.class);
 
         suite.addTestSuite(IgniteExceptionInNioWorkerSelfTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryBasicTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryBasicTestSuite.java
new file mode 100644
index 0000000..cbb87fa
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryBasicTestSuite.java
@@ -0,0 +1,95 @@
+/*
+ * 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.testsuites;
+
+import java.util.HashSet;
+import java.util.Set;
+import junit.framework.TestSuite;
+import org.apache.ignite.internal.ClusterGroupSelfTest;
+import org.apache.ignite.internal.GridReleaseTypeSelfTest;
+import org.apache.ignite.internal.GridVersionSelfTest;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.internal.managers.deployment.GridDeploymentMessageCountSelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheP2pUnmarshallingErrorTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheP2pUnmarshallingNearErrorTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheP2pUnmarshallingRebalanceErrorTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheP2pUnmarshallingTxErrorTest;
+import org.apache.ignite.internal.processors.cache.IgniteDaemonNodeMarshallerCacheTest;
+import org.apache.ignite.internal.util.GridStartupWithSpecifiedWorkDirectorySelfTest;
+import org.apache.ignite.internal.util.IgniteUtilsSelfTest;
+import org.apache.ignite.internal.util.io.GridUnsafeDataOutputArraySizingSelfTest;
+import org.apache.ignite.internal.util.nio.GridNioSelfTest;
+import org.apache.ignite.internal.util.nio.GridNioSslSelfTest;
+import org.apache.ignite.marshaller.jdk.GridJdkMarshallerSelfTest;
+import org.apache.ignite.marshaller.optimized.OptimizedMarshallerEnumSelfTest;
+import org.apache.ignite.marshaller.optimized.OptimizedMarshallerNodeFailoverTest;
+import org.apache.ignite.marshaller.optimized.OptimizedMarshallerSelfTest;
+import org.apache.ignite.marshaller.optimized.OptimizedMarshallerSerialPersistentFieldsSelfTest;
+import org.apache.ignite.marshaller.optimized.OptimizedMarshallerTest;
+import org.apache.ignite.marshaller.optimized.OptimizedObjectStreamSelfTest;
+import org.apache.ignite.messaging.GridMessagingNoPeerClassLoadingSelfTest;
+import org.apache.ignite.messaging.GridMessagingSelfTest;
+import org.apache.ignite.testframework.config.GridTestProperties;
+
+/**
+ * Basic test suite.
+ */
+public class IgniteBinaryBasicTestSuite extends TestSuite {
+    /**
+     * @return Test suite.
+     * @throws Exception Thrown in case of the failure.
+     */
+    public static TestSuite suite() throws Exception {
+        GridTestProperties.setProperty(GridTestProperties.MARSH_CLASS_NAME, BinaryMarshaller.class.getName());
+
+        TestSuite suite = new TestSuite("GridGain Binary Basic Test Suite");
+
+        Set<Class> ignoredTests = new HashSet<>();
+
+        // Tests that are not ready to be used with PortableMarshaller
+        ignoredTests.add(GridJdkMarshallerSelfTest.class);
+        ignoredTests.add(OptimizedMarshallerEnumSelfTest.class);
+        ignoredTests.add(OptimizedMarshallerSelfTest.class);
+        ignoredTests.add(OptimizedMarshallerTest.class);
+        ignoredTests.add(OptimizedObjectStreamSelfTest.class);
+        ignoredTests.add(GridUnsafeDataOutputArraySizingSelfTest.class);
+        ignoredTests.add(OptimizedMarshallerNodeFailoverTest.class);
+        ignoredTests.add(OptimizedMarshallerSerialPersistentFieldsSelfTest.class);
+        ignoredTests.add(GridNioSslSelfTest.class);
+        ignoredTests.add(GridNioSelfTest.class);
+        ignoredTests.add(IgniteCacheP2pUnmarshallingErrorTest.class);
+        ignoredTests.add(IgniteCacheP2pUnmarshallingTxErrorTest.class);
+        ignoredTests.add(IgniteCacheP2pUnmarshallingNearErrorTest.class);
+        ignoredTests.add(IgniteCacheP2pUnmarshallingRebalanceErrorTest.class);
+        ignoredTests.add(GridReleaseTypeSelfTest.class);
+        ignoredTests.add(GridStartupWithSpecifiedWorkDirectorySelfTest.class);
+        ignoredTests.add(IgniteUtilsSelfTest.class);
+        ignoredTests.add(ClusterGroupSelfTest.class);
+        ignoredTests.add(GridMessagingNoPeerClassLoadingSelfTest.class);
+        ignoredTests.add(GridMessagingSelfTest.class);
+        ignoredTests.add(GridVersionSelfTest.class);
+        ignoredTests.add(GridDeploymentMessageCountSelfTest.class);
+
+        // TODO: check and delete if pass.
+        ignoredTests.add(IgniteDaemonNodeMarshallerCacheTest.class);
+
+        suite.addTest(IgniteBasicTestSuite.suite(ignoredTests));
+
+        return suite;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsSimpleNameMapperComputeGridTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsSimpleNameMapperComputeGridTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsSimpleNameMapperComputeGridTestSuite.java
new file mode 100644
index 0000000..d3ea755
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsSimpleNameMapperComputeGridTestSuite.java
@@ -0,0 +1,38 @@
+/*
+ * 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.testsuites;
+
+import junit.framework.TestSuite;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.testframework.config.GridTestProperties;
+
+/**
+ *
+ */
+public class IgniteBinaryObjectsSimpleNameMapperComputeGridTestSuite {
+    /**
+     * @return Test suite.
+     * @throws Exception If failed.
+     */
+    public static TestSuite suite() throws Exception {
+        GridTestProperties.setProperty(GridTestProperties.MARSH_CLASS_NAME, BinaryMarshaller.class.getName());
+        GridTestProperties.setProperty(GridTestProperties.BINARY_MARSHALLER_USE_SIMPLE_NAME_MAPPER, "true");
+
+        return IgniteComputeGridTestSuite.suite();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java
index 9651f6d..5eb7b66 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java
@@ -18,6 +18,9 @@
 package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
+import org.apache.ignite.internal.binary.BinaryBasicIdMapperSelfTest;
+import org.apache.ignite.internal.binary.BinaryBasicNameMapperSelfTest;
+import org.apache.ignite.internal.binary.BinaryConfigurationConsistencySelfTest;
 import org.apache.ignite.internal.binary.BinaryEnumsSelfTest;
 import org.apache.ignite.internal.binary.BinaryFieldsHeapSelfTest;
 import org.apache.ignite.internal.binary.BinaryFieldsOffheapSelfTest;
@@ -25,19 +28,24 @@ import org.apache.ignite.internal.binary.BinaryFooterOffsetsHeapSelfTest;
 import org.apache.ignite.internal.binary.BinaryFooterOffsetsOffheapSelfTest;
 import org.apache.ignite.internal.binary.BinaryMarshallerSelfTest;
 import org.apache.ignite.internal.binary.BinaryObjectBuilderAdditionalSelfTest;
-import org.apache.ignite.internal.binary.BinaryObjectBuilderSelfTest;
+import org.apache.ignite.internal.binary.BinaryObjectBuilderDefaultMappersSelfTest;
+import org.apache.ignite.internal.binary.BinaryObjectBuilderSimpleNameLowerCaseMappersSelfTest;
+import org.apache.ignite.internal.binary.BinarySimpleNameTestPropertySelfTest;
 import org.apache.ignite.internal.binary.GridBinaryAffinityKeySelfTest;
 import org.apache.ignite.internal.binary.GridBinaryMarshallerCtxDisabledSelfTest;
-import org.apache.ignite.internal.binary.GridBinaryMetaDataSelfTest;
 import org.apache.ignite.internal.binary.GridBinaryWildcardsSelfTest;
+import org.apache.ignite.internal.binary.GridDefaultBinaryMappersBinaryMetaDataSelfTest;
+import org.apache.ignite.internal.binary.GridSimpleLowerCaseBinaryMappersBinaryMetaDataSelfTest;
 import org.apache.ignite.internal.binary.noncompact.BinaryFieldsHeapNonCompactSelfTest;
 import org.apache.ignite.internal.binary.noncompact.BinaryFieldsOffheapNonCompactSelfTest;
 import org.apache.ignite.internal.binary.noncompact.BinaryFooterOffsetsHeapNonCompactSelfTest;
 import org.apache.ignite.internal.binary.noncompact.BinaryFooterOffsetsOffheapNonCompactSelfTest;
 import org.apache.ignite.internal.binary.noncompact.BinaryMarshallerNonCompactSelfTest;
 import org.apache.ignite.internal.binary.noncompact.BinaryObjectBuilderAdditionalNonCompactSelfTest;
-import org.apache.ignite.internal.binary.noncompact.BinaryObjectBuilderNonCompactSelfTest;
-import org.apache.ignite.internal.processors.cache.binary.GridCacheBinaryStoreBinariesSelfTest;
+import org.apache.ignite.internal.binary.noncompact.BinaryObjectBuilderNonCompactDefaultMappersSelfTest;
+import org.apache.ignite.internal.binary.noncompact.BinaryObjectBuilderNonCompactSimpleNameLowerCaseMappersSelfTest;
+import org.apache.ignite.internal.processors.cache.binary.GridCacheBinaryStoreBinariesDefaultMappersSelfTest;
+import org.apache.ignite.internal.processors.cache.binary.GridCacheBinaryStoreBinariesSimpleNameMappersSelfTest;
 import org.apache.ignite.internal.processors.cache.binary.GridCacheBinaryStoreObjectsSelfTest;
 import org.apache.ignite.internal.processors.cache.binary.GridCacheClientNodeBinaryObjectMetadataMultinodeTest;
 import org.apache.ignite.internal.processors.cache.binary.GridCacheClientNodeBinaryObjectMetadataTest;
@@ -68,22 +76,31 @@ public class IgniteBinaryObjectsTestSuite extends TestSuite {
     public static TestSuite suite() throws Exception {
         TestSuite suite = new TestSuite("Ignite Binary Objects Test Suite");
 
+        suite.addTestSuite(BinarySimpleNameTestPropertySelfTest.class);
+        
+        suite.addTestSuite(BinaryBasicIdMapperSelfTest.class);
+        suite.addTestSuite(BinaryBasicNameMapperSelfTest.class);
+
         suite.addTestSuite(BinaryMarshallerSelfTest.class);
+        suite.addTestSuite(BinaryConfigurationConsistencySelfTest.class);
         suite.addTestSuite(GridBinaryMarshallerCtxDisabledSelfTest.class);
-        suite.addTestSuite(BinaryObjectBuilderSelfTest.class);
+        suite.addTestSuite(BinaryObjectBuilderDefaultMappersSelfTest.class);
+        suite.addTestSuite(BinaryObjectBuilderSimpleNameLowerCaseMappersSelfTest.class);
         suite.addTestSuite(BinaryObjectBuilderAdditionalSelfTest.class);
         suite.addTestSuite(BinaryFieldsHeapSelfTest.class);
         suite.addTestSuite(BinaryFieldsOffheapSelfTest.class);
         suite.addTestSuite(BinaryFooterOffsetsHeapSelfTest.class);
         suite.addTestSuite(BinaryFooterOffsetsOffheapSelfTest.class);
         suite.addTestSuite(BinaryEnumsSelfTest.class);
-        suite.addTestSuite(GridBinaryMetaDataSelfTest.class);
+        suite.addTestSuite(GridDefaultBinaryMappersBinaryMetaDataSelfTest.class);
+        suite.addTestSuite(GridSimpleLowerCaseBinaryMappersBinaryMetaDataSelfTest.class);
         suite.addTestSuite(GridBinaryAffinityKeySelfTest.class);
         suite.addTestSuite(GridBinaryWildcardsSelfTest.class);
 
         // Tests for objects with non-compact footers.
         suite.addTestSuite(BinaryMarshallerNonCompactSelfTest.class);
-        suite.addTestSuite(BinaryObjectBuilderNonCompactSelfTest.class);
+        suite.addTestSuite(BinaryObjectBuilderNonCompactDefaultMappersSelfTest.class);
+        suite.addTestSuite(BinaryObjectBuilderNonCompactSimpleNameLowerCaseMappersSelfTest.class);
         suite.addTestSuite(BinaryObjectBuilderAdditionalNonCompactSelfTest.class);
         suite.addTestSuite(BinaryFieldsHeapNonCompactSelfTest.class);
         suite.addTestSuite(BinaryFieldsOffheapNonCompactSelfTest.class);
@@ -105,7 +122,8 @@ public class IgniteBinaryObjectsTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheBinaryObjectsPartitionedNearDisabledOffheapTieredSelfTest.class);
 
         suite.addTestSuite(GridCacheBinaryStoreObjectsSelfTest.class);
-        suite.addTestSuite(GridCacheBinaryStoreBinariesSelfTest.class);
+        suite.addTestSuite(GridCacheBinaryStoreBinariesDefaultMappersSelfTest.class);
+        suite.addTestSuite(GridCacheBinaryStoreBinariesSimpleNameMappersSelfTest.class);
 
         suite.addTestSuite(GridCacheClientNodeBinaryObjectMetadataTest.class);
         suite.addTestSuite(GridCacheClientNodeBinaryObjectMetadataMultinodeTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinarySimpleNameMapperBasicTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinarySimpleNameMapperBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinarySimpleNameMapperBasicTestSuite.java
new file mode 100644
index 0000000..77020bd
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinarySimpleNameMapperBasicTestSuite.java
@@ -0,0 +1,38 @@
+/*
+ * 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.testsuites;
+
+import junit.framework.TestSuite;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.testframework.config.GridTestProperties;
+
+/**
+ * Basic test suite.
+ */
+public class IgniteBinarySimpleNameMapperBasicTestSuite extends TestSuite {
+    /**
+     * @return Test suite.
+     * @throws Exception Thrown in case of the failure.
+     */
+    public static TestSuite suite() throws Exception {
+        GridTestProperties.setProperty(GridTestProperties.MARSH_CLASS_NAME, BinaryMarshaller.class.getName());
+        GridTestProperties.setProperty(GridTestProperties.BINARY_MARSHALLER_USE_SIMPLE_NAME_MAPPER, "true");
+
+        return IgniteBinaryBasicTestSuite.suite();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinarySimpleNameMapperCacheFullApiTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinarySimpleNameMapperCacheFullApiTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinarySimpleNameMapperCacheFullApiTestSuite.java
new file mode 100644
index 0000000..bbf4297
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinarySimpleNameMapperCacheFullApiTestSuite.java
@@ -0,0 +1,39 @@
+/*
+ * 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.testsuites;
+
+import junit.framework.TestSuite;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.testframework.config.GridTestProperties;
+
+/**
+ * Cache full API suite with binary marshaller and simple name mapper.
+ */
+public class IgniteBinarySimpleNameMapperCacheFullApiTestSuite extends TestSuite {
+    /**
+     * @return Suite.
+     * @throws Exception In case of error.
+     */
+    public static TestSuite suite() throws Exception {
+        GridTestProperties.setProperty(GridTestProperties.MARSH_CLASS_NAME, BinaryMarshaller.class.getName());
+        GridTestProperties.setProperty(GridTestProperties.BINARY_MARSHALLER_USE_SIMPLE_NAME_MAPPER, "true");
+
+        return IgniteCacheFullApiSelfTestSuite.suite();
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
index 3bab1f9..89ead35 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
@@ -21,90 +21,10 @@ import junit.framework.TestSuite;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
 import org.apache.ignite.internal.processors.cache.BinarySerializationQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.BinarySerializationQueryWithReflectiveSerializerSelfTest;
-import org.apache.ignite.internal.processors.cache.CacheLocalQueryMetricsSelfTest;
-import org.apache.ignite.internal.processors.cache.CachePartitionedQueryMetricsDistributedSelfTest;
-import org.apache.ignite.internal.processors.cache.CachePartitionedQueryMetricsLocalSelfTest;
-import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryMetricsDistributedSelfTest;
-import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryMetricsLocalSelfTest;
-import org.apache.ignite.internal.processors.cache.CacheScanPartitionQueryFallbackSelfTest;
-import org.apache.ignite.internal.processors.cache.GridCacheCrossCacheQuerySelfTest;
-import org.apache.ignite.internal.processors.cache.GridCacheQueryIndexDisabledSelfTest;
-import org.apache.ignite.internal.processors.cache.GridCacheQueryIndexingDisabledSelfTest;
-import org.apache.ignite.internal.processors.cache.GridCacheQueryInternalKeysSelfTest;
-import org.apache.ignite.internal.processors.cache.GridCacheQuerySerializationSelfTest;
-import org.apache.ignite.internal.processors.cache.GridCacheReduceQueryMultithreadedSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteBinaryObjectFieldsQuerySelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteBinaryWrappedObjectFieldsQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheBinaryObjectsScanSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheCollocatedQuerySelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheDuplicateEntityConfigurationSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheFieldsQueryNoDataSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheLargeResultSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheNoClassQuerySelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapEvictQueryTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapTieredMultithreadedSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheP2pUnmarshallingQueryErrorTest;
-import org.apache.ignite.internal.processors.cache.IgniteCachePartitionedQueryMultiThreadedSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheQueryEvictsMultiThreadedSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheQueryIndexSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheQueryLoadSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheQueryMultiThreadedOffHeapTieredSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheQueryMultiThreadedSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheQueryOffheapMultiThreadedSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheSqlQueryMultiThreadedSelfTest;
-import org.apache.ignite.internal.processors.cache.SqlFieldsQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.binary.distributed.dht.GridCacheBinaryDuplicateIndexObjectPartitionedAtomicSelfTest;
 import org.apache.ignite.internal.processors.cache.binary.distributed.dht.GridCacheBinaryDuplicateIndexObjectPartitionedTransactionalSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheAtomicFieldsQuerySelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheAtomicNearEnabledFieldsQuerySelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheAtomicNearEnabledQuerySelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheAtomicQuerySelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheClientQueryReplicatedNodeRestartSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedFieldsQueryP2PEnabledSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedFieldsQuerySelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedQueryP2PDisabledSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedQuerySelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedSnapshotEnabledQuerySelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNodeRestartSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNodeRestartSelfTest2;
-import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedFieldsQueryP2PEnabledSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedFieldsQuerySelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedQueryP2PDisabledSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedQuerySelfTest;
-import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalAtomicQuerySelfTest;
-import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalFieldsQuerySelfTest;
-import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryLostPartitionTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverAtomicPrimaryWriteOrderSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverAtomicReplicatedSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverTxReplicatedSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverTxSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicNearEnabledSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicP2PDisabledSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryLocalAtomicSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryLocalSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryPartitionedOnlySelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryPartitionedP2PDisabledSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryPartitionedSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryReplicatedAtomicSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryReplicatedOneNodeSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryReplicatedP2PDisabledSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryReplicatedSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryTxSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryClientReconnectTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryClientTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryClientTxReconnectTest;
-import org.apache.ignite.internal.processors.cache.reducefields.GridCacheReduceFieldsQueryAtomicSelfTest;
-import org.apache.ignite.internal.processors.cache.reducefields.GridCacheReduceFieldsQueryLocalSelfTest;
-import org.apache.ignite.internal.processors.cache.reducefields.GridCacheReduceFieldsQueryPartitionedSelfTest;
-import org.apache.ignite.internal.processors.cache.reducefields.GridCacheReduceFieldsQueryReplicatedSelfTest;
-import org.apache.ignite.internal.processors.query.IgniteSqlSplitterSelfTest;
-import org.apache.ignite.internal.processors.query.h2.sql.BaseH2CompareQueryTest;
-import org.apache.ignite.internal.processors.query.h2.sql.GridQueryParsingTest;
-import org.apache.ignite.internal.processors.query.h2.sql.H2CompareBigQueryTest;
-import org.apache.ignite.spi.communication.tcp.GridOrderedMessageCancelSelfTest;
 import org.apache.ignite.testframework.config.GridTestProperties;
 
 /**
@@ -118,122 +38,24 @@ public class IgniteBinaryCacheQueryTestSuite extends TestSuite {
     public static TestSuite suite() throws Exception {
         GridTestProperties.setProperty(GridTestProperties.MARSH_CLASS_NAME, BinaryMarshaller.class.getName());
 
-        TestSuite suite = new TestSuite("Grid Cache Query Test Suite using BinaryMarshaller");
+        TestSuite suite = IgniteCacheQuerySelfTestSuite.suite();
 
         // Serialization.
         suite.addTestSuite(BinarySerializationQuerySelfTest.class);
         suite.addTestSuite(BinarySerializationQueryWithReflectiveSerializerSelfTest.class);
 
-        // Parsing.
-        suite.addTestSuite(GridQueryParsingTest.class);
-
-        // Config.
-        suite.addTestSuite(IgniteCacheDuplicateEntityConfigurationSelfTest.class);
-
-        // Queries tests.
-        suite.addTestSuite(IgniteSqlSplitterSelfTest.class);
-        suite.addTestSuite(GridCacheQueryIndexDisabledSelfTest.class);
-        suite.addTestSuite(IgniteCacheQueryLoadSelfTest.class);
-        suite.addTestSuite(IgniteCacheLocalQuerySelfTest.class);
-        suite.addTestSuite(IgniteCacheLocalAtomicQuerySelfTest.class);
-        suite.addTestSuite(IgniteCacheReplicatedQuerySelfTest.class);
-        suite.addTestSuite(IgniteCacheReplicatedQueryP2PDisabledSelfTest.class);
-        suite.addTestSuite(IgniteCachePartitionedQuerySelfTest.class);
-        suite.addTestSuite(IgniteCachePartitionedSnapshotEnabledQuerySelfTest.class);
-        suite.addTestSuite(IgniteCacheAtomicQuerySelfTest.class);
-        suite.addTestSuite(IgniteCacheAtomicNearEnabledQuerySelfTest.class);
-        suite.addTestSuite(IgniteCachePartitionedQueryP2PDisabledSelfTest.class);
-        suite.addTestSuite(IgniteCachePartitionedQueryMultiThreadedSelfTest.class);
-        suite.addTestSuite(IgniteCacheQueryIndexSelfTest.class);
-        suite.addTestSuite(IgniteCacheCollocatedQuerySelfTest.class);
-        suite.addTestSuite(IgniteCacheLargeResultSelfTest.class);
-        suite.addTestSuite(GridCacheQueryInternalKeysSelfTest.class);
-        suite.addTestSuite(IgniteCacheQueryMultiThreadedSelfTest.class);
-        suite.addTestSuite(IgniteCacheQueryMultiThreadedOffHeapTieredSelfTest.class);
-        suite.addTestSuite(IgniteCacheQueryEvictsMultiThreadedSelfTest.class);
-        suite.addTestSuite(IgniteCacheQueryOffheapMultiThreadedSelfTest.class);
-        suite.addTestSuite(IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.class);
-        suite.addTestSuite(IgniteCacheOffheapEvictQueryTest.class);
-        suite.addTestSuite(IgniteCacheSqlQueryMultiThreadedSelfTest.class);
-        suite.addTestSuite(IgniteCacheOffheapTieredMultithreadedSelfTest.class);
-        suite.addTestSuite(IgniteCacheQueryNodeRestartSelfTest.class);
-        suite.addTestSuite(IgniteCacheQueryNodeRestartSelfTest2.class);
-        suite.addTestSuite(IgniteCacheClientQueryReplicatedNodeRestartSelfTest.class);
-        suite.addTestSuite(GridCacheReduceQueryMultithreadedSelfTest.class);
-        suite.addTestSuite(GridCacheCrossCacheQuerySelfTest.class);
-        suite.addTestSuite(GridCacheQuerySerializationSelfTest.class);
         suite.addTestSuite(IgniteCacheBinaryObjectsScanSelfTest.class);
-        suite.addTestSuite(IgniteBinaryObjectFieldsQuerySelfTest.class);
-        suite.addTestSuite(IgniteBinaryWrappedObjectFieldsQuerySelfTest.class);
-
-        // Scan queries.
-        suite.addTestSuite(CacheScanPartitionQueryFallbackSelfTest.class);
-
-        // Fields queries.
-        suite.addTestSuite(SqlFieldsQuerySelfTest.class);
-        suite.addTestSuite(IgniteCacheLocalFieldsQuerySelfTest.class);
-        suite.addTestSuite(IgniteCacheReplicatedFieldsQuerySelfTest.class);
-        suite.addTestSuite(IgniteCacheReplicatedFieldsQueryP2PEnabledSelfTest.class);
-        suite.addTestSuite(IgniteCachePartitionedFieldsQuerySelfTest.class);
-        suite.addTestSuite(IgniteCacheAtomicFieldsQuerySelfTest.class);
-        suite.addTestSuite(IgniteCacheAtomicNearEnabledFieldsQuerySelfTest.class);
-        suite.addTestSuite(IgniteCachePartitionedFieldsQueryP2PEnabledSelfTest.class);
-        suite.addTestSuite(IgniteCacheFieldsQueryNoDataSelfTest.class);
-
-        // Continuous queries.
-        suite.addTestSuite(GridCacheContinuousQueryLocalSelfTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryLocalAtomicSelfTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryReplicatedSelfTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryReplicatedAtomicSelfTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryReplicatedP2PDisabledSelfTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryPartitionedSelfTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryPartitionedOnlySelfTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryPartitionedP2PDisabledSelfTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryTxSelfTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryAtomicSelfTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryAtomicNearEnabledSelfTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryAtomicP2PDisabledSelfTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryReplicatedOneNodeSelfTest.class);
-        suite.addTestSuite(IgniteCacheContinuousQueryClientTest.class);
-        suite.addTestSuite(IgniteCacheContinuousQueryClientReconnectTest.class);
-        suite.addTestSuite(IgniteCacheContinuousQueryClientTxReconnectTest.class);
-        suite.addTestSuite(CacheContinuousQueryFailoverAtomicPrimaryWriteOrderSelfTest.class);
-        suite.addTestSuite(CacheContinuousQueryFailoverAtomicReplicatedSelfTest.class);
-        suite.addTestSuite(CacheContinuousQueryFailoverTxSelfTest.class);
-        suite.addTestSuite(CacheContinuousQueryFailoverTxReplicatedSelfTest.class);
         suite.addTestSuite(CacheContinuousQueryLostPartitionTest.class);
 
-        // Reduce fields queries.
-        suite.addTestSuite(GridCacheReduceFieldsQueryLocalSelfTest.class);
-        suite.addTestSuite(GridCacheReduceFieldsQueryPartitionedSelfTest.class);
-        suite.addTestSuite(GridCacheReduceFieldsQueryAtomicSelfTest.class);
-        suite.addTestSuite(GridCacheReduceFieldsQueryReplicatedSelfTest.class);
-
-        suite.addTestSuite(GridCacheQueryIndexingDisabledSelfTest.class);
-
         //Should be adjusted. Not ready to be used with BinaryMarshaller.
         //suite.addTestSuite(GridCacheBinarySwapScanQuerySelfTest.class);
 
-        suite.addTestSuite(GridOrderedMessageCancelSelfTest.class);
-
-        // Ignite cache and H2 comparison.
-        suite.addTestSuite(BaseH2CompareQueryTest.class);
-        suite.addTestSuite(H2CompareBigQueryTest.class);
-
-        // Metrics tests
-        suite.addTestSuite(CacheLocalQueryMetricsSelfTest.class);
-        suite.addTestSuite(CachePartitionedQueryMetricsDistributedSelfTest.class);
-        suite.addTestSuite(CachePartitionedQueryMetricsLocalSelfTest.class);
-        suite.addTestSuite(CacheReplicatedQueryMetricsDistributedSelfTest.class);
-        suite.addTestSuite(CacheReplicatedQueryMetricsLocalSelfTest.class);
-
-        //Unmarshallig query test.
-        suite.addTestSuite(IgniteCacheP2pUnmarshallingQueryErrorTest.class);
-        suite.addTestSuite(IgniteCacheNoClassQuerySelfTest.class);
-
         suite.addTestSuite(GridCacheBinaryDuplicateIndexObjectPartitionedAtomicSelfTest.class);
         suite.addTestSuite(GridCacheBinaryDuplicateIndexObjectPartitionedTransactionalSelfTest.class);
 
+        //TODO: the following tests= was never tested with binary. Exclude or pass?
+//        suite.addTestSuite(IgniteSqlSchemaIndexingTest.class);
+
         return suite;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinarySimpleNameMapperCacheQueryTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinarySimpleNameMapperCacheQueryTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinarySimpleNameMapperCacheQueryTestSuite.java
new file mode 100644
index 0000000..3eed980
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinarySimpleNameMapperCacheQueryTestSuite.java
@@ -0,0 +1,38 @@
+/*
+ * 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.testsuites;
+
+import junit.framework.TestSuite;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.testframework.config.GridTestProperties;
+
+/**
+ * Cache query suite with binary marshaller.
+ */
+public class IgniteBinarySimpleNameMapperCacheQueryTestSuite extends TestSuite {
+    /**
+     * @return Suite.
+     * @throws Exception In case of error.
+     */
+    public static TestSuite suite() throws Exception {
+        GridTestProperties.setProperty(GridTestProperties.MARSH_CLASS_NAME, BinaryMarshaller.class.getName());
+        GridTestProperties.setProperty(GridTestProperties.BINARY_MARSHALLER_USE_SIMPLE_NAME_MAPPER, "true");
+
+        return IgniteBinaryCacheQueryTestSuite.suite();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid2.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid2.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid2.xml
index 434f468..ef29a89 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid2.xml
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid2.xml
@@ -45,6 +45,27 @@
             </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" />
+                            <property name="enum" value="true" />
+                        </bean>
+                    </list>
+                </property>
+
+            </bean>
+        </property>
+
         <property name="discoverySpi">
             <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
                 <property name="ipFinder">

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid3.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid3.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid3.xml
index 31ccdf0..d1c96b6 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid3.xml
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid3.xml
@@ -25,14 +25,35 @@
       <property name="localHost" value="127.0.0.1"/>
         <property name="connectorConfiguration"><null/></property>
 
-      <property name="clientMode" value="true"/>
+        <property name="clientMode" value="true"/>
 
-      <property name="gridName" value="grid3"/>
+        <property name="gridName" value="grid3"/>
 
-      <property name="metricsUpdateFrequency" value="1000"/>
-      <property name="metricsLogFrequency" value="0"/>
+        <property name="metricsUpdateFrequency" value="1000"/>
+        <property name="metricsLogFrequency" value="0"/>
 
-      <property name="discoverySpi">
+        <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" />
+                            <property name="enum" value="true" />
+                        </bean>
+                    </list>
+                </property>
+
+            </bean>
+        </property>
+
+        <property name="discoverySpi">
             <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
                 <property name="forceServerMode" value="true"/>
 


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

Posted by ag...@apache.org.
IGNITE-2191 - Support classes with the same simple name for Binary marshaller - Fixes #398.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


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

Branch: refs/heads/master
Commit: 62502b2c44d7d35584b31c8f1284b810840f9a8d
Parents: 58dce88
Author: ashutak <as...@gridgain.com>
Authored: Mon Feb 1 18:12:03 2016 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Feb 1 18:12:03 2016 +0300

----------------------------------------------------------------------
 .../ignite/binary/BinaryBasicIdMapper.java      |  167 +++
 .../ignite/binary/BinaryBasicNameMapper.java    |  141 +++
 .../apache/ignite/binary/BinaryIdMapper.java    |   19 +-
 .../apache/ignite/binary/BinaryNameMapper.java  |   47 +
 .../ignite/binary/BinaryTypeConfiguration.java  |   23 +-
 .../configuration/BinaryConfiguration.java      |   28 +
 .../configuration/IgniteConfiguration.java      |    8 +-
 .../ignite/internal/IgniteNodeAttributes.java   |    3 +
 .../internal/binary/BinaryClassDescriptor.java  |   23 +-
 .../ignite/internal/binary/BinaryContext.java   |  347 ++++--
 .../internal/binary/BinaryEnumObjectImpl.java   |   11 +-
 .../internal/binary/BinaryInternalIdMapper.java |  161 ---
 .../internal/binary/BinaryInternalMapper.java   |  131 ++
 .../binary/BinaryMetadataCollector.java         |   22 +-
 .../internal/binary/BinaryReaderExImpl.java     |   14 +-
 .../internal/binary/BinaryWriterExImpl.java     |   30 +-
 .../binary/builder/BinaryBuilderSerializer.java |   16 +-
 .../binary/builder/BinaryObjectBuilderImpl.java |    2 +-
 .../binary/CacheObjectBinaryProcessorImpl.java  |  123 +-
 .../cpp/PlatformCppConfigurationClosure.java    |   32 +
 .../PlatformDotNetConfigurationClosure.java     |   34 +-
 .../binary/BinaryBasicIdMapperSelfTest.java     |   51 +
 .../binary/BinaryBasicNameMapperSelfTest.java   |   50 +
 .../BinaryConfigurationConsistencySelfTest.java |  231 ++++
 .../internal/binary/BinaryEnumsSelfTest.java    |   12 +-
 .../binary/BinaryMarshallerSelfTest.java        |  508 ++++++--
 .../BinaryObjectBuilderAdditionalSelfTest.java  |    4 +-
 ...naryObjectBuilderDefaultMappersSelfTest.java | 1149 ++++++++++++++++++
 .../binary/BinaryObjectBuilderSelfTest.java     | 1108 -----------------
 ...ilderSimpleNameLowerCaseMappersSelfTest.java |   41 +
 .../BinarySimpleNameTestPropertySelfTest.java   |   94 ++
 .../binary/GridBinaryMetaDataSelfTest.java      |  371 ------
 .../binary/GridBinaryWildcardsSelfTest.java     |  338 +++++-
 ...aultBinaryMappersBinaryMetaDataSelfTest.java |  389 ++++++
 ...CaseBinaryMappersBinaryMetaDataSelfTest.java |   41 +
 .../internal/binary/TestMappedObject.java       |   25 +
 ...BuilderNonCompactDefaultMappersSelfTest.java |   30 +
 .../BinaryObjectBuilderNonCompactSelfTest.java  |   30 -
 ...mpactSimpleNameLowerCaseMappersSelfTest.java |   31 +
 .../GridCacheConditionalDeploymentSelfTest.java |    4 +-
 .../GridCacheBinaryObjectsAbstractSelfTest.java |   17 +-
 .../GridCacheBinaryStoreAbstractSelfTest.java   |   10 +
 ...naryStoreBinariesDefaultMappersSelfTest.java |   81 ++
 .../GridCacheBinaryStoreBinariesSelfTest.java   |   66 -
 ...yStoreBinariesSimpleNameMappersSelfTest.java |   40 +
 .../config/GridTestProperties.java              |    5 +-
 .../testframework/junits/GridAbstractTest.java  |   23 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |    2 +-
 .../testsuites/IgniteBinaryBasicTestSuite.java  |   95 ++
 ...ctsSimpleNameMapperComputeGridTestSuite.java |   38 +
 .../IgniteBinaryObjectsTestSuite.java           |   34 +-
 ...iteBinarySimpleNameMapperBasicTestSuite.java |   38 +
 ...rySimpleNameMapperCacheFullApiTestSuite.java |   39 +
 .../IgniteBinaryCacheQueryTestSuite.java        |  186 +--
 ...narySimpleNameMapperCacheQueryTestSuite.java |   38 +
 .../Config/Compute/compute-grid2.xml            |   21 +
 .../Config/Compute/compute-grid3.xml            |   31 +-
 .../org/apache/ignite/yardstick/IgniteNode.java |    3 +
 58 files changed, 4381 insertions(+), 2275 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/main/java/org/apache/ignite/binary/BinaryBasicIdMapper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryBasicIdMapper.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryBasicIdMapper.java
new file mode 100644
index 0000000..da31751
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryBasicIdMapper.java
@@ -0,0 +1,167 @@
+/*
+ * 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.binary;
+
+import org.apache.ignite.internal.util.typedef.internal.A;
+
+/**
+ * Base binary ID mapper implementation.
+ */
+public class BinaryBasicIdMapper implements BinaryIdMapper {
+    /** Default lower case flag setting. */
+    public static final boolean DFLT_LOWER_CASE = true;
+
+    /** Maximum lower-case character. */
+    private static final char MAX_LOWER_CASE_CHAR = 0x7e;
+
+    /** Cached lower-case characters. */
+    private static final char[] LOWER_CASE_CHARS;
+
+    /** */
+    private boolean isLowerCase = DFLT_LOWER_CASE;
+
+    /**
+     * Static initializer.
+     */
+    static {
+        LOWER_CASE_CHARS = new char[MAX_LOWER_CASE_CHAR + 1];
+
+        for (char c = 0; c <= MAX_LOWER_CASE_CHAR; c++)
+            LOWER_CASE_CHARS[c] = Character.toLowerCase(c);
+    }
+
+    /**
+     * Default constructor.
+     */
+    public BinaryBasicIdMapper() {
+    }
+
+    /**
+     * @param isLowerCase Whether to use strings in lower case or not.
+     * <p>
+     * Defaults to {@link #DFLT_LOWER_CASE}.
+     */
+    public BinaryBasicIdMapper(boolean isLowerCase) {
+        this.isLowerCase = isLowerCase;
+    }
+
+    /**
+     * Get type ID.
+     *
+     * @param typeName Type name.
+     * @return Type ID.
+     */
+    public int typeId(String typeName) {
+        A.notNull(typeName, "typeName");
+
+        int id = isLowerCase ? lowerCaseHashCode(typeName) : typeName.hashCode();
+
+        if (id != 0)
+            return id;
+        else {
+            throw new BinaryObjectException("Binary ID mapper resolved type ID to zero " +
+                "(either change type's name or use custom ID mapper) " +
+                "[name=" + typeName + ", isLowerCase=" + isLowerCase + "]");
+        }
+    }
+
+    /**
+     * Get field ID.
+     *
+     * @param typeId Type ID.
+     * @param fieldName Field name.
+     * @return Field ID.
+     */
+    public int fieldId(int typeId, String fieldName) {
+        A.notNull(fieldName, "fieldName");
+
+        int id = isLowerCase ? lowerCaseHashCode(fieldName) : fieldName.hashCode();
+
+        if (id != 0)
+            return id;
+        else {
+            throw new BinaryObjectException("Binary ID mapper resolved field ID to zero " +
+                "(either change filed's name or use custom ID mapper) " +
+                "[name=" + fieldName + ", isLowerCase=" + isLowerCase + "]");
+        }
+    }
+
+    /**
+     * Gets whether to use strings in lower case or not.
+     *
+     * @return Whether to use strings in lower case or not.
+     */
+    public boolean isLowerCase() {
+        return isLowerCase;
+    }
+
+    /**
+     * Sets whether to use strings in lower case or not.
+     *
+     * @param isLowerCase Whether to use strings in lower case or not.
+     */
+    public void setLowerCase(boolean isLowerCase) {
+        this.isLowerCase = isLowerCase;
+    }
+
+    /**
+     * Routine to calculate string hash code an
+     *
+     * @param str String.
+     * @return Hash code for given string converted to lower case.
+     */
+    private static int lowerCaseHashCode(String str) {
+        int len = str.length();
+
+        int h = 0;
+
+        for (int i = 0; i < len; i++) {
+            int c = str.charAt(i);
+
+            c = c <= MAX_LOWER_CASE_CHAR ? LOWER_CASE_CHARS[c] : Character.toLowerCase(c);
+
+            h = 31 * h + c;
+        }
+
+        return h;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (!(o instanceof BinaryBasicIdMapper))
+            return false;
+
+        BinaryBasicIdMapper mapper = (BinaryBasicIdMapper)o;
+
+        return isLowerCase == mapper.isLowerCase;
+
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return (isLowerCase ? 1 : 0);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return "BinaryBaseIdMapper [isLowerCase=" + isLowerCase + ']';
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/main/java/org/apache/ignite/binary/BinaryBasicNameMapper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryBasicNameMapper.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryBasicNameMapper.java
new file mode 100644
index 0000000..62420f7
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryBasicNameMapper.java
@@ -0,0 +1,141 @@
+/*
+ * 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.binary;
+
+import org.apache.ignite.internal.util.typedef.internal.A;
+
+/**
+ * Base binary name mapper implementation.
+ */
+public class BinaryBasicNameMapper implements BinaryNameMapper {
+    /** Default use simple name flag setting. */
+    public static final boolean DFLT_SIMPLE_NAME = false;
+
+    /** */
+    private boolean isSimpleName = DFLT_SIMPLE_NAME;
+
+    /**
+     * Default constructor.
+     */
+    public BinaryBasicNameMapper() {
+        // No-op.
+    }
+
+    /**
+     * @param isSimpleName Whether to use simple name of class or not.
+     * <p>
+     * Defaults to {@link #DFLT_SIMPLE_NAME}.
+     */
+    public BinaryBasicNameMapper(boolean isSimpleName) {
+        this.isSimpleName = isSimpleName;
+    }
+
+    /**
+     * Gets whether to use simple name of class or not.
+     *
+     * @return Whether to use simple name of class or not.
+     */
+    public boolean isSimpleName() {
+        return isSimpleName;
+    }
+
+    /**
+     * Sets whether to use simple name of class or not.
+     *
+     * @param isSimpleName Whether to use simple name of class or not.
+     */
+    public void setLowerCase(boolean isSimpleName) {
+        this.isSimpleName = isSimpleName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String typeName(String clsName) {
+        A.notNull(clsName, "clsName");
+
+        return isSimpleName ? simpleName(clsName) : clsName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String fieldName(String fieldName) {
+        A.notNull(fieldName, "fieldName");
+
+        return fieldName;
+    }
+
+    /**
+     * @param clsName Class name.
+     * @return Type name.
+     */
+    private static String simpleName(String clsName) {
+        assert clsName != null;
+
+        int idx = clsName.lastIndexOf('$');
+
+        if (idx == clsName.length() - 1)
+            // This is a regular (not inner) class name that ends with '$'. Common use case for Scala classes.
+            idx = -1;
+        else if (idx >= 0) {
+            String typeName = clsName.substring(idx + 1);
+
+            try {
+                Integer.parseInt(typeName);
+
+                // This is an anonymous class. Don't cut off enclosing class name for it.
+                idx = -1;
+            }
+            catch (NumberFormatException ignore) {
+                // This is a lambda class.
+                if (clsName.indexOf("$$Lambda$") > 0)
+                    idx = -1;
+                else
+                    return typeName;
+            }
+        }
+
+        if (idx < 0)
+            idx = clsName.lastIndexOf('.');
+
+        return idx >= 0 ? clsName.substring(idx + 1) : clsName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (!(o instanceof BinaryBasicNameMapper))
+            return false;
+
+        BinaryBasicNameMapper mapper = (BinaryBasicNameMapper)o;
+
+        if (isSimpleName != mapper.isSimpleName)
+            return false;
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return (isSimpleName ? 1 : 0);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return "BinaryBaseNameMapper [isSimpleName=" + isSimpleName + ']';
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/main/java/org/apache/ignite/binary/BinaryIdMapper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryIdMapper.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryIdMapper.java
index 4c31140..dac5c44 100644
--- a/modules/core/src/main/java/org/apache/ignite/binary/BinaryIdMapper.java
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryIdMapper.java
@@ -30,26 +30,29 @@ package org.apache.ignite.binary;
  * Binary ID mapper can be configured for all binary objects via
  * {@link org.apache.ignite.configuration.BinaryConfiguration#getIdMapper()} method,
  * or for a specific binary type via {@link BinaryTypeConfiguration#getIdMapper()} method.
+ * @see BinaryNameMapper
  */
 public interface BinaryIdMapper {
     /**
-     * Gets type ID for provided class name.
+     * Gets type ID for provided type name.
      * <p>
      * If {@code 0} is returned, hash code of class simple name will be used.
      *
-     * @param clsName Class name.
+     * @param typeName Type name. Type name is a result of {@link BinaryNameMapper#typeName(String)} call for an
+     *        initial class or type name.
      * @return Type ID.
+     * @see BinaryNameMapper#typeName(String)
      */
-    public int typeId(String clsName);
+    public int typeId(String typeName);
 
     /**
-     * Gets ID for provided field.
-     * <p>
-     * If {@code 0} is returned, hash code of field name will be used.
+     * Gets ID for provided field name. <p> If {@code 0} is returned, hash code of field name will be used.
      *
      * @param typeId Type ID.
-     * @param fieldName Field name.
+     * @param fieldName Field name. Filed anme is a result of {@link BinaryNameMapper#fieldName(String)} call for an
+     *        initial filed name.
      * @return Field ID.
+     * @see BinaryNameMapper#fieldName(String)
      */
     public int fieldId(int typeId, String fieldName);
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/main/java/org/apache/ignite/binary/BinaryNameMapper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryNameMapper.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryNameMapper.java
new file mode 100644
index 0000000..2db0aeb
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryNameMapper.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.binary;
+
+import org.apache.ignite.configuration.BinaryConfiguration;
+
+/**
+ * Maps type and field names to different names. Prepares class/type names
+ * and field names before pass them to {@link BinaryIdMapper}.
+ * <p>
+ * Binary name mapper can be configured for all binary objects via
+ * {@link BinaryConfiguration#getNameMapper()} method,
+ * or for a specific binary type via {@link BinaryTypeConfiguration#getNameMapper()} method.
+ * @see BinaryIdMapper
+ */
+public interface BinaryNameMapper {
+    /**
+     * Gets type clsName.
+     *
+     * @param clsName Class came
+     * @return Type name.
+     */
+    String typeName(String clsName);
+
+    /**
+     * Gets field name.
+     *
+     * @param fieldName Field name.
+     * @return Field name.
+     */
+    String fieldName(String fieldName);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/main/java/org/apache/ignite/binary/BinaryTypeConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryTypeConfiguration.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryTypeConfiguration.java
index a00c061..fea0af7 100644
--- a/modules/core/src/main/java/org/apache/ignite/binary/BinaryTypeConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryTypeConfiguration.java
@@ -36,6 +36,9 @@ public class BinaryTypeConfiguration {
     /** ID mapper. */
     private BinaryIdMapper idMapper;
 
+    /** Name mapper. */
+    private BinaryNameMapper nameMapper;
+
     /** Serializer. */
     private BinarySerializer serializer;
 
@@ -107,6 +110,24 @@ public class BinaryTypeConfiguration {
     }
 
     /**
+     * Gets name mapper.
+     *
+     * @return Name mapper.
+     */
+    public BinaryNameMapper getNameMapper() {
+        return nameMapper;
+    }
+
+    /**
+     * Sets name mapper.
+     *
+     * @param nameMapper Name mapper.
+     */
+    public void setNameMapper(BinaryNameMapper nameMapper) {
+        this.nameMapper = nameMapper;
+    }
+
+    /**
      * Gets serializer.
      *
      * @return Serializer.
@@ -146,4 +167,4 @@ public class BinaryTypeConfiguration {
     @Override public String toString() {
         return S.toString(BinaryTypeConfiguration.class, this, super.toString());
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/main/java/org/apache/ignite/configuration/BinaryConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/BinaryConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/BinaryConfiguration.java
index 1151245..30d77de 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/BinaryConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/BinaryConfiguration.java
@@ -20,8 +20,10 @@ package org.apache.ignite.configuration;
 import java.util.ArrayList;
 import java.util.Collection;
 import org.apache.ignite.binary.BinaryIdMapper;
+import org.apache.ignite.binary.BinaryNameMapper;
 import org.apache.ignite.binary.BinarySerializer;
 import org.apache.ignite.binary.BinaryTypeConfiguration;
+import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
  * Configuration object for Ignite Binary Objects.
@@ -34,6 +36,9 @@ public class BinaryConfiguration {
     /** ID mapper. */
     private BinaryIdMapper idMapper;
 
+    /** Name mapper. */
+    private BinaryNameMapper nameMapper;
+
     /** Serializer. */
     private BinarySerializer serializer;
 
@@ -75,6 +80,24 @@ public class BinaryConfiguration {
     }
 
     /**
+     * Gets name mapper.
+     *
+     * @return Name mapper.
+     */
+    public BinaryNameMapper getNameMapper() {
+        return nameMapper;
+    }
+
+    /**
+     * Sets name mapper.
+     *
+     * @param nameMapper Name mapper.
+     */
+    public void setNameMapper(BinaryNameMapper nameMapper) {
+        this.nameMapper = nameMapper;
+    }
+
+    /**
      * Gets serializer.
      *
      * @return Serializer.
@@ -136,4 +159,9 @@ public class BinaryConfiguration {
     public void setCompactFooter(boolean compactFooter) {
         this.compactFooter = compactFooter;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(BinaryConfiguration.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index 6598dc0..f705638 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -2002,8 +2002,10 @@ public class IgniteConfiguration {
      *
      * @param cacheKeyCfg Cache key configuration.
      */
-    public void setCacheKeyConfiguration(CacheKeyConfiguration... cacheKeyCfg) {
+    public IgniteConfiguration setCacheKeyConfiguration(CacheKeyConfiguration... cacheKeyCfg) {
         this.cacheKeyCfg = cacheKeyCfg;
+
+        return this;
     }
 
     /**
@@ -2020,8 +2022,10 @@ public class IgniteConfiguration {
      *
      * @param binaryCfg Binary configuration object.
      */
-    public void setBinaryConfiguration(BinaryConfiguration binaryCfg) {
+    public IgniteConfiguration setBinaryConfiguration(BinaryConfiguration binaryCfg) {
         this.binaryCfg = binaryCfg;
+
+        return this;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/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 3c5f960..da6f40d 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
@@ -147,6 +147,9 @@ public final class IgniteNodeAttributes {
     /** Update notifier enabled. */
     public static final String ATTR_UPDATE_NOTIFIER_ENABLED = ATTR_PREFIX + ".update.notifier.enabled";
 
+    /** Binary configuration. */
+    public static final String ATTR_BINARY_CONFIGURATION = ATTR_PREFIX + ".binary.config";
+
     /**
      * Enforces singleton.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
index eb45ac6..233e74b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
@@ -34,13 +34,14 @@ import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.binary.BinaryIdMapper;
 import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.binary.BinaryReflectiveSerializer;
 import org.apache.ignite.binary.BinarySerializer;
 import org.apache.ignite.binary.Binarylizable;
 import org.apache.ignite.internal.processors.cache.CacheObjectImpl;
 import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.internal.util.tostring.GridToStringExclude;
+import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.marshaller.MarshallerExclusions;
 import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
@@ -55,6 +56,7 @@ public class BinaryClassDescriptor {
     public static final Unsafe UNSAFE = GridUnsafe.unsafe();
 
     /** */
+    @GridToStringExclude
     private final BinaryContext ctx;
 
     /** */
@@ -64,7 +66,7 @@ public class BinaryClassDescriptor {
     private final BinarySerializer serializer;
 
     /** ID mapper. */
-    private final BinaryIdMapper idMapper;
+    private final BinaryInternalMapper mapper;
 
     /** */
     private final BinaryWriteMode mode;
@@ -118,7 +120,7 @@ public class BinaryClassDescriptor {
      * @param typeId Type ID.
      * @param typeName Type name.
      * @param affKeyFieldName Affinity key field name.
-     * @param idMapper ID mapper.
+     * @param mapper Mapper.
      * @param serializer Serializer.
      * @param metaDataEnabled Metadata enabled flag.
      * @param registered Whether typeId has been successfully registered by MarshallerContext or not.
@@ -131,14 +133,14 @@ public class BinaryClassDescriptor {
         int typeId,
         String typeName,
         @Nullable String affKeyFieldName,
-        @Nullable BinaryIdMapper idMapper,
+        @Nullable BinaryInternalMapper mapper,
         @Nullable BinarySerializer serializer,
         boolean metaDataEnabled,
         boolean registered
     ) throws BinaryObjectException {
         assert ctx != null;
         assert cls != null;
-        assert idMapper != null;
+        assert mapper != null;
 
         // If serializer is not defined at this point, then we have to user OptimizedMarshaller.
         useOptMarshaller = serializer == null;
@@ -154,7 +156,7 @@ public class BinaryClassDescriptor {
         this.typeName = typeName;
         this.affKeyFieldName = affKeyFieldName;
         this.serializer = serializer;
-        this.idMapper = idMapper;
+        this.mapper = mapper;
         this.registered = registered;
 
         schemaReg = ctx.schemaRegistry(typeId);
@@ -269,7 +271,7 @@ public class BinaryClassDescriptor {
 
                             assert added : name;
 
-                            int fieldId = idMapper.fieldId(typeId, name);
+                            int fieldId = this.mapper.fieldId(typeId, name);
 
                             if (!ids.add(fieldId))
                                 throw new BinaryObjectException("Duplicate field ID: " + name);
@@ -635,7 +637,7 @@ public class BinaryClassDescriptor {
                             if (schemaReg.schema(schemaId) == null) {
                                 // This is new schema, let's update metadata.
                                 BinaryMetadataCollector collector =
-                                    new BinaryMetadataCollector(typeId, typeName, idMapper);
+                                    new BinaryMetadataCollector(typeId, typeName, mapper);
 
                                 if (serializer != null)
                                     serializer.writeBinary(obj, collector);
@@ -804,4 +806,9 @@ public class BinaryClassDescriptor {
             throw new BinaryObjectException("Failed to get constructor for class: " + cls.getName(), e);
         }
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(BinaryClassDescriptor.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
index 163532d..cf19bdf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
@@ -17,10 +17,39 @@
 
 package org.apache.ignite.internal.binary;
 
+import java.io.Externalizable;
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.math.BigDecimal;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentMap;
+import java.util.jar.JarEntry;
+import java.util.jar.JarFile;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.binary.BinaryBasicIdMapper;
+import org.apache.ignite.binary.BinaryBasicNameMapper;
 import org.apache.ignite.binary.BinaryIdMapper;
 import org.apache.ignite.binary.BinaryInvalidTypeException;
+import org.apache.ignite.binary.BinaryNameMapper;
 import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.binary.BinaryReflectiveSerializer;
 import org.apache.ignite.binary.BinarySerializer;
@@ -31,7 +60,6 @@ import org.apache.ignite.cache.affinity.AffinityKey;
 import org.apache.ignite.cache.affinity.AffinityKeyMapped;
 import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.IgnitionEx;
 import org.apache.ignite.internal.processors.cache.binary.BinaryMetadataKey;
 import org.apache.ignite.internal.processors.datastructures.CollocatedQueueItemKey;
 import org.apache.ignite.internal.processors.datastructures.CollocatedSetItemKey;
@@ -46,32 +74,6 @@ import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 
-import java.io.Externalizable;
-import java.io.File;
-import java.io.IOException;
-import java.lang.reflect.Field;
-import java.math.BigDecimal;
-import java.net.URISyntaxException;
-import java.net.URL;
-import java.net.URLClassLoader;
-import java.sql.Timestamp;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Date;
-import java.util.Enumeration;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
-import java.util.LinkedHashSet;
-import java.util.LinkedList;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentMap;
-import java.util.jar.JarEntry;
-import java.util.jar.JarFile;
-
 /**
  * Binary context.
  */
@@ -80,6 +82,14 @@ public class BinaryContext {
     private static final ClassLoader dfltLdr = U.gridClassLoader();
 
     /** */
+    private static final BinaryInternalMapper DFLT_MAPPER =
+        new BinaryInternalMapper(new BinaryBasicNameMapper(false), new BinaryBasicIdMapper(true), false);
+
+    /** */
+    static final BinaryInternalMapper SIMPLE_NAME_LOWER_CASE_MAPPER =
+        new BinaryInternalMapper(new BinaryBasicNameMapper(true), new BinaryBasicIdMapper(true), false);
+
+    /** */
     private final ConcurrentMap<Class<?>, BinaryClassDescriptor> descByCls = new ConcurrentHashMap8<>();
 
     /** Holds classes loaded by default class loader only. */
@@ -97,14 +107,14 @@ public class BinaryContext {
     /** */
     private final Map<Class<? extends Map>, Byte> mapTypes = new HashMap<>();
 
-    /** */
-    private final ConcurrentMap<Integer, BinaryIdMapper> mappers = new ConcurrentHashMap8<>(0);
+    /** Maps typeId to mappers. */
+    private final ConcurrentMap<Integer, BinaryInternalMapper> typeId2Mapper = 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);
+    /** Maps className to mapper */
+    private final ConcurrentMap<String, BinaryInternalMapper> cls2Mappers = new ConcurrentHashMap8<>(0);
 
     /** */
     private BinaryMetadataHandler metaHnd;
@@ -271,6 +281,7 @@ public class BinaryContext {
         optmMarsh.setContext(marshCtx);
 
         configure(
+            binaryCfg.getNameMapper(),
             binaryCfg.getIdMapper(),
             binaryCfg.getSerializer(),
             binaryCfg.getTypeConfigurations()
@@ -286,6 +297,7 @@ public class BinaryContext {
      * @throws BinaryObjectException In case of error.
      */
     private void configure(
+        BinaryNameMapper globalNameMapper,
         BinaryIdMapper globalIdMapper,
         BinarySerializer globalSerializer,
         Collection<BinaryTypeConfiguration> typeCfgs
@@ -306,13 +318,20 @@ public class BinaryContext {
                 if (clsName == null)
                     throw new BinaryObjectException("Class name is required for binary type configuration.");
 
+                // Resolve mapper.
                 BinaryIdMapper idMapper = globalIdMapper;
 
                 if (typeCfg.getIdMapper() != null)
                     idMapper = typeCfg.getIdMapper();
 
-                idMapper = BinaryInternalIdMapper.create(idMapper);
+                BinaryNameMapper nameMapper = globalNameMapper;
+
+                if (typeCfg.getNameMapper() != null)
+                    nameMapper = typeCfg.getNameMapper();
+
+                BinaryInternalMapper mapper = resolveMapper(nameMapper, idMapper);
 
+                // Resolve serializer.
                 BinarySerializer serializer = globalSerializer;
 
                 if (typeCfg.getSerializer() != null)
@@ -322,25 +341,25 @@ public class BinaryContext {
                     String pkgName = clsName.substring(0, clsName.length() - 2);
 
                     for (String clsName0 : classesInPackage(pkgName))
-                        descs.add(clsName0, idMapper, serializer, affFields.get(clsName0),
+                        descs.add(clsName0, mapper, serializer, affFields.get(clsName0),
                             typeCfg.isEnum(), true);
                 }
                 else
-                    descs.add(clsName, idMapper, serializer, affFields.get(clsName),
+                    descs.add(clsName, mapper, serializer, affFields.get(clsName),
                         typeCfg.isEnum(), false);
             }
         }
 
         for (TypeDescriptor desc : descs.descriptors())
-            registerUserType(desc.clsName, desc.idMapper, desc.serializer, desc.affKeyFieldName, desc.isEnum);
+            registerUserType(desc.clsName, desc.mapper, desc.serializer, desc.affKeyFieldName, desc.isEnum);
 
-        BinaryInternalIdMapper dfltMapper = BinaryInternalIdMapper.create(globalIdMapper);
+        BinaryInternalMapper globalMapper = resolveMapper(globalNameMapper, globalIdMapper);
 
         // Put affinity field names for unconfigured types.
         for (Map.Entry<String, String> entry : affFields.entrySet()) {
             String typeName = entry.getKey();
 
-            int typeId = dfltMapper.typeId(typeName);
+            int typeId = globalMapper.typeId(typeName);
 
             affKeyFieldNames.putIfAbsent(typeId, entry.getValue());
         }
@@ -350,6 +369,52 @@ public class BinaryContext {
     }
 
     /**
+     * @param nameMapper Name mapper.
+     * @param idMapper ID mapper.
+     * @return Mapper.
+     */
+    private static BinaryInternalMapper resolveMapper(BinaryNameMapper nameMapper, BinaryIdMapper idMapper) {
+        if ((nameMapper == null || (DFLT_MAPPER.nameMapper().equals(nameMapper)))
+            && (idMapper == null || DFLT_MAPPER.idMapper().equals(idMapper)))
+            return DFLT_MAPPER;
+
+        if (nameMapper != null && nameMapper instanceof BinaryBasicNameMapper
+            && ((BinaryBasicNameMapper)nameMapper).isSimpleName()
+            && idMapper != null && idMapper instanceof BinaryBasicIdMapper
+            && ((BinaryBasicIdMapper)idMapper).isLowerCase())
+            return SIMPLE_NAME_LOWER_CASE_MAPPER;
+
+        if (nameMapper == null)
+            nameMapper = DFLT_MAPPER.nameMapper();
+
+        if (idMapper == null)
+            idMapper = DFLT_MAPPER.idMapper();
+
+        return new BinaryInternalMapper(nameMapper, idMapper, true);
+    }
+
+    /**
+     * @return Intenal mpper used as default.
+     */
+    public static BinaryInternalMapper defaultMapper() {
+        return DFLT_MAPPER;
+    }
+
+    /**
+     * @return ID mapper used as default.
+     */
+    public static BinaryIdMapper defaultIdMapper() {
+        return DFLT_MAPPER.idMapper();
+    }
+
+    /**
+     * @return Name mapper used as default.
+     */
+    public static BinaryNameMapper defaultNameMapper() {
+        return DFLT_MAPPER.nameMapper();
+    }
+
+    /**
      * @param cls Class.
      */
     private void addSystemClassAffinityKey(Class<?> cls) {
@@ -502,7 +567,8 @@ public class BinaryContext {
         if (desc == null) {
             desc = registerClassDescriptor(cls, deserialize);
 
-            assert desc.typeId() == typeId;
+            assert desc.typeId() == typeId : "Duplicate typeId [typeId=" + typeId + ", cls=" + cls
+                + ", desc=" + desc + "]";
         }
 
         return desc;
@@ -526,7 +592,7 @@ public class BinaryContext {
                 clsName.hashCode(),
                 clsName,
                 null,
-                BinaryInternalIdMapper.defaultInstance(),
+                SIMPLE_NAME_LOWER_CASE_MAPPER,
                 null,
                 false,
                 true /* registered */
@@ -552,11 +618,13 @@ public class BinaryContext {
     private BinaryClassDescriptor registerUserClassDescriptor(Class<?> cls, boolean deserialize) {
         boolean registered;
 
-        String typeName = typeName(cls.getName());
+        final String clsName = cls.getName();
+
+        BinaryInternalMapper mapper = userTypeMapper(clsName);
 
-        BinaryIdMapper idMapper = userTypeIdMapper(typeName);
+        final String typeName = mapper.typeName(clsName);
 
-        int typeId = idMapper.typeId(typeName);
+        final int typeId = mapper.typeId(clsName);
 
         try {
             registered = marshCtx.registerClass(typeId, cls);
@@ -575,7 +643,7 @@ public class BinaryContext {
             typeId,
             typeName,
             affFieldName,
-            idMapper,
+            mapper,
             serializer,
             true,
             registered
@@ -595,7 +663,7 @@ public class BinaryContext {
 
         descByCls.put(cls, desc);
 
-        mappers.putIfAbsent(typeId, idMapper);
+        typeId2Mapper.putIfAbsent(typeId, mapper);
 
         return desc;
     }
@@ -656,9 +724,7 @@ public class BinaryContext {
      * @return Type ID.
      */
     public int typeId(String typeName) {
-        String typeName0 = typeName(typeName);
-
-        Integer id = predefinedTypeNames.get(typeName0);
+        Integer id = predefinedTypeNames.get(SIMPLE_NAME_LOWER_CASE_MAPPER.typeName(typeName));
 
         if (id != null)
             return id;
@@ -666,7 +732,9 @@ public class BinaryContext {
         if (marshCtx.isSystemType(typeName))
             return typeName.hashCode();
 
-        return userTypeIdMapper(typeName0).typeId(typeName0);
+        BinaryInternalMapper mapper = userTypeMapper(typeName);
+
+        return mapper.typeId(typeName);
     }
 
     /**
@@ -675,27 +743,107 @@ public class BinaryContext {
      * @return Field ID.
      */
     public int fieldId(int typeId, String fieldName) {
-        return userTypeIdMapper(typeId).fieldId(typeId, fieldName);
+        BinaryInternalMapper mapper = userTypeMapper(typeId);
+
+        return mapper.fieldId(typeId, fieldName);
     }
 
     /**
      * @param typeId Type ID.
      * @return Instance of ID mapper.
      */
-    public BinaryIdMapper userTypeIdMapper(int typeId) {
-        BinaryIdMapper idMapper = mappers.get(typeId);
+    public BinaryInternalMapper userTypeMapper(int typeId) {
+        BinaryInternalMapper mapper = typeId2Mapper.get(typeId);
 
-        return idMapper != null ? idMapper : BinaryInternalIdMapper.defaultInstance();
+        return mapper != null ? mapper : SIMPLE_NAME_LOWER_CASE_MAPPER;
     }
 
     /**
-     * @param typeName Type name.
+     * @param clsName Type name.
      * @return Instance of ID mapper.
      */
-    private BinaryIdMapper userTypeIdMapper(String typeName) {
-        BinaryIdMapper idMapper = typeMappers.get(typeName);
+    private BinaryInternalMapper userTypeMapper(String clsName) {
+        BinaryInternalMapper mapper = cls2Mappers.get(clsName);
+
+        if (mapper != null)
+            return mapper;
+
+        mapper = resolveMapper(clsName, igniteCfg.getBinaryConfiguration());
+
+        BinaryInternalMapper prevMap = cls2Mappers.putIfAbsent(clsName, mapper);
+
+        if (prevMap != null && !mapper.equals(prevMap))
+            throw new IgniteException("Different mappers [clsName=" + clsName + ", newMapper=" + mapper
+            + ", prevMap=" + prevMap + "]");
+
+        prevMap = typeId2Mapper.putIfAbsent(mapper.typeId(clsName), mapper);
 
-        return idMapper != null ? idMapper : BinaryInternalIdMapper.defaultInstance();
+        if (prevMap != null && !mapper.equals(prevMap))
+            throw new IgniteException("Different mappers [clsName=" + clsName + ", newMapper=" + mapper
+                + ", prevMap=" + prevMap + "]");
+
+        return mapper;
+    }
+
+    /**
+     * @param clsName Type name.
+     * @param cfg Binary configuration.
+     * @return Mapper according to configuration.
+     */
+    private static BinaryInternalMapper resolveMapper(String clsName, BinaryConfiguration cfg) {
+        assert clsName != null;
+
+        if (cfg == null)
+            return DFLT_MAPPER;
+
+        BinaryIdMapper globalIdMapper = cfg.getIdMapper();
+        BinaryNameMapper globalNameMapper = cfg.getNameMapper();
+
+        Collection<BinaryTypeConfiguration> typeCfgs = cfg.getTypeConfigurations();
+
+        if (typeCfgs != null) {
+            for (BinaryTypeConfiguration typeCfg : typeCfgs) {
+                String typeCfgName = typeCfg.getTypeName();
+
+                // Pattern.
+                if (typeCfgName != null && typeCfgName.endsWith(".*")) {
+                    String pkgName = typeCfgName.substring(0, typeCfgName.length() - 2);
+
+                    int dotIndex = clsName.lastIndexOf('.');
+
+                    if (dotIndex > 0) {
+                        String typePkgName = clsName.substring(0, dotIndex);
+
+                        if (pkgName.equals(typePkgName)) {
+                            // Resolve mapper.
+                            BinaryIdMapper idMapper = globalIdMapper;
+
+                            if (typeCfg.getIdMapper() != null)
+                                idMapper = typeCfg.getIdMapper();
+
+                            BinaryNameMapper nameMapper = globalNameMapper;
+
+                            if (typeCfg.getNameMapper() != null)
+                                nameMapper = typeCfg.getNameMapper();
+
+                            return resolveMapper(nameMapper, idMapper);
+                        }
+                    }
+                }
+            }
+        }
+
+        return resolveMapper(globalNameMapper, globalIdMapper);
+    }
+
+    /**
+     * @param clsName Class name.
+     * @return Type name.
+     */
+    public String userTypeName(String clsName) {
+        BinaryInternalMapper mapper = userTypeMapper(clsName);
+
+        return mapper.typeName(clsName);
     }
 
     /**
@@ -728,25 +876,25 @@ public class BinaryContext {
      * @return GridBinaryClassDescriptor.
      */
     public BinaryClassDescriptor registerPredefinedType(Class<?> cls, int id, String affFieldName) {
-        String typeName = typeName(cls.getName());
+        String simpleClsName = SIMPLE_NAME_LOWER_CASE_MAPPER.typeName(cls.getName());
 
         if (id == 0)
-            id = BinaryInternalIdMapper.defaultInstance().typeId(typeName);
+            id = SIMPLE_NAME_LOWER_CASE_MAPPER.typeId(simpleClsName);
 
         BinaryClassDescriptor desc = new BinaryClassDescriptor(
             this,
             cls,
             false,
             id,
-            typeName,
+            simpleClsName,
             affFieldName,
-            BinaryInternalIdMapper.defaultInstance(),
+            SIMPLE_NAME_LOWER_CASE_MAPPER,
             new BinaryReflectiveSerializer(),
             false,
             true /* registered */
         );
 
-        predefinedTypeNames.put(typeName, id);
+        predefinedTypeNames.put(simpleClsName, id);
         predefinedTypes.put(id, desc);
 
         descByCls.put(cls, desc);
@@ -759,7 +907,7 @@ public class BinaryContext {
 
     /**
      * @param clsName Class name.
-     * @param idMapper ID mapper.
+     * @param mapper ID mapper.
      * @param serializer Serializer.
      * @param affKeyFieldName Affinity key field name.
      * @param isEnum If enum.
@@ -767,12 +915,12 @@ public class BinaryContext {
      */
     @SuppressWarnings("ErrorNotRethrown")
     public void registerUserType(String clsName,
-        BinaryIdMapper idMapper,
+        BinaryInternalMapper mapper,
         @Nullable BinarySerializer serializer,
         @Nullable String affKeyFieldName,
         boolean isEnum)
         throws BinaryObjectException {
-        assert idMapper != null;
+        assert mapper != null;
 
         Class<?> cls = null;
 
@@ -783,15 +931,15 @@ public class BinaryContext {
             // No-op.
         }
 
-        String typeName = typeName(clsName);
+        String typeName = mapper.typeName(clsName);
 
-        int id = idMapper.typeId(typeName);
+        int id = mapper.typeId(clsName);
 
         //Workaround for IGNITE-1358
         if (predefinedTypes.get(id) != null)
             throw new BinaryObjectException("Duplicate type ID [clsName=" + clsName + ", id=" + id + ']');
 
-        if (mappers.put(id, idMapper) != null)
+        if (typeId2Mapper.put(id, mapper) != null)
             throw new BinaryObjectException("Duplicate type ID [clsName=" + clsName + ", id=" + id + ']');
 
         if (affKeyFieldName != null) {
@@ -799,7 +947,7 @@ public class BinaryContext {
                 throw new BinaryObjectException("Duplicate type ID [clsName=" + clsName + ", id=" + id + ']');
         }
 
-        typeMappers.put(typeName, idMapper);
+        cls2Mappers.put(clsName, mapper);
 
         Map<String, Integer> fieldsMeta = null;
         Collection<BinarySchema> schemas = null;
@@ -820,7 +968,7 @@ public class BinaryContext {
                 id,
                 typeName,
                 affKeyFieldName,
-                idMapper,
+                mapper,
                 serializer,
                 true,
                 true /* registered */
@@ -858,7 +1006,9 @@ public class BinaryContext {
     public BinaryFieldImpl createField(int typeId, String fieldName) {
         BinarySchemaRegistry schemaReg = schemaRegistry(typeId);
 
-        int fieldId = userTypeIdMapper(typeId).fieldId(typeId, fieldName);
+        BinaryInternalMapper mapper = userTypeMapper(typeId);
+
+        int fieldId = mapper.fieldId(typeId, fieldName);
 
         return new BinaryFieldImpl(typeId, schemaReg, fieldName, fieldId);
     }
@@ -954,43 +1104,6 @@ public class BinaryContext {
     }
 
     /**
-     * @param clsName Class name.
-     * @return Type name.
-     */
-    @SuppressWarnings("ResultOfMethodCallIgnored")
-    public static String typeName(String clsName) {
-        assert clsName != null;
-
-        int idx = clsName.lastIndexOf('$');
-
-        if (idx == clsName.length() - 1)
-            // This is a regular (not inner) class name that ends with '$'. Common use case for Scala classes.
-            idx = -1;
-        else if (idx >= 0) {
-            String typeName = clsName.substring(idx + 1);
-
-            try {
-                Integer.parseInt(typeName);
-
-                // This is an anonymous class. Don't cut off enclosing class name for it.
-                idx = -1;
-            }
-            catch (NumberFormatException ignore) {
-                // This is a lambda class.
-                if (clsName.indexOf("$$Lambda$") > 0)
-                    idx = -1;
-                else
-                    return typeName;
-            }
-        }
-
-        if (idx < 0)
-            idx = clsName.lastIndexOf('.');
-
-        return idx >= 0 ? clsName.substring(idx + 1) : clsName;
-    }
-
-    /**
      * Undeployment callback invoked when class loader is being undeployed.
      *
      * Some marshallers may want to clean their internal state that uses the undeployed class loader somehow.
@@ -1017,7 +1130,7 @@ public class BinaryContext {
          * Add type descriptor.
          *
          * @param clsName Class name.
-         * @param idMapper ID mapper.
+         * @param mapper Mapper.
          * @param serializer Serializer.
          * @param affKeyFieldName Affinity key field name.
          * @param isEnum Enum flag.
@@ -1025,14 +1138,14 @@ public class BinaryContext {
          * @throws BinaryObjectException If failed.
          */
         private void add(String clsName,
-            BinaryIdMapper idMapper,
+            BinaryInternalMapper mapper,
             BinarySerializer serializer,
             String affKeyFieldName,
             boolean isEnum,
             boolean canOverride)
             throws BinaryObjectException {
             TypeDescriptor desc = new TypeDescriptor(clsName,
-                idMapper,
+                mapper,
                 serializer,
                 affKeyFieldName,
                 isEnum,
@@ -1063,8 +1176,8 @@ public class BinaryContext {
         /** Class name. */
         private final String clsName;
 
-        /** ID mapper. */
-        private BinaryIdMapper idMapper;
+        /** Mapper. */
+        private BinaryInternalMapper mapper;
 
         /** Serializer. */
         private BinarySerializer serializer;
@@ -1082,16 +1195,16 @@ public class BinaryContext {
          * Constructor.
          *
          * @param clsName Class name.
-         * @param idMapper ID mapper.
+         * @param mapper ID mapper.
          * @param serializer Serializer.
          * @param affKeyFieldName Affinity key field name.
          * @param isEnum Enum type.
          * @param canOverride Whether this descriptor can be override.
          */
-        private TypeDescriptor(String clsName, BinaryIdMapper idMapper, BinarySerializer serializer,
-            String affKeyFieldName, boolean isEnum, boolean canOverride) {
+        private TypeDescriptor(String clsName, BinaryInternalMapper mapper,
+            BinarySerializer serializer, String affKeyFieldName, boolean isEnum, boolean canOverride) {
             this.clsName = clsName;
-            this.idMapper = idMapper;
+            this.mapper = mapper;
             this.serializer = serializer;
             this.affKeyFieldName = affKeyFieldName;
             this.isEnum = isEnum;
@@ -1108,7 +1221,7 @@ public class BinaryContext {
             assert clsName.equals(other.clsName);
 
             if (canOverride) {
-                idMapper = other.idMapper;
+                mapper = other.mapper;
                 serializer = other.serializer;
                 affKeyFieldName = other.affKeyFieldName;
                 isEnum = other.isEnum;

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
index 180e20a..fd4a4d8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
@@ -17,6 +17,11 @@
 
 package org.apache.ignite.internal.binary;
 
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryObjectBuilder;
@@ -31,12 +36,6 @@ import org.apache.ignite.plugin.extensions.communication.MessageReader;
 import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 import org.jetbrains.annotations.Nullable;
 
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.nio.ByteBuffer;
-
 /**
  * Binary enum object.
  */

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryInternalIdMapper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryInternalIdMapper.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryInternalIdMapper.java
deleted file mode 100644
index 14d7d6f..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryInternalIdMapper.java
+++ /dev/null
@@ -1,161 +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.binary;
-
-import org.apache.ignite.binary.BinaryIdMapper;
-import org.apache.ignite.binary.BinaryObjectException;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Internal ID mapper. Mimics ID mapper interface, but provides default implementation and offers slightly better
- * performance on micro-level in default case because it doesn't need virtual calls.
- */
-public class BinaryInternalIdMapper implements BinaryIdMapper {
-    /** Maximum lower-case character. */
-    private static final char MAX_LOWER_CASE_CHAR = 0x7e;
-
-    /** Cached lower-case characters. */
-    private static final char[] LOWER_CASE_CHARS;
-
-    /** Default implementation. */
-    private static final BinaryInternalIdMapper DFLT = new BinaryInternalIdMapper();
-
-    /**
-     * Static initializer.
-     */
-    static {
-        LOWER_CASE_CHARS = new char[MAX_LOWER_CASE_CHAR + 1];
-
-        for (char c = 0; c <= MAX_LOWER_CASE_CHAR; c++)
-            LOWER_CASE_CHARS[c] = Character.toLowerCase(c);
-    }
-
-    /**
-     * Get default instance.
-     *
-     * @return Default instance.
-     */
-    public static BinaryInternalIdMapper defaultInstance() {
-        return DFLT;
-    }
-
-    /**
-     * Create internal mapper.
-     *
-     * @param mapper Public mapper.
-     * @return Internal mapper.
-     */
-    public static BinaryInternalIdMapper create(@Nullable BinaryIdMapper mapper) {
-        return mapper == null ? DFLT : new Wrapper(mapper);
-    }
-
-    /**
-     * Private constructor.
-     */
-    protected BinaryInternalIdMapper() {
-        // No-op.
-    }
-
-    /**
-     * Get type ID.
-     *
-     * @param typeName Type name.
-     * @return Type ID.
-     */
-    public int typeId(String typeName) {
-        assert typeName != null;
-
-        return lowerCaseHashCode(typeName, true);
-    }
-
-    /**
-     * Get field ID.
-     *
-     * @param typeId Type ID.
-     * @param fieldName Field name.
-     * @return Field ID.
-     */
-    public int fieldId(int typeId, String fieldName) {
-        assert fieldName != null;
-
-        return lowerCaseHashCode(fieldName, false);
-    }
-
-    /**
-     * Routine to calculate string hash code an
-     *
-     * @param str String.
-     * @param type {@code True} if this is type name, false otherwise.
-     * @return Hash code for given string converted to lower case.
-     */
-    private static int lowerCaseHashCode(String str, boolean type) {
-        int len = str.length();
-
-        int h = 0;
-
-        for (int i = 0; i < len; i++) {
-            int c = str.charAt(i);
-
-            c = c <= MAX_LOWER_CASE_CHAR ? LOWER_CASE_CHARS[c] : Character.toLowerCase(c);
-
-            h = 31 * h + c;
-        }
-
-        if (h != 0)
-            return h;
-        else {
-            String what = type ? "type" : "field";
-
-            throw new BinaryObjectException("Default binary ID mapper resolved " + what + " ID to zero " +
-                "(either change " + what + "'s name or use custom ID mapper) [name=" + str + ']');
-        }
-    }
-
-    /**
-     * Wrapping ID mapper.
-     */
-    private static class Wrapper extends BinaryInternalIdMapper {
-        /** Delegate. */
-        private final BinaryIdMapper mapper;
-
-        /**
-         * Constructor.
-         *
-         * @param mapper Delegate.
-         */
-        private Wrapper(BinaryIdMapper mapper) {
-            assert mapper != null;
-
-            this.mapper = mapper;
-        }
-
-        /** {@inheritDoc} */
-        @Override public int typeId(String typeName) {
-            int id = mapper.typeId(typeName);
-
-            return id != 0 ? id : super.typeId(typeName);
-        }
-
-        /** {@inheritDoc} */
-        @Override public int fieldId(int typeId, String fieldName) {
-            int id = mapper.fieldId(typeId, fieldName);
-
-            return id != 0 ? id : super.fieldId(typeId, fieldName);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryInternalMapper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryInternalMapper.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryInternalMapper.java
new file mode 100644
index 0000000..1adc8c7
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryInternalMapper.java
@@ -0,0 +1,131 @@
+/*
+ * 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.BinaryIdMapper;
+import org.apache.ignite.binary.BinaryNameMapper;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * Internal binary mapper.
+ */
+public class BinaryInternalMapper {
+    /** */
+    private final BinaryNameMapper nameMapper;
+
+    /** */
+    private final BinaryIdMapper idMapper;
+
+    /** */
+    private boolean checkOnZeroId;
+
+    /**
+     * @param nameMapper Name mapper.
+     * @param idMapper Id mapper.
+     * @param checkOnZeroId Whether checks on zero id or not.
+     */
+    public BinaryInternalMapper(BinaryNameMapper nameMapper, BinaryIdMapper idMapper, boolean checkOnZeroId) {
+        assert nameMapper != null;
+        assert idMapper != null;
+
+        this.nameMapper = nameMapper;
+        this.idMapper = idMapper;
+        this.checkOnZeroId = checkOnZeroId;
+    }
+
+    /**
+     * @return Name mapper.
+     */
+    public BinaryNameMapper nameMapper() {
+        return nameMapper;
+    }
+
+    /**
+     * @return ID mapper.
+     */
+    public BinaryIdMapper idMapper() {
+        return idMapper;
+    }
+
+    /**
+     * @param clsName Class name.
+     * @return Type ID.
+     */
+    public int typeId(String clsName) {
+        int id = idMapper.typeId(nameMapper.typeName(clsName));
+
+        if (!checkOnZeroId)
+            return id;
+
+        return id != 0 ? id : BinaryContext.SIMPLE_NAME_LOWER_CASE_MAPPER.typeId(clsName);
+    }
+
+    /**
+     * @param typeId Type ID.
+     * @param fieldName Field name.
+     * @return Field ID.
+     */
+    public int fieldId(int typeId, String fieldName) {
+        int id = idMapper.fieldId(typeId, nameMapper.fieldName(fieldName));
+
+        if (!checkOnZeroId)
+            return id;
+
+        return id != 0 ? id : BinaryContext.SIMPLE_NAME_LOWER_CASE_MAPPER.fieldId(typeId, fieldName);
+    }
+
+    /**
+     * @param clsName Class name.
+     * @return Type name.
+     */
+    public String typeName(String clsName) {
+        return nameMapper.typeName(clsName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (!(o instanceof BinaryInternalMapper))
+            return false;
+
+        BinaryInternalMapper mapper = (BinaryInternalMapper)o;
+
+        return checkOnZeroId == mapper.checkOnZeroId 
+            && idMapper.equals(mapper.idMapper) 
+            && nameMapper.equals(mapper.nameMapper);
+
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        int res = nameMapper.hashCode();
+        
+        res = 31 * res + idMapper.hashCode();
+        
+        res = 31 * res + (checkOnZeroId ? 1 : 0);
+        
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(BinaryInternalMapper.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadataCollector.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadataCollector.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadataCollector.java
index 54f2b13..e8574ef 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadataCollector.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadataCollector.java
@@ -17,12 +17,6 @@
 
 package org.apache.ignite.internal.binary;
 
-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;
@@ -33,6 +27,10 @@ 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.
@@ -44,8 +42,8 @@ class BinaryMetadataCollector implements BinaryWriter {
     /** Type name. */
     private final String typeName;
 
-    /** ID mapper. */
-    private final BinaryIdMapper idMapper;
+    /** Name mapper. */
+    private final BinaryInternalMapper mapper;
 
     /** Collected metadata. */
     private final Map<String, Integer> meta = new HashMap<>();
@@ -58,12 +56,12 @@ class BinaryMetadataCollector implements BinaryWriter {
      *
      * @param typeId Type ID.
      * @param typeName Type name.
-     * @param idMapper ID mapper.
+     * @param mapper Name mapper.
      */
-    BinaryMetadataCollector(int typeId, String typeName, BinaryIdMapper idMapper) {
+    BinaryMetadataCollector(int typeId, String typeName, BinaryInternalMapper mapper) {
         this.typeId = typeId;
         this.typeName = typeName;
-        this.idMapper = idMapper;
+        this.mapper = mapper;
     }
 
     /**
@@ -272,6 +270,6 @@ class BinaryMetadataCollector implements BinaryWriter {
             );
         }
 
-        schemaBuilder.addField(idMapper.fieldId(typeId, name));
+        schemaBuilder.addField(mapper.fieldId(typeId, name));
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java
index 8f9cc92..607dabc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java
@@ -26,9 +26,7 @@ import java.util.Collection;
 import java.util.Date;
 import java.util.Map;
 import java.util.UUID;
-
 import org.apache.ignite.binary.BinaryCollectionFactory;
-import org.apache.ignite.binary.BinaryIdMapper;
 import org.apache.ignite.binary.BinaryInvalidTypeException;
 import org.apache.ignite.binary.BinaryMapFactory;
 import org.apache.ignite.binary.BinaryObject;
@@ -40,6 +38,7 @@ import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.BINARY_OBJ;
 import static org.apache.ignite.internal.binary.GridBinaryMarshaller.BOOLEAN;
 import static org.apache.ignite.internal.binary.GridBinaryMarshaller.BOOLEAN_ARR;
 import static org.apache.ignite.internal.binary.GridBinaryMarshaller.BYTE;
@@ -69,7 +68,6 @@ import static org.apache.ignite.internal.binary.GridBinaryMarshaller.NULL;
 import static org.apache.ignite.internal.binary.GridBinaryMarshaller.OBJ;
 import static org.apache.ignite.internal.binary.GridBinaryMarshaller.OBJ_ARR;
 import static org.apache.ignite.internal.binary.GridBinaryMarshaller.OPTM_MARSH;
-import static org.apache.ignite.internal.binary.GridBinaryMarshaller.BINARY_OBJ;
 import static org.apache.ignite.internal.binary.GridBinaryMarshaller.SHORT;
 import static org.apache.ignite.internal.binary.GridBinaryMarshaller.SHORT_ARR;
 import static org.apache.ignite.internal.binary.GridBinaryMarshaller.STRING;
@@ -115,8 +113,8 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
     /** Footer end. */
     private final int footerLen;
 
-    /** ID mapper. */
-    private final BinaryIdMapper idMapper;
+    /** Mapper. */
+    private final BinaryInternalMapper mapper;
 
     /** Schema Id. */
     private final int schemaId;
@@ -247,7 +245,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
                 dataStart = start + DFLT_HDR_LEN;
             }
 
-            idMapper = userType ? ctx.userTypeIdMapper(typeId) : BinaryInternalIdMapper.defaultInstance();
+            mapper = userType ? ctx.userTypeMapper(typeId) : BinaryContext.defaultMapper();
             schema = BinaryUtils.hasSchema(flags) ? getOrCreateSchema() : null;
         }
         else {
@@ -256,7 +254,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
             rawOff = 0;
             footerStart = 0;
             footerLen = 0;
-            idMapper = null;
+            mapper = null;
             schemaId = 0;
             userType = false;
             fieldIdLen = 0;
@@ -1653,7 +1651,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
     private int fieldId(String name) {
         assert name != null;
 
-        return idMapper.fieldId(typeId, name);
+        return mapper.fieldId(typeId, name);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java
index 6e05b40..877a2db 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java
@@ -17,16 +17,6 @@
 
 package org.apache.ignite.internal.binary;
 
-import org.apache.ignite.IgniteCheckedException;
-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.apache.ignite.internal.binary.streams.BinaryOutputStream;
-import org.apache.ignite.internal.binary.streams.BinaryHeapOutputStream;
-import org.apache.ignite.internal.util.typedef.internal.A;
-import org.jetbrains.annotations.Nullable;
-
 import java.io.IOException;
 import java.io.ObjectOutput;
 import java.lang.reflect.InvocationTargetException;
@@ -37,6 +27,14 @@ import java.util.Collection;
 import java.util.Date;
 import java.util.Map;
 import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryRawWriter;
+import org.apache.ignite.binary.BinaryWriter;
+import org.apache.ignite.internal.binary.streams.BinaryHeapOutputStream;
+import org.apache.ignite.internal.binary.streams.BinaryOutputStream;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.jetbrains.annotations.Nullable;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 
@@ -77,8 +75,8 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
     /** Amount of written fields. */
     private int fieldCnt;
 
-    /** ID mapper. */
-    private BinaryIdMapper idMapper;
+    /** */
+    private BinaryInternalMapper mapper;
 
     /**
      * @param ctx Context.
@@ -1642,10 +1640,12 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
         if (rawOffPos != 0)
             throw new BinaryObjectException("Individual field can't be written after raw writer is acquired.");
 
-        if (idMapper == null)
-            idMapper = ctx.userTypeIdMapper(typeId);
+        if (mapper == null)
+            mapper = ctx.userTypeMapper(typeId);
+
+        assert mapper != null;
 
-        int id = idMapper.fieldId(typeId, fieldName);
+        int id = mapper.fieldId(typeId, fieldName);
 
         writeFieldId(id);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderSerializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderSerializer.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderSerializer.java
index a095242..6e5a621 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderSerializer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderSerializer.java
@@ -17,17 +17,15 @@
 
 package org.apache.ignite.internal.binary.builder;
 
+import java.util.Collection;
+import java.util.IdentityHashMap;
+import java.util.Map;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.internal.binary.BinaryMetadata;
 import org.apache.ignite.internal.binary.BinaryObjectExImpl;
+import org.apache.ignite.internal.binary.BinaryUtils;
 import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 import org.apache.ignite.internal.binary.GridBinaryMarshaller;
-import org.apache.ignite.internal.binary.BinaryContext;
-import org.apache.ignite.internal.binary.BinaryUtils;
-
-import java.util.Collection;
-import java.util.IdentityHashMap;
-import java.util.Map;
 
 /**
  *
@@ -100,8 +98,10 @@ class BinaryBuilderSerializer {
         }
 
         if (val.getClass().isEnum()) {
-            String typeName = BinaryContext.typeName(val.getClass().getName());
-            int typeId = writer.context().typeId(typeName);
+            String clsName = val.getClass().getName();
+
+            int typeId = writer.context().typeId(clsName);
+            String typeName = writer.context().userTypeName(clsName);
 
             BinaryMetadata meta = new BinaryMetadata(typeId, typeName, null, null, null, true);
             writer.context().updateMetadata(typeId, meta);

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java
index 8353cdb..e2290d6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java
@@ -90,7 +90,7 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
      * @param ctx Binary context.
      */
     public BinaryObjectBuilderImpl(BinaryContext ctx, String clsName) {
-        this(ctx, ctx.typeId(clsName), BinaryContext.typeName(clsName));
+        this(ctx, ctx.typeId(clsName), ctx.userTypeName(clsName));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
index c9d6dad..a21331e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
@@ -22,9 +22,10 @@ import java.io.IOException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.Map;
-import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CountDownLatch;
@@ -39,16 +40,20 @@ import javax.cache.processor.MutableEntry;
 import org.apache.ignite.IgniteBinary;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.binary.BinaryBasicNameMapper;
 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.binary.BinaryTypeConfiguration;
 import org.apache.ignite.cache.CacheEntryEventSerializableFilter;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.cluster.ClusterTopologyException;
+import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.IgniteNodeAttributes;
+import org.apache.ignite.internal.binary.BinaryContext;
 import org.apache.ignite.internal.binary.BinaryEnumObjectImpl;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
 import org.apache.ignite.internal.binary.BinaryMetadata;
@@ -57,12 +62,12 @@ import org.apache.ignite.internal.binary.BinaryObjectEx;
 import org.apache.ignite.internal.binary.BinaryObjectImpl;
 import org.apache.ignite.internal.binary.BinaryObjectOffheapImpl;
 import org.apache.ignite.internal.binary.BinaryTypeImpl;
-import org.apache.ignite.internal.binary.GridBinaryMarshaller;
-import org.apache.ignite.internal.binary.BinaryContext;
 import org.apache.ignite.internal.binary.BinaryUtils;
+import org.apache.ignite.internal.binary.GridBinaryMarshaller;
 import org.apache.ignite.internal.binary.builder.BinaryObjectBuilderImpl;
 import org.apache.ignite.internal.binary.streams.BinaryInputStream;
 import org.apache.ignite.internal.binary.streams.BinaryOffheapInputStream;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheEntryPredicate;
 import org.apache.ignite.internal.processors.cache.CacheEntryPredicateAdapter;
@@ -92,11 +97,16 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteClosure;
+import org.apache.ignite.lang.IgniteProductVersion;
 import org.apache.ignite.marshaller.Marshaller;
+import org.apache.ignite.spi.IgniteNodeValidationResult;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 import sun.misc.Unsafe;
 
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK;
+import static org.apache.ignite.IgniteSystemProperties.getBoolean;
+
 /**
  * Binary processor implementation.
  */
@@ -106,6 +116,9 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
     private static final Unsafe UNSAFE = GridUnsafe.unsafe();
 
     /** */
+    public static final IgniteProductVersion BINARY_CFG_CHECK_SINCE = IgniteProductVersion.fromString("1.5.6");
+
+    /** */
     private final CountDownLatch startLatch = new CountDownLatch(1);
 
     /** */
@@ -204,16 +217,46 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
                 }
             };
 
-            BinaryMarshaller pMarh0 = (BinaryMarshaller)marsh;
+            BinaryMarshaller bMarsh0 = (BinaryMarshaller)marsh;
 
             binaryCtx = new BinaryContext(metaHnd, ctx.config(), ctx.log(BinaryContext.class));
 
-            IgniteUtils.invoke(BinaryMarshaller.class, pMarh0, "setBinaryContext", binaryCtx,
-                ctx.config());
+            IgniteUtils.invoke(BinaryMarshaller.class, bMarsh0, "setBinaryContext", binaryCtx, ctx.config());
 
             binaryMarsh = new GridBinaryMarshaller(binaryCtx);
 
             binaries = new IgniteBinaryImpl(ctx, this);
+
+            if (!getBoolean(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK)) {
+                BinaryConfiguration bCfg = ctx.config().getBinaryConfiguration();
+
+                if (bCfg != null) {
+                    Map<String, Object> map = new HashMap<>();
+
+                    map.put("globIdMapper", bCfg.getIdMapper() != null ? bCfg.getIdMapper().getClass().getName() : null);
+                    map.put("globSerializer", bCfg.getSerializer() != null ? bCfg.getSerializer().getClass() : null);
+                    map.put("compactFooter", bCfg.isCompactFooter());
+
+                    if (bCfg.getTypeConfigurations() != null) {
+                        Map<Object, Object> typeCfgsMap = new HashMap<>();
+
+                        for (BinaryTypeConfiguration c : bCfg.getTypeConfigurations()) {
+                            typeCfgsMap.put(
+                                c.getTypeName() != null,
+                                Arrays.asList(
+                                    c.getIdMapper() != null ? c.getIdMapper().getClass() : null,
+                                    c.getSerializer() != null ? c.getSerializer().getClass() : null,
+                                    c.isEnum()
+                                )
+                            );
+                        }
+
+                        map.put("typeCfgs", typeCfgsMap);
+                    }
+
+                    ctx.addNodeAttribute(IgniteNodeAttributes.ATTR_BINARY_CONFIGURATION, map);
+                }
+            }
         }
     }
 
@@ -294,6 +337,37 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
     }
 
     /** {@inheritDoc} */
+    @Override public void onKernalStart() throws IgniteCheckedException {
+        super.onKernalStart();
+
+        if (!getBoolean(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK) && marsh instanceof BinaryMarshaller) {
+            BinaryConfiguration bcfg = ctx.config().getBinaryConfiguration();
+
+            for (ClusterNode rmtNode : ctx.discovery().remoteNodes()) {
+                if (rmtNode.version().compareTo(BINARY_CFG_CHECK_SINCE) < 0) {
+                    if (bcfg == null || bcfg.getNameMapper() == null) {
+                        throw new IgniteCheckedException("When BinaryMarshaller is used and topology contains old " +
+                            "nodes, then " + BinaryBasicNameMapper.class.getName() + " mapper have to be set " +
+                            "explicitely into binary configuration and 'simpleName' property of the mapper " +
+                            "have to be set to 'true'.");
+                    }
+
+                    if (!(bcfg.getNameMapper() instanceof BinaryBasicNameMapper)
+                        || !((BinaryBasicNameMapper)bcfg.getNameMapper()).isSimpleName()) {
+                        U.quietAndWarn(log, "When BinaryMarshaller is used and topology contains old" +
+                            " nodes, it's strongly recommended, to set " + BinaryBasicNameMapper.class.getName() +
+                            " mapper into binary configuration explicitely " +
+                            " and 'simpleName' property of the mapper set to 'true' (fix configuration or set " +
+                            "-D" + IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK + "=true system property).");
+                    }
+
+                    break;
+                }
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public void onKernalStop(boolean cancel) {
         super.onKernalStop(cancel);
 
@@ -578,10 +652,10 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
 
     /** {@inheritDoc} */
     @Override public BinaryObject buildEnum(String typeName, int ord) throws IgniteException {
-        typeName = BinaryContext.typeName(typeName);
-
         int typeId = binaryCtx.typeId(typeName);
 
+        typeName = binaryCtx.userTypeName(typeName);
+
         updateMetadata(typeId, typeName, null, null, true);
 
         return new BinaryEnumObjectImpl(binaryCtx, typeId, null, ord);
@@ -784,6 +858,37 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
         return marshalToBinary(obj);
     }
 
+    /** {@inheritDoc} */
+    @Nullable @Override public IgniteNodeValidationResult validateNode(ClusterNode rmtNode) {
+        IgniteNodeValidationResult res = super.validateNode(rmtNode);
+
+        if (res != null)
+            return res;
+
+        if (getBoolean(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK) || !(marsh instanceof BinaryMarshaller))
+            return null;
+
+        Object rmtBinaryCfg = rmtNode.attribute(IgniteNodeAttributes.ATTR_BINARY_CONFIGURATION);
+
+        if (rmtNode.version().compareTo(BINARY_CFG_CHECK_SINCE) < 0)
+            return null;
+
+        ClusterNode locNode = ctx.discovery().localNode();
+
+        Object locBinaryCfg = locNode.attribute(IgniteNodeAttributes.ATTR_BINARY_CONFIGURATION);
+
+        if (!F.eq(locBinaryCfg, rmtBinaryCfg)) {
+            String msg = "Local node's binary configuration is not equal to remote node's binary configuration " +
+                "[locNodeId=%s, rmtNodeId=%s, locBinaryCfg=%s, rmtBinaryCfg=%s]";
+
+            return new IgniteNodeValidationResult(rmtNode.id(),
+                String.format(msg, locNode.id(), rmtNode.id(), locBinaryCfg, rmtBinaryCfg),
+                String.format(msg, rmtNode.id(), locNode.id(), rmtBinaryCfg, locBinaryCfg));
+        }
+
+        return null;
+    }
+
     /**
      * Processor responsible for metadata update.
      */


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

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderSelfTest.java
deleted file mode 100644
index 3b6d0b0..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderSelfTest.java
+++ /dev/null
@@ -1,1108 +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.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.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.binary.mutabletest.GridBinaryTestClasses;
-import org.apache.ignite.internal.binary.builder.BinaryObjectBuilderImpl;
-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 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);
-
-        BinaryTypeConfiguration customTypeCfg = new BinaryTypeConfiguration();
-
-        customTypeCfg.setTypeName(CustomIdMapper.class.getName());
-        customTypeCfg.setIdMapper(new BinaryIdMapper() {
-            @Override public int typeId(String clsName) {
-                return ~BinaryInternalIdMapper.defaultInstance().typeId(clsName);
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                return typeId + ~BinaryInternalIdMapper.defaultInstance().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));
-
-        cfg.setBinaryConfiguration(bCfg);
-
-        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() {
-        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("class".hashCode(), 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("class".hashCode(), 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("class".hashCode(), 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("class".hashCode(), 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("class".hashCode(), 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("class".hashCode(), 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("class".hashCode(), po.type().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.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("class".hashCode(), 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("class".hashCode(), 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("class".hashCode(), 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("class".hashCode(), 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("class".hashCode(), 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("class".hashCode(), 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("class".hashCode(), 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("class".hashCode(), 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("class".hashCode(), 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("class".hashCode(), 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("class".hashCode(), 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("class".hashCode(), 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("class".hashCode(), 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("class".hashCode(), 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("class".hashCode(), 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("class".hashCode(), 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("class".hashCode(), 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("class".hashCode(), 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("class".hashCode(), 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("class".hashCode(), 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 po = builder.build();
-
-        assertEquals("value".hashCode(), po.type().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(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;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderSimpleNameLowerCaseMappersSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderSimpleNameLowerCaseMappersSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderSimpleNameLowerCaseMappersSelfTest.java
new file mode 100644
index 0000000..da87072
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderSimpleNameLowerCaseMappersSelfTest.java
@@ -0,0 +1,41 @@
+/*
+ * 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.binary.BinaryBasicNameMapper;
+import org.apache.ignite.configuration.BinaryConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+
+/**
+ * Binary builder test.
+ */
+@SuppressWarnings("ResultOfMethodCallIgnored")
+public class BinaryObjectBuilderSimpleNameLowerCaseMappersSelfTest extends BinaryObjectBuilderDefaultMappersSelfTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        BinaryConfiguration bCfg = cfg.getBinaryConfiguration();
+
+        bCfg.setIdMapper(new BinaryBasicIdMapper(true));
+        bCfg.setNameMapper(new BinaryBasicNameMapper(true));
+
+        return cfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/test/java/org/apache/ignite/internal/binary/BinarySimpleNameTestPropertySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinarySimpleNameTestPropertySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinarySimpleNameTestPropertySelfTest.java
new file mode 100644
index 0000000..8f2df9e
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinarySimpleNameTestPropertySelfTest.java
@@ -0,0 +1,94 @@
+/*
+ * 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.IgniteBinary;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.binary.BinaryObjectBuilder;
+import org.apache.ignite.testframework.config.GridTestProperties;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.testframework.config.GridTestProperties.BINARY_MARSHALLER_USE_SIMPLE_NAME_MAPPER;
+import static org.apache.ignite.testframework.config.GridTestProperties.MARSH_CLASS_NAME;
+
+/**
+ * Tests testing framewrok, epecially BINARY_MARSHALLER_USE_SIMPLE_NAME_MAPPER test property.
+ */
+public class BinarySimpleNameTestPropertySelfTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPropertyEnabled() throws Exception {
+        String useSimpleNameBackup = GridTestProperties.getProperty(BINARY_MARSHALLER_USE_SIMPLE_NAME_MAPPER);
+
+        try {
+            GridTestProperties.setProperty(BINARY_MARSHALLER_USE_SIMPLE_NAME_MAPPER, "true");
+
+            checkProperty("TestClass");
+        }
+        finally {
+            if (useSimpleNameBackup != null)
+                GridTestProperties.setProperty(BINARY_MARSHALLER_USE_SIMPLE_NAME_MAPPER, "true");
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPropertyDisabled() throws Exception {
+        checkProperty("org.ignite.test.TestClass");
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBinaryDisabled() throws Exception {
+        assertNull(startGrid().binary());
+    }
+
+    /**
+     * @param expTypeName Type name.
+     * @throws Exception If failed.
+     */
+    private void checkProperty(String expTypeName) throws Exception {
+        String marshBackup = GridTestProperties.getProperty(MARSH_CLASS_NAME);
+
+        try {
+            GridTestProperties.setProperty(MARSH_CLASS_NAME, BinaryMarshaller.class.getName());
+
+            IgniteBinary binary = startGrid().binary();
+
+            BinaryObjectBuilder builder = binary.builder("org.ignite.test.TestClass");
+
+            BinaryObject bObj = builder.build();
+
+            assertEquals(expTypeName, bObj.type().typeName());
+        }
+        finally {
+            if (marshBackup != null)
+                GridTestProperties.setProperty(MARSH_CLASS_NAME, marshBackup);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryMetaDataSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryMetaDataSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryMetaDataSelfTest.java
deleted file mode 100644
index 5c5d6c3..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryMetaDataSelfTest.java
+++ /dev/null
@@ -1,371 +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.binary;
-
-import java.math.BigDecimal;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Date;
-import java.util.HashMap;
-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.binary.BinaryObjectException;
-import org.apache.ignite.binary.Binarylizable;
-import org.apache.ignite.binary.BinaryType;
-import org.apache.ignite.binary.BinaryObject;
-import org.apache.ignite.binary.BinaryRawWriter;
-import org.apache.ignite.binary.BinaryReader;
-import org.apache.ignite.binary.BinaryWriter;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-
-/**
- * Binary meta data test.
- */
-public class GridBinaryMetaDataSelfTest extends GridCommonAbstractTest {
-    /** */
-    private static int idx;
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        BinaryConfiguration bCfg = new BinaryConfiguration();
-
-        bCfg.setClassNames(Arrays.asList(TestObject1.class.getName(), TestObject2.class.getName()));
-
-        cfg.setBinaryConfiguration(bCfg);
-
-        cfg.setMarshaller(new BinaryMarshaller());
-
-        CacheConfiguration ccfg = new CacheConfiguration();
-
-        cfg.setCacheConfiguration(ccfg);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        idx = 0;
-
-        startGrid();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        stopGrid();
-    }
-
-    /**
-     * @return Binaries API.
-     */
-    protected IgniteBinary binaries() {
-        return grid().binary();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testGetAll() throws Exception {
-        binaries().toBinary(new TestObject2());
-
-        Collection<BinaryType> metas = binaries().types();
-
-        assertEquals(2, metas.size());
-
-        for (BinaryType meta : metas) {
-            Collection<String> fields;
-
-            switch (meta.typeName()) {
-                case "TestObject1":
-                    fields = meta.fieldNames();
-
-                    assertEquals(7, fields.size());
-
-                    assertTrue(fields.contains("intVal"));
-                    assertTrue(fields.contains("strVal"));
-                    assertTrue(fields.contains("arrVal"));
-                    assertTrue(fields.contains("obj1Val"));
-                    assertTrue(fields.contains("obj2Val"));
-                    assertTrue(fields.contains("decVal"));
-                    assertTrue(fields.contains("decArrVal"));
-
-                    assertEquals("int", meta.fieldTypeName("intVal"));
-                    assertEquals("String", meta.fieldTypeName("strVal"));
-                    assertEquals("byte[]", meta.fieldTypeName("arrVal"));
-                    assertEquals("Object", meta.fieldTypeName("obj1Val"));
-                    assertEquals("Object", meta.fieldTypeName("obj2Val"));
-                    assertEquals("decimal", meta.fieldTypeName("decVal"));
-                    assertEquals("decimal[]", meta.fieldTypeName("decArrVal"));
-
-                    break;
-
-                case "TestObject2":
-                    fields = meta.fieldNames();
-
-                    assertEquals(7, fields.size());
-
-                    assertTrue(fields.contains("boolVal"));
-                    assertTrue(fields.contains("dateVal"));
-                    assertTrue(fields.contains("uuidArrVal"));
-                    assertTrue(fields.contains("objVal"));
-                    assertTrue(fields.contains("mapVal"));
-                    assertTrue(fields.contains("decVal"));
-                    assertTrue(fields.contains("decArrVal"));
-
-                    assertEquals("boolean", meta.fieldTypeName("boolVal"));
-                    assertEquals("Date", meta.fieldTypeName("dateVal"));
-                    assertEquals("UUID[]", meta.fieldTypeName("uuidArrVal"));
-                    assertEquals("Object", meta.fieldTypeName("objVal"));
-                    assertEquals("Map", meta.fieldTypeName("mapVal"));
-                    assertEquals("decimal", meta.fieldTypeName("decVal"));
-                    assertEquals("decimal[]", meta.fieldTypeName("decArrVal"));
-
-                    break;
-
-                default:
-                    assert false : meta.typeName();
-            }
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testNoConfiguration() throws Exception {
-        binaries().toBinary(new TestObject3());
-
-        assertNotNull(binaries().type(TestObject3.class));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testReflection() throws Exception {
-        BinaryType meta = binaries().type(TestObject1.class);
-
-        assertNotNull(meta);
-
-        assertEquals("TestObject1", meta.typeName());
-
-        Collection<String> fields = meta.fieldNames();
-
-        assertEquals(7, fields.size());
-
-        assertTrue(fields.contains("intVal"));
-        assertTrue(fields.contains("strVal"));
-        assertTrue(fields.contains("arrVal"));
-        assertTrue(fields.contains("obj1Val"));
-        assertTrue(fields.contains("obj2Val"));
-        assertTrue(fields.contains("decVal"));
-        assertTrue(fields.contains("decArrVal"));
-
-        assertEquals("int", meta.fieldTypeName("intVal"));
-        assertEquals("String", meta.fieldTypeName("strVal"));
-        assertEquals("byte[]", meta.fieldTypeName("arrVal"));
-        assertEquals("Object", meta.fieldTypeName("obj1Val"));
-        assertEquals("Object", meta.fieldTypeName("obj2Val"));
-        assertEquals("decimal", meta.fieldTypeName("decVal"));
-        assertEquals("decimal[]", meta.fieldTypeName("decArrVal"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testBinaryMarshalAware() throws Exception {
-        binaries().toBinary(new TestObject2());
-
-        BinaryType meta = binaries().type(TestObject2.class);
-
-        assertNotNull(meta);
-
-        assertEquals("TestObject2", meta.typeName());
-
-        Collection<String> fields = meta.fieldNames();
-
-        assertEquals(7, fields.size());
-
-        assertTrue(fields.contains("boolVal"));
-        assertTrue(fields.contains("dateVal"));
-        assertTrue(fields.contains("uuidArrVal"));
-        assertTrue(fields.contains("objVal"));
-        assertTrue(fields.contains("mapVal"));
-        assertTrue(fields.contains("decVal"));
-        assertTrue(fields.contains("decArrVal"));
-
-        assertEquals("boolean", meta.fieldTypeName("boolVal"));
-        assertEquals("Date", meta.fieldTypeName("dateVal"));
-        assertEquals("UUID[]", meta.fieldTypeName("uuidArrVal"));
-        assertEquals("Object", meta.fieldTypeName("objVal"));
-        assertEquals("Map", meta.fieldTypeName("mapVal"));
-        assertEquals("decimal", meta.fieldTypeName("decVal"));
-        assertEquals("decimal[]", meta.fieldTypeName("decArrVal"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMerge() throws Exception {
-        binaries().toBinary(new TestObject2());
-
-        idx = 1;
-
-        binaries().toBinary(new TestObject2());
-
-        BinaryType meta = binaries().type(TestObject2.class);
-
-        assertNotNull(meta);
-
-        assertEquals("TestObject2", meta.typeName());
-
-        Collection<String> fields = meta.fieldNames();
-
-        assertEquals(9, fields.size());
-
-        assertTrue(fields.contains("boolVal"));
-        assertTrue(fields.contains("dateVal"));
-        assertTrue(fields.contains("uuidArrVal"));
-        assertTrue(fields.contains("objVal"));
-        assertTrue(fields.contains("mapVal"));
-        assertTrue(fields.contains("charVal"));
-        assertTrue(fields.contains("colVal"));
-        assertTrue(fields.contains("decVal"));
-        assertTrue(fields.contains("decArrVal"));
-
-        assertEquals("boolean", meta.fieldTypeName("boolVal"));
-        assertEquals("Date", meta.fieldTypeName("dateVal"));
-        assertEquals("UUID[]", meta.fieldTypeName("uuidArrVal"));
-        assertEquals("Object", meta.fieldTypeName("objVal"));
-        assertEquals("Map", meta.fieldTypeName("mapVal"));
-        assertEquals("char", meta.fieldTypeName("charVal"));
-        assertEquals("Collection", meta.fieldTypeName("colVal"));
-        assertEquals("decimal", meta.fieldTypeName("decVal"));
-        assertEquals("decimal[]", meta.fieldTypeName("decArrVal"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testSerializedObject() throws Exception {
-        TestObject1 obj = new TestObject1();
-
-        obj.intVal = 10;
-        obj.strVal = "str";
-        obj.arrVal = new byte[] {2, 4, 6};
-        obj.obj1Val = null;
-        obj.obj2Val = new TestObject2();
-        obj.decVal = BigDecimal.ZERO;
-        obj.decArrVal = new BigDecimal[] { BigDecimal.ONE };
-
-        BinaryObject po = binaries().toBinary(obj);
-
-        info(po.toString());
-
-        BinaryType meta = po.type();
-
-        assertNotNull(meta);
-
-        assertEquals("TestObject1", meta.typeName());
-
-        Collection<String> fields = meta.fieldNames();
-
-        assertEquals(7, fields.size());
-
-        assertTrue(fields.contains("intVal"));
-        assertTrue(fields.contains("strVal"));
-        assertTrue(fields.contains("arrVal"));
-        assertTrue(fields.contains("obj1Val"));
-        assertTrue(fields.contains("obj2Val"));
-        assertTrue(fields.contains("decVal"));
-        assertTrue(fields.contains("decArrVal"));
-
-        assertEquals("int", meta.fieldTypeName("intVal"));
-        assertEquals("String", meta.fieldTypeName("strVal"));
-        assertEquals("byte[]", meta.fieldTypeName("arrVal"));
-        assertEquals("Object", meta.fieldTypeName("obj1Val"));
-        assertEquals("Object", meta.fieldTypeName("obj2Val"));
-        assertEquals("decimal", meta.fieldTypeName("decVal"));
-        assertEquals("decimal[]", meta.fieldTypeName("decArrVal"));
-    }
-
-    /**
-     */
-    @SuppressWarnings("UnusedDeclaration")
-    private static class TestObject1 {
-        /** */
-        private int intVal;
-
-        /** */
-        private String strVal;
-
-        /** */
-        private byte[] arrVal;
-
-        /** */
-        private TestObject1 obj1Val;
-
-        /** */
-        private TestObject2 obj2Val;
-
-        /** */
-        private BigDecimal decVal;
-
-        /** */
-        private BigDecimal[] decArrVal;
-    }
-
-    /**
-     */
-    private static class TestObject2 implements Binarylizable {
-        /** {@inheritDoc} */
-        @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
-            writer.writeBoolean("boolVal", false);
-            writer.writeDate("dateVal", new Date());
-            writer.writeUuidArray("uuidArrVal", null);
-            writer.writeObject("objVal", null);
-            writer.writeMap("mapVal", new HashMap<>());
-            writer.writeDecimal("decVal", BigDecimal.ZERO);
-            writer.writeDecimalArray("decArrVal", new BigDecimal[] { BigDecimal.ONE });
-
-            if (idx == 1) {
-                writer.writeChar("charVal", (char)0);
-                writer.writeCollection("colVal", null);
-            }
-
-            BinaryRawWriter raw = writer.rawWriter();
-
-            raw.writeChar((char)0);
-            raw.writeCollection(null);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
-            // No-op.
-        }
-    }
-
-    /**
-     */
-    @SuppressWarnings("UnusedDeclaration")
-    private static class TestObject3 {
-        /** */
-        private int intVal;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/62502b2c/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryWildcardsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryWildcardsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryWildcardsSelfTest.java
index ae0cfa0..d0d63b3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryWildcardsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryWildcardsSelfTest.java
@@ -17,31 +17,67 @@
 
 package org.apache.ignite.internal.binary;
 
+import java.util.Arrays;
 import java.util.Collection;
+import java.util.Map;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.binary.BinaryBasicIdMapper;
 import org.apache.ignite.binary.BinaryIdMapper;
+import org.apache.ignite.binary.BinaryNameMapper;
+import org.apache.ignite.binary.BinaryBasicNameMapper;
 import org.apache.ignite.binary.BinarySerializer;
 import org.apache.ignite.binary.BinaryTypeConfiguration;
 import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.binary.test.GridBinaryTestClass1;
+import org.apache.ignite.internal.binary.test.GridBinaryTestClass2;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.logger.NullLogger;
 import org.apache.ignite.marshaller.MarshallerContextTestImpl;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
-import java.util.Arrays;
-import java.util.Map;
-
 /**
  * Wildcards test.
  */
 public class GridBinaryWildcardsSelfTest extends GridCommonAbstractTest {
+    /** */
+    public static final String CLASS1_FULL_NAME = GridBinaryTestClass1.class.getName();
+
+    /** */
+    public static final String CLASS2_FULL_NAME = GridBinaryTestClass2.class.getName();
+
+    /** */
+    public static final String INNER_CLASS_FULL_NAME = GridBinaryTestClass1.class.getName() + "$InnerClass";
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClassNamesFullNameMapper() throws Exception {
+        checkClassNames(new BinaryBasicNameMapper(false), new BinaryBasicIdMapper(false));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClassNamesSimpleNameMapper() throws Exception {
+        checkClassNames(new BinaryBasicNameMapper(true), new BinaryBasicIdMapper(true));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClassNamesMixedMappers() throws Exception {
+        checkClassNames(new BinaryBasicNameMapper(false), new BinaryBasicIdMapper(true));
+    }
+
     /**
      * @throws Exception If failed.
+     * @param nameMapper Name mapper.
+     * @param mapper ID mapper.
      */
-    public void testClassNames() throws Exception {
-        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(
+    private void checkClassNames(BinaryNameMapper nameMapper, BinaryIdMapper mapper) throws Exception {
+        BinaryMarshaller marsh = binaryMarshaller(nameMapper, mapper, Arrays.asList(
             new BinaryTypeConfiguration("org.apache.ignite.internal.binary.test.*"),
             new BinaryTypeConfiguration("unknown.*")
         ));
@@ -52,16 +88,16 @@ public class GridBinaryWildcardsSelfTest extends GridCommonAbstractTest {
 
         assertEquals(3, typeIds.size());
 
-        assertTrue(typeIds.containsKey("gridbinarytestclass1".hashCode()));
-        assertTrue(typeIds.containsKey("gridbinarytestclass2".hashCode()));
-        assertTrue(typeIds.containsKey("innerclass".hashCode()));
+        assertTrue(typeIds.containsKey(typeId(CLASS1_FULL_NAME, nameMapper, mapper)));
+        assertTrue(typeIds.containsKey(typeId(CLASS2_FULL_NAME, nameMapper, mapper)));
+        assertTrue(typeIds.containsKey(typeId(INNER_CLASS_FULL_NAME, nameMapper, mapper)));
     }
 
     /**
      * @throws Exception If failed.
      */
-    public void testClassNamesWithMapper() throws Exception {
-        BinaryMarshaller marsh = binaryMarshaller(new BinaryIdMapper() {
+    public void testClassNamesCustomMappers() throws Exception {
+        BinaryMarshaller marsh = binaryMarshaller(null, new BinaryIdMapper() {
             @SuppressWarnings("IfMayBeConditional")
             @Override public int typeId(String clsName) {
                 if (clsName.endsWith("1"))
@@ -84,20 +120,37 @@ public class GridBinaryWildcardsSelfTest extends GridCommonAbstractTest {
 
         BinaryContext ctx = binaryContext(marsh);
 
-        Map<String, BinaryIdMapper> typeMappers = U.field(ctx, "typeMappers");
+        Map<String, org.apache.ignite.internal.binary.BinaryInternalMapper> typeMappers = U.field(ctx, "cls2Mappers");
 
         assertEquals(3, typeMappers.size());
 
-        assertEquals(300, typeMappers.get("GridBinaryTestClass1").typeId("GridBinaryTestClass1"));
-        assertEquals(400, typeMappers.get("GridBinaryTestClass2").typeId("GridBinaryTestClass2"));
-        assertEquals(500, typeMappers.get("InnerClass").typeId("InnerClass"));
+        assertEquals(300, typeMappers.get(CLASS1_FULL_NAME).idMapper().typeId(CLASS1_FULL_NAME));
+        assertEquals(400, typeMappers.get(CLASS2_FULL_NAME).idMapper().typeId(CLASS2_FULL_NAME));
+        assertEquals(500, typeMappers.get(INNER_CLASS_FULL_NAME).idMapper().typeId(INNER_CLASS_FULL_NAME));
     }
 
     /**
      * @throws Exception If failed.
      */
-    public void testTypeConfigurations() throws Exception {
-        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(
+    public void testTypeConfigurationsSimpleNameIdMapper() throws Exception {
+        checkTypeConfigurations(new BinaryBasicNameMapper(true), new BinaryBasicIdMapper(true));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTypeConfigurationsFullNameIdMapper() throws Exception {
+        checkTypeConfigurations(new BinaryBasicNameMapper(false), new BinaryBasicIdMapper(false));
+    }
+
+    /**
+     *
+     * @param nameMapper Name mapper.
+     * @param idMapper ID mapper.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void checkTypeConfigurations(BinaryNameMapper nameMapper, BinaryIdMapper idMapper) throws IgniteCheckedException {
+        BinaryMarshaller marsh = binaryMarshaller(nameMapper, idMapper, Arrays.asList(
             new BinaryTypeConfiguration("org.apache.ignite.internal.binary.test.*"),
             new BinaryTypeConfiguration("unknown.*")
         ));
@@ -108,16 +161,31 @@ public class GridBinaryWildcardsSelfTest extends GridCommonAbstractTest {
 
         assertEquals(3, typeIds.size());
 
-        assertTrue(typeIds.containsKey("gridbinarytestclass1".hashCode()));
-        assertTrue(typeIds.containsKey("gridbinarytestclass2".hashCode()));
-        assertTrue(typeIds.containsKey("innerclass".hashCode()));
+        assertTrue(typeIds.containsKey(typeId(CLASS1_FULL_NAME, nameMapper, idMapper)));
+        assertTrue(typeIds.containsKey(typeId(CLASS2_FULL_NAME, nameMapper, idMapper)));
+        assertTrue(typeIds.containsKey(typeId(INNER_CLASS_FULL_NAME, nameMapper, idMapper)));
+    }
+
+    /**
+     * @param typeName Type name.
+     * @param nameMapper Name mapper.
+     * @param mapper ID mapper.  @return Type ID.
+     */
+    private int typeId(String typeName, BinaryNameMapper nameMapper, BinaryIdMapper mapper) {
+        if (mapper == null)
+            mapper = BinaryContext.defaultIdMapper();
+
+        if (nameMapper == null)
+            nameMapper = BinaryContext.defaultNameMapper();
+
+        return mapper.typeId(nameMapper.typeName(typeName));
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testTypeConfigurationsWithGlobalMapper() throws Exception {
-        BinaryMarshaller marsh = binaryMarshaller(new BinaryIdMapper() {
+        BinaryMarshaller marsh = binaryMarshaller(new BinaryBasicNameMapper(false), new BinaryIdMapper() {
             @SuppressWarnings("IfMayBeConditional")
             @Override public int typeId(String clsName) {
                 if (clsName.endsWith("1"))
@@ -140,20 +208,20 @@ public class GridBinaryWildcardsSelfTest extends GridCommonAbstractTest {
 
         BinaryContext ctx = binaryContext(marsh);
 
-        Map<String, BinaryIdMapper> typeMappers = U.field(ctx, "typeMappers");
+        Map<String, org.apache.ignite.internal.binary.BinaryInternalMapper> typeMappers = U.field(ctx, "cls2Mappers");
 
         assertEquals(3, typeMappers.size());
 
-        assertEquals(300, typeMappers.get("GridBinaryTestClass1").typeId("GridBinaryTestClass1"));
-        assertEquals(400, typeMappers.get("GridBinaryTestClass2").typeId("GridBinaryTestClass2"));
-        assertEquals(500, typeMappers.get("InnerClass").typeId("InnerClass"));
+        assertEquals(300, typeMappers.get(CLASS1_FULL_NAME).idMapper().typeId(CLASS1_FULL_NAME));
+        assertEquals(400, typeMappers.get(CLASS2_FULL_NAME).idMapper().typeId(CLASS2_FULL_NAME));
+        assertEquals(500, typeMappers.get(INNER_CLASS_FULL_NAME).idMapper().typeId(INNER_CLASS_FULL_NAME));
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testTypeConfigurationsWithNonGlobalMapper() throws Exception {
-        BinaryMarshaller marsh = binaryMarshaller(new BinaryIdMapper() {
+        BinaryMarshaller marsh = binaryMarshaller(new BinaryBasicNameMapper(true), new BinaryIdMapper() {
             @SuppressWarnings("IfMayBeConditional")
             @Override public int typeId(String clsName) {
                 if (clsName.endsWith("1"))
@@ -176,22 +244,39 @@ public class GridBinaryWildcardsSelfTest extends GridCommonAbstractTest {
 
         BinaryContext ctx = binaryContext(marsh);
 
-        Map<String, BinaryIdMapper> typeMappers = U.field(ctx, "typeMappers");
+        Map<String, org.apache.ignite.internal.binary.BinaryInternalMapper> typeMappers = U.field(ctx, "cls2Mappers");
 
         assertEquals(3, typeMappers.size());
 
-        assertEquals(300, typeMappers.get("GridBinaryTestClass1").typeId("GridBinaryTestClass1"));
-        assertEquals(400, typeMappers.get("GridBinaryTestClass2").typeId("GridBinaryTestClass2"));
-        assertEquals(500, typeMappers.get("InnerClass").typeId("InnerClass"));
+        assertEquals(300, typeMappers.get(CLASS1_FULL_NAME).idMapper().typeId(CLASS1_FULL_NAME));
+        assertEquals(400, typeMappers.get(CLASS2_FULL_NAME).idMapper().typeId(CLASS2_FULL_NAME));
+        assertEquals(500, typeMappers.get(INNER_CLASS_FULL_NAME).idMapper().typeId(INNER_CLASS_FULL_NAME));
     }
 
     /**
      * @throws Exception If failed.
      */
-    public void testOverride() throws Exception {
+    public void testOverrideIdMapperSimpleNameMapper() throws Exception {
+        checkOverrideNameMapper(new BinaryBasicNameMapper(true), new BinaryBasicIdMapper(true));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testOverrideIdMapperFullNameMapper() throws Exception {
+        checkOverrideNameMapper(new BinaryBasicNameMapper(false), new BinaryBasicIdMapper(false));
+    }
+
+    /**
+     *
+     * @param nameMapper Name mapper.
+     * @param mapper Mapper.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void checkOverrideIdMapper(BinaryNameMapper nameMapper, BinaryIdMapper mapper) throws IgniteCheckedException {
         BinaryTypeConfiguration typeCfg = new BinaryTypeConfiguration();
 
-        typeCfg.setTypeName("GridBinaryTestClass2");
+        typeCfg.setTypeName(CLASS2_FULL_NAME);
         typeCfg.setIdMapper(new BinaryIdMapper() {
             @Override public int typeId(String clsName) {
                 return 100;
@@ -202,7 +287,7 @@ public class GridBinaryWildcardsSelfTest extends GridCommonAbstractTest {
             }
         });
 
-        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(
+        BinaryMarshaller marsh = binaryMarshaller(nameMapper, mapper, Arrays.asList(
             new BinaryTypeConfiguration("org.apache.ignite.internal.binary.test.*"),
             typeCfg));
 
@@ -212,20 +297,90 @@ public class GridBinaryWildcardsSelfTest extends GridCommonAbstractTest {
 
         assertEquals(3, typeIds.size());
 
-        assertTrue(typeIds.containsKey("gridbinarytestclass1".hashCode()));
-        assertTrue(typeIds.containsKey("innerclass".hashCode()));
-        assertFalse(typeIds.containsKey(100));
+        assertTrue(typeIds.containsKey(typeId(CLASS1_FULL_NAME, nameMapper, mapper)));
+        assertTrue(typeIds.containsKey(typeId(INNER_CLASS_FULL_NAME, nameMapper, mapper)));
+        assertTrue(typeIds.containsKey(100));
 
-        Map<String, BinaryIdMapper> typeMappers = U.field(ctx, "typeMappers");
+        Map<String, org.apache.ignite.internal.binary.BinaryInternalMapper> typeMappers = U.field(ctx, "cls2Mappers");
 
-        assertEquals(100, typeMappers.get("GridBinaryTestClass2").typeId("GridBinaryTestClass2"));
+        assertEquals(100, typeMappers.get(CLASS2_FULL_NAME).idMapper().typeId(CLASS2_FULL_NAME));
     }
 
     /**
      * @throws Exception If failed.
      */
-    public void testClassNamesJar() throws Exception {
-        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(
+    public void testOverrideNameMapperSimpleNameMapper() throws Exception {
+        checkOverrideNameMapper(new BinaryBasicNameMapper(true), new BinaryBasicIdMapper(true));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testOverrideNameMapperFullNameMapper() throws Exception {
+        checkOverrideNameMapper(new BinaryBasicNameMapper(false), new BinaryBasicIdMapper(false));
+    }
+
+    /**
+     *
+     * @param nameMapper Name mapper.
+     * @param mapper Mapper.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void checkOverrideNameMapper(BinaryNameMapper nameMapper, BinaryIdMapper mapper) throws IgniteCheckedException {
+        BinaryTypeConfiguration typeCfg = new BinaryTypeConfiguration();
+
+        typeCfg.setTypeName(CLASS2_FULL_NAME);
+        typeCfg.setNameMapper(new BinaryNameMapper() {
+            @Override public String typeName(String clsName) {
+                return "type2";
+            }
+
+            @Override public String fieldName(String fieldName) {
+                return "field2";
+            }
+        });
+
+        BinaryMarshaller marsh = binaryMarshaller(nameMapper, mapper, Arrays.asList(
+            new BinaryTypeConfiguration("org.apache.ignite.internal.binary.test.*"),
+            typeCfg));
+
+        BinaryContext ctx = binaryContext(marsh);
+
+        Map<Integer, Class> typeIds = U.field(ctx, "userTypes");
+
+        assertEquals(3, typeIds.size());
+
+        assertTrue(typeIds.containsKey(typeId(CLASS1_FULL_NAME, nameMapper, mapper)));
+        assertTrue(typeIds.containsKey(typeId(INNER_CLASS_FULL_NAME, nameMapper, mapper)));
+        assertTrue(typeIds.containsKey("type2".hashCode()));
+
+        Map<String, org.apache.ignite.internal.binary.BinaryInternalMapper> typeMappers = U.field(ctx, "cls2Mappers");
+
+        assertEquals("type2", typeMappers.get(CLASS2_FULL_NAME).nameMapper().typeName(CLASS2_FULL_NAME));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClassNamesJarFullNameMapper() throws Exception {
+        checkClassNamesJar(new BinaryBasicNameMapper(false), new BinaryBasicIdMapper(false));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClassNamesJarSimpleNameMapper() throws Exception {
+        checkClassNamesJar(new BinaryBasicNameMapper(true), new BinaryBasicIdMapper(true));
+    }
+
+    /**
+     *
+     * @param nameMapper Name mapper.
+     * @param idMapper Mapper.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void checkClassNamesJar(BinaryNameMapper nameMapper, BinaryIdMapper idMapper) throws IgniteCheckedException {
+        BinaryMarshaller marsh = binaryMarshaller(nameMapper, idMapper, Arrays.asList(
             new BinaryTypeConfiguration("org.apache.ignite.internal.binary.test.*"),
             new BinaryTypeConfiguration("unknown.*")
         ));
@@ -236,15 +391,15 @@ public class GridBinaryWildcardsSelfTest extends GridCommonAbstractTest {
 
         assertEquals(3, typeIds.size());
 
-        assertTrue(typeIds.containsKey("gridbinarytestclass1".hashCode()));
-        assertTrue(typeIds.containsKey("gridbinarytestclass2".hashCode()));
+        assertTrue(typeIds.containsKey(typeId(CLASS1_FULL_NAME, nameMapper, idMapper)));
+        assertTrue(typeIds.containsKey(typeId(CLASS2_FULL_NAME, nameMapper, idMapper)));
     }
 
     /**
      * @throws Exception If failed.
      */
-    public void testClassNamesWithMapperJar() throws Exception {
-        BinaryMarshaller marsh = binaryMarshaller(new BinaryIdMapper() {
+    public void testClassNamesWithCustomMapperJar() throws Exception {
+        BinaryMarshaller marsh = binaryMarshaller(new BinaryBasicNameMapper(false), new BinaryIdMapper() {
             @SuppressWarnings("IfMayBeConditional")
             @Override public int typeId(String clsName) {
                 if (clsName.endsWith("1"))
@@ -265,19 +420,40 @@ public class GridBinaryWildcardsSelfTest extends GridCommonAbstractTest {
 
         BinaryContext ctx = binaryContext(marsh);
 
-        Map<String, BinaryIdMapper> typeMappers = U.field(ctx, "typeMappers");
+        Map<String, org.apache.ignite.internal.binary.BinaryInternalMapper> typeMappers = U.field(ctx, "cls2Mappers");
 
         assertEquals(3, typeMappers.size());
 
-        assertEquals(300, typeMappers.get("GridBinaryTestClass1").typeId("GridBinaryTestClass1"));
-        assertEquals(400, typeMappers.get("GridBinaryTestClass2").typeId("GridBinaryTestClass2"));
+        assertFalse(((BinaryBasicNameMapper)typeMappers.get(CLASS1_FULL_NAME).nameMapper()).isSimpleName());
+        assertEquals(300, typeMappers.get(CLASS1_FULL_NAME).idMapper().typeId(CLASS1_FULL_NAME));
+
+        assertFalse(((BinaryBasicNameMapper)typeMappers.get(CLASS2_FULL_NAME).nameMapper()).isSimpleName());
+        assertEquals(400, typeMappers.get(CLASS2_FULL_NAME).idMapper().typeId(CLASS2_FULL_NAME));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTypeConfigurationsJarSimpleNameMapper() throws Exception {
+        checkTypeConfigurationJar(new BinaryBasicNameMapper(true), new BinaryBasicIdMapper(true));
     }
 
     /**
      * @throws Exception If failed.
      */
-    public void testTypeConfigurationsJar() throws Exception {
-        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(
+    public void testTypeConfigurationsJarFullNameMapper() throws Exception {
+        checkTypeConfigurationJar(new BinaryBasicNameMapper(false), new BinaryBasicIdMapper(false));
+    }
+
+    /**
+     *
+     * @param nameMapper
+     * @param idMapper Mapper.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void checkTypeConfigurationJar(BinaryNameMapper nameMapper, BinaryIdMapper idMapper)
+        throws IgniteCheckedException {
+        BinaryMarshaller marsh = binaryMarshaller(nameMapper, idMapper, Arrays.asList(
             new BinaryTypeConfiguration("org.apache.ignite.internal.binary.test.*"),
             new BinaryTypeConfiguration("unknown.*")
         ));
@@ -288,15 +464,15 @@ public class GridBinaryWildcardsSelfTest extends GridCommonAbstractTest {
 
         assertEquals(3, typeIds.size());
 
-        assertTrue(typeIds.containsKey("gridbinarytestclass1".hashCode()));
-        assertTrue(typeIds.containsKey("gridbinarytestclass2".hashCode()));
+        assertTrue(typeIds.containsKey(typeId(CLASS1_FULL_NAME, nameMapper, idMapper)));
+        assertTrue(typeIds.containsKey(typeId(CLASS2_FULL_NAME, nameMapper, idMapper)));
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testTypeConfigurationsWithGlobalMapperJar() throws Exception {
-        BinaryMarshaller marsh = binaryMarshaller(new BinaryIdMapper() {
+        BinaryMarshaller marsh = binaryMarshaller(new BinaryBasicNameMapper(false), new BinaryIdMapper() {
             @SuppressWarnings("IfMayBeConditional")
             @Override public int typeId(String clsName) {
                 if (clsName.endsWith("1"))
@@ -317,19 +493,22 @@ public class GridBinaryWildcardsSelfTest extends GridCommonAbstractTest {
 
         BinaryContext ctx = binaryContext(marsh);
 
-        Map<String, BinaryIdMapper> typeMappers = U.field(ctx, "typeMappers");
+        Map<String, org.apache.ignite.internal.binary.BinaryInternalMapper> typeMappers = U.field(ctx, "cls2Mappers");
 
         assertEquals(3, typeMappers.size());
 
-        assertEquals(300, typeMappers.get("GridBinaryTestClass1").typeId("GridBinaryTestClass1"));
-        assertEquals(400, typeMappers.get("GridBinaryTestClass2").typeId("GridBinaryTestClass2"));
+        assertFalse(((BinaryBasicNameMapper)typeMappers.get(CLASS1_FULL_NAME).nameMapper()).isSimpleName());
+        assertEquals(300, typeMappers.get(CLASS1_FULL_NAME).idMapper().typeId(CLASS1_FULL_NAME));
+
+        assertFalse(((BinaryBasicNameMapper)typeMappers.get(CLASS2_FULL_NAME).nameMapper()).isSimpleName());
+        assertEquals(400, typeMappers.get(CLASS2_FULL_NAME).idMapper().typeId(CLASS2_FULL_NAME));
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testTypeConfigurationsWithNonGlobalMapperJar() throws Exception {
-        BinaryMarshaller marsh = binaryMarshaller(new BinaryIdMapper() {
+        BinaryMarshaller marsh = binaryMarshaller(new BinaryBasicNameMapper(false), new BinaryIdMapper() {
             @SuppressWarnings("IfMayBeConditional")
             @Override public int typeId(String clsName) {
                 if (clsName.endsWith("1"))
@@ -350,18 +529,38 @@ public class GridBinaryWildcardsSelfTest extends GridCommonAbstractTest {
 
         BinaryContext ctx = binaryContext(marsh);
 
-        Map<String, BinaryIdMapper> typeMappers = U.field(ctx, "typeMappers");
+        Map<String, org.apache.ignite.internal.binary.BinaryInternalMapper> typeMappers = U.field(ctx, "cls2Mappers");
 
         assertEquals(3, typeMappers.size());
 
-        assertEquals(300, typeMappers.get("GridBinaryTestClass1").typeId("GridBinaryTestClass1"));
-        assertEquals(400, typeMappers.get("GridBinaryTestClass2").typeId("GridBinaryTestClass2"));
+        assertFalse(((BinaryBasicNameMapper)typeMappers.get(CLASS1_FULL_NAME).nameMapper()).isSimpleName());
+        assertEquals(300, typeMappers.get(CLASS1_FULL_NAME).idMapper().typeId(CLASS1_FULL_NAME));
+
+        assertFalse(((BinaryBasicNameMapper)typeMappers.get(CLASS2_FULL_NAME).nameMapper()).isSimpleName());
+        assertEquals(400, typeMappers.get(CLASS2_FULL_NAME).idMapper().typeId(CLASS2_FULL_NAME));
     }
 
     /**
      * @throws Exception If failed.
      */
-    public void testOverrideJar() throws Exception {
+    public void testOverrideJarSimpleNameMapper() throws Exception {
+        checkOverrideJar(new BinaryBasicNameMapper(true), new BinaryBasicIdMapper(true));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testOverrideJarFullNameMapper() throws Exception {
+        checkOverrideJar(new BinaryBasicNameMapper(false), new BinaryBasicIdMapper(false));
+    }
+
+    /**
+     *
+     * @param nameMapper Name mapper.
+     * @param idMapper Mapper.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void checkOverrideJar(BinaryNameMapper nameMapper, BinaryIdMapper idMapper) throws IgniteCheckedException {
         BinaryTypeConfiguration typeCfg = new BinaryTypeConfiguration(
             "org.apache.ignite.internal.binary.test.GridBinaryTestClass2");
 
@@ -375,7 +574,7 @@ public class GridBinaryWildcardsSelfTest extends GridCommonAbstractTest {
             }
         });
 
-        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(
+        BinaryMarshaller marsh = binaryMarshaller(nameMapper, idMapper, Arrays.asList(
             new BinaryTypeConfiguration("org.apache.ignite.internal.binary.test.*"),
             typeCfg));
 
@@ -385,13 +584,14 @@ public class GridBinaryWildcardsSelfTest extends GridCommonAbstractTest {
 
         assertEquals(3, typeIds.size());
 
-        assertTrue(typeIds.containsKey("gridbinarytestclass1".hashCode()));
+        assertTrue(typeIds.containsKey(typeId(CLASS1_FULL_NAME, nameMapper, idMapper)));
 
-        Map<String, BinaryIdMapper> typeMappers = U.field(ctx, "typeMappers");
+        Map<String, org.apache.ignite.internal.binary.BinaryInternalMapper> typeMappers = U.field(ctx, "cls2Mappers");
 
         assertEquals(3, typeMappers.size());
 
-        assertEquals(100, typeMappers.get("GridBinaryTestClass2").typeId("GridBinaryTestClass2"));
+        assertEquals(nameMapper, typeMappers.get(CLASS2_FULL_NAME).nameMapper());
+        assertEquals(100, typeMappers.get(CLASS2_FULL_NAME).idMapper().typeId(CLASS2_FULL_NAME));
     }
 
     /**
@@ -423,9 +623,10 @@ public class GridBinaryWildcardsSelfTest extends GridCommonAbstractTest {
     /**
      *
      */
-    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);
     }
 
     /**
@@ -433,10 +634,14 @@ public class GridBinaryWildcardsSelfTest extends GridCommonAbstractTest {
      */
     protected BinaryMarshaller binaryMarshaller(BinarySerializer serializer, Collection<BinaryTypeConfiguration> cfgs)
         throws IgniteCheckedException {
-        return binaryMarshaller(null, serializer, cfgs);
+        return binaryMarshaller(null, null, serializer, cfgs);
     }
 
+    /**
+     *
+     */
     protected BinaryMarshaller binaryMarshaller(
+        BinaryNameMapper nameMapper,
         BinaryIdMapper mapper,
         BinarySerializer serializer,
         Collection<BinaryTypeConfiguration> cfgs
@@ -445,6 +650,7 @@ public class GridBinaryWildcardsSelfTest extends GridCommonAbstractTest {
 
         BinaryConfiguration bCfg = new BinaryConfiguration();
 
+        bCfg.setNameMapper(nameMapper);
         bCfg.setIdMapper(mapper);
         bCfg.setSerializer(serializer);