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/06/10 18:27:33 UTC

[10/31] incubator-ignite git commit: ignite-471-2: huge merge from sprint-6

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d78aa15/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java
new file mode 100644
index 0000000..e5d30b6
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java
@@ -0,0 +1,632 @@
+/*
+ * 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.distributed;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.affinity.*;
+import org.apache.ignite.cache.affinity.fair.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.events.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.managers.communication.*;
+import org.apache.ignite.internal.managers.discovery.*;
+import org.apache.ignite.internal.processors.affinity.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.internal.processors.cache.distributed.dht.*;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.*;
+import org.apache.ignite.internal.util.lang.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.plugin.extensions.communication.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.spi.communication.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ *
+ */
+public class IgniteCacheClientNodePartitionsExchangeTest extends GridCommonAbstractTest {
+    /** */
+    protected static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private boolean client;
+
+    /** */
+    private boolean fairAffinity;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder).setForceServerMode(true);
+
+        cfg.setClientMode(client);
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        if (fairAffinity)
+            ccfg.setAffinity(new FairAffinityFunction());
+
+        cfg.setCacheConfiguration(ccfg);
+
+        cfg.setCommunicationSpi(new TestCommunicationSpi());
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testServerNodeLeave() throws Exception {
+        Ignite ignite0 = startGrid(0);
+
+        client = true;
+
+        final Ignite ignite1 = startGrid(1);
+
+        waitForTopologyUpdate(2, 2);
+
+        final Ignite ignite2 = startGrid(2);
+
+        waitForTopologyUpdate(3, 3);
+
+        ignite0.close();
+
+        waitForTopologyUpdate(2, 4);
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                ignite1.cache(null).get(1);
+
+                return null;
+            }
+        }, CacheServerNotFoundException.class, null);
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                ignite2.cache(null).get(1);
+
+                return null;
+            }
+        }, CacheServerNotFoundException.class, null);
+
+        ignite1.close();
+
+        waitForTopologyUpdate(1, 5);
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                ignite2.cache(null).get(1);
+
+                return null;
+            }
+        }, CacheServerNotFoundException.class, null);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSkipPreload() throws Exception {
+        Ignite ignite0 = startGrid(0);
+
+        final CountDownLatch evtLatch0 = new CountDownLatch(1);
+
+        ignite0.events().localListen(new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                log.info("Rebalance event: " + evt);
+
+                evtLatch0.countDown();
+
+                return true;
+            }
+        }, EventType.EVT_CACHE_REBALANCE_STARTED, EventType.EVT_CACHE_REBALANCE_STOPPED);
+
+        client = true;
+
+        Ignite ignite1 = startGrid(1);
+
+        assertTrue(evtLatch0.await(1000, TimeUnit.MILLISECONDS));
+
+        ignite1.close();
+
+        assertTrue(evtLatch0.await(1000, TimeUnit.MILLISECONDS));
+
+        ignite1 = startGrid(1);
+
+        final CountDownLatch evtLatch1 = new CountDownLatch(1);
+
+        ignite1.events().localListen(new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                log.info("Rebalance event: " + evt);
+
+                evtLatch1.countDown();
+
+                return true;
+            }
+        }, EventType.EVT_CACHE_REBALANCE_STARTED, EventType.EVT_CACHE_REBALANCE_STOPPED);
+
+        assertTrue(evtLatch0.await(1000, TimeUnit.MILLISECONDS));
+
+        client = false;
+
+        startGrid(2);
+
+        assertTrue(evtLatch0.await(1000, TimeUnit.MILLISECONDS));
+        assertFalse(evtLatch1.await(1000, TimeUnit.MILLISECONDS));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPartitionsExchange() throws Exception {
+        partitionsExchange();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPartitionsExchangeFairAffinity() throws Exception {
+        fairAffinity = true;
+
+        partitionsExchange();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void partitionsExchange() throws Exception {
+        Ignite ignite0 = startGrid(0);
+
+        TestCommunicationSpi spi0 = (TestCommunicationSpi)ignite0.configuration().getCommunicationSpi();
+
+        Ignite ignite1 = startGrid(1);
+
+        waitForTopologyUpdate(2, 2);
+
+        TestCommunicationSpi spi1 = (TestCommunicationSpi)ignite1.configuration().getCommunicationSpi();
+
+        assertEquals(0, spi0.partitionsSingleMessages());
+        assertEquals(1, spi0.partitionsFullMessages());
+
+        assertEquals(1, spi1.partitionsSingleMessages());
+        assertEquals(0, spi1.partitionsFullMessages());
+
+        spi0.reset();
+        spi1.reset();
+
+        client = true;
+
+        log.info("Start client node1.");
+
+        Ignite ignite2 = startGrid(2);
+
+        waitForTopologyUpdate(3, 3);
+
+        TestCommunicationSpi spi2 = (TestCommunicationSpi)ignite2.configuration().getCommunicationSpi();
+
+        assertEquals(0, spi0.partitionsSingleMessages());
+        assertEquals(1, spi0.partitionsFullMessages());
+
+        assertEquals(0, spi1.partitionsSingleMessages());
+        assertEquals(0, spi1.partitionsFullMessages());
+
+        assertEquals(1, spi2.partitionsSingleMessages());
+        assertEquals(0, spi2.partitionsFullMessages());
+
+        spi0.reset();
+        spi1.reset();
+        spi2.reset();
+
+        log.info("Start client node2.");
+
+        Ignite ignite3 = startGrid(3);
+
+        waitForTopologyUpdate(4, 4);
+
+        TestCommunicationSpi spi3 = (TestCommunicationSpi)ignite3.configuration().getCommunicationSpi();
+
+        assertEquals(0, spi0.partitionsSingleMessages());
+        assertEquals(1, spi0.partitionsFullMessages());
+
+        assertEquals(0, spi1.partitionsSingleMessages());
+        assertEquals(0, spi1.partitionsFullMessages());
+
+        assertEquals(0, spi2.partitionsSingleMessages());
+        assertEquals(0, spi2.partitionsFullMessages());
+
+        assertEquals(1, spi3.partitionsSingleMessages());
+        assertEquals(0, spi3.partitionsFullMessages());
+
+        spi0.reset();
+        spi1.reset();
+        spi2.reset();
+        spi3.reset();
+
+        log.info("Start one more server node.");
+
+        client = false;
+
+        Ignite ignite4 = startGrid(4);
+
+        waitForTopologyUpdate(5, 5);
+
+        TestCommunicationSpi spi4 = (TestCommunicationSpi)ignite4.configuration().getCommunicationSpi();
+
+        assertEquals(0, spi0.partitionsSingleMessages());
+        assertEquals(4, spi0.partitionsFullMessages());
+
+        assertEquals(1, spi1.partitionsSingleMessages());
+        assertEquals(0, spi1.partitionsFullMessages());
+
+        assertEquals(1, spi2.partitionsSingleMessages());
+        assertEquals(0, spi2.partitionsFullMessages());
+
+        assertEquals(1, spi3.partitionsSingleMessages());
+        assertEquals(0, spi3.partitionsFullMessages());
+
+        assertEquals(1, spi4.partitionsSingleMessages());
+        assertEquals(0, spi4.partitionsFullMessages());
+
+        spi0.reset();
+        spi1.reset();
+        spi2.reset();
+        spi3.reset();
+
+        log.info("Stop server node.");
+
+        ignite4.close();
+
+        waitForTopologyUpdate(4, 6);
+
+        assertEquals(0, spi0.partitionsSingleMessages());
+        assertEquals(3, spi0.partitionsFullMessages());
+
+        assertEquals(1, spi1.partitionsSingleMessages());
+        assertEquals(0, spi1.partitionsFullMessages());
+
+        assertEquals(1, spi2.partitionsSingleMessages());
+        assertEquals(0, spi2.partitionsFullMessages());
+
+        assertEquals(1, spi3.partitionsSingleMessages());
+        assertEquals(0, spi3.partitionsFullMessages());
+
+        spi0.reset();
+        spi1.reset();
+        spi2.reset();
+
+        log.info("Stop client node2.");
+
+        ignite3.close();
+
+        waitForTopologyUpdate(3, 7);
+
+        assertEquals(0, spi0.partitionsSingleMessages());
+        assertEquals(0, spi0.partitionsFullMessages());
+
+        assertEquals(0, spi1.partitionsSingleMessages());
+        assertEquals(0, spi1.partitionsFullMessages());
+
+        assertEquals(0, spi2.partitionsSingleMessages());
+        assertEquals(0, spi2.partitionsFullMessages());
+
+        spi0.reset();
+        spi1.reset();
+
+        log.info("Stop client node1.");
+
+        ignite2.close();
+
+        waitForTopologyUpdate(2, 8);
+
+        assertEquals(0, spi0.partitionsSingleMessages());
+        assertEquals(0, spi0.partitionsFullMessages());
+
+        assertEquals(0, spi1.partitionsSingleMessages());
+        assertEquals(0, spi1.partitionsFullMessages());
+
+        log.info("Stop server node.");
+
+        ignite1.close();
+
+        waitForTopologyUpdate(1, 9);
+
+        assertEquals(0, spi0.partitionsSingleMessages());
+        assertEquals(0, spi0.partitionsFullMessages());
+    }
+
+    /**
+     * @param expNodes Expected number of nodes.
+     * @param topVer Expected topology version.
+     * @throws Exception If failed.
+     */
+    private void waitForTopologyUpdate(int expNodes, int topVer) throws Exception {
+        final AffinityTopologyVersion ver = new AffinityTopologyVersion(topVer, 0);
+
+        waitForTopologyUpdate(expNodes, ver);
+    }
+
+    /**
+     * @param expNodes Expected number of nodes.
+     * @param topVer Expected topology version.
+     * @throws Exception If failed.
+     */
+    private void waitForTopologyUpdate(int expNodes, final AffinityTopologyVersion topVer) throws Exception {
+        List<Ignite> nodes = G.allGrids();
+
+        assertEquals(expNodes, nodes.size());
+
+        for (Ignite ignite : nodes) {
+            final IgniteKernal kernal = (IgniteKernal)ignite;
+
+            GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                @Override public boolean apply() {
+                    return topVer.equals(kernal.context().cache().context().exchange().readyAffinityVersion());
+                }
+            }, 10_000);
+
+            assertEquals("Unexpected affinity version for " + ignite.name(),
+                topVer,
+                kernal.context().cache().context().exchange().readyAffinityVersion());
+        }
+
+        Iterator<Ignite> it = nodes.iterator();
+
+        Ignite ignite0 = it.next();
+
+        Affinity<Integer> aff0 = ignite0.affinity(null);
+
+        while (it.hasNext()) {
+            Ignite ignite = it.next();
+
+            Affinity<Integer> aff = ignite.affinity(null);
+
+            assertEquals(aff0.partitions(), aff.partitions());
+
+            for (int part = 0; part < aff.partitions(); part++)
+                assertEquals(aff0.mapPartitionToPrimaryAndBackups(part), aff.mapPartitionToPrimaryAndBackups(part));
+        }
+
+        for (Ignite ignite : nodes) {
+            final IgniteKernal kernal = (IgniteKernal)ignite;
+
+            for (IgniteInternalCache cache : kernal.context().cache().caches()) {
+                GridDhtPartitionTopology top = cache.context().topology();
+
+                assertEquals("Unexpected topology version [node=" + ignite.name() + ", cache=" + cache.name() + ']',
+                    topVer,
+                    top.topologyVersion());
+            }
+        }
+
+        awaitPartitionMapExchange();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientOnlyCacheStart() throws Exception {
+        clientOnlyCacheStart(false, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNearOnlyCacheStart() throws Exception {
+        clientOnlyCacheStart(true, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientOnlyCacheStartFromServerNode() throws Exception {
+        clientOnlyCacheStart(false, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNearOnlyCacheStartFromServerNode() throws Exception {
+        clientOnlyCacheStart(true, true);
+    }
+
+    /**
+     * @param nearCache If {@code true} creates near cache on client.
+     * @throws Exception If failed.
+     */
+    private void clientOnlyCacheStart(boolean nearCache, boolean srvNode) throws Exception {
+        Ignite ignite0 = startGrid(0);
+        Ignite ignite1 = startGrid(1);
+
+        waitForTopologyUpdate(2, 2);
+
+        final String CACHE_NAME1 = "cache1";
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setName(CACHE_NAME1);
+
+        if (srvNode)
+            ccfg.setNodeFilter(new TestFilter(getTestGridName(2)));
+
+        ignite0.createCache(ccfg);
+
+        client = !srvNode;
+
+        Ignite ignite2 = startGrid(2);
+
+        waitForTopologyUpdate(3, 3);
+
+        TestCommunicationSpi spi0 = (TestCommunicationSpi)ignite0.configuration().getCommunicationSpi();
+        TestCommunicationSpi spi1 = (TestCommunicationSpi)ignite1.configuration().getCommunicationSpi();
+        TestCommunicationSpi spi2 = (TestCommunicationSpi)ignite2.configuration().getCommunicationSpi();
+
+        spi0.reset();
+        spi1.reset();
+        spi2.reset();
+
+        assertNull(((IgniteKernal)ignite2).context().cache().context().cache().internalCache("cache1"));
+
+        if (nearCache)
+            ignite2.getOrCreateNearCache(CACHE_NAME1, new NearCacheConfiguration<>());
+        else
+            ignite2.cache(CACHE_NAME1);
+
+        waitForTopologyUpdate(3, new AffinityTopologyVersion(3, 1));
+
+        GridCacheAdapter cache = ((IgniteKernal)ignite2).context().cache().context().cache().internalCache("cache1");
+
+        assertNotNull(cache);
+        assertEquals(nearCache, cache.context().isNear());
+
+        assertEquals(0, spi0.partitionsSingleMessages());
+        assertEquals(1, spi0.partitionsFullMessages());
+        assertEquals(0, spi1.partitionsSingleMessages());
+        assertEquals(0, spi1.partitionsFullMessages());
+        assertEquals(1, spi2.partitionsSingleMessages());
+        assertEquals(0, spi2.partitionsFullMessages());
+
+        ClusterNode clientNode = ((IgniteKernal)ignite2).localNode();
+
+        for (Ignite ignite : Ignition.allGrids()) {
+            GridDiscoveryManager disco = ((IgniteKernal)ignite).context().discovery();
+
+            assertTrue(disco.cacheNode(clientNode, CACHE_NAME1));
+            assertFalse(disco.cacheAffinityNode(clientNode, CACHE_NAME1));
+            assertEquals(nearCache, disco.cacheNearNode(clientNode, CACHE_NAME1));
+        }
+
+        spi0.reset();
+        spi1.reset();
+        spi2.reset();
+
+        final String CACHE_NAME2 = "cache2";
+
+        ccfg = new CacheConfiguration();
+
+        ccfg.setName(CACHE_NAME2);
+
+        ignite2.createCache(ccfg);
+
+        waitForTopologyUpdate(3, new AffinityTopologyVersion(3, 2));
+
+        assertEquals(0, spi0.partitionsSingleMessages());
+        assertEquals(2, spi0.partitionsFullMessages());
+        assertEquals(1, spi1.partitionsSingleMessages());
+        assertEquals(0, spi1.partitionsFullMessages());
+        assertEquals(1, spi2.partitionsSingleMessages());
+        assertEquals(0, spi2.partitionsFullMessages());
+    }
+
+    /**
+     *
+     */
+    private static class TestFilter implements IgnitePredicate<ClusterNode> {
+        /** */
+        private String exclNodeName;
+
+        /**
+         * @param exclNodeName Node name to exclude.
+         */
+        public TestFilter(String exclNodeName) {
+            this.exclNodeName = exclNodeName;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean apply(ClusterNode clusterNode) {
+            return !exclNodeName.equals(clusterNode.attribute(IgniteNodeAttributes.ATTR_GRID_NAME));
+        }
+    }
+
+    /**
+     * Test communication SPI.
+     */
+    private static class TestCommunicationSpi extends TcpCommunicationSpi {
+        /** */
+        private AtomicInteger partSingleMsgs = new AtomicInteger();
+
+        /** */
+        private AtomicInteger partFullMsgs = new AtomicInteger();
+
+        /** */
+        @LoggerResource
+        private IgniteLogger log;
+
+        /** {@inheritDoc} */
+        @Override public void sendMessage(ClusterNode node, Message msg) {
+            super.sendMessage(node, msg);
+
+            Object msg0 = ((GridIoMessage)msg).message();
+
+            if (msg0 instanceof GridDhtPartitionsSingleMessage) {
+                if (((GridDhtPartitionsSingleMessage)msg0).exchangeId() != null) {
+                    log.info("Partitions message: " + msg0.getClass().getSimpleName());
+
+                    partSingleMsgs.incrementAndGet();
+                }
+            }
+            else if (msg0 instanceof GridDhtPartitionsFullMessage) {
+                if (((GridDhtPartitionsFullMessage)msg0).exchangeId() != null) {
+                    log.info("Partitions message: " + msg0.getClass().getSimpleName());
+
+                    partFullMsgs.incrementAndGet();
+                }
+            }
+        }
+
+        /**
+         *
+         */
+        void reset() {
+            partSingleMsgs.set(0);
+            partFullMsgs.set(0);
+        }
+
+        /**
+         * @return Sent partitions single messages.
+         */
+        int partitionsSingleMessages() {
+            return partSingleMsgs.get();
+        }
+
+        /**
+         * @return Sent partitions full messages.
+         */
+        int partitionsFullMessages() {
+            return partFullMsgs.get();
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d78aa15/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCrossCacheTxStoreSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCrossCacheTxStoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCrossCacheTxStoreSelfTest.java
index cb32b13..f2de8ce 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCrossCacheTxStoreSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCrossCacheTxStoreSelfTest.java
@@ -101,19 +101,28 @@ public class IgniteCrossCacheTxStoreSelfTest extends GridCommonAbstractTest {
         grid(0).cache("cacheA").removeAll();
         grid(0).cache("cacheB").removeAll();
         grid(0).cache("cacheC").removeAll();
+
+        for (CacheStore store : firstStores.values())
+            ((TestStore)store).clear();
+
+        for (CacheStore store : secondStores.values())
+            ((TestStore)store).clear();
     }
 
     /**
      * @throws Exception If failed.
      */
-    public void testWriteThrough() throws Exception {
+    public void testSameStore() throws Exception {
         IgniteEx grid = grid(0);
 
         TestStore firstStore = (TestStore)firstStores.get(grid.name());
+        TestStore secondStore = (TestStore)secondStores.get(grid.name());
 
         assertNotNull(firstStore);
+        assertNotNull(secondStore);
 
-        Collection<String> evts = firstStore.events();
+        Collection<String> firstStoreEvts = firstStore.events();
+        Collection<String> secondStoreEvts = secondStore.events();
 
         try (Transaction tx = grid.transactions().txStart()) {
             IgniteCache<Object, Object> cacheA = grid.cache("cacheA");
@@ -138,82 +147,122 @@ public class IgniteCrossCacheTxStoreSelfTest extends GridCommonAbstractTest {
         }
 
         assertEqualsCollections(F.asList(
-                "writeAll cacheA 2",
-                "writeAll cacheB 2",
-                "deleteAll cacheA 2",
-                "deleteAll cacheB 2",
-                "write cacheA",
-                "delete cacheA",
-                "write cacheB",
-                "sessionEnd true"
-            ),
-            evts);
+            "writeAll cacheA 2",
+            "writeAll cacheB 2",
+            "deleteAll cacheA 2",
+            "deleteAll cacheB 2",
+            "write cacheA",
+            "delete cacheA",
+            "write cacheB",
+            "sessionEnd true"
+        ),
+        firstStoreEvts);
+
+        assertEquals(0, secondStoreEvts.size());
     }
 
     /**
      * @throws Exception If failed.
      */
-    public void testIncompatibleCaches1() throws Exception {
+    public void testDifferentStores() throws Exception {
         IgniteEx grid = grid(0);
 
-        try (Transaction ignored = grid.transactions().txStart()) {
+        TestStore firstStore = (TestStore)firstStores.get(grid.name());
+        TestStore secondStore = (TestStore)secondStores.get(grid.name());
+
+        assertNotNull(firstStore);
+        assertNotNull(secondStore);
+
+        Collection<String> firstStoreEvts = firstStore.events();
+        Collection<String> secondStoreEvts = secondStore.events();
+
+        try (Transaction tx = grid.transactions().txStart()) {
             IgniteCache<Object, Object> cacheA = grid.cache("cacheA");
             IgniteCache<Object, Object> cacheC = grid.cache("cacheC");
 
-            cacheA.put("1", "2");
+            cacheA.put("1", "1");
+            cacheA.put("2", "2");
+            cacheC.put("1", "1");
+            cacheC.put("2", "2");
 
-            cacheC.put("1", "2");
+            cacheA.remove("3");
+            cacheA.remove("4");
+            cacheC.remove("3");
+            cacheC.remove("4");
 
-            fail("Must not allow to enlist caches with different stores to one transaction");
-        }
-        catch (CacheException e) {
-            assertTrue(e.getMessage().contains("Failed to enlist new cache to existing transaction"));
+            cacheA.put("5", "5");
+            cacheA.remove("6");
+
+            cacheC.put("7", "7");
+
+            tx.commit();
         }
+
+        assertEqualsCollections(F.asList(
+            "writeAll cacheA 2",
+            "deleteAll cacheA 2",
+            "write cacheA",
+            "delete cacheA",
+            "sessionEnd true"
+        ),
+        firstStoreEvts);
+
+        assertEqualsCollections(F.asList(
+            "writeAll cacheC 2",
+            "deleteAll cacheC 2",
+            "write cacheC",
+            "sessionEnd true"
+        ),
+        secondStoreEvts);
     }
 
     /**
      * @throws Exception If failed.
      */
-    public void testIncompatibleCaches2() throws Exception {
+    public void testNonPersistentCache() throws Exception {
         IgniteEx grid = grid(0);
 
-        try (Transaction ignored = grid.transactions().txStart()) {
-            IgniteCache<Object, Object> cacheA = grid.cache("cacheA");
-            IgniteCache<Object, Object> cacheC = grid.cache("cacheD");
+        TestStore firstStore = (TestStore)firstStores.get(grid.name());
+        TestStore secondStore = (TestStore)secondStores.get(grid.name());
 
-            cacheA.put("1", "2");
+        assertNotNull(firstStore);
+        assertNotNull(secondStore);
 
-            cacheC.put("1", "2");
+        Collection<String> firstStoreEvts = firstStore.events();
+        Collection<String> secondStoreEvts = secondStore.events();
 
-            fail("Must not allow to enlist caches with different stores to one transaction");
-        }
-        catch (CacheException e) {
-            assertTrue(e.getMessage().contains("Failed to enlist new cache to existing transaction"));
-        }
-    }
-
-    /**
-     * @param col1 Collection 1.
-     * @param col2 Collection 2.
-     */
-    private static void assertEqualsCollections(Collection<?> col1, Collection<?> col2) {
-        if (col1.size() != col2.size())
-            fail("Collections are not equal:\nExpected:\t" + col1 + "\nActual:\t" + col2);
+        try (Transaction tx = grid.transactions().txStart()) {
+            IgniteCache<Object, Object> cacheA = grid.cache("cacheA");
+            IgniteCache<Object, Object> cacheD = grid.cache("cacheD");
 
-        Iterator<?> it1 = col1.iterator();
-        Iterator<?> it2 = col2.iterator();
+            cacheA.put("1", "1");
+            cacheA.put("2", "2");
+            cacheD.put("1", "1");
+            cacheD.put("2", "2");
 
-        int idx = 0;
+            cacheA.remove("3");
+            cacheA.remove("4");
+            cacheD.remove("3");
+            cacheD.remove("4");
 
-        while (it1.hasNext()) {
-            Object item1 = it1.next();
-            Object item2 = it2.next();
+            cacheA.put("5", "5");
+            cacheA.remove("6");
 
-            if (!F.eq(item1, item2))
-                fail("Collections are not equal (position " + idx + "):\nExpected: " + col1 + "\nActual:   " + col2);
+            cacheD.put("7", "7");
 
-            idx++;
+            tx.commit();
         }
+
+        assertEqualsCollections(F.asList(
+            "writeAll cacheA 2",
+            "deleteAll cacheA 2",
+            "write cacheA",
+            "delete cacheA",
+            "sessionEnd true"
+        ),
+        firstStoreEvts);
+
+        assertEquals(0, secondStoreEvts.size());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d78aa15/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheClientOnlySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheClientOnlySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheClientOnlySelfTest.java
index 0d4d607..24f72a9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheClientOnlySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheClientOnlySelfTest.java
@@ -17,20 +17,72 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.dht;
 
+import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
 
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+
 /**
  * Client only test.
  */
-public class GridCacheClientOnlySelfTest extends GridCacheClientModesAbstractSelfTest {
+@SuppressWarnings("RedundantMethodOverride")
+public abstract class GridCacheClientOnlySelfTest extends GridCacheClientModesAbstractSelfTest {
     /** {@inheritDoc} */
     @Override protected NearCacheConfiguration nearConfiguration() {
         return null;
     }
 
-    /** {@inheritDoc} */
-    @Override protected boolean clientOnly() {
-        return true;
+    /** */
+    public static class CaseReplicatedAtomic extends GridCacheClientOnlySelfTest {
+        /** {@inheritDoc} */
+        @Override protected CacheMode cacheMode() {
+            return REPLICATED;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected CacheAtomicityMode atomicityMode() {
+            return ATOMIC;
+        }
+    }
+
+    /** */
+    public static class CaseReplicatedTransactional extends GridCacheClientOnlySelfTest {
+        /** {@inheritDoc} */
+        @Override protected CacheMode cacheMode() {
+            return REPLICATED;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected CacheAtomicityMode atomicityMode() {
+            return TRANSACTIONAL;
+        }
+    }
+
+    /** */
+    public static class CasePartitionedAtomic extends GridCacheClientOnlySelfTest {
+        /** {@inheritDoc} */
+        @Override protected CacheMode cacheMode() {
+            return PARTITIONED;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected CacheAtomicityMode atomicityMode() {
+            return ATOMIC;
+        }
+    }
+
+    /** */
+    public static class CasePartitionedTransactional extends GridCacheClientOnlySelfTest {
+        /** {@inheritDoc} */
+        @Override protected CacheMode cacheMode() {
+            return PARTITIONED;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected CacheAtomicityMode atomicityMode() {
+            return TRANSACTIONAL;
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d78aa15/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedFailoverSelfTest.java
index 568b08f..74a107d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedFailoverSelfTest.java
@@ -28,6 +28,11 @@ import static org.apache.ignite.cache.CacheMode.*;
  */
 public class GridCacheColocatedFailoverSelfTest extends GridCacheAbstractFailoverTxSelfTest {
     /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-631 or IGNITE-157 or IGNITE-882");
+    }
+
+    /** {@inheritDoc} */
     @Override protected CacheMode cacheMode() {
         return PARTITIONED;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d78aa15/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedTxExceptionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedTxExceptionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedTxExceptionSelfTest.java
index 645c59f..0fd4bc8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedTxExceptionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedTxExceptionSelfTest.java
@@ -28,6 +28,11 @@ import static org.apache.ignite.cache.CacheMode.*;
  */
 public class GridCacheColocatedTxExceptionSelfTest extends IgniteTxExceptionAbstractSelfTest {
     /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-257");
+    }
+
+    /** {@inheritDoc} */
     @Override protected CacheMode cacheMode() {
         return PARTITIONED;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d78aa15/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtClientRemoveFailureTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtClientRemoveFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtClientRemoveFailureTest.java
new file mode 100644
index 0000000..09c643b
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtClientRemoveFailureTest.java
@@ -0,0 +1,28 @@
+/*
+ * 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.distributed.dht;
+
+/**
+ *
+ */
+public class GridCacheDhtClientRemoveFailureTest extends GridCacheDhtRemoveFailureTest {
+    /** {@inheritDoc} */
+    @Override protected boolean testClientNode() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d78aa15/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java
index aff5512..7adeba8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java
@@ -77,11 +77,18 @@ public class GridCacheDhtEvictionNearReadersSelfTest extends GridCommonAbstractT
         // Set eviction queue size explicitly.
         cacheCfg.setEvictSynchronizedKeyBufferSize(1);
         cacheCfg.setEvictMaxOverflowRatio(0);
-        cacheCfg.setEvictionPolicy(new FifoEvictionPolicy(10));
+
+        FifoEvictionPolicy plc = new FifoEvictionPolicy();
+        plc.setMaxSize(10);
+
+        cacheCfg.setEvictionPolicy(plc);
 
         NearCacheConfiguration nearCfg = new NearCacheConfiguration();
 
-        nearCfg.setNearEvictionPolicy(new FifoEvictionPolicy(10));
+        FifoEvictionPolicy nearPlc = new FifoEvictionPolicy();
+        nearPlc.setMaxSize(10);
+
+        nearCfg.setNearEvictionPolicy(nearPlc);
 
         cacheCfg.setNearConfiguration(nearCfg);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d78aa15/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionSelfTest.java
index 97e8657..8eada9c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionSelfTest.java
@@ -76,14 +76,21 @@ public class GridCacheDhtEvictionSelfTest extends GridCommonAbstractTest {
         cacheCfg.setBackups(1);
 
         NearCacheConfiguration nearCfg = new NearCacheConfiguration();
-        nearCfg.setNearEvictionPolicy(new FifoEvictionPolicy(10000));
+
+        FifoEvictionPolicy nearPlc = new FifoEvictionPolicy();
+        nearPlc.setMaxSize(10000);
+
+        nearCfg.setNearEvictionPolicy(nearPlc);
         cacheCfg.setNearConfiguration(nearCfg);
 
         // Set eviction queue size explicitly.
         cacheCfg.setEvictMaxOverflowRatio(0);
         cacheCfg.setEvictSynchronizedKeyBufferSize(1);
-        cacheCfg.setEvictionPolicy(new FifoEvictionPolicy(10000));
 
+        FifoEvictionPolicy plc = new FifoEvictionPolicy();
+        plc.setMaxSize(10000);
+
+        cacheCfg.setEvictionPolicy(plc);
 
         cfg.setCacheConfiguration(cacheCfg);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d78aa15/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheExColocatedFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheExColocatedFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheExColocatedFullApiSelfTest.java
deleted file mode 100644
index 55b1f92..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheExColocatedFullApiSelfTest.java
+++ /dev/null
@@ -1,33 +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.distributed.dht;
-
-import org.apache.ignite.cache.*;
-import org.apache.ignite.internal.processors.cache.*;
-
-import static org.apache.ignite.cache.CacheMode.*;
-
-/**
- * Tests private cache interface on colocated cache.
- */
-public class GridCacheExColocatedFullApiSelfTest extends GridCacheExAbstractFullApiSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return PARTITIONED;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d78aa15/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledMetricsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledMetricsSelfTest.java
index 17bbead..fd04f4f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledMetricsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledMetricsSelfTest.java
@@ -75,7 +75,9 @@ public class GridCachePartitionedNearDisabledMetricsSelfTest extends GridCacheAb
     /**
      * @throws Exception If failed.
      */
-    public void _testGettingRemovedKey() throws Exception {
+    public void testGettingRemovedKey() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-819");
+
         IgniteCache<Integer, Integer> cache = grid(0).cache(null);
 
         cache.put(0, 0);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d78aa15/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedTopologyChangeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedTopologyChangeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedTopologyChangeSelfTest.java
index 7bd3876..812ee29 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedTopologyChangeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedTopologyChangeSelfTest.java
@@ -61,6 +61,11 @@ public class GridCachePartitionedTopologyChangeSelfTest extends GridCommonAbstra
     private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
     /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-807");
+    }
+
+    /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration c = super.getConfiguration(gridName);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d78aa15/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java
index 5983c1b..9e54673 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.dht;
 
-import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.eviction.lru.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
@@ -33,6 +32,10 @@ import org.apache.ignite.testframework.junits.common.*;
 import java.util.*;
 import java.util.concurrent.*;
 
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+
 /**
  * Tests explicit lock.
  */
@@ -46,6 +49,9 @@ public class IgniteCacheMultiTxLockSelfTest extends GridCommonAbstractTest {
     /** */
     private volatile boolean run = true;
 
+    /** */
+    private boolean client;
+
     /** {@inheritDoc} */
     @Override protected void afterTestsStopped() throws Exception {
         stopAllGrids();
@@ -66,16 +72,22 @@ public class IgniteCacheMultiTxLockSelfTest extends GridCommonAbstractTest {
         CacheConfiguration ccfg = new CacheConfiguration();
 
         ccfg.setName(CACHE_NAME);
-        ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
-        ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.PRIMARY_SYNC);
+        ccfg.setAtomicityMode(TRANSACTIONAL);
+        ccfg.setWriteSynchronizationMode(PRIMARY_SYNC);
         ccfg.setBackups(2);
-        ccfg.setCacheMode(CacheMode.PARTITIONED);
+        ccfg.setCacheMode(PARTITIONED);
         ccfg.setStartSize(100000);
-        ccfg.setEvictionPolicy(new LruEvictionPolicy(100000));
+
+        LruEvictionPolicy plc = new LruEvictionPolicy();
+        plc.setMaxSize(100000);
+
+        ccfg.setEvictionPolicy(plc);
         ccfg.setEvictSynchronized(true);
 
         c.setCacheConfiguration(ccfg);
 
+        c.setClientMode(client);
+
         return c;
     }
 
@@ -83,33 +95,50 @@ public class IgniteCacheMultiTxLockSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testExplicitLockOneKey() throws Exception {
-        checkExplicitLock(1);
+        checkExplicitLock(1, false);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testExplicitLockManyKeys() throws Exception {
-        checkExplicitLock(4);
+        checkExplicitLock(4, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testExplicitLockManyKeysWithClient() throws Exception {
+        checkExplicitLock(4, true);
     }
 
     /**
+     * @param keys Number of keys.
+     * @param testClient If {@code true} uses one client node.
      * @throws Exception If failed.
      */
-    public void checkExplicitLock(int keys) throws Exception {
+    public void checkExplicitLock(int keys, boolean testClient) throws Exception {
         Collection<Thread> threads = new ArrayList<>();
 
         try {
             // Start grid 1.
             IgniteEx grid1 = startGrid(1);
 
+            assertFalse(grid1.configuration().isClientMode());
+
             threads.add(runCacheOperations(grid1.cachex(CACHE_NAME), keys));
 
             TimeUnit.SECONDS.sleep(3L);
 
+            client = testClient; // If test client start on node in client mode.
+
             // Start grid 2.
             IgniteEx grid2 = startGrid(2);
 
+            assertEquals((Object)testClient, grid2.configuration().isClientMode());
+
+            client = false;
+
             threads.add(runCacheOperations(grid2.cachex(CACHE_NAME), keys));
 
             TimeUnit.SECONDS.sleep(3L);
@@ -117,6 +146,11 @@ public class IgniteCacheMultiTxLockSelfTest extends GridCommonAbstractTest {
             // Start grid 3.
             IgniteEx grid3 = startGrid(3);
 
+            assertFalse(grid3.configuration().isClientMode());
+
+            if (testClient)
+                log.info("Started client node: " + grid3.name());
+
             threads.add(runCacheOperations(grid3.cachex(CACHE_NAME), keys));
 
             TimeUnit.SECONDS.sleep(3L);
@@ -124,6 +158,8 @@ public class IgniteCacheMultiTxLockSelfTest extends GridCommonAbstractTest {
             // Start grid 4.
             IgniteEx grid4 = startGrid(4);
 
+            assertFalse(grid4.configuration().isClientMode());
+
             threads.add(runCacheOperations(grid4.cachex(CACHE_NAME), keys));
 
             TimeUnit.SECONDS.sleep(3L);
@@ -158,6 +194,7 @@ public class IgniteCacheMultiTxLockSelfTest extends GridCommonAbstractTest {
 
     /**
      * @param cache Cache.
+     * @param keys Number of keys.
      * @return Running thread.
      */
     @SuppressWarnings("TypeMayBeWeakened")

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d78aa15/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicClientInvalidPartitionHandlingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicClientInvalidPartitionHandlingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicClientInvalidPartitionHandlingSelfTest.java
new file mode 100644
index 0000000..64414a4
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicClientInvalidPartitionHandlingSelfTest.java
@@ -0,0 +1,29 @@
+/*
+ * 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.distributed.dht.atomic;
+
+/**
+ *
+ */
+public class GridCacheAtomicClientInvalidPartitionHandlingSelfTest
+    extends GridCacheAtomicInvalidPartitionHandlingSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean testClientNode() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d78aa15/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicClientRemoveFailureTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicClientRemoveFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicClientRemoveFailureTest.java
new file mode 100644
index 0000000..2edb125
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicClientRemoveFailureTest.java
@@ -0,0 +1,28 @@
+/*
+ * 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.distributed.dht.atomic;
+
+/**
+ *
+ */
+public class GridCacheAtomicClientRemoveFailureTest extends GridCacheAtomicRemoveFailureTest {
+    /** {@inheritDoc} */
+    @Override protected boolean testClientNode() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d78aa15/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java
index a68423b..054a110 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java
@@ -68,16 +68,15 @@ public class GridCacheAtomicInvalidPartitionHandlingSelfTest extends GridCommonA
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
-
-        discoSpi.setIpFinder(IP_FINDER);
-
-        cfg.setDiscoverySpi(discoSpi);
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER).setForceServerMode(true));
 
         cfg.setCacheConfiguration(cacheConfiguration());
 
         cfg.setCommunicationSpi(new DelayCommunicationSpi());
 
+        if (testClientNode() && getTestGridName(0).equals(gridName))
+            cfg.setClientMode(true);
+
         return cfg;
     }
 
@@ -109,6 +108,13 @@ public class GridCacheAtomicInvalidPartitionHandlingSelfTest extends GridCommonA
     }
 
     /**
+     * @return {@code True} if test updates from client node.
+     */
+    protected boolean testClientNode() {
+        return false;
+    }
+
+    /**
      * @throws Exception If failed.
      */
     public void testClockFullSync() throws Exception {
@@ -167,6 +173,8 @@ public class GridCacheAtomicInvalidPartitionHandlingSelfTest extends GridCommonA
         awaitPartitionMapExchange();
 
         try {
+            assertEquals(testClientNode(), (boolean)grid(0).configuration().isClientMode());
+
             final IgniteCache<Object, Object> cache = grid(0).cache(null);
 
             final int range = 100_000;
@@ -321,7 +329,10 @@ public class GridCacheAtomicInvalidPartitionHandlingSelfTest extends GridCommonA
                                     assertEquals("Failed to check value for key [key=" + k + ", node=" +
                                         locNode.id() + ", primary=" + primary + ", recNodeId=" + nodeId + ']',
                                         val, CU.value(entry.rawGetOrUnmarshal(false), entry.context(), false));
-                                    assertEquals(ver, entry.version());
+
+                                    assertEquals("Failed to check version for key [key=" + k + ", node=" +
+                                        locNode.id() + ", primary=" + primary + ", recNodeId=" + nodeId + ']',
+                                        ver, entry.version());
                                 }
                             }
                             else

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d78aa15/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAffinityFunctionExcludeNeighborsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAffinityFunctionExcludeNeighborsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAffinityFunctionExcludeNeighborsAbstractSelfTest.java
index fcc15e0..7cdf265 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAffinityFunctionExcludeNeighborsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAffinityFunctionExcludeNeighborsAbstractSelfTest.java
@@ -55,7 +55,8 @@ public abstract class GridCacheAffinityFunctionExcludeNeighborsAbstractSelfTest
 
         // Override node attributes in discovery spi.
         TcpDiscoverySpi spi = new TcpDiscoverySpi() {
-            @Override public void setNodeAttributes(Map<String, Object> attrs, IgniteProductVersion ver) {
+            @Override public void setNodeAttributes(Map<String, Object> attrs,
+                IgniteProductVersion ver) {
                 super.setNodeAttributes(attrs, ver);
 
                 // Set unique mac addresses for every group of three nodes.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d78aa15/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearOnlySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearOnlySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearOnlySelfTest.java
deleted file mode 100644
index 7b51d9e..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearOnlySelfTest.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.distributed.near;
-
-import org.apache.ignite.cache.*;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.*;
-
-/**
- * Test for atomic cache.
- */
-public class GridCacheAtomicNearOnlySelfTest extends GridCacheNearOnlySelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheAtomicityMode atomicityMode() {
-        return ATOMIC;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d78aa15/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheExNearFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheExNearFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheExNearFullApiSelfTest.java
deleted file mode 100644
index 11ec96a..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheExNearFullApiSelfTest.java
+++ /dev/null
@@ -1,39 +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.distributed.near;
-
-import org.apache.ignite.cache.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.internal.processors.cache.*;
-
-import static org.apache.ignite.cache.CacheMode.*;
-
-/**
- * Tests private cache interface on partitioned cache with near enabled.
- */
-public class GridCacheExNearFullApiSelfTest extends GridCacheExAbstractFullApiSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return PARTITIONED;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected NearCacheConfiguration nearConfiguration() {
-        return new NearCacheConfiguration();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d78aa15/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyMultiNodeFullApiSelfTest.java
index 337a921..e78b782 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyMultiNodeFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyMultiNodeFullApiSelfTest.java
@@ -83,6 +83,8 @@ public class GridCacheNearOnlyMultiNodeFullApiSelfTest extends GridCachePartitio
             info("Use grid '" + gridName + "' as near-only.");
 
             cfg.setClientMode(true);
+
+            cfg.setCacheConfiguration();
         }
 
         return cfg;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d78aa15/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlySelfTest.java
index da7d3ce..f3a62f2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlySelfTest.java
@@ -21,15 +21,14 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
 
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+
 /**
  * Near only self test.
  */
-public class GridCacheNearOnlySelfTest extends GridCacheClientModesAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected boolean clientOnly() {
-        return false;
-    }
-
+@SuppressWarnings("RedundantMethodOverride")
+public abstract class GridCacheNearOnlySelfTest extends GridCacheClientModesAbstractSelfTest {
     /**
      * @throws Exception If failed.
      */
@@ -59,4 +58,56 @@ public class GridCacheNearOnlySelfTest extends GridCacheClientModesAbstractSelfT
             assertEquals(i * i, nearOnlyCache.get(i));
         }
     }
+
+    /** */
+    public static class CaseReplicatedAtomic extends GridCacheNearOnlySelfTest {
+        /** {@inheritDoc} */
+        @Override protected CacheMode cacheMode() {
+            return REPLICATED;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected CacheAtomicityMode atomicityMode() {
+            return ATOMIC;
+        }
+    }
+
+    /** */
+    public static class CaseReplicatedTransactional extends GridCacheNearOnlySelfTest {
+        /** {@inheritDoc} */
+        @Override protected CacheMode cacheMode() {
+            return REPLICATED;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected CacheAtomicityMode atomicityMode() {
+            return TRANSACTIONAL;
+        }
+    }
+
+    /** */
+    public static class CasePartitionedAtomic extends GridCacheNearOnlySelfTest {
+        /** {@inheritDoc} */
+        @Override protected CacheMode cacheMode() {
+            return PARTITIONED;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected CacheAtomicityMode atomicityMode() {
+            return ATOMIC;
+        }
+    }
+
+    /** */
+    public static class CasePartitionedTransactional extends GridCacheNearOnlySelfTest {
+        /** {@inheritDoc} */
+        @Override protected CacheMode cacheMode() {
+            return PARTITIONED;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected CacheAtomicityMode atomicityMode() {
+            return TRANSACTIONAL;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d78aa15/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java
index 2893678..b6bc56e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java
@@ -70,6 +70,7 @@ public class GridCacheNearOnlyTopologySelfTest extends GridCommonAbstractTest {
 
         TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
 
+        discoSpi.setForceServerMode(true);
         discoSpi.setIpFinder(IP_FINDER);
 
         cfg.setDiscoverySpi(discoSpi);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d78aa15/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxExceptionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxExceptionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxExceptionSelfTest.java
index 00154e1..7fd30f4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxExceptionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxExceptionSelfTest.java
@@ -27,6 +27,11 @@ import static org.apache.ignite.cache.CacheMode.*;
  */
 public class GridCacheNearTxExceptionSelfTest extends IgniteTxExceptionAbstractSelfTest {
     /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-257");
+    }
+
+    /** {@inheritDoc} */
     @Override protected CacheMode cacheMode() {
         return PARTITIONED;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d78aa15/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java
index c79c32a..6b7d1ad 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java
@@ -22,6 +22,7 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.cluster.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.lang.*;
+import org.apache.ignite.spi.discovery.tcp.*;
 
 import java.util.*;
 
@@ -38,6 +39,7 @@ public class GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest extends Grid
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
         cfg.setClientMode(true);
 
         return cfg;
@@ -63,7 +65,8 @@ public class GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest extends Grid
     @Override protected IgniteClosure<Throwable, Throwable> errorHandler() {
         return new IgniteClosure<Throwable, Throwable>() {
             @Override public Throwable apply(Throwable e) {
-                if (e instanceof IgniteException || e instanceof IgniteCheckedException || X.hasCause(e, ClusterTopologyCheckedException.class)) {
+                if (e instanceof IgniteException || e instanceof IgniteCheckedException ||
+                    X.hasCause(e, ClusterTopologyCheckedException.class)) {
                     info("Discarding exception: " + e);
 
                     return null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d78aa15/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEvictionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEvictionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEvictionSelfTest.java
index 28ad7a8..c0d36e2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEvictionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEvictionSelfTest.java
@@ -78,8 +78,15 @@ public class GridCachePartitionedEvictionSelfTest extends GridCacheAbstractSelfT
 
         cc.setCacheMode(PARTITIONED);
         cc.setWriteSynchronizationMode(FULL_SYNC);
-        cc.setEvictionPolicy(new FifoEvictionPolicy(EVICT_CACHE_SIZE));
-        cc.getNearConfiguration().setNearEvictionPolicy(new FifoEvictionPolicy(EVICT_CACHE_SIZE));
+
+        FifoEvictionPolicy plc = new FifoEvictionPolicy();
+        plc.setMaxSize(EVICT_CACHE_SIZE);
+        cc.setEvictionPolicy(plc);
+
+        FifoEvictionPolicy nearPlc = new FifoEvictionPolicy();
+        nearPlc.setMaxSize(EVICT_CACHE_SIZE);
+        cc.getNearConfiguration().setNearEvictionPolicy(nearPlc);
+
         cc.setSwapEnabled(false);
 
         // We set 1 backup explicitly.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d78aa15/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFailoverSelfTest.java
index 553d748..9825cfd 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFailoverSelfTest.java
@@ -27,6 +27,11 @@ import static org.apache.ignite.cache.CacheMode.*;
  */
 public class GridCachePartitionedFailoverSelfTest extends GridCacheAbstractFailoverTxSelfTest {
     /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-631 or IGNITE-157 or IGNITE-882");
+    }
+
+    /** {@inheritDoc} */
     @Override protected CacheMode cacheMode() {
         return PARTITIONED;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d78aa15/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiMultithreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiMultithreadedSelfTest.java
index 7d4a61e..74df95f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiMultithreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiMultithreadedSelfTest.java
@@ -27,6 +27,11 @@ import static org.apache.ignite.cache.CacheMode.*;
  */
 public class GridCachePartitionedFullApiMultithreadedSelfTest extends GridCacheAbstractFullApiMultithreadedSelfTest {
     /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-805");
+    }
+
+    /** {@inheritDoc} */
     @Override protected int gridCount() {
         return 3;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d78aa15/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiSelfTest.java
index 7e16ac7..da5d1bb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiSelfTest.java
@@ -17,12 +17,15 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.near;
 
+import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 
+import javax.cache.*;
+
 import static org.apache.ignite.cache.CacheMode.*;
 
 /**
@@ -72,4 +75,33 @@ public class GridCachePartitionedFullApiSelfTest extends GridCacheAbstractFullAp
         for (int i = 0 ; i < aff.partitions(); i++)
             String.valueOf(cache.entrySet(i));
     }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testUpdate() throws Exception {
+        if (gridCount() > 1) {
+            IgniteCache<Object, Object> cache = grid(0).cache(null);
+
+            Integer key = nearKey(cache);
+
+            primaryCache(key, null).put(key, 1);
+
+            assertEquals(1, cache.get(key));
+
+            primaryCache(key, null).put(key, 2);
+
+            if (cache.getConfiguration(CacheConfiguration.class).getNearConfiguration() != null)
+                assertEquals(2, cache.localPeek(key));
+
+            assertEquals(2, cache.get(key));
+
+            int cnt = 0;
+
+            for (Cache.Entry e : cache)
+                cnt++;
+
+            assertEquals(1, cnt);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d78aa15/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiThreadedPutGetSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiThreadedPutGetSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiThreadedPutGetSelfTest.java
index 8131d3e..feb94b2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiThreadedPutGetSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiThreadedPutGetSelfTest.java
@@ -66,7 +66,11 @@ public class GridCachePartitionedMultiThreadedPutGetSelfTest extends GridCommonA
         cc.setCacheMode(PARTITIONED);
         cc.setBackups(1);
         cc.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        cc.setEvictionPolicy(new FifoEvictionPolicy<>(1000));
+
+        FifoEvictionPolicy plc = new FifoEvictionPolicy();
+        plc.setMaxMemorySize(1000);
+
+        cc.setEvictionPolicy(plc);
         cc.setSwapEnabled(false);
         cc.setAtomicityMode(TRANSACTIONAL);
         cc.setEvictSynchronized(false);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d78aa15/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNodeRestartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNodeRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNodeRestartTest.java
index 2096836..9529f9d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNodeRestartTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNodeRestartTest.java
@@ -30,6 +30,11 @@ import static org.apache.ignite.transactions.TransactionConcurrency.*;
  */
 public class GridCachePartitionedNodeRestartTest extends GridCacheAbstractNodeRestartSelfTest {
     /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-882");
+    }
+
+    /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration c = super.getConfiguration(gridName);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d78aa15/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOffHeapTieredMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOffHeapTieredMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOffHeapTieredMultiNodeFullApiSelfTest.java
index 4392365..61c3563 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOffHeapTieredMultiNodeFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOffHeapTieredMultiNodeFullApiSelfTest.java
@@ -65,7 +65,7 @@ public class GridCachePartitionedOffHeapTieredMultiNodeFullApiSelfTest extends G
         cache.put(key, 5); // Put from near to add reader on primary.
 
         assertEquals(5, primaryCache.localPeek(key, CachePeekMode.ONHEAP).intValue());
-        assertEquals(5, primaryCache.localPeek(key, CachePeekMode.OFFHEAP).intValue());
+        assertNull(primaryCache.localPeek(key, CachePeekMode.OFFHEAP));
         assertEquals(5, cache.get(key).intValue());
         assertEquals(5, map.get(key));
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d78aa15/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOptimisticTxNodeRestartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOptimisticTxNodeRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOptimisticTxNodeRestartTest.java
index 82da2ac..62cc6b1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOptimisticTxNodeRestartTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOptimisticTxNodeRestartTest.java
@@ -31,6 +31,11 @@ import static org.apache.ignite.transactions.TransactionConcurrency.*;
  */
 public class GridCachePartitionedOptimisticTxNodeRestartTest extends GridCacheAbstractNodeRestartSelfTest {
     /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-882");
+    }
+
+    /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration c = super.getConfiguration(gridName);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d78aa15/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxMultiThreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxMultiThreadedSelfTest.java
index e6042e9..4b20f02 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxMultiThreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxMultiThreadedSelfTest.java
@@ -38,6 +38,11 @@ public class GridCachePartitionedTxMultiThreadedSelfTest extends IgniteTxMultiTh
     private TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
     /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-806");
+    }
+
+    /** {@inheritDoc} */
     @SuppressWarnings({"ConstantConditions"})
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration c = super.getConfiguration(gridName);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d78aa15/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheRendezvousAffinityClientSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheRendezvousAffinityClientSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheRendezvousAffinityClientSelfTest.java
index 80651bf..3bab050 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheRendezvousAffinityClientSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheRendezvousAffinityClientSelfTest.java
@@ -24,6 +24,7 @@ import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.testframework.junits.common.*;
 
 import java.util.*;
@@ -35,9 +36,12 @@ public class GridCacheRendezvousAffinityClientSelfTest extends GridCommonAbstrac
     /** Client node. */
     private boolean client;
 
+    /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
+
         CacheConfiguration ccfg = defaultCacheConfiguration();
 
         ccfg.setCacheMode(CacheMode.PARTITIONED);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d78aa15/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheExReplicatedFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheExReplicatedFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheExReplicatedFullApiSelfTest.java
deleted file mode 100644
index 32ee784..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheExReplicatedFullApiSelfTest.java
+++ /dev/null
@@ -1,33 +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.distributed.replicated;
-
-import org.apache.ignite.cache.*;
-import org.apache.ignite.internal.processors.cache.*;
-
-import static org.apache.ignite.cache.CacheMode.*;
-
-/**
- * Tests private cache interface on replicated cache.
- */
-public class GridCacheExReplicatedFullApiSelfTest extends GridCacheExAbstractFullApiSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return REPLICATED;
-    }
-}