You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2017/04/10 14:18:00 UTC

[10/17] ignite git commit: IGNITE-4535 - Add option to store deserialized values on-heap

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractNodeRestartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractNodeRestartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractNodeRestartSelfTest.java
index e2cf734..d268e47 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractNodeRestartSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractNodeRestartSelfTest.java
@@ -36,6 +36,7 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteTransactions;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheRebalanceMode;
+import org.apache.ignite.cache.eviction.lru.LruEvictionPolicy;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
@@ -100,6 +101,9 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
     protected CacheRebalanceMode rebalancMode = ASYNC;
 
     /** */
+    protected boolean evict = false;
+
+    /** */
     protected int rebalancBatchSize = DFLT_BATCH_SIZE;
 
     /** Number of partitions. */
@@ -115,9 +119,6 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
     private int retries = DFLT_RETRIES;
 
     /** */
-    private GridTestUtils.TestMemoryMode memMode = GridTestUtils.TestMemoryMode.HEAP;
-
-    /** */
     private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
     /** {@inheritDoc} */
@@ -139,7 +140,14 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
 
         CacheConfiguration ccfg = cacheConfiguration();
 
-        GridTestUtils.setMemoryMode(c, ccfg, memMode, 100, 1024);
+        if (evict) {
+            LruEvictionPolicy plc = new LruEvictionPolicy();
+
+            plc.setMaxSize(100);
+
+            ccfg.setEvictionPolicy(plc);
+            ccfg.setOnheapCacheEnabled(true);
+        }
 
         c.setCacheConfiguration(ccfg);
 
@@ -166,6 +174,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
         backups = DFLT_BACKUPS;
         partitions = DFLT_PARTITIONS;
         rebalancMode = ASYNC;
+        evict = false;
         rebalancBatchSize = DFLT_BATCH_SIZE;
         nodeCnt = DFLT_NODE_CNT;
         keyCnt = DFLT_KEY_CNT;
@@ -188,7 +197,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
     /**
      * @throws Exception If failed.
      */
