You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2015/12/14 07:00:24 UTC

[05/29] ignite git commit: ignite-2065: rename "portable" classes to "binary"

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientPortableMetaData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientPortableMetaData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientPortableMetaData.java
deleted file mode 100644
index 5522095..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientPortableMetaData.java
+++ /dev/null
@@ -1,71 +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.rest.client.message;
-
-import java.util.Map;
-import org.apache.ignite.internal.util.typedef.internal.S;
-
-/**
- * Portable meta data sent from client.
- */
-public class GridClientPortableMetaData {
-    /** */
-    private int typeId;
-
-    /** */
-    private String typeName;
-
-    /** */
-    private Map<String, Integer> fields;
-
-    /** */
-    private String affKeyFieldName;
-
-    /**
-     * @return Type ID.
-     */
-    public int typeId() {
-        return typeId;
-    }
-
-    /**
-     * @return Type name.
-     */
-    public String typeName() {
-        return typeName;
-    }
-
-    /**
-     * @return Fields.
-     */
-    public Map<String, Integer> fields() {
-        return fields;
-    }
-
-    /**
-     * @return Affinity key field name.
-     */
-    public String affinityKeyFieldName() {
-        return affKeyFieldName;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridClientPortableMetaData.class, this);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldsAbstractSelfTest.java
index f12bb92..7d53705 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldsAbstractSelfTest.java
@@ -47,7 +47,7 @@ public abstract class BinaryFieldsAbstractSelfTest extends GridCommonAbstractTes
      * @throws Exception If failed.
      */
     protected BinaryMarshaller createMarshaller() throws Exception {
-        PortableContext ctx = new PortableContext(BinaryCachingMetadataHandler.create(), new IgniteConfiguration());
+        BinaryContext ctx = new BinaryContext(BinaryCachingMetadataHandler.create(), new IgniteConfiguration());
 
         BinaryMarshaller marsh = new BinaryMarshaller();
 
@@ -85,8 +85,8 @@ public abstract class BinaryFieldsAbstractSelfTest extends GridCommonAbstractTes
      * @param marsh Marshaller.
      * @return Portable context.
      */
-    protected static PortableContext portableContext(BinaryMarshaller marsh) {
-        GridPortableMarshaller impl = U.field(marsh, "impl");
+    protected static BinaryContext portableContext(BinaryMarshaller marsh) {
+        GridBinaryMarshaller impl = U.field(marsh, "impl");
 
         return impl.context();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFooterOffsetsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFooterOffsetsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFooterOffsetsAbstractSelfTest.java
index 1e0f375..43609b8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFooterOffsetsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFooterOffsetsAbstractSelfTest.java
@@ -40,13 +40,13 @@ public abstract class BinaryFooterOffsetsAbstractSelfTest extends GridCommonAbst
     protected BinaryMarshaller marsh;
 
     /** Portable context. */
-    protected PortableContext ctx;
+    protected BinaryContext ctx;
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
         super.beforeTest();
 
-        ctx = new PortableContext(BinaryCachingMetadataHandler.create(), new IgniteConfiguration());
+        ctx = new BinaryContext(BinaryCachingMetadataHandler.create(), new IgniteConfiguration());
 
         marsh = new BinaryMarshaller();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/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 d4f3acf..52fecbf 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
@@ -84,7 +84,7 @@ import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 import sun.misc.Unsafe;
 
-import static org.apache.ignite.internal.binary.streams.PortableMemoryAllocator.INSTANCE;
+import static org.apache.ignite.internal.binary.streams.BinaryMemoryAllocator.INSTANCE;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertNotEquals;
 
@@ -1244,7 +1244,7 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
                 String typeName;
 
                 try {
-                    Method mtd = PortableContext.class.getDeclaredMethod("typeName", String.class);
+                    Method mtd = BinaryContext.class.getDeclaredMethod("typeName", String.class);
 
                     mtd.setAccessible(true);
 
@@ -1343,7 +1343,7 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
             customType4
         ));
 
-        PortableContext ctx = portableContext(marsh);
+        BinaryContext ctx = portableContext(marsh);
 
         assertEquals("notconfiguredclass".hashCode(), ctx.typeId("NotConfiguredClass"));
         assertEquals("key".hashCode(), ctx.typeId("Key"));
@@ -1406,7 +1406,7 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
             customType1,
             customType2));
 
-        PortableContext ctx = portableContext(marsh);
+        BinaryContext ctx = portableContext(marsh);
 
         assertEquals("val".hashCode(), ctx.fieldId("key".hashCode(), "val"));
         assertEquals("val".hashCode(), ctx.fieldId("nonexistentclass2".hashCode(), "val"));
@@ -1942,7 +1942,7 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
     public void testOffheapPortable() throws Exception {
         BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(new BinaryTypeConfiguration(SimpleObject.class.getName())));
 
-        PortableContext ctx = portableContext(marsh);
+        BinaryContext ctx = portableContext(marsh);
 
         SimpleObject simpleObj = simpleObject();
 
@@ -2273,7 +2273,7 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
     public void testPredefinedTypeIds() throws Exception {
         BinaryMarshaller marsh = binaryMarshaller();
 
-        PortableContext pCtx = portableContext(marsh);
+        BinaryContext pCtx = portableContext(marsh);
 
         Field field = pCtx.getClass().getDeclaredField("predefinedTypeNames");
 
@@ -2286,10 +2286,10 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
         for (Map.Entry<String, Integer> entry : map.entrySet()) {
             int id = entry.getValue();
 
-            if (id == GridPortableMarshaller.UNREGISTERED_TYPE_ID)
+            if (id == GridBinaryMarshaller.UNREGISTERED_TYPE_ID)
                 continue;
 
-            PortableClassDescriptor desc = pCtx.descriptorForTypeId(false, entry.getValue(), null, false);
+            BinaryClassDescriptor desc = pCtx.descriptorForTypeId(false, entry.getValue(), null, false);
 
             assertEquals(desc.typeId(), pCtx.typeId(desc.describedClass().getName()));
             assertEquals(desc.typeId(), pCtx.typeId(pCtx.typeName(desc.describedClass().getName())));
@@ -2652,7 +2652,7 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
     private <T> BinaryObjectImpl marshal(T obj, BinaryMarshaller marsh) throws IgniteCheckedException {
         byte[] bytes = marsh.marshal(obj);
 
-        return new BinaryObjectImpl(U.<GridPortableMarshaller>field(marsh, "impl").context(),
+        return new BinaryObjectImpl(U.<GridBinaryMarshaller>field(marsh, "impl").context(),
             bytes, 0);
     }
 
@@ -2667,8 +2667,8 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
      * @param marsh Marshaller.
      * @return Portable context.
      */
-    protected PortableContext portableContext(BinaryMarshaller marsh) {
-        GridPortableMarshaller impl = U.field(marsh, "impl");
+    protected BinaryContext portableContext(BinaryMarshaller marsh) {
+        GridBinaryMarshaller impl = U.field(marsh, "impl");
 
         return impl.context();
     }
@@ -2725,7 +2725,7 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
 
         iCfg.setBinaryConfiguration(bCfg);
 
-        PortableContext ctx = new PortableContext(BinaryCachingMetadataHandler.create(), iCfg);
+        BinaryContext ctx = new BinaryContext(BinaryCachingMetadataHandler.create(), iCfg);
 
         BinaryMarshaller marsh = new BinaryMarshaller();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/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 50f74eb..bf329a5 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
@@ -32,7 +32,7 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.binary.mutabletest.GridPortableTestClasses;
 import org.apache.ignite.internal.binary.builder.BinaryObjectBuilderImpl;
-import org.apache.ignite.internal.binary.builder.PortableBuilderEnum;
+import org.apache.ignite.internal.binary.builder.BinaryBuilderEnum;
 import org.apache.ignite.internal.binary.mutabletest.GridBinaryMarshalerAwareTestClass;
 import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
 import org.apache.ignite.internal.processors.cache.binary.IgniteBinaryImpl;
@@ -128,11 +128,11 @@ public class BinaryObjectBuilderAdditionalSelfTest extends GridCommonAbstractTes
 
             switch (field.getName()) {
                 case "anEnum":
-                    assertEquals(((PortableBuilderEnum)actVal).getOrdinal(), ((Enum)expVal).ordinal());
+                    assertEquals(((BinaryBuilderEnum)actVal).getOrdinal(), ((Enum)expVal).ordinal());
                     break;
 
                 case "enumArr": {
-                    PortableBuilderEnum[] actArr = (PortableBuilderEnum[])actVal;
+                    BinaryBuilderEnum[] actArr = (BinaryBuilderEnum[])actVal;
                     Enum[] expArr = (Enum[])expVal;
 
                     assertEquals(expArr.length, actArr.length);
@@ -847,8 +847,8 @@ public class BinaryObjectBuilderAdditionalSelfTest extends GridCommonAbstractTes
 
         BinaryObjectBuilderImpl mutObj = wrap(obj);
 
-        PortableBuilderEnum[] arr = mutObj.getField("enumArr");
-        arr[0] = new PortableBuilderEnum(mutObj.typeId(), GridPortableTestClasses.TestObjectEnum.B);
+        BinaryBuilderEnum[] arr = mutObj.getField("enumArr");
+        arr[0] = new BinaryBuilderEnum(mutObj.typeId(), GridPortableTestClasses.TestObjectEnum.B);
 
         GridPortableTestClasses.TestObjectAllTypes res = mutObj.build().deserialize();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryAffinityKeySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryAffinityKeySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryAffinityKeySelfTest.java
new file mode 100644
index 0000000..2528f17
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryAffinityKeySelfTest.java
@@ -0,0 +1,234 @@
+/*
+ * 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.Collections;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.binary.BinaryObjectBuilder;
+import org.apache.ignite.cache.CacheKeyConfiguration;
+import org.apache.ignite.cache.affinity.Affinity;
+import org.apache.ignite.configuration.BinaryConfiguration;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.processors.affinity.GridAffinityProcessor;
+import org.apache.ignite.internal.processors.cache.CacheObject;
+import org.apache.ignite.internal.processors.cache.CacheObjectContext;
+import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
+import org.apache.ignite.lang.IgniteCallable;
+import org.apache.ignite.lang.IgniteRunnable;
+import org.apache.ignite.binary.BinaryTypeConfiguration;
+import org.apache.ignite.resources.IgniteInstanceResource;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+
+/**
+ * Test for portable object affinity key.
+ */
+public class GridBinaryAffinityKeySelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final AtomicReference<UUID> nodeId = new AtomicReference<>();
+
+    /** VM ip finder for TCP discovery. */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static int GRID_CNT = 5;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        BinaryTypeConfiguration typeCfg = new BinaryTypeConfiguration();
+
+        typeCfg.setTypeName(TestObject.class.getName());
+
+        BinaryConfiguration bCfg = new BinaryConfiguration();
+
+        bCfg.setTypeConfigurations(Collections.singleton(typeCfg));
+
+        cfg.setBinaryConfiguration(bCfg);
+
+        CacheKeyConfiguration keyCfg = new CacheKeyConfiguration(TestObject.class.getName(), "affKey");
+        CacheKeyConfiguration keyCfg2 = new CacheKeyConfiguration("TestObject2", "affKey");
+
+        cfg.setCacheKeyConfiguration(keyCfg, keyCfg2);
+
+        cfg.setMarshaller(new BinaryMarshaller());
+
+        if (!gridName.equals(getTestGridName(GRID_CNT))) {
+            CacheConfiguration cacheCfg = new CacheConfiguration();
+
+            cacheCfg.setCacheMode(PARTITIONED);
+
+            cfg.setCacheConfiguration(cacheCfg);
+        }
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGridsMultiThreaded(GRID_CNT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAffinity() throws Exception {
+        checkAffinity(grid(0));
+
+        try (Ignite igniteNoCache = startGrid(GRID_CNT)) {
+            try {
+                igniteNoCache.cache(null);
+            }
+            catch (IllegalArgumentException ignore) {
+                // Expected error.
+            }
+
+            checkAffinity(igniteNoCache);
+        }
+    }
+
+    /**
+     * @param ignite Ignite.
+     * @throws Exception If failed.
+     */
+    private void checkAffinity(Ignite ignite) throws Exception {
+        Affinity<Object> aff = ignite.affinity(null);
+
+        GridAffinityProcessor affProc = ((IgniteKernal)ignite).context().affinity();
+
+        IgniteCacheObjectProcessor cacheObjProc = ((IgniteKernal)ignite).context().cacheObjects();
+
+        CacheObjectContext cacheObjCtx = cacheObjProc.contextForCache(
+            ignite.cache(null).getConfiguration(CacheConfiguration.class));
+
+        for (int i = 0; i < 1000; i++) {
+            assertEquals(i, aff.affinityKey(i));
+
+            assertEquals(i, aff.affinityKey(new TestObject(i)));
+
+            assertEquals(i, aff.affinityKey(ignite.binary().toBinary(new TestObject(i))));
+
+            BinaryObjectBuilder bldr = ignite.binary().builder("TestObject2");
+
+            bldr.setField("affKey", i);
+
+            assertEquals(i, aff.affinityKey(bldr.build()));
+
+            CacheObject cacheObj = cacheObjProc.toCacheObject(cacheObjCtx, new TestObject(i), true);
+
+            assertEquals(i, aff.affinityKey(cacheObj));
+
+            assertEquals(aff.mapKeyToNode(i), aff.mapKeyToNode(new TestObject(i)));
+
+            assertEquals(aff.mapKeyToNode(i), aff.mapKeyToNode(cacheObj));
+
+            assertEquals(i, affProc.affinityKey(null, i));
+
+            assertEquals(i, affProc.affinityKey(null, new TestObject(i)));
+
+            assertEquals(i, affProc.affinityKey(null, cacheObj));
+
+            assertEquals(affProc.mapKeyToNode(null, i), affProc.mapKeyToNode(null, new TestObject(i)));
+
+            assertEquals(affProc.mapKeyToNode(null, i), affProc.mapKeyToNode(null, cacheObj));
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAffinityRun() throws Exception {
+        Affinity<Object> aff = grid(0).affinity(null);
+
+        for (int i = 0; i < 1000; i++) {
+            nodeId.set(null);
+
+            grid(0).compute().affinityRun(null, new TestObject(i), new IgniteRunnable() {
+                @IgniteInstanceResource
+                private Ignite ignite;
+
+                @Override public void run() {
+                    nodeId.set(ignite.configuration().getNodeId());
+                }
+            });
+
+            assertEquals(aff.mapKeyToNode(i).id(), nodeId.get());
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAffinityCall() throws Exception {
+        Affinity<Object> aff = grid(0).affinity(null);
+
+        for (int i = 0; i < 1000; i++) {
+            nodeId.set(null);
+
+            grid(0).compute().affinityCall(null, new TestObject(i), new IgniteCallable<Object>() {
+                @IgniteInstanceResource
+                private Ignite ignite;
+
+                @Override public Object call() {
+                    nodeId.set(ignite.configuration().getNodeId());
+
+                    return null;
+                }
+            });
+
+            assertEquals(aff.mapKeyToNode(i).id(), nodeId.get());
+        }
+    }
+
+    /**
+     */
+    private static class TestObject {
+        /** */
+        @SuppressWarnings("UnusedDeclaration")
+        private int affKey;
+
+        /**
+         */
+        private TestObject() {
+            // No-op.
+        }
+
+        /**
+         * @param affKey Affinity key.
+         */
+        private TestObject(int affKey) {
+            this.affKey = affKey;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryMarshallerCtxDisabledSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryMarshallerCtxDisabledSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryMarshallerCtxDisabledSelfTest.java
new file mode 100644
index 0000000..c48b056
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryMarshallerCtxDisabledSelfTest.java
@@ -0,0 +1,247 @@
+/*
+ * 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.IgniteCheckedException;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryReader;
+import org.apache.ignite.binary.BinaryWriter;
+import org.apache.ignite.binary.Binarylizable;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.MarshallerContextAdapter;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.Arrays;
+
+/**
+ *
+ */
+public class GridBinaryMarshallerCtxDisabledSelfTest extends GridCommonAbstractTest {
+    /**
+     * @throws Exception If failed.
+     */
+    public void testObjectExchange() throws Exception {
+        BinaryMarshaller marsh = new BinaryMarshaller();
+        marsh.setContext(new MarshallerContextWithNoStorage());
+
+        IgniteConfiguration cfg = new IgniteConfiguration();
+
+        BinaryContext context = new BinaryContext(BinaryCachingMetadataHandler.create(), cfg);
+
+        IgniteUtils.invoke(BinaryMarshaller.class, marsh, "setPortableContext", context, cfg);
+
+        SimpleObject simpleObj = new SimpleObject();
+
+        simpleObj.b = 2;
+        simpleObj.bArr = new byte[] {2, 3, 4, 5, 5};
+        simpleObj.c = 'A';
+        simpleObj.enumVal = TestEnum.D;
+        simpleObj.objArr = new Object[] {"hello", "world", "from", "me"};
+        simpleObj.enumArr = new TestEnum[] {TestEnum.C, TestEnum.B};
+
+        SimpleObject otherObj = new SimpleObject();
+
+        otherObj.b = 3;
+        otherObj.bArr = new byte[] {5, 3, 4};
+
+        simpleObj.otherObj = otherObj;
+
+        assertEquals(simpleObj, marsh.unmarshal(marsh.marshal(simpleObj), null));
+
+        SimpleBinary simplePortable = new SimpleBinary();
+
+        simplePortable.str = "portable";
+        simplePortable.arr = new long[] {100, 200, 300};
+
+        assertEquals(simplePortable, marsh.unmarshal(marsh.marshal(simplePortable), null));
+
+        SimpleExternalizable simpleExtr = new SimpleExternalizable();
+
+        simpleExtr.str = "externalizable";
+        simpleExtr.arr = new long[] {20000, 300000, 400000};
+
+        assertEquals(simpleExtr, marsh.unmarshal(marsh.marshal(simpleExtr), null));
+    }
+
+    /**
+     * Marshaller context with no storage. Platform has to work in such environment as well by marshalling class name of
+     * a portable object.
+     */
+    private static class MarshallerContextWithNoStorage extends MarshallerContextAdapter {
+        /** */
+        public MarshallerContextWithNoStorage() {
+            super(null);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected boolean registerClassName(int id, String clsName) throws IgniteCheckedException {
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected String className(int id) throws IgniteCheckedException {
+            return null;
+        }
+    }
+
+    /**
+     */
+    private enum TestEnum {
+        A, B, C, D, E
+    }
+
+    /**
+     */
+    private static class SimpleObject {
+        /** */
+        private byte b;
+
+        /** */
+        private char c;
+
+        /** */
+        private byte[] bArr;
+
+        /** */
+        private Object[] objArr;
+
+        /** */
+        private TestEnum enumVal;
+
+        /** */
+        private TestEnum[] enumArr;
+
+        private SimpleObject otherObj;
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            SimpleObject object = (SimpleObject)o;
+
+            if (b != object.b)
+                return false;
+
+            if (c != object.c)
+                return false;
+
+            if (!Arrays.equals(bArr, object.bArr))
+                return false;
+
+            // Probably incorrect - comparing Object[] arrays with Arrays.equals
+            if (!Arrays.equals(objArr, object.objArr))
+                return false;
+
+            if (enumVal != object.enumVal)
+                return false;
+
+            // Probably incorrect - comparing Object[] arrays with Arrays.equals
+            if (!Arrays.equals(enumArr, object.enumArr))
+                return false;
+
+            return !(otherObj != null ? !otherObj.equals(object.otherObj) : object.otherObj != null);
+        }
+    }
+
+    /**
+     *
+     */
+    private static class SimpleBinary implements Binarylizable {
+        /** */
+        private String str;
+
+        /** */
+        private long[] arr;
+
+        /** {@inheritDoc} */
+        @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+            writer.writeString("str", str);
+            writer.writeLongArray("longArr", arr);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+            str = reader.readString("str");
+            arr = reader.readLongArray("longArr");
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            SimpleBinary that = (SimpleBinary)o;
+
+            if (str != null ? !str.equals(that.str) : that.str != null)
+                return false;
+
+            return Arrays.equals(arr, that.arr);
+        }
+    }
+
+    /**
+     *
+     */
+    private static class SimpleExternalizable implements Externalizable {
+        /** */
+        private String str;
+
+        /** */
+        private long[] arr;
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            out.writeUTF(str);
+            out.writeObject(arr);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            str = in.readUTF();
+            arr = (long[])in.readObject();
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            SimpleExternalizable that = (SimpleExternalizable)o;
+
+            if (str != null ? !str.equals(that.str) : that.str != null)
+                return false;
+
+            return Arrays.equals(arr, that.arr);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/test/java/org/apache/ignite/internal/binary/GridPortableAffinityKeySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/GridPortableAffinityKeySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/GridPortableAffinityKeySelfTest.java
deleted file mode 100644
index 74941d0..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/binary/GridPortableAffinityKeySelfTest.java
+++ /dev/null
@@ -1,234 +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.util.Collections;
-import java.util.UUID;
-import java.util.concurrent.atomic.AtomicReference;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.binary.BinaryObjectBuilder;
-import org.apache.ignite.cache.CacheKeyConfiguration;
-import org.apache.ignite.cache.affinity.Affinity;
-import org.apache.ignite.configuration.BinaryConfiguration;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.IgniteKernal;
-import org.apache.ignite.internal.processors.affinity.GridAffinityProcessor;
-import org.apache.ignite.internal.processors.cache.CacheObject;
-import org.apache.ignite.internal.processors.cache.CacheObjectContext;
-import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
-import org.apache.ignite.lang.IgniteCallable;
-import org.apache.ignite.lang.IgniteRunnable;
-import org.apache.ignite.binary.BinaryTypeConfiguration;
-import org.apache.ignite.resources.IgniteInstanceResource;
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-
-import static org.apache.ignite.cache.CacheMode.PARTITIONED;
-
-/**
- * Test for portable object affinity key.
- */
-public class GridPortableAffinityKeySelfTest extends GridCommonAbstractTest {
-    /** */
-    private static final AtomicReference<UUID> nodeId = new AtomicReference<>();
-
-    /** VM ip finder for TCP discovery. */
-    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
-
-    /** */
-    private static int GRID_CNT = 5;
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        BinaryTypeConfiguration typeCfg = new BinaryTypeConfiguration();
-
-        typeCfg.setTypeName(TestObject.class.getName());
-
-        BinaryConfiguration bCfg = new BinaryConfiguration();
-
-        bCfg.setTypeConfigurations(Collections.singleton(typeCfg));
-
-        cfg.setBinaryConfiguration(bCfg);
-
-        CacheKeyConfiguration keyCfg = new CacheKeyConfiguration(TestObject.class.getName(), "affKey");
-        CacheKeyConfiguration keyCfg2 = new CacheKeyConfiguration("TestObject2", "affKey");
-
-        cfg.setCacheKeyConfiguration(keyCfg, keyCfg2);
-
-        cfg.setMarshaller(new BinaryMarshaller());
-
-        if (!gridName.equals(getTestGridName(GRID_CNT))) {
-            CacheConfiguration cacheCfg = new CacheConfiguration();
-
-            cacheCfg.setCacheMode(PARTITIONED);
-
-            cfg.setCacheConfiguration(cacheCfg);
-        }
-
-        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        startGridsMultiThreaded(GRID_CNT);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopAllGrids();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAffinity() throws Exception {
-        checkAffinity(grid(0));
-
-        try (Ignite igniteNoCache = startGrid(GRID_CNT)) {
-            try {
-                igniteNoCache.cache(null);
-            }
-            catch (IllegalArgumentException ignore) {
-                // Expected error.
-            }
-
-            checkAffinity(igniteNoCache);
-        }
-    }
-
-    /**
-     * @param ignite Ignite.
-     * @throws Exception If failed.
-     */
-    private void checkAffinity(Ignite ignite) throws Exception {
-        Affinity<Object> aff = ignite.affinity(null);
-
-        GridAffinityProcessor affProc = ((IgniteKernal)ignite).context().affinity();
-
-        IgniteCacheObjectProcessor cacheObjProc = ((IgniteKernal)ignite).context().cacheObjects();
-
-        CacheObjectContext cacheObjCtx = cacheObjProc.contextForCache(
-            ignite.cache(null).getConfiguration(CacheConfiguration.class));
-
-        for (int i = 0; i < 1000; i++) {
-            assertEquals(i, aff.affinityKey(i));
-
-            assertEquals(i, aff.affinityKey(new TestObject(i)));
-
-            assertEquals(i, aff.affinityKey(ignite.binary().toBinary(new TestObject(i))));
-
-            BinaryObjectBuilder bldr = ignite.binary().builder("TestObject2");
-
-            bldr.setField("affKey", i);
-
-            assertEquals(i, aff.affinityKey(bldr.build()));
-
-            CacheObject cacheObj = cacheObjProc.toCacheObject(cacheObjCtx, new TestObject(i), true);
-
-            assertEquals(i, aff.affinityKey(cacheObj));
-
-            assertEquals(aff.mapKeyToNode(i), aff.mapKeyToNode(new TestObject(i)));
-
-            assertEquals(aff.mapKeyToNode(i), aff.mapKeyToNode(cacheObj));
-
-            assertEquals(i, affProc.affinityKey(null, i));
-
-            assertEquals(i, affProc.affinityKey(null, new TestObject(i)));
-
-            assertEquals(i, affProc.affinityKey(null, cacheObj));
-
-            assertEquals(affProc.mapKeyToNode(null, i), affProc.mapKeyToNode(null, new TestObject(i)));
-
-            assertEquals(affProc.mapKeyToNode(null, i), affProc.mapKeyToNode(null, cacheObj));
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAffinityRun() throws Exception {
-        Affinity<Object> aff = grid(0).affinity(null);
-
-        for (int i = 0; i < 1000; i++) {
-            nodeId.set(null);
-
-            grid(0).compute().affinityRun(null, new TestObject(i), new IgniteRunnable() {
-                @IgniteInstanceResource
-                private Ignite ignite;
-
-                @Override public void run() {
-                    nodeId.set(ignite.configuration().getNodeId());
-                }
-            });
-
-            assertEquals(aff.mapKeyToNode(i).id(), nodeId.get());
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAffinityCall() throws Exception {
-        Affinity<Object> aff = grid(0).affinity(null);
-
-        for (int i = 0; i < 1000; i++) {
-            nodeId.set(null);
-
-            grid(0).compute().affinityCall(null, new TestObject(i), new IgniteCallable<Object>() {
-                @IgniteInstanceResource
-                private Ignite ignite;
-
-                @Override public Object call() {
-                    nodeId.set(ignite.configuration().getNodeId());
-
-                    return null;
-                }
-            });
-
-            assertEquals(aff.mapKeyToNode(i).id(), nodeId.get());
-        }
-    }
-
-    /**
-     */
-    private static class TestObject {
-        /** */
-        @SuppressWarnings("UnusedDeclaration")
-        private int affKey;
-
-        /**
-         */
-        private TestObject() {
-            // No-op.
-        }
-
-        /**
-         * @param affKey Affinity key.
-         */
-        private TestObject(int affKey) {
-            this.affKey = affKey;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/test/java/org/apache/ignite/internal/binary/GridPortableMarshallerCtxDisabledSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/GridPortableMarshallerCtxDisabledSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/GridPortableMarshallerCtxDisabledSelfTest.java
deleted file mode 100644
index a30d09f..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/binary/GridPortableMarshallerCtxDisabledSelfTest.java
+++ /dev/null
@@ -1,247 +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.IgniteCheckedException;
-import org.apache.ignite.binary.BinaryObjectException;
-import org.apache.ignite.binary.BinaryReader;
-import org.apache.ignite.binary.BinaryWriter;
-import org.apache.ignite.binary.Binarylizable;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.MarshallerContextAdapter;
-import org.apache.ignite.internal.util.IgniteUtils;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.util.Arrays;
-
-/**
- *
- */
-public class GridPortableMarshallerCtxDisabledSelfTest extends GridCommonAbstractTest {
-    /**
-     * @throws Exception If failed.
-     */
-    public void testObjectExchange() throws Exception {
-        BinaryMarshaller marsh = new BinaryMarshaller();
-        marsh.setContext(new MarshallerContextWithNoStorage());
-
-        IgniteConfiguration cfg = new IgniteConfiguration();
-
-        PortableContext context = new PortableContext(BinaryCachingMetadataHandler.create(), cfg);
-
-        IgniteUtils.invoke(BinaryMarshaller.class, marsh, "setPortableContext", context, cfg);
-
-        SimpleObject simpleObj = new SimpleObject();
-
-        simpleObj.b = 2;
-        simpleObj.bArr = new byte[] {2, 3, 4, 5, 5};
-        simpleObj.c = 'A';
-        simpleObj.enumVal = TestEnum.D;
-        simpleObj.objArr = new Object[] {"hello", "world", "from", "me"};
-        simpleObj.enumArr = new TestEnum[] {TestEnum.C, TestEnum.B};
-
-        SimpleObject otherObj = new SimpleObject();
-
-        otherObj.b = 3;
-        otherObj.bArr = new byte[] {5, 3, 4};
-
-        simpleObj.otherObj = otherObj;
-
-        assertEquals(simpleObj, marsh.unmarshal(marsh.marshal(simpleObj), null));
-
-        SimpleBinary simplePortable = new SimpleBinary();
-
-        simplePortable.str = "portable";
-        simplePortable.arr = new long[] {100, 200, 300};
-
-        assertEquals(simplePortable, marsh.unmarshal(marsh.marshal(simplePortable), null));
-
-        SimpleExternalizable simpleExtr = new SimpleExternalizable();
-
-        simpleExtr.str = "externalizable";
-        simpleExtr.arr = new long[] {20000, 300000, 400000};
-
-        assertEquals(simpleExtr, marsh.unmarshal(marsh.marshal(simpleExtr), null));
-    }
-
-    /**
-     * Marshaller context with no storage. Platform has to work in such environment as well by marshalling class name of
-     * a portable object.
-     */
-    private static class MarshallerContextWithNoStorage extends MarshallerContextAdapter {
-        /** */
-        public MarshallerContextWithNoStorage() {
-            super(null);
-        }
-
-        /** {@inheritDoc} */
-        @Override protected boolean registerClassName(int id, String clsName) throws IgniteCheckedException {
-            return false;
-        }
-
-        /** {@inheritDoc} */
-        @Override protected String className(int id) throws IgniteCheckedException {
-            return null;
-        }
-    }
-
-    /**
-     */
-    private enum TestEnum {
-        A, B, C, D, E
-    }
-
-    /**
-     */
-    private static class SimpleObject {
-        /** */
-        private byte b;
-
-        /** */
-        private char c;
-
-        /** */
-        private byte[] bArr;
-
-        /** */
-        private Object[] objArr;
-
-        /** */
-        private TestEnum enumVal;
-
-        /** */
-        private TestEnum[] enumArr;
-
-        private SimpleObject otherObj;
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object o) {
-            if (this == o)
-                return true;
-
-            if (o == null || getClass() != o.getClass())
-                return false;
-
-            SimpleObject object = (SimpleObject)o;
-
-            if (b != object.b)
-                return false;
-
-            if (c != object.c)
-                return false;
-
-            if (!Arrays.equals(bArr, object.bArr))
-                return false;
-
-            // Probably incorrect - comparing Object[] arrays with Arrays.equals
-            if (!Arrays.equals(objArr, object.objArr))
-                return false;
-
-            if (enumVal != object.enumVal)
-                return false;
-
-            // Probably incorrect - comparing Object[] arrays with Arrays.equals
-            if (!Arrays.equals(enumArr, object.enumArr))
-                return false;
-
-            return !(otherObj != null ? !otherObj.equals(object.otherObj) : object.otherObj != null);
-        }
-    }
-
-    /**
-     *
-     */
-    private static class SimpleBinary implements Binarylizable {
-        /** */
-        private String str;
-
-        /** */
-        private long[] arr;
-
-        /** {@inheritDoc} */
-        @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
-            writer.writeString("str", str);
-            writer.writeLongArray("longArr", arr);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
-            str = reader.readString("str");
-            arr = reader.readLongArray("longArr");
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object o) {
-            if (this == o)
-                return true;
-
-            if (o == null || getClass() != o.getClass())
-                return false;
-
-            SimpleBinary that = (SimpleBinary)o;
-
-            if (str != null ? !str.equals(that.str) : that.str != null)
-                return false;
-
-            return Arrays.equals(arr, that.arr);
-        }
-    }
-
-    /**
-     *
-     */
-    private static class SimpleExternalizable implements Externalizable {
-        /** */
-        private String str;
-
-        /** */
-        private long[] arr;
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            out.writeUTF(str);
-            out.writeObject(arr);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            str = in.readUTF();
-            arr = (long[])in.readObject();
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object o) {
-            if (this == o)
-                return true;
-
-            if (o == null || getClass() != o.getClass())
-                return false;
-
-            SimpleExternalizable that = (SimpleExternalizable)o;
-
-            if (str != null ? !str.equals(that.str) : that.str != null)
-                return false;
-
-            return Arrays.equals(arr, that.arr);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/test/java/org/apache/ignite/internal/binary/GridPortableWildcardsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/GridPortableWildcardsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/GridPortableWildcardsSelfTest.java
index a91e350..9226272 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/binary/GridPortableWildcardsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/GridPortableWildcardsSelfTest.java
@@ -45,7 +45,7 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
             new BinaryTypeConfiguration("unknown.*")
         ));
 
-        PortableContext ctx = portableContext(marsh);
+        BinaryContext ctx = portableContext(marsh);
 
         Map<Integer, Class> typeIds = U.field(ctx, "userTypes");
 
@@ -81,7 +81,7 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
             new BinaryTypeConfiguration("unknown.*")
         ));
 
-        PortableContext ctx = portableContext(marsh);
+        BinaryContext ctx = portableContext(marsh);
 
         Map<String, BinaryIdMapper> typeMappers = U.field(ctx, "typeMappers");
 
@@ -101,7 +101,7 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
             new BinaryTypeConfiguration("unknown.*")
         ));
 
-        PortableContext ctx = portableContext(marsh);
+        BinaryContext ctx = portableContext(marsh);
 
         Map<Integer, Class> typeIds = U.field(ctx, "userTypes");
 
@@ -137,7 +137,7 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
             new BinaryTypeConfiguration("unknown.*")
         ));
 
-        PortableContext ctx = portableContext(marsh);
+        BinaryContext ctx = portableContext(marsh);
 
         Map<String, BinaryIdMapper> typeMappers = U.field(ctx, "typeMappers");
 
@@ -173,7 +173,7 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
             new BinaryTypeConfiguration("unknown.*")
         ));
 
-        PortableContext ctx = portableContext(marsh);
+        BinaryContext ctx = portableContext(marsh);
 
         Map<String, BinaryIdMapper> typeMappers = U.field(ctx, "typeMappers");
 
@@ -205,7 +205,7 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
             new BinaryTypeConfiguration("org.apache.ignite.internal.binary.test.*"),
             typeCfg));
 
-        PortableContext ctx = portableContext(marsh);
+        BinaryContext ctx = portableContext(marsh);
 
         Map<Integer, Class> typeIds = U.field(ctx, "userTypes");
 
@@ -229,7 +229,7 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
             new BinaryTypeConfiguration("unknown.*")
         ));
 
-        PortableContext ctx = portableContext(marsh);
+        BinaryContext ctx = portableContext(marsh);
 
         Map<Integer, Class> typeIds = U.field(ctx, "userTypes");
 
@@ -262,7 +262,7 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
             new BinaryTypeConfiguration("unknown.*")
         ));
 
-        PortableContext ctx = portableContext(marsh);
+        BinaryContext ctx = portableContext(marsh);
 
         Map<String, BinaryIdMapper> typeMappers = U.field(ctx, "typeMappers");
 
@@ -281,7 +281,7 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
             new BinaryTypeConfiguration("unknown.*")
         ));
 
-        PortableContext ctx = portableContext(marsh);
+        BinaryContext ctx = portableContext(marsh);
 
         Map<Integer, Class> typeIds = U.field(ctx, "userTypes");
 
@@ -314,7 +314,7 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
             new BinaryTypeConfiguration("unknown.*")
         ));
 
-        PortableContext ctx = portableContext(marsh);
+        BinaryContext ctx = portableContext(marsh);
 
         Map<String, BinaryIdMapper> typeMappers = U.field(ctx, "typeMappers");
 
@@ -347,7 +347,7 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
             new BinaryTypeConfiguration("unknown.*")
         ));
 
-        PortableContext ctx = portableContext(marsh);
+        BinaryContext ctx = portableContext(marsh);
 
         Map<String, BinaryIdMapper> typeMappers = U.field(ctx, "typeMappers");
 
@@ -378,7 +378,7 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
             new BinaryTypeConfiguration("org.apache.ignite.binary.testjar.*"),
             typeCfg));
 