-    private void startGrids() throws  Exception {
+    private void startGrids() throws Exception {
         for (int i = 0; i < nodeCnt; i++) {
             startGrid(i);
 
@@ -282,6 +291,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
         keyCnt = 10;
         partitions = 29;
         rebalancMode = ASYNC;
+        evict = false;
 
         long duration = 30000;
 
@@ -297,6 +307,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
         keyCnt = 10;
         partitions = 29;
         rebalancMode = ASYNC;
+        evict = false;
 
         long duration = 30000;
 
@@ -312,6 +323,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
         keyCnt = 10;
         partitions = 29;
         rebalancMode = ASYNC;
+        evict = false;
 
         long duration = 30000;
 
@@ -327,6 +339,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
         keyCnt = 10;
         partitions = 29;
         rebalancMode = ASYNC;
+        evict = false;
 
         long duration = 30000;
 
@@ -342,6 +355,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
         keyCnt = 10;
         partitions = 29;
         rebalancMode = ASYNC;
+        evict = false;
 
         long duration = 60000;
 
@@ -357,6 +371,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
         keyCnt = 10;
         partitions = 29;
         rebalancMode = ASYNC;
+        evict = false;
 
         long duration = 60000;
 
@@ -372,6 +387,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
         keyCnt = 10;
         partitions = 29;
         rebalancMode = ASYNC;
+        evict = false;
 
         long duration = 60000;
 
@@ -381,52 +397,15 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
     /**
      * @throws Exception If failed.
      */
-    public void testRestartWithPutFourNodesOneBackupsSwap() throws Throwable {
-        restartWithPutFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode.SWAP);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testRestartWithPutFourNodesOneBackupsOffheapTiered() throws Throwable {
-        restartWithPutFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode.OFFHEAP_TIERED);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testRestartWithPutFourNodesOneBackupsOffheapTieredSwap() throws Throwable {
-        restartWithPutFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode.OFFHEAP_TIERED_SWAP);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
     public void testRestartWithPutFourNodesOneBackupsOffheapEvict() throws Throwable {
-        restartWithPutFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode.OFFHEAP_EVICT);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testRestartWithPutFourNodesOneBackupsOffheapEvictSwap() throws Throwable {
-        restartWithPutFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode.OFFHEAP_EVICT_SWAP);
-    }
-
-    /**
-     * @param memMode Memory mode.
-     * @throws Throwable If failed.
-     */
-    private void restartWithPutFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode memMode)
-        throws Throwable {
         backups = 1;
         nodeCnt = 4;
-        keyCnt = 100_000;
+        keyCnt = 10;
         partitions = 29;
         rebalancMode = ASYNC;
-        this.memMode = memMode;
+        evict = true;
 
-        long duration = 30_000;
+        long duration = 60000;
 
         checkRestartWithPut(duration, 2, 2);
     }
@@ -440,6 +419,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
         keyCnt = 10;
         partitions = 29;
         rebalancMode = ASYNC;
+        evict = false;
 
         long duration = 60000;
 
@@ -449,49 +429,13 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
     /**
      * @throws Exception If failed.
      */
-    public void testRestartWithTxFourNodesOneBackupsSwap() throws Throwable {
-        restartWithTxFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode.SWAP);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testRestartWithTxFourNodesOneBackupsOffheapTiered() throws Throwable {
-        restartWithTxFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode.OFFHEAP_TIERED);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testRestartWithTxFourNodesOneBackupsOffheapTieredSwap() throws Throwable {
-        restartWithTxFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode.OFFHEAP_TIERED_SWAP);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
     public void testRestartWithTxFourNodesOneBackupsOffheapEvict() throws Throwable {
-        restartWithTxFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode.OFFHEAP_EVICT);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testRestartWithTxFourNodesOneBackupsOffheapEvictSwap() throws Throwable {
-        restartWithTxFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode.OFFHEAP_EVICT_SWAP);
-    }
-
-    /**
-     * @param memMode Memory mode.
-     * @throws Throwable If failed.
-     */
-    private void restartWithTxFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode memMode) throws Throwable {
         backups = 1;
         nodeCnt = 4;
         keyCnt = 100_000;
         partitions = 29;
         rebalancMode = ASYNC;
-        this.memMode = memMode;
+        evict = true;
 
         long duration = 30_000;
 
@@ -507,6 +451,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
         keyCnt = 10;
         partitions = 29;
         rebalancMode = ASYNC;
+        evict = false;
 
         long duration = 90000;
 
@@ -522,6 +467,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
         keyCnt = 10;
         partitions = 29;
         rebalancMode = ASYNC;
+        evict = false;
 
         long duration = 90000;
 
@@ -537,6 +483,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
         keyCnt = 10;
         partitions = 29;
         rebalancMode = ASYNC;
+        evict = false;
 
         long duration = 90000;
 
@@ -552,6 +499,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
         keyCnt = 10;
         partitions = 29;
         rebalancMode = ASYNC;
+        evict = false;
 
         long duration = 90000;
 
@@ -567,6 +515,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
         keyCnt = 10;
         partitions = 29;
         rebalancMode = ASYNC;
+        evict = false;
 
         long duration = 90000;
 
@@ -582,6 +531,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
         keyCnt = 10;
         partitions = 29;
         rebalancMode = ASYNC;
+        evict = false;
 
         long duration = 90000;
 
@@ -597,6 +547,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
         keyCnt = 10;
         partitions = 29;
         rebalancMode = ASYNC;
+        evict = false;
 
         long duration = 90000;
 
@@ -612,6 +563,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
         keyCnt = 10;
         partitions = 29;
         rebalancMode = ASYNC;
+        evict = false;
 
         long duration = 90000;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractPartitionedByteArrayValuesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractPartitionedByteArrayValuesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractPartitionedByteArrayValuesSelfTest.java
index ba412d7..16ebf7e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractPartitionedByteArrayValuesSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractPartitionedByteArrayValuesSelfTest.java
@@ -23,8 +23,6 @@ import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.configuration.TransactionConfiguration;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
-import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_TIERED;
-import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_VALUES;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 
@@ -55,37 +53,6 @@ public abstract class GridCacheAbstractPartitionedByteArrayValuesSelfTest extend
         cfg.setNearConfiguration(nearConfiguration());
         cfg.setBackups(1);
         cfg.setWriteSynchronizationMode(FULL_SYNC);
-        cfg.setEvictSynchronized(false);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheConfiguration offheapCacheConfiguration0() {
-        CacheConfiguration cfg = new CacheConfiguration();
-
-        cfg.setCacheMode(PARTITIONED);
-        cfg.setAtomicityMode(TRANSACTIONAL);
-        cfg.setNearConfiguration(nearConfiguration());
-        cfg.setBackups(1);
-        cfg.setWriteSynchronizationMode(FULL_SYNC);
-        cfg.setMemoryMode(OFFHEAP_VALUES);
-        cfg.setOffHeapMaxMemory(100 * 1024 * 1024);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheConfiguration offheapTieredCacheConfiguration0() {
-        CacheConfiguration cfg = new CacheConfiguration();
-
-        cfg.setCacheMode(PARTITIONED);
-        cfg.setAtomicityMode(TRANSACTIONAL);
-        cfg.setNearConfiguration(nearConfiguration());
-        cfg.setBackups(1);
-        cfg.setWriteSynchronizationMode(FULL_SYNC);
-        cfg.setMemoryMode(OFFHEAP_TIERED);
-        cfg.setOffHeapMaxMemory(100 * 1024 * 1024);
 
         return cfg;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicNodeRestartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicNodeRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicNodeRestartTest.java
index 37ed866..6b62912 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicNodeRestartTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicNodeRestartTest.java
@@ -35,9 +35,4 @@ public class IgniteCacheAtomicNodeRestartTest extends GridCachePartitionedNodeRe
     @Override public void testRestartWithPutFourNodesNoBackups() {
         fail("https://issues.apache.org/jira/browse/IGNITE-1587");
     }
-
-    /** {@inheritDoc} */
-    @Override public void testRestartWithPutFourNodesOneBackupsOffheapTiered() {
-        fail("https://issues.apache.org/jira/browse/IGNITE-1587");
-    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheNearOffheapGetSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheNearOffheapGetSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheNearOffheapGetSelfTest.java
deleted file mode 100644
index 0635b2d..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheNearOffheapGetSelfTest.java
+++ /dev/null
@@ -1,136 +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;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgniteState;
-import org.apache.ignite.Ignition;
-import org.apache.ignite.cache.CacheMemoryMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.cache.affinity.Affinity;
-import org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicy;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.configuration.NearCacheConfiguration;
-import org.apache.ignite.internal.IgniteEx;
-import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest;
-
-import static org.apache.ignite.cache.CacheMode.PARTITIONED;
-
-/**
- *
- */
-public class IgniteCacheNearOffheapGetSelfTest extends GridCacheAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 4;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        super.beforeTestsStarted();
-
-        if (nearEnabled())
-            grid(gridCount() - 1).getOrCreateCache(new CacheConfiguration(), nearConfiguration());
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override protected NearCacheConfiguration nearConfiguration() {
-        NearCacheConfiguration nearCfg = super.nearConfiguration();
-
-        nearCfg.setNearEvictionPolicy(new FifoEvictionPolicy(100));
-
-        return nearCfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
-
-        if (getTestIgniteInstanceName(gridCount() - 1).equals(igniteInstanceName)) {
-            cfg.setClientMode(true);
-
-            cfg.setCacheConfiguration();
-        }
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override protected CacheConfiguration cacheConfiguration(String igniteInstanceName) throws Exception {
-        CacheConfiguration cfg = super.cacheConfiguration(igniteInstanceName);
-
-        cfg.setBackups(1);
-        cfg.setMemoryMode(CacheMemoryMode.OFFHEAP_TIERED);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return PARTITIONED;
-    }
-
-    @Override
-    protected long getTestTimeout() {
-        return Long.MAX_VALUE;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testGetFromNear() throws Exception {
-
-        IgniteCache<Object, Object> nearOnly = ignite(gridCount() - 1).cache(null);
-
-        // Start extra node.
-        IgniteEx ignite = startGrid(gridCount());
-
-        try {
-            final int keyCnt = 30;
-
-            for (int i = 0; i < keyCnt; i++)
-                ignite(0).cache(null).put(i, i);
-
-            for (int i = 0; i < keyCnt; i++)
-                assertEquals(i, nearOnly.get(i));
-
-            Collection<Integer> invalidatedKeys = new ArrayList<>();
-
-            Affinity<Object> cacheAff = ignite.affinity(null);
-
-            // Going to stop the last node.
-            for (int i = 0; i < keyCnt; i++) {
-                if (cacheAff.mapKeyToNode(i).equals(ignite.localNode()))
-                    invalidatedKeys.add(i);
-            }
-
-            stopGrid(gridCount());
-
-            for (Integer key : invalidatedKeys)
-                assertEquals(key, nearOnly.get(key));
-        }
-        finally {
-            if (Ignition.state(getTestIgniteInstanceName(gridCount())) == IgniteState.STARTED)
-                stopGrid(gridCount());
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheTxIteratorSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheTxIteratorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheTxIteratorSelfTest.java
index 2d9442c..cdf996d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheTxIteratorSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheTxIteratorSelfTest.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.cache.distributed;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMemoryMode;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicy;
@@ -62,7 +61,6 @@ public class IgniteCacheTxIteratorSelfTest extends GridCommonAbstractTest {
     private CacheConfiguration<String, TestClass> cacheConfiguration(
         CacheMode mode,
         CacheAtomicityMode atomMode,
-        CacheMemoryMode memMode,
         boolean nearEnabled,
         boolean useEvictPlc
     ) {
@@ -70,15 +68,14 @@ public class IgniteCacheTxIteratorSelfTest extends GridCommonAbstractTest {
 
         ccfg.setAtomicityMode(atomMode);
         ccfg.setCacheMode(mode);
-        ccfg.setMemoryMode(memMode);
         ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
 
         if (nearEnabled)
             ccfg.setNearConfiguration(new NearCacheConfiguration<String, TestClass>());
 
-        if (memMode == CacheMemoryMode.ONHEAP_TIERED && useEvictPlc) {
-            ccfg.setOffHeapMaxMemory(10 * 1024 * 1024);
+        if (useEvictPlc) {
             ccfg.setEvictionPolicy(new FifoEvictionPolicy(50));
+            ccfg.setOnheapCacheEnabled(true);
         }
 
         return ccfg;
@@ -107,17 +104,14 @@ public class IgniteCacheTxIteratorSelfTest extends GridCommonAbstractTest {
         try {
             for (CacheMode mode : CacheMode.values()) {
                 for (CacheAtomicityMode atomMode : CacheAtomicityMode.values()) {
-                    for (CacheMemoryMode memMode : CacheMemoryMode.values()) {
-                        if (mode == CacheMode.PARTITIONED) {
-                            // Near cache makes sense only for partitioned cache.
-                            checkTxCache(CacheMode.PARTITIONED, atomMode, memMode, true, false);
-                        }
+                    if (mode == CacheMode.PARTITIONED) {
+                        // Near cache makes sense only for partitioned cache.
+                        checkTxCache(CacheMode.PARTITIONED, atomMode, true, false);
+                    }
 
-                        if (memMode == CacheMemoryMode.ONHEAP_TIERED)
-                            checkTxCache(mode, atomMode, CacheMemoryMode.ONHEAP_TIERED, false, true);
+                    checkTxCache(CacheMode.PARTITIONED, atomMode, false, true);
 
-                        checkTxCache(mode, atomMode, memMode, false, false);
-                    }
+                    checkTxCache(CacheMode.PARTITIONED, atomMode, false, false);
                 }
             }
         }
@@ -132,7 +126,6 @@ public class IgniteCacheTxIteratorSelfTest extends GridCommonAbstractTest {
     private void checkTxCache(
         CacheMode mode,
         CacheAtomicityMode atomMode,
-        CacheMemoryMode memMode,
         boolean nearEnabled,
         boolean useEvicPlc
     ) throws Exception {
@@ -141,14 +134,13 @@ public class IgniteCacheTxIteratorSelfTest extends GridCommonAbstractTest {
         final CacheConfiguration<String, TestClass> ccfg = cacheConfiguration(
             mode,
             atomMode,
-            memMode,
             nearEnabled,
             useEvicPlc);
 
         final IgniteCache<String, TestClass> cache = ignite.createCache(ccfg);
 
-        info("Checking cache [mode=" + mode + ", atomMode=" + atomMode + ", memMode=" + memMode +
-            ", near=" + nearEnabled + ']');
+        info("Checking cache [mode=" + mode + ", atomMode=" + atomMode + ", near=" + nearEnabled +
+            ", evict=" + useEvicPlc + ']');
 
         try {
             for (int i = 0; i < 30; i++) {
@@ -213,8 +205,10 @@ public class IgniteCacheTxIteratorSelfTest extends GridCommonAbstractTest {
          */
         @Override
         public boolean equals(final Object o) {
-            if (this == o) return true;
-            if (o == null || getClass() != o.getClass()) return false;
+            if (this == o)
+                return true;
+            if (o == null || getClass() != o.getClass())
+                return false;
 
             final TestClass testCls = (TestClass)o;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheAbstractPartitionedOnlyByteArrayValuesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheAbstractPartitionedOnlyByteArrayValuesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheAbstractPartitionedOnlyByteArrayValuesSelfTest.java
index 0151be5..affac28 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheAbstractPartitionedOnlyByteArrayValuesSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheAbstractPartitionedOnlyByteArrayValuesSelfTest.java
@@ -38,18 +38,9 @@ public abstract class GridCacheAbstractPartitionedOnlyByteArrayValuesSelfTest ex
     /** Offheap cache name. */
     protected static final String CACHE_ATOMIC_OFFHEAP = "cache_atomic_offheap";
 
-    /** Offheap tiered cache name. */
-    protected static final String CACHE_ATOMIC_OFFHEAP_TIERED = "cache_atomic_offheap_tiered";
-
     /** Atomic caches. */
     private static IgniteCache<Integer, Object>[] cachesAtomic;
 
-    /** Atomic offheap caches. */
-    private static IgniteCache<Integer, Object>[] cachesAtomicOffheap;
-
-    /** Atomic offheap caches. */
-    private static IgniteCache<Integer, Object>[] cachesAtomicOffheapTiered;
-
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration c = super.getConfiguration(igniteInstanceName);
@@ -60,24 +51,7 @@ public abstract class GridCacheAbstractPartitionedOnlyByteArrayValuesSelfTest ex
         atomicCacheCfg.setAtomicityMode(ATOMIC);
         atomicCacheCfg.setAtomicWriteOrderMode(PRIMARY);
 
-        CacheConfiguration atomicOffheapCacheCfg = offheapCacheConfiguration0();
-
-        atomicOffheapCacheCfg.setName(CACHE_ATOMIC_OFFHEAP);
-        atomicOffheapCacheCfg.setAtomicityMode(ATOMIC);
-        atomicOffheapCacheCfg.setAtomicWriteOrderMode(PRIMARY);
-
-        CacheConfiguration atomicOffheapTieredCacheCfg = offheapTieredCacheConfiguration();
-
-        atomicOffheapTieredCacheCfg.setName(CACHE_ATOMIC_OFFHEAP_TIERED);
-        atomicOffheapTieredCacheCfg.setAtomicityMode(ATOMIC);
-        atomicOffheapTieredCacheCfg.setAtomicWriteOrderMode(PRIMARY);
-
-        c.setCacheConfiguration(cacheConfiguration(),
-            offheapCacheConfiguration(),
-            offheapTieredCacheConfiguration(),
-            atomicCacheCfg,
-            atomicOffheapCacheCfg,
-            atomicOffheapTieredCacheCfg);
+        c.setCacheConfiguration(cacheConfiguration(), atomicCacheCfg);
 
         c.setPeerClassLoadingEnabled(peerClassLoading());
 
@@ -97,21 +71,14 @@ public abstract class GridCacheAbstractPartitionedOnlyByteArrayValuesSelfTest ex
         int gridCnt = gridCount();
 
         cachesAtomic = new IgniteCache[gridCnt];
-        cachesAtomicOffheap = new IgniteCache[gridCnt];
-        cachesAtomicOffheapTiered = new IgniteCache[gridCnt];
 
-        for (int i = 0; i < gridCount(); i++) {
+        for (int i = 0; i < gridCount(); i++)
             cachesAtomic[i] = ignites[i].cache(CACHE_ATOMIC);
-            cachesAtomicOffheap[i] = ignites[i].cache(CACHE_ATOMIC_OFFHEAP);
-            cachesAtomicOffheapTiered[i] = ignites[i].cache(CACHE_ATOMIC_OFFHEAP_TIERED);
-        }
     }
 
     /** {@inheritDoc} */
     @Override protected void afterTestsStopped() throws Exception {
         cachesAtomic = null;
-        cachesAtomicOffheap = null;
-        cachesAtomicOffheapTiered = null;
 
         super.afterTestsStopped();
     }
@@ -126,24 +93,6 @@ public abstract class GridCacheAbstractPartitionedOnlyByteArrayValuesSelfTest ex
     }
 
     /**
-     * Test atomic offheap cache.
-     *
-     * @throws Exception If failed.
-     */
-    public void testAtomicOffheap() throws Exception {
-        testAtomic0(cachesAtomicOffheap);
-    }
-
-    /**
-     * Test atomic offheap cache.
-     *
-     * @throws Exception If failed.
-     */
-    public void testAtomicOffheapTiered() throws Exception {
-        testAtomic0(cachesAtomicOffheapTiered);
-    }
-
-    /**
      * INternal routine for ATOMIC cache testing.
      *
      * @param caches Caches.

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedOptimisticTransactionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedOptimisticTransactionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedOptimisticTransactionSelfTest.java
index f056825..e32d4a1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedOptimisticTransactionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedOptimisticTransactionSelfTest.java
@@ -77,7 +77,6 @@ public class GridCacheColocatedOptimisticTransactionSelfTest extends GridCommonA
         cc.setNearConfiguration(null);
         cc.setBackups(1);
         cc.setWriteSynchronizationMode(FULL_SYNC);
-        cc.setEvictSynchronized(false);
 
         c.setDiscoverySpi(disco);
         c.setCacheConfiguration(cc);

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/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 2d2ef41..20162a7 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
@@ -76,19 +76,15 @@ public class GridCacheDhtEvictionNearReadersSelfTest extends GridCommonAbstractT
 
         cacheCfg.setCacheMode(PARTITIONED);
         cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        cacheCfg.setEvictSynchronized(true);
         cacheCfg.setRebalanceMode(SYNC);
         cacheCfg.setAtomicityMode(atomicityMode());
         cacheCfg.setBackups(1);
 
-        // Set eviction queue size explicitly.
-        cacheCfg.setEvictSynchronizedKeyBufferSize(1);
-        cacheCfg.setEvictMaxOverflowRatio(0);
-
         FifoEvictionPolicy plc = new FifoEvictionPolicy();
         plc.setMaxSize(10);
 
         cacheCfg.setEvictionPolicy(plc);
+        cacheCfg.setOnheapCacheEnabled(true);
 
         NearCacheConfiguration nearCfg = new NearCacheConfiguration();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/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
deleted file mode 100644
index b177a86..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionSelfTest.java
+++ /dev/null
@@ -1,356 +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 java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.UUID;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.cache.CacheWriteSynchronizationMode;
-import org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicy;
-import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.configuration.NearCacheConfiguration;
-import org.apache.ignite.events.Event;
-import org.apache.ignite.internal.IgniteKernal;
-import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheAdapter;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.P1;
-import org.apache.ignite.lang.IgniteFuture;
-import org.apache.ignite.lang.IgnitePredicate;
-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.CacheAtomicityMode.TRANSACTIONAL;
-import static org.apache.ignite.cache.CacheMode.PARTITIONED;
-import static org.apache.ignite.cache.CacheRebalanceMode.NONE;
-import static org.apache.ignite.events.EventType.EVT_CACHE_ENTRY_EVICTED;
-
-/**
- * Tests for dht cache eviction.
- */
-public class GridCacheDhtEvictionSelfTest extends GridCommonAbstractTest {
-    /** */
-    private static final int GRID_CNT = 2;
-
-    /** */
-    private TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
-
-    /** Default constructor. */
-    public GridCacheDhtEvictionSelfTest() {
-        super(false /* don't start grid. */);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
-
-        TcpDiscoverySpi disco = new TcpDiscoverySpi();
-
-        disco.setIpFinder(ipFinder);
-
-        cfg.setDiscoverySpi(disco);
-
-        CacheConfiguration cacheCfg = defaultCacheConfiguration();
-
-        cacheCfg.setCacheMode(PARTITIONED);
-        cacheCfg.setRebalanceMode(NONE);
-        cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        cacheCfg.setEvictSynchronized(true);
-        cacheCfg.setAtomicityMode(TRANSACTIONAL);
-        cacheCfg.setBackups(1);
-
-        NearCacheConfiguration nearCfg = new NearCacheConfiguration();
-
-        FifoEvictionPolicy nearPlc = new FifoEvictionPolicy();
-        nearPlc.setMaxSize(10000);
-
-        nearCfg.setNearEvictionPolicy(nearPlc);
-        cacheCfg.setNearConfiguration(nearCfg);
-
-        // Set eviction queue size explicitly.
-        cacheCfg.setEvictMaxOverflowRatio(0);
-        cacheCfg.setEvictSynchronizedKeyBufferSize(1);
-
-        FifoEvictionPolicy plc = new FifoEvictionPolicy();
-        plc.setMaxSize(10000);
-
-        cacheCfg.setEvictionPolicy(plc);
-
-        cfg.setCacheConfiguration(cacheCfg);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings({"ConstantConditions"})
-    @Override protected void beforeTestsStarted() throws Exception {
-        super.beforeTestsStarted();
-
-        if (GRID_CNT < 2)
-            throw new IgniteCheckedException("GRID_CNT must not be less than 2.");
-
-        startGrids(GRID_CNT);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        super.afterTestsStopped();
-
-        stopAllGrids();
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings({"SizeReplaceableByIsEmpty"})
-    @Override protected void beforeTest() throws Exception {
-        for (int i = 0; i < GRID_CNT; i++) {
-            assert near(grid(i)).size() == 0;
-            assert dht(grid(i)).size() == 0;
-
-            assert near(grid(i)).isEmpty();
-            assert dht(grid(i)).isEmpty();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings({"unchecked"})
-    @Override protected void afterTest() throws Exception {
-        for (int i = 0; i < GRID_CNT; i++) {
-            near(grid(i)).removeAll();
-
-            assert near(grid(i)).isEmpty() : "Near cache is not empty [idx=" + i + "]";
-            assert dht(grid(i)).isEmpty() : "Dht cache is not empty [idx=" + i + "]";
-        }
-    }
-
-    /**
-     * @param g Grid.
-     * @return Near cache.
-     */
-    @SuppressWarnings({"unchecked"})
-    private GridNearCacheAdapter<Integer, String> near(Ignite g) {
-        return (GridNearCacheAdapter)((IgniteKernal)g).internalCache();
-    }
-
-    /**
-     * @param g Grid.
-     * @return Dht cache.
-     */
-    @SuppressWarnings({"unchecked", "TypeMayBeWeakened"})
-    private GridDhtCacheAdapter<Integer, String> dht(Ignite g) {
-        return ((GridNearCacheAdapter)((IgniteKernal)g).internalCache()).dht();
-    }
-
-    /**
-     * @param key Key.
-     * @return Primary node for the given key.
-     */
-    private Collection<ClusterNode> keyNodes(Object key) {
-        return grid(0).affinity(null).mapKeyToPrimaryAndBackups(key);
-    }
-
-    /**
-     * @param nodeId Node id.
-     * @return Predicate for events belonging to specified node.
-     */
-    private IgnitePredicate<Event> nodeEvent(final UUID nodeId) {
-        assert nodeId != null;
-
-        return new P1<Event>() {
-            @Override public boolean apply(Event e) {
-                info("Predicate called [e.nodeId()=" + e.node().id() + ", nodeId=" + nodeId + ']');
-
-                return e.node().id().equals(nodeId);
-            }
-        };
-    }
-
-    /**
-     * JUnit.
-     *
-     * @throws Exception If failed.
-     */
-    @SuppressWarnings("NullArgumentToVariableArgMethod")
-    public void testSingleKey() throws Exception {
-        Integer key = 1;
-
-        Collection<ClusterNode> nodes = new ArrayList<>(keyNodes(key));
-
-        ClusterNode primary = F.first(nodes);
-
-        assert primary != null;
-
-        nodes.remove(primary);
-
-        ClusterNode backup = F.first(nodes);
-
-        assert backup != null;
-
-        assert !F.eqNodes(primary, backup);
-
-        info("Key primary node: " + primary.id());
-        info("Key backup node: " + backup.id());
-
-        GridNearCacheAdapter<Integer, String> nearPrimary = near(grid(primary));
-        GridDhtCacheAdapter<Integer, String> dhtPrimary = dht(grid(primary));
-
-        GridNearCacheAdapter<Integer, String> nearBackup = near(grid(backup));
-        GridDhtCacheAdapter<Integer, String> dhtBackup = dht(grid(backup));
-
-        String val = "v1";
-
-        // Put on primary node.
-        nearPrimary.getAndPut(key, val, null);
-
-        assertEquals(val, localPeek(nearPrimary, key));
-        assertEquals(val, localPeek(dhtPrimary, key));
-
-        assertEquals(val, localPeek(nearBackup, key));
-        assertEquals(val, localPeek(dhtBackup, key));
-
-        GridDhtCacheEntry entryPrimary = (GridDhtCacheEntry)dhtPrimary.peekEx(key);
-        GridDhtCacheEntry entryBackup = (GridDhtCacheEntry)dhtBackup.peekEx(key);
-
-        assert entryPrimary != null;
-        assert entryBackup != null;
-
-        assertTrue(entryPrimary.readers().isEmpty());
-        assertTrue(entryBackup.readers().isEmpty());
-
-        IgniteFuture<Event> futBackup =
-            waitForLocalEvent(grid(backup).events(), nodeEvent(backup.id()), EVT_CACHE_ENTRY_EVICTED);
-
-        IgniteFuture<Event> futPrimary =
-            waitForLocalEvent(grid(primary).events(), nodeEvent(primary.id()), EVT_CACHE_ENTRY_EVICTED);
-
-        // Evict on primary node.
-        // It should trigger dht eviction and eviction on backup node.
-        grid(primary).cache(null).localEvict(Collections.<Object>singleton(key));
-
-        // Give some time for eviction event to occur on backup and primary node.
-        futBackup.get(10_000);
-        futPrimary.get(10_000);
-
-        assertEquals(0, nearPrimary.size());
-
-        assertNull(nearPrimary.peekEx(key));
-        assertNull(dhtPrimary.peekEx(key));
-
-        assertNull(nearBackup.peekEx(key));
-        assertNull(dhtBackup.peekEx(key));
-    }
-
-    /**
-     * JUnit.
-     *
-     * @throws Exception If failed.
-     */
-    @SuppressWarnings("NullArgumentToVariableArgMethod")
-    public void testMultipleKeys() throws Exception {
-        final int keyCnt = 1000;
-
-        final Ignite primaryIgnite = grid(0);
-        final Ignite backupIgnite = grid(1);
-
-        GridNearCacheAdapter<Integer, String> nearPrimary = near(primaryIgnite);
-        GridDhtCacheAdapter<Integer, String> dhtPrimary = dht(primaryIgnite);
-
-        GridNearCacheAdapter<Integer, String> nearBackup = near(backupIgnite);
-        GridDhtCacheAdapter<Integer, String> dhtBackup = dht(backupIgnite);
-
-        Collection<Integer> keys = new ArrayList<>(keyCnt);
-
-        for (int key = 0; keys.size() < keyCnt; key++) {
-            if (F.eqNodes(primaryIgnite.cluster().localNode(), F.first(keyNodes(key))))
-                keys.add(key++);
-        }
-
-        info("Test keys: " + keys);
-
-        // Put on primary node.
-        for (Integer key : keys)
-            nearPrimary.getAndPut(key, "v" + key, null);
-
-        for (Integer key : keys) {
-            String val = "v" + key;
-
-            assertEquals(val, localPeek(nearPrimary, key));
-            assertEquals(val, localPeek(dhtPrimary, key));
-
-            assertEquals(val, localPeek(nearBackup, key));
-            assertEquals(val, localPeek(dhtBackup, key));
-        }
-
-        final AtomicInteger cntBackup = new AtomicInteger();
-
-        IgniteFuture<Event> futBackup = waitForLocalEvent(backupIgnite.events(), new P1<Event>() {
-            @Override public boolean apply(Event e) {
-                return e.node().id().equals(backupIgnite.cluster().localNode().id()) &&
-                    cntBackup.incrementAndGet() == keyCnt;
-            }
-        }, EVT_CACHE_ENTRY_EVICTED);
-
-        final AtomicInteger cntPrimary = new AtomicInteger();
-
-        IgniteFuture<Event> futPrimary = waitForLocalEvent(primaryIgnite.events(), new P1<Event>() {
-            @Override public boolean apply(Event e) {
-                return e.node().id().equals(primaryIgnite.cluster().localNode().id()) &&
-                    cntPrimary.incrementAndGet() == keyCnt;
-            }
-        }, EVT_CACHE_ENTRY_EVICTED);
-
-        // Evict on primary node.
-        // Eviction of the last key should trigger queue processing.
-        for (Integer key : keys)
-            primaryIgnite.cache(null).localEvict(Collections.<Object>singleton(key));
-
-        // Give some time for eviction events to occur on backup and primary node.
-        futBackup.get(10_000);
-        futPrimary.get(10_000);
-
-        info("nearBackupSize: " + nearBackup.size());
-        info("dhtBackupSize: " + dhtBackup.size());
-        info("nearPrimarySize: " + nearPrimary.size());
-        info("dhtPrimarySize: " + dhtPrimary.size());
-
-        // Check backup node first.
-        for (Integer key : keys) {
-            String msg = "Failed key: " + key;
-
-            assertNull(msg, localPeek(nearBackup, key));
-            assertNull(msg, localPeek(dhtBackup, key));
-            assertNull(msg, nearBackup.peekEx(key));
-            assertNull(msg, dhtBackup.peekEx(key));
-        }
-
-        for (Integer key : keys) {
-            String msg = "Failed key: " + key;
-
-            assertNull(msg, localPeek(nearPrimary, key));
-            assertNull(msg, localPeek(dhtPrimary, key));
-            assertNull(msg, nearPrimary.peekEx(key));
-            assertNull(dhtPrimary.peekEx(key));
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadOffHeapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadOffHeapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadOffHeapSelfTest.java
deleted file mode 100644
index 2680a4f..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadOffHeapSelfTest.java
+++ /dev/null
@@ -1,38 +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.configuration.CacheConfiguration;
-import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloader;
-
-import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_VALUES;
-
-/**
- * Test cases for partitioned cache {@link GridDhtPreloader preloader} with off-heap value storage.
- */
-public class GridCacheDhtPreloadOffHeapSelfTest extends GridCacheDhtPreloadSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration(String igniteInstanceName) {
-        CacheConfiguration cacheCfg = super.cacheConfiguration(igniteInstanceName);
-
-        cacheCfg.setMemoryMode(OFFHEAP_VALUES);
-        cacheCfg.setOffHeapMaxMemory(0);
-
-        return cacheCfg;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadOnheapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadOnheapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadOnheapSelfTest.java
new file mode 100644
index 0000000..07a858e
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadOnheapSelfTest.java
@@ -0,0 +1,26 @@
+/*
+ * 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 GridCacheDhtPreloadOnheapSelfTest extends GridCacheDhtPreloadSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean onheapCacheEnabled() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
index 9987a8d..43aa10b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
@@ -136,6 +136,7 @@ public class GridCacheDhtPreloadSelfTest extends GridCommonAbstractTest {
         cacheCfg.setRebalanceMode(preloadMode);
         cacheCfg.setAffinity(new RendezvousAffinityFunction(false, partitions));
         cacheCfg.setBackups(backups);
+        cacheCfg.setOnheapCacheEnabled(onheapCacheEnabled());
 
         return cacheCfg;
     }
@@ -163,6 +164,13 @@ public class GridCacheDhtPreloadSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @return {@code True} if on-heap cache is enabled.
+     */
+    protected boolean onheapCacheEnabled() {
+        return false;
+    }
+
+    /**
      * @throws Exception If failed.
      */
     public void testActivePartitionTransferSyncSameCoordinator() throws Exception {
@@ -344,7 +352,8 @@ public class GridCacheDhtPreloadSelfTest extends GridCommonAbstractTest {
             error("Test failed.", e);
 
             throw e;
-        } finally {
+        }
+        finally {
             stopAllGrids();
         }
     }
@@ -432,7 +441,6 @@ public class GridCacheDhtPreloadSelfTest extends GridCommonAbstractTest {
         checkNodes(1000, 4, false, false);
     }
 
-
     /**
      * @throws Exception If failed.
      */
@@ -549,17 +557,16 @@ public class GridCacheDhtPreloadSelfTest extends GridCommonAbstractTest {
 
                 for (Ignite gg : ignites)
                     futs.add(waitForLocalEvent(gg.events(), new P1<Event>() {
-                            @Override public boolean apply(Event e) {
-                                CacheRebalancingEvent evt = (CacheRebalancingEvent)e;
+                        @Override public boolean apply(Event e) {
+                            CacheRebalancingEvent evt = (CacheRebalancingEvent)e;
 
-                                ClusterNode node = evt.discoveryNode();
+                            ClusterNode node = evt.discoveryNode();
 
-                                return evt.type() == EVT_CACHE_REBALANCE_STOPPED && node.id().equals(nodeId) &&
-                                    (evt.discoveryEventType() == EVT_NODE_LEFT ||
+                            return evt.type() == EVT_CACHE_REBALANCE_STOPPED && node.id().equals(nodeId) &&
+                                (evt.discoveryEventType() == EVT_NODE_LEFT ||
                                     evt.discoveryEventType() == EVT_NODE_FAILED);
-                            }
-                        }, EVT_CACHE_REBALANCE_STOPPED));
-
+                        }
+                    }, EVT_CACHE_REBALANCE_STOPPED));
 
                 info("Before grid stop [name=" + g.name() + ", fullTop=" + top2string(ignites));
 
@@ -607,7 +614,8 @@ public class GridCacheDhtPreloadSelfTest extends GridCommonAbstractTest {
             error("Test failed.", e);
 
             throw e;
-        } finally {
+        }
+        finally {
             stopAllGrids();
         }
     }
@@ -652,9 +660,9 @@ public class GridCacheDhtPreloadSelfTest extends GridCommonAbstractTest {
                 boolean primary = primaryNode.equals(loc);
 
                 assertEquals("Key check failed [igniteInstanceName=" + ignite.name() +
-                    ", cache=" + cache.getName() + ", key=" + i + ", expected=" + i + ", actual=" + val +
-                    ", part=" + aff.partition(i) + ", primary=" + primary + ", affNodes=" + U.nodeIds(affNodes) +
-                    ", locId=" + loc.id() + ", allNodes=" + U.nodeIds(nodes) + ", allParts=" + top2string(grids) + ']',
+                        ", cache=" + cache.getName() + ", key=" + i + ", expected=" + i + ", actual=" + val +
+                        ", part=" + aff.partition(i) + ", primary=" + primary + ", affNodes=" + U.nodeIds(affNodes) +
+                        ", locId=" + loc.id() + ", allNodes=" + U.nodeIds(nodes) + ", allParts=" + top2string(grids) + ']',
                     Integer.toString(i), val);
             }
         }
@@ -664,7 +672,7 @@ public class GridCacheDhtPreloadSelfTest extends GridCommonAbstractTest {
      * @param grids Grids
      * @return String representation of all partitions and their state.
      */
-    @SuppressWarnings( {"ConstantConditions"})
+    @SuppressWarnings({"ConstantConditions"})
     private String top2string(Iterable<Ignite> grids) {
         Map<String, String> map = new HashMap<>();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledAtomicOffHeapTieredMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledAtomicOffHeapTieredMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledAtomicOffHeapTieredMultiNodeFullApiSelfTest.java
deleted file mode 100644
index 4bddbf9..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledAtomicOffHeapTieredMultiNodeFullApiSelfTest.java
+++ /dev/null
@@ -1,43 +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.CacheAtomicityMode;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
-
-/**
- * Tests colocated cache with off-heap tiered mode.
- */
-public class GridCachePartitionedNearDisabledAtomicOffHeapTieredMultiNodeFullApiSelfTest extends
-    GridCachePartitionedNearDisabledOffHeapTieredMultiNodeFullApiSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheAtomicityMode atomicityMode() {
-        return ATOMIC;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected boolean txEnabled() {
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected boolean lockingEnabled() {
-        return false;
-    }
-}
\ No newline at end of file

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledAtomicOnheapMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledAtomicOnheapMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledAtomicOnheapMultiNodeFullApiSelfTest.java
new file mode 100644
index 0000000..20e7b7a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledAtomicOnheapMultiNodeFullApiSelfTest.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;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+
+public class GridCachePartitionedNearDisabledAtomicOnheapMultiNodeFullApiSelfTest extends GridCachePartitionedNearDisabledOnheapMultiNodeFullApiSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return CacheAtomicityMode.ATOMIC;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledOffHeapFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledOffHeapFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledOffHeapFullApiSelfTest.java
deleted file mode 100644
index 6f64c9a..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledOffHeapFullApiSelfTest.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.CacheMemoryMode;
-
-import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_VALUES;
-
-/**
- * Tests colocated cache with values being stored off-heap.
- */
-public class GridCachePartitionedNearDisabledOffHeapFullApiSelfTest extends
-    GridCachePartitionedNearDisabledFullApiSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMemoryMode memoryMode() {
-        return OFFHEAP_VALUES;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledOffHeapMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledOffHeapMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledOffHeapMultiNodeFullApiSelfTest.java
deleted file mode 100644
index b559395..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledOffHeapMultiNodeFullApiSelfTest.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.CacheMemoryMode;
-
-import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_VALUES;
-
-/**
- * Tests colocated cache with values being stored off-heap.
- */
-public class GridCachePartitionedNearDisabledOffHeapMultiNodeFullApiSelfTest
-    extends GridCachePartitionedNearDisabledMultiNodeFullApiSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMemoryMode memoryMode() {
-        return OFFHEAP_VALUES;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledOffHeapTieredAtomicFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledOffHeapTieredAtomicFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledOffHeapTieredAtomicFullApiSelfTest.java
deleted file mode 100644
index 7e3220f..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledOffHeapTieredAtomicFullApiSelfTest.java
+++ /dev/null
@@ -1,57 +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.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMemoryMode;
-import org.apache.ignite.configuration.NearCacheConfiguration;
-import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedFullApiSelfTest;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
-import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_TIERED;
-
-/**
- * Tests colocated cache with off-heap tiered mode.
- */
-public class GridCachePartitionedNearDisabledOffHeapTieredAtomicFullApiSelfTest extends
-    GridCachePartitionedFullApiSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheAtomicityMode atomicityMode() {
-        return ATOMIC;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected boolean txEnabled() {
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected boolean lockingEnabled() {
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected NearCacheConfiguration nearConfiguration() {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheMemoryMode memoryMode() {
-        return OFFHEAP_TIERED;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledOffHeapTieredFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledOffHeapTieredFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledOffHeapTieredFullApiSelfTest.java
deleted file mode 100644
index 1ff7702..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledOffHeapTieredFullApiSelfTest.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.CacheMemoryMode;
-
-import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_TIERED;
-
-/**
- * Tests colocated cache with off-heap tiered mode.
- */
-public class GridCachePartitionedNearDisabledOffHeapTieredFullApiSelfTest extends
-    GridCachePartitionedNearDisabledOffHeapFullApiSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMemoryMode memoryMode() {
-        return OFFHEAP_TIERED;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledOffHeapTieredMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledOffHeapTieredMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledOffHeapTieredMultiNodeFullApiSelfTest.java
deleted file mode 100644
index 7fc98cc..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledOffHeapTieredMultiNodeFullApiSelfTest.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.CacheMemoryMode;
-
-import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_TIERED;
-
-/**
- * Tests colocated cache with off-heap tiered mode.
- */
-public class GridCachePartitionedNearDisabledOffHeapTieredMultiNodeFullApiSelfTest extends
-    GridCachePartitionedNearDisabledOffHeapMultiNodeFullApiSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMemoryMode memoryMode() {
-        return OFFHEAP_TIERED;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledOnheapFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledOnheapFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledOnheapFullApiSelfTest.java
new file mode 100644
index 0000000..620db8a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledOnheapFullApiSelfTest.java
@@ -0,0 +1,27 @@
+/*
+ * 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 GridCachePartitionedNearDisabledOnheapFullApiSelfTest
+    extends GridCachePartitionedNearDisabledFullApiSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean onheapCacheEnabled() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledOnheapMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledOnheapMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledOnheapMultiNodeFullApiSelfTest.java
new file mode 100644
index 0000000..3769103
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledOnheapMultiNodeFullApiSelfTest.java
@@ -0,0 +1,27 @@
+/*
+ * 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 GridCachePartitionedNearDisabledOnheapMultiNodeFullApiSelfTest
+    extends GridCachePartitionedNearDisabledMultiNodeFullApiSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean onheapCacheEnabled() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheConcurrentPutGetRemove.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheConcurrentPutGetRemove.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheConcurrentPutGetRemove.java
index 436eec8..ca7b3ed 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheConcurrentPutGetRemove.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheConcurrentPutGetRemove.java
@@ -24,7 +24,6 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMemoryMode;
 import org.apache.ignite.cache.affinity.Affinity;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -38,8 +37,6 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.PRIMARY;
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
-import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_TIERED;
-import static org.apache.ignite.cache.CacheMemoryMode.ONHEAP_TIERED;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 
 /**
@@ -72,28 +69,14 @@ public class IgniteCacheConcurrentPutGetRemove extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testPutGetRemoveAtomic() throws Exception {
-        putGetRemove(cacheConfiguration(ATOMIC, ONHEAP_TIERED, 1));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPutGetRemoveAtomicOffheap() throws Exception {
-        putGetRemove(cacheConfiguration(ATOMIC, OFFHEAP_TIERED, 1));
+        putGetRemove(cacheConfiguration(ATOMIC, 1));
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testPutGetRemoveTx() throws Exception {
-        putGetRemove(cacheConfiguration(TRANSACTIONAL, ONHEAP_TIERED, 1));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPutGetRemoveTxOffheap() throws Exception {
-        putGetRemove(cacheConfiguration(TRANSACTIONAL, OFFHEAP_TIERED, 1));
+        putGetRemove(cacheConfiguration(TRANSACTIONAL, 1));
     }
 
     /**
@@ -181,17 +164,13 @@ public class IgniteCacheConcurrentPutGetRemove extends GridCommonAbstractTest {
 
     /**
      * @param atomicityMode Cache atomicity mode.
-     * @param memoryMode Cache memory mode.
      * @param backups Backups number.
      * @return Cache configuration.
      */
-    private CacheConfiguration cacheConfiguration(CacheAtomicityMode atomicityMode,
-        CacheMemoryMode memoryMode,
-        int backups) {
+    private CacheConfiguration cacheConfiguration(CacheAtomicityMode atomicityMode, int backups) {
         CacheConfiguration ccfg = new CacheConfiguration();
 
         ccfg.setAtomicityMode(atomicityMode);
-        ccfg.setMemoryMode(memoryMode);
         ccfg.setBackups(backups);
         ccfg.setWriteSynchronizationMode(FULL_SYNC);
         ccfg.setAtomicWriteOrderMode(PRIMARY);

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheCrossCacheTxFailoverTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheCrossCacheTxFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheCrossCacheTxFailoverTest.java
index 9a1aaff..0831d1e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheCrossCacheTxFailoverTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheCrossCacheTxFailoverTest.java
@@ -53,9 +53,7 @@ import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
-import static org.apache.ignite.testframework.GridTestUtils.TestMemoryMode;
 import static org.apache.ignite.testframework.GridTestUtils.runMultiThreadedAsync;
-import static org.apache.ignite.testframework.GridTestUtils.setMemoryMode;
 import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC;
 import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
 import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
@@ -115,13 +113,11 @@ public class IgniteCacheCrossCacheTxFailoverTest extends GridCommonAbstractTest
      * @param name Cache name.
      * @param cacheMode Cache mode.
      * @param parts Number of partitions.
-     * @param memMode Memory mode.
      * @return Cache configuration.
      */
     private CacheConfiguration cacheConfiguration(String name,
         CacheMode cacheMode,
-        int parts,
-        TestMemoryMode memMode) {
+        int parts) {
         CacheConfiguration ccfg = new CacheConfiguration();
 
         ccfg.setName(name);
@@ -134,8 +130,6 @@ public class IgniteCacheCrossCacheTxFailoverTest extends GridCommonAbstractTest
 
         ccfg.setAffinity(new RendezvousAffinityFunction(false, parts));
 
-        setMemoryMode(null, ccfg, memMode, 100, 1024);
-
         return ccfg;
     }
 
@@ -148,70 +142,56 @@ public class IgniteCacheCrossCacheTxFailoverTest extends GridCommonAbstractTest
      * @throws Exception If failed.
      */
     public void testCrossCachePessimisticTxFailover() throws Exception {
-        crossCacheTxFailover(PARTITIONED, true, PESSIMISTIC, REPEATABLE_READ, TestMemoryMode.HEAP);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCrossCachePessimisticTxFailoverOffheapSwap() throws Exception {
-        crossCacheTxFailover(PARTITIONED, true, PESSIMISTIC, REPEATABLE_READ, TestMemoryMode.OFFHEAP_EVICT_SWAP);
+        crossCacheTxFailover(PARTITIONED, true, PESSIMISTIC, REPEATABLE_READ);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testCrossCachePessimisticTxFailoverDifferentAffinity() throws Exception {
-        crossCacheTxFailover(PARTITIONED, false, PESSIMISTIC, REPEATABLE_READ, TestMemoryMode.HEAP);
+        crossCacheTxFailover(PARTITIONED, false, PESSIMISTIC, REPEATABLE_READ);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testCrossCacheOptimisticTxFailover() throws Exception {
-        crossCacheTxFailover(PARTITIONED, true, OPTIMISTIC, REPEATABLE_READ, TestMemoryMode.HEAP);
+        crossCacheTxFailover(PARTITIONED, true, OPTIMISTIC, REPEATABLE_READ);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testCrossCacheOptimisticSerializableTxFailover() throws Exception {
-        crossCacheTxFailover(PARTITIONED, true, OPTIMISTIC, SERIALIZABLE, TestMemoryMode.HEAP);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCrossCacheOptimisticTxFailoverOffheapSwap() throws Exception {
-        crossCacheTxFailover(PARTITIONED, true, OPTIMISTIC, REPEATABLE_READ, TestMemoryMode.OFFHEAP_EVICT_SWAP);
+        crossCacheTxFailover(PARTITIONED, true, OPTIMISTIC, SERIALIZABLE);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testCrossCacheOptimisticTxFailoverDifferentAffinity() throws Exception {
-        crossCacheTxFailover(PARTITIONED, false, OPTIMISTIC, REPEATABLE_READ, TestMemoryMode.HEAP);
+        crossCacheTxFailover(PARTITIONED, false, OPTIMISTIC, REPEATABLE_READ);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testCrossCachePessimisticTxFailoverReplicated() throws Exception {
-        crossCacheTxFailover(REPLICATED, true, PESSIMISTIC, REPEATABLE_READ, TestMemoryMode.HEAP);
+        crossCacheTxFailover(REPLICATED, true, PESSIMISTIC, REPEATABLE_READ);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testCrossCacheOptimisticTxFailoverReplicated() throws Exception {
-        crossCacheTxFailover(REPLICATED, true, OPTIMISTIC, REPEATABLE_READ, TestMemoryMode.HEAP);
+        crossCacheTxFailover(REPLICATED, true, OPTIMISTIC, REPEATABLE_READ);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testCrossCachePessimisticTxFailoverDifferentAffinityReplicated() throws Exception {
-        crossCacheTxFailover(PARTITIONED, false, PESSIMISTIC, REPEATABLE_READ, TestMemoryMode.HEAP);
+        crossCacheTxFailover(PARTITIONED, false, PESSIMISTIC, REPEATABLE_READ);
     }
 
     /**
@@ -219,21 +199,19 @@ public class IgniteCacheCrossCacheTxFailoverTest extends GridCommonAbstractTest
      * @param sameAff If {@code false} uses different number of partitions for caches.
      * @param concurrency Transaction concurrency.
      * @param isolation Transaction isolation.
-     * @param memMode Memory mode.
      * @throws Exception If failed.
      */
     private void crossCacheTxFailover(CacheMode cacheMode,
         boolean sameAff,
         final TransactionConcurrency concurrency,
-        final TransactionIsolation isolation,
-        TestMemoryMode memMode) throws Exception {
+        final TransactionIsolation isolation) throws Exception {
         IgniteKernal ignite0 = (IgniteKernal)ignite(0);
 
         final AtomicBoolean stop = new AtomicBoolean();
 
         try {
-            ignite0.createCache(cacheConfiguration(CACHE1, cacheMode, 256, memMode));
-            ignite0.createCache(cacheConfiguration(CACHE2, cacheMode, sameAff ? 256 : 128, memMode));
+            ignite0.createCache(cacheConfiguration(CACHE1, cacheMode, 256));
+            ignite0.createCache(cacheConfiguration(CACHE2, cacheMode, sameAff ? 256 : 128));
 
             final AtomicInteger threadIdx = new AtomicInteger();