-        PortableContext ctx = portableContext(marsh);
+        BinaryContext ctx = portableContext(marsh);
 
         Map<Integer, Class> typeIds = U.field(ctx, "userTypes");
 
@@ -397,8 +397,8 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
      * @param marsh Marshaller.
      * @return Portable context.
      */
-    protected PortableContext portableContext(BinaryMarshaller marsh) {
-        GridPortableMarshaller impl = U.field(marsh, "impl");
+    protected BinaryContext portableContext(BinaryMarshaller marsh) {
+        GridBinaryMarshaller impl = U.field(marsh, "impl");
 
         return impl.context();
     }
@@ -451,7 +451,7 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
 
         iCfg.setBinaryConfiguration(bCfg);
 
-        PortableContext ctx = new PortableContext(BinaryNoopMetadataHandler.instance(), iCfg);
+        BinaryContext ctx = new BinaryContext(BinaryNoopMetadataHandler.instance(), iCfg);
 
         BinaryMarshaller marsh = new BinaryMarshaller();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridBinaryCacheEntryMemorySizeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridBinaryCacheEntryMemorySizeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridBinaryCacheEntryMemorySizeSelfTest.java
new file mode 100644
index 0000000..128e316
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridBinaryCacheEntryMemorySizeSelfTest.java
@@ -0,0 +1,48 @@
+/*
+ * 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.IgniteCheckedException;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.binary.BinaryNoopMetadataHandler;
+import org.apache.ignite.internal.binary.BinaryContext;
+import org.apache.ignite.internal.processors.cache.GridCacheEntryMemorySizeSelfTest;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.marshaller.Marshaller;
+import org.apache.ignite.marshaller.MarshallerContextTestImpl;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+
+/**
+ *
+ */
+public class GridBinaryCacheEntryMemorySizeSelfTest extends GridCacheEntryMemorySizeSelfTest {
+    /** {@inheritDoc} */
+    @Override protected Marshaller createMarshaller() throws IgniteCheckedException {
+        BinaryMarshaller marsh = new BinaryMarshaller();
+
+        marsh.setContext(new MarshallerContextTestImpl(null));
+
+        IgniteConfiguration iCfg = new IgniteConfiguration();
+
+        BinaryContext pCtx = new BinaryContext(BinaryNoopMetadataHandler.instance(), iCfg);
+
+        IgniteUtils.invoke(BinaryMarshaller.class, marsh, "setPortableContext", pCtx, iCfg);
+
+        return marsh;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridBinaryDuplicateIndexObjectsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridBinaryDuplicateIndexObjectsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridBinaryDuplicateIndexObjectsAbstractSelfTest.java
new file mode 100644
index 0000000..9306958
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridBinaryDuplicateIndexObjectsAbstractSelfTest.java
@@ -0,0 +1,161 @@
+/*
+ * 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.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CacheTypeMetadata;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.configuration.BinaryConfiguration;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.binary.BinaryObject;
+
+/**
+ * Tests that portable object is the same in cache entry and in index.
+ */
+public abstract class GridBinaryDuplicateIndexObjectsAbstractSelfTest extends GridCacheAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        BinaryConfiguration bCfg = new BinaryConfiguration();
+
+        bCfg.setClassNames(Collections.singletonList(TestPortable.class.getName()));
+
+        cfg.setBinaryConfiguration(bCfg);
+
+        cfg.setMarshaller(new BinaryMarshaller());
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
+        CacheConfiguration ccfg = super.cacheConfiguration(gridName);
+
+        ccfg.setCopyOnRead(false);
+
+        CacheTypeMetadata meta = new CacheTypeMetadata();
+
+        meta.setKeyType(Integer.class);
+        meta.setValueType(TestPortable.class.getName());
+
+        Map<String, Class<?>> idx = new HashMap<>();
+
+        idx.put("fieldOne", String.class);
+        idx.put("fieldTwo", Integer.class);
+
+        meta.setAscendingFields(idx);
+
+        ccfg.setTypeMetadata(Collections.singletonList(meta));
+
+        return ccfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override public abstract CacheAtomicityMode atomicityMode();
+
+    /** {@inheritDoc} */
+    @Override public abstract CacheMode cacheMode();
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIndexReferences() throws Exception {
+        IgniteCache<Integer, TestPortable> cache = grid(0).cache(null);
+
+        String fieldOneVal = "123";
+        int fieldTwoVal = 123;
+        int key = 0;
+
+        cache.put(key, new TestPortable(fieldOneVal, fieldTwoVal));
+
+        IgniteCache<Integer, BinaryObject> prj = grid(0).cache(null).withKeepBinary();
+
+        BinaryObject cacheVal = prj.get(key);
+
+        assertEquals(fieldOneVal, cacheVal.field("fieldOne"));
+        assertEquals(new Integer(fieldTwoVal), cacheVal.field("fieldTwo"));
+
+        List<?> row = F.first(prj.query(new SqlFieldsQuery("select _val from " +
+            "TestPortable where _key = ?").setArgs(key)).getAll());
+
+        assertEquals(1, row.size());
+
+        BinaryObject qryVal = (BinaryObject)row.get(0);
+
+        assertEquals(fieldOneVal, qryVal.field("fieldOne"));
+        assertEquals(new Integer(fieldTwoVal), qryVal.field("fieldTwo"));
+        assertSame(cacheVal, qryVal);
+    }
+
+    /**
+     * Test portable object.
+     */
+    private static class TestPortable {
+        /** */
+        private String fieldOne;
+
+        /** */
+        private int fieldTwo;
+
+        /**
+         *
+         */
+        private TestPortable() {
+            // No-op.
+        }
+
+        /**
+         * @param fieldOne Field one.
+         * @param fieldTwo Field two.
+         */
+        private TestPortable(String fieldOne, int fieldTwo) {
+            this.fieldOne = fieldOne;
+            this.fieldTwo = fieldTwo;
+        }
+
+        /**
+         * @return Field one.
+         */
+        public String fieldOne() {
+            return fieldOne;
+        }
+
+        /**
+         * @return Field two.
+         */
+        public int fieldTwo() {
+            return fieldTwo;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/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
new file mode 100644
index 0000000..058dc0b
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryStoreAbstractSelfTest.java
@@ -0,0 +1,300 @@
+/*
+ * 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 com.google.common.collect.ImmutableSet;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import javax.cache.Cache;
+import org.apache.ignite.cache.store.CacheStoreAdapter;
+import org.apache.ignite.configuration.BinaryConfiguration;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.jetbrains.annotations.Nullable;
+import org.jsr166.ConcurrentHashMap8;
+
+/**
+ * Tests for cache store with binary.
+ */
+public abstract class GridCacheBinaryStoreAbstractSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final TestStore STORE = new TestStore();
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        BinaryConfiguration bCfg = new BinaryConfiguration();
+
+        bCfg.setClassNames(Arrays.asList(Key.class.getName(), Value.class.getName()));
+
+        cfg.setBinaryConfiguration(bCfg);
+
+        cfg.setMarshaller(new BinaryMarshaller());
+
+        CacheConfiguration cacheCfg = new CacheConfiguration();
+
+        cacheCfg.setCacheStoreFactory(singletonFactory(STORE));
+        cacheCfg.setKeepBinaryInStore(keepPortableInStore());
+        cacheCfg.setReadThrough(true);
+        cacheCfg.setWriteThrough(true);
+        cacheCfg.setLoadPreviousValue(true);
+
+        cfg.setCacheConfiguration(cacheCfg);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disco);
+
+        return cfg;
+    }
+
+    /**
+     * @return Keep binary in store flag.
+     */
+    protected abstract boolean keepPortableInStore();
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGrid();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopGrid();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        STORE.map().clear();
+
+        jcache().clear();
+
+        assert jcache().size() == 0;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPut() throws Exception {
+        jcache().put(new Key(1), new Value(1));
+
+        checkMap(STORE.map(), 1);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAll() throws Exception {
+        Map<Object, Object> map = new HashMap<>();
+
+        for (int i = 1; i <= 3; i++)
+            map.put(new Key(i), new Value(i));
+
+        jcache().putAll(map);
+
+        checkMap(STORE.map(), 1, 2, 3);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLoad() throws Exception {
+        populateMap(STORE.map(), 1);
+
+        Object val = jcache().get(new Key(1));
+
+        assertTrue(String.valueOf(val), val instanceof Value);
+
+        assertEquals(1, ((Value)val).index());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLoadAll() throws Exception {
+        populateMap(STORE.map(), 1, 2, 3);
+
+        Set<Object> keys = new HashSet<>();
+
+        for (int i = 1; i <= 3; i++)
+            keys.add(new Key(i));
+
+        Map<Object, Object> res = jcache().getAll(keys);
+
+        assertEquals(3, res.size());
+
+        for (int i = 1; i <= 3; i++) {
+            Object val = res.get(new Key(i));
+
+            assertTrue(String.valueOf(val), val instanceof Value);
+
+            assertEquals(i, ((Value)val).index());
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRemove() throws Exception {
+        for (int i = 1; i <= 3; i++)
+            jcache().put(new Key(i), new Value(i));
+
+        jcache().remove(new Key(1));
+
+        checkMap(STORE.map(), 2, 3);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRemoveAll() throws Exception {
+        for (int i = 1; i <= 3; i++)
+            jcache().put(new Key(i), new Value(i));
+
+        jcache().removeAll(ImmutableSet.of(new Key(1), new Key(2)));
+
+        checkMap(STORE.map(), 3);
+    }
+
+    /**
+     * @param map Map.
+     * @param idxs Indexes.
+     */
+    protected abstract void populateMap(Map<Object, Object> map, int... idxs);
+
+    /**
+     * @param map Map.
+     * @param idxs Indexes.
+     */
+    protected abstract void checkMap(Map<Object, Object> map, int... idxs);
+
+    /**
+     */
+    protected static class Key {
+        /** */
+        private int idx;
+
+        /**
+         * @param idx Index.
+         */
+        public Key(int idx) {
+            this.idx = idx;
+        }
+
+        /**
+         * @return Index.
+         */
+        int index() {
+            return idx;
+        }
+
+        /** {@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 idx == key.idx;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return idx;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return "Key [idx=" + idx + ']';
+        }
+    }
+
+    /**
+     */
+    protected static class Value {
+        /** */
+        private int idx;
+
+        /**
+         * @param idx Index.
+         */
+        public Value(int idx) {
+            this.idx = idx;
+        }
+
+        /**
+         * @return Index.
+         */
+        int index() {
+            return idx;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return "Value [idx=" + idx + ']';
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestStore extends CacheStoreAdapter<Object, Object> {
+        /** */
+        private final Map<Object, Object> map = new ConcurrentHashMap8<>();
+
+        /** {@inheritDoc} */
+        @Nullable @Override public Object load(Object key) {
+            return map.get(key);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Cache.Entry<?, ?> e) {
+            map.put(e.getKey(), e.getValue());
+        }
+
+        /** {@inheritDoc} */
+        @Override public void delete(Object key) {
+            map.remove(key);
+        }
+
+        /**
+         * @return Map.
+         */
+        Map<Object, Object> map() {
+            return map;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryStoreObjectsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryStoreObjectsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryStoreObjectsSelfTest.java
new file mode 100644
index 0000000..a4d08ec
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryStoreObjectsSelfTest.java
@@ -0,0 +1,55 @@
+/*
+ * 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;
+
+/**
+ * Tests for cache store with binary.
+ */
+public class GridCacheBinaryStoreObjectsSelfTest extends GridCacheBinaryStoreAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean keepPortableInStore() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void populateMap(Map<Object, Object> map, int... idxs) {
+        assert map != null;
+        assert idxs != null;
+
+        for (int idx : idxs)
+            map.put(new Key(idx), 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(new Key(idx));
+
+            assertTrue(String.valueOf(val), val instanceof Value);
+
+            assertEquals(idx, ((Value)val).index());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryStorePortablesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryStorePortablesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryStorePortablesSelfTest.java
new file mode 100644
index 0000000..db15f08
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryStorePortablesSelfTest.java
@@ -0,0 +1,66 @@
+/*
+ * 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 GridCacheBinaryStorePortablesSelfTest extends GridCacheBinaryStoreAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean keepPortableInStore() {
+        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(portable(new Key(idx)), portable(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(portable(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 Portable object.
+     */
+    private Object portable(Object obj) {
+        return grid().binary().toBinary(obj);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCachePortableStoreAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCachePortableStoreAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCachePortableStoreAbstractSelfTest.java
deleted file mode 100644
index b9d7b5d..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCachePortableStoreAbstractSelfTest.java
+++ /dev/null
@@ -1,300 +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 com.google.common.collect.ImmutableSet;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import javax.cache.Cache;
-import org.apache.ignite.cache.store.CacheStoreAdapter;
-import org.apache.ignite.configuration.BinaryConfiguration;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.binary.BinaryMarshaller;
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.jetbrains.annotations.Nullable;
-import org.jsr166.ConcurrentHashMap8;
-
-/**
- * Tests for cache store with binary.
- */
-public abstract class GridCachePortableStoreAbstractSelfTest extends GridCommonAbstractTest {
-    /** */
-    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
-
-    /** */
-    private static final TestStore STORE = new TestStore();
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        BinaryConfiguration bCfg = new BinaryConfiguration();
-
-        bCfg.setClassNames(Arrays.asList(Key.class.getName(), Value.class.getName()));
-
-        cfg.setBinaryConfiguration(bCfg);
-
-        cfg.setMarshaller(new BinaryMarshaller());
-
-        CacheConfiguration cacheCfg = new CacheConfiguration();
-
-        cacheCfg.setCacheStoreFactory(singletonFactory(STORE));
-        cacheCfg.setKeepBinaryInStore(keepPortableInStore());
-        cacheCfg.setReadThrough(true);
-        cacheCfg.setWriteThrough(true);
-        cacheCfg.setLoadPreviousValue(true);
-
-        cfg.setCacheConfiguration(cacheCfg);
-
-        TcpDiscoverySpi disco = new TcpDiscoverySpi();
-
-        disco.setIpFinder(IP_FINDER);
-
-        cfg.setDiscoverySpi(disco);
-
-        return cfg;
-    }
-
-    /**
-     * @return Keep binary in store flag.
-     */
-    protected abstract boolean keepPortableInStore();
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        startGrid();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopGrid();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        STORE.map().clear();
-
-        jcache().clear();
-
-        assert jcache().size() == 0;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPut() throws Exception {
-        jcache().put(new Key(1), new Value(1));
-
-        checkMap(STORE.map(), 1);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPutAll() throws Exception {
-        Map<Object, Object> map = new HashMap<>();
-
-        for (int i = 1; i <= 3; i++)
-            map.put(new Key(i), new Value(i));
-
-        jcache().putAll(map);
-
-        checkMap(STORE.map(), 1, 2, 3);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testLoad() throws Exception {
-        populateMap(STORE.map(), 1);
-
-        Object val = jcache().get(new Key(1));
-
-        assertTrue(String.valueOf(val), val instanceof Value);
-
-        assertEquals(1, ((Value)val).index());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testLoadAll() throws Exception {
-        populateMap(STORE.map(), 1, 2, 3);
-
-        Set<Object> keys = new HashSet<>();
-
-        for (int i = 1; i <= 3; i++)
-            keys.add(new Key(i));
-
-        Map<Object, Object> res = jcache().getAll(keys);
-
-        assertEquals(3, res.size());
-
-        for (int i = 1; i <= 3; i++) {
-            Object val = res.get(new Key(i));
-
-            assertTrue(String.valueOf(val), val instanceof Value);
-
-            assertEquals(i, ((Value)val).index());
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testRemove() throws Exception {
-        for (int i = 1; i <= 3; i++)
-            jcache().put(new Key(i), new Value(i));
-
-        jcache().remove(new Key(1));
-
-        checkMap(STORE.map(), 2, 3);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testRemoveAll() throws Exception {
-        for (int i = 1; i <= 3; i++)
-            jcache().put(new Key(i), new Value(i));
-
-        jcache().removeAll(ImmutableSet.of(new Key(1), new Key(2)));
-
-        checkMap(STORE.map(), 3);
-    }
-
-    /**
-     * @param map Map.
-     * @param idxs Indexes.
-     */
-    protected abstract void populateMap(Map<Object, Object> map, int... idxs);
-
-    /**
-     * @param map Map.
-     * @param idxs Indexes.
-     */
-    protected abstract void checkMap(Map<Object, Object> map, int... idxs);
-
-    /**
-     */
-    protected static class Key {
-        /** */
-        private int idx;
-
-        /**
-         * @param idx Index.
-         */
-        public Key(int idx) {
-            this.idx = idx;
-        }
-
-        /**
-         * @return Index.
-         */
-        int index() {
-            return idx;
-        }
-
-        /** {@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 idx == key.idx;
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            return idx;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return "Key [idx=" + idx + ']';
-        }
-    }
-
-    /**
-     */
-    protected static class Value {
-        /** */
-        private int idx;
-
-        /**
-         * @param idx Index.
-         */
-        public Value(int idx) {
-            this.idx = idx;
-        }
-
-        /**
-         * @return Index.
-         */
-        int index() {
-            return idx;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return "Value [idx=" + idx + ']';
-        }
-    }
-
-    /**
-     *
-     */
-    private static class TestStore extends CacheStoreAdapter<Object, Object> {
-        /** */
-        private final Map<Object, Object> map = new ConcurrentHashMap8<>();
-
-        /** {@inheritDoc} */
-        @Nullable @Override public Object load(Object key) {
-            return map.get(key);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void write(Cache.Entry<?, ?> e) {
-            map.put(e.getKey(), e.getValue());
-        }
-
-        /** {@inheritDoc} */
-        @Override public void delete(Object key) {
-            map.remove(key);
-        }
-
-        /**
-         * @return Map.
-         */
-        Map<Object, Object> map() {
-            return map;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCachePortableStoreObjectsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCachePortableStoreObjectsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCachePortableStoreObjectsSelfTest.java
deleted file mode 100644
index 00d1ae4..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCachePortableStoreObjectsSelfTest.java
+++ /dev/null
@@ -1,55 +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;
-
-/**
- * Tests for cache store with binary.
- */
-public class GridCachePortableStoreObjectsSelfTest extends GridCachePortableStoreAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected boolean keepPortableInStore() {
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void populateMap(Map<Object, Object> map, int... idxs) {
-        assert map != null;
-        assert idxs != null;
-
-        for (int idx : idxs)
-            map.put(new Key(idx), 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(new Key(idx));
-
-            assertTrue(String.valueOf(val), val instanceof Value);
-
-            assertEquals(idx, ((Value)val).index());
-        }
-    }
-}