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:04 UTC

[14/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/CacheSerializableTransactionsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSerializableTransactionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSerializableTransactionsTest.java
index 251e171..bfd7806 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSerializableTransactionsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSerializableTransactionsTest.java
@@ -51,6 +51,7 @@ import org.apache.ignite.IgniteTransactions;
 import org.apache.ignite.cache.CacheEntry;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.cache.eviction.lru.LruEvictionPolicy;
 import org.apache.ignite.cache.store.CacheStore;
 import org.apache.ignite.cache.store.CacheStoreAdapter;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -82,7 +83,6 @@ import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
 import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
 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.transactions.TransactionConcurrency.OPTIMISTIC;
 import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
@@ -2578,8 +2578,7 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
         String readCacheName,
         String writeCacheName,
         final boolean entry,
-        final AtomicInteger putKey) throws Exception
-    {
+        final AtomicInteger putKey) throws Exception {
         final int THREADS = 64;
 
         final IgniteCache<Integer, Integer> readCache = ignite.cache(readCacheName);
@@ -3065,7 +3064,7 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
             }
 
             try (Transaction tx = client2.transactions().txStart(OPTIMISTIC, SERIALIZABLE)) {
-                assertEquals(1, (Object) cache2.get(key));
+                assertEquals(1, (Object)cache2.get(key));
                 cache2.put(key, 2);
 
                 tx.commit();
@@ -3266,7 +3265,7 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
 
             ignite0.createCache(ccfg1);
 
-            CacheConfiguration<Integer, Integer> ccfg2=
+            CacheConfiguration<Integer, Integer> ccfg2 =
                 cacheConfiguration(PARTITIONED, FULL_SYNC, 1, false, false);
 
             ccfg2.setName(CACHE2);
@@ -4008,42 +4007,35 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testAccountTx1() throws Exception {
-        accountTx(false, false, false, false, TestMemoryMode.HEAP);
+        accountTx(false, false, false, false);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testAccountTx2() throws Exception {
-        accountTx(true, false, false, false, TestMemoryMode.HEAP);
+        accountTx(true, false, false, false);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testAccountTxWithNonSerializable() throws Exception {
-        accountTx(false, false, true, false, TestMemoryMode.HEAP);
+        accountTx(false, false, true, false);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testAccountTxNearCache() throws Exception {
-        accountTx(false, true, false, false, TestMemoryMode.HEAP);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAccountTxOffheapTiered() throws Exception {
-        accountTx(false, false, false, false, TestMemoryMode.OFFHEAP_TIERED);
+        accountTx(false, true, false, false);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testAccountTxNodeRestart() throws Exception {
-        accountTx(false, false, false, true, TestMemoryMode.HEAP);
+        accountTx(false, false, false, true);
     }
 
     /**
@@ -4051,20 +4043,16 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
      * @param nearCache If {@code true} near cache is enabled.
      * @param nonSer If {@code true} starts threads executing non-serializable transactions.
      * @param restart If {@code true} restarts one node.
-     * @param memMode Test memory mode.
      * @throws Exception If failed.
      */
     private void accountTx(final boolean getAll,
         final boolean nearCache,
         final boolean nonSer,
-        final boolean restart,
-        TestMemoryMode memMode) throws Exception {
+        final boolean restart) throws Exception {
         final Ignite srv = ignite(1);
 
         CacheConfiguration<Integer, Integer> ccfg = cacheConfiguration(PARTITIONED, FULL_SYNC, 1, false, false);
 
-        GridTestUtils.setMemoryMode(null, ccfg, memMode, 1, 64);
-
         final String cacheName = srv.createCache(ccfg).getName();
 
         try {
@@ -4346,12 +4334,18 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
             }
 
             {
-                // Offheap.
+                // Eviction.
                 CacheConfiguration<Integer, Integer> ccfg = cacheConfiguration(PARTITIONED, FULL_SYNC, 1, false, false);
                 ccfg.setName("cache3");
                 ccfg.setRebalanceMode(SYNC);
 
-                GridTestUtils.setMemoryMode(null, ccfg, TestMemoryMode.OFFHEAP_TIERED, 1, 64);
+                LruEvictionPolicy plc = new LruEvictionPolicy();
+
+                plc.setMaxSize(100);
+
+                ccfg.setEvictionPolicy(plc);
+
+                ccfg.setOnheapCacheEnabled(true);
 
                 srv.createCache(ccfg);
 
@@ -4542,7 +4536,7 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
             }
         }
         finally {
-                destroyCache(cacheName);
+            destroyCache(cacheName);
         }
     }
 
@@ -4881,15 +4875,6 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
         ccfgs.add(cacheConfiguration(PARTITIONED, FULL_SYNC, 1, true, true));
         ccfgs.add(cacheConfiguration(PARTITIONED, FULL_SYNC, 2, true, true));
 
-        // Swap and offheap enabled.
-        for (GridTestUtils.TestMemoryMode memMode : GridTestUtils.TestMemoryMode.values()) {
-            CacheConfiguration<Integer, Integer> ccfg = cacheConfiguration(PARTITIONED, FULL_SYNC, 1, false, false);
-
-            GridTestUtils.setMemoryMode(null, ccfg, memMode, 1, 64);
-
-            ccfgs.add(ccfg);
-        }
-
         return ccfgs;
     }
 
@@ -4903,7 +4888,6 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
             ", near=" + (ccfg.getNearConfiguration() != null) +
             ", store=" + ccfg.isWriteThrough() +
             ", evictPlc=" + (ccfg.getEvictionPolicy() != null) +
-            ", maxOffheap=" + ccfg.getOffHeapMaxMemory()  +
             ']');
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSwapUnswapGetTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSwapUnswapGetTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSwapUnswapGetTest.java
deleted file mode 100644
index 9a9b675..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSwapUnswapGetTest.java
+++ /dev/null
@@ -1,310 +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;
-
-import java.util.Collections;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.atomic.AtomicBoolean;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMemoryMode;
-import org.apache.ignite.cache.CachePeekMode;
-import org.apache.ignite.cache.eviction.lru.LruEvictionPolicy;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.internal.util.typedef.CAX;
-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.GridTestUtils;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.eclipse.jetty.util.ConcurrentHashSet;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
-import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
-
-/**
- *
- */
-public class CacheSwapUnswapGetTest extends GridCommonAbstractTest {
-    /** */
-    private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
-
-    /** */
-    private static final long DURATION = 30_000;
-
-    /** */
-    private static final long OFFHEAP_MEM = 1000;
-
-    /** */
-    private static final int MAX_HEAP_SIZE = 100;
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
-
-        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
-
-        return cfg;
-    }
-
-    /**
-     * @param atomicityMode Cache atomicity mode.
-     * @param memMode Cache memory mode.
-     * @param swap {@code True} if swap enabled.
-     * @return Cache configuration.
-     */
-    private CacheConfiguration<Integer, String> cacheConfiguration(CacheAtomicityMode atomicityMode,
-        CacheMemoryMode memMode,
-        boolean swap) {
-        CacheConfiguration<Integer, String> ccfg = new CacheConfiguration<>();
-
-        ccfg.setAtomicityMode(atomicityMode);
-        ccfg.setWriteSynchronizationMode(FULL_SYNC);
-        ccfg.setMemoryMode(memMode);
-
-        if (memMode == CacheMemoryMode.ONHEAP_TIERED) {
-            LruEvictionPolicy plc = new LruEvictionPolicy();
-            plc.setMaxSize(MAX_HEAP_SIZE);
-
-            ccfg.setEvictionPolicy(plc);
-        }
-
-        // TODO GG-11148.
-//        if (swap) {
-//            ccfg.setSwapEnabled(true);
-//
-//            ccfg.setOffHeapMaxMemory(OFFHEAP_MEM);
-//        }
-//        else
-//            ccfg.setOffHeapMaxMemory(0);
-
-        return ccfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        super.beforeTestsStarted();
-
-        startGrid(0);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        super.afterTestsStopped();
-
-        stopAllGrids();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected long getTestTimeout() {
-        return DURATION + 60_000;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testTxCacheOffheapEvict() throws Exception {
-        swapUnswap(TRANSACTIONAL, CacheMemoryMode.ONHEAP_TIERED, false);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testTxCacheOffheapTiered() throws Exception {
-        swapUnswap(TRANSACTIONAL, CacheMemoryMode.OFFHEAP_TIERED, false);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testTxCacheOffheapSwapEvict() throws Exception {
-        swapUnswap(TRANSACTIONAL, CacheMemoryMode.ONHEAP_TIERED, true);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testTxCacheOffheapTieredSwapEvict() throws Exception {
-        swapUnswap(TRANSACTIONAL, CacheMemoryMode.OFFHEAP_TIERED, true);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAtomicCacheOffheapEvict() throws Exception {
-        swapUnswap(ATOMIC, CacheMemoryMode.ONHEAP_TIERED, false);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAtomicCacheOffheapTiered() throws Exception {
-        swapUnswap(ATOMIC, CacheMemoryMode.OFFHEAP_TIERED, false);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAtomicCacheOffheapSwapEvict() throws Exception {
-        swapUnswap(ATOMIC, CacheMemoryMode.ONHEAP_TIERED, true);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAtomicCacheOffheapTieredSwapEvict() throws Exception {
-        swapUnswap(ATOMIC, CacheMemoryMode.OFFHEAP_TIERED, true);
-    }
-
-    /**
-     * @param atomicityMode Cache atomicity mode.
-     * @param memMode Cache memory mode.
-     * @param swap {@code True} if swap enabled.
-     * @throws Exception If failed.
-     */
-    private void swapUnswap(CacheAtomicityMode atomicityMode, CacheMemoryMode memMode, boolean swap) throws Exception {
-        log.info("Start test [mode=" + atomicityMode + ", swap=" + swap + ']');
-
-        int threadCnt = 20;
-        final int keyCnt = 1000;
-        final int valCnt = 10000;
-
-        final Ignite g = grid(0);
-
-        final ConcurrentHashSet<Integer> keys = new ConcurrentHashSet<>();
-
-        final AtomicBoolean done = new AtomicBoolean();
-
-        g.destroyCache(null);
-
-        final IgniteCache<Integer, String> cache = g.createCache(cacheConfiguration(atomicityMode, memMode, swap));
-
-        try {
-            IgniteInternalFuture<?> fut = multithreadedAsync(new CAX() {
-                @Override public void applyx() throws IgniteCheckedException {
-                    ThreadLocalRandom rnd = ThreadLocalRandom.current();
-
-                    while (!done.get()) {
-                        Integer key = rnd.nextInt(keyCnt);
-
-                        switch (rnd.nextInt(3)) {
-                            case 0:
-                                cache.put(key, String.valueOf(rnd.nextInt(valCnt)));
-
-                                keys.add(key);
-
-                                break;
-
-                            case 1:
-                                cache.localEvict(Collections.singletonList(key));
-
-                                break;
-
-                            case 2:
-                                if (keys.contains(key)) {
-                                    String val = cache.get(key);
-
-                                    assertNotNull(val);
-                                }
-
-                                break;
-
-                            default:
-                                assert false;
-                        }
-                    }
-                }
-            }, threadCnt, "update-thread");
-
-            IgniteInternalFuture<?> getFut = GridTestUtils.runAsync(new Callable<Void>() {
-                @Override public Void call() throws Exception {
-                    Thread.currentThread().setName("get-thread");
-
-                    while (!done.get()) {
-                        for (Integer key : keys) {
-                            String val = cache.get(key);
-
-                            assertNotNull(val);
-                        }
-                    }
-
-                    return null;
-                }
-            });
-
-            long endTime = System.currentTimeMillis() + DURATION;
-
-            while (System.currentTimeMillis() < endTime) {
-                Thread.sleep(5000);
-
-                log.info("Cache size [heap=" + cache.localSize(CachePeekMode.ONHEAP) +
-                    ", offheap=" + cache.localSize(CachePeekMode.OFFHEAP) +
-                    ", total=" + cache.localSize() +
-                    ", offheapMem=" + cache.localMetrics().getOffHeapAllocatedSize() + ']');
-            }
-
-            done.set(true);
-
-            fut.get();
-            getFut.get();
-
-            for (Integer key : keys) {
-                String val = cache.get(key);
-
-                assertNotNull(val);
-            }
-
-            int onheapSize = cache.localSize(CachePeekMode.ONHEAP);
-            int offheapSize = cache.localSize(CachePeekMode.OFFHEAP);
-            int swapSize = cache.localSize(CachePeekMode.SWAP);
-            int total = cache.localSize();
-            long offheapMem = cache.localMetrics().getOffHeapAllocatedSize();
-
-            log.info("Cache size [heap=" + onheapSize +
-                ", offheap=" + offheapSize +
-                ", swap=" + swapSize +
-                ", total=" + total +
-                ", offheapMem=" + offheapMem +  ']');
-
-            assertTrue(total > 0);
-
-            assertEquals(onheapSize + offheapSize + swapSize, total);
-
-            if (memMode == CacheMemoryMode.OFFHEAP_TIERED)
-                assertEquals(0, onheapSize);
-            else
-                assertEquals(MAX_HEAP_SIZE, onheapSize);
-
-            if (swap) {
-                assertTrue(swapSize > 0);
-                assertTrue(offheapMem <= OFFHEAP_MEM);
-            }
-            else
-                assertEquals(0, swapSize);
-        }
-        finally {
-            done.set(true);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java
index 0812fcc..52c98c2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java
@@ -27,7 +27,6 @@ import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheInterceptor;
 import org.apache.ignite.cache.CacheInterceptorAdapter;
-import org.apache.ignite.cache.CacheMemoryMode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteEx;
@@ -102,11 +101,6 @@ public abstract class GridAbstractCacheInterceptorRebalanceTest extends GridComm
      */
     protected abstract CacheAtomicityMode atomicityMode();
 
-    /**
-     * @return Cache memory mode;
-     */
-    protected abstract CacheMemoryMode memoryMode();
-
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         stopAllGrids();

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractByteArrayValuesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractByteArrayValuesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractByteArrayValuesSelfTest.java
index abbf98d..fb8cc22 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractByteArrayValuesSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractByteArrayValuesSelfTest.java
@@ -30,12 +30,6 @@ public abstract class GridCacheAbstractByteArrayValuesSelfTest extends GridCommo
     /** Regular cache name. */
     protected static final String CACHE_REGULAR = "cache";
 
-    /** Offheap cache name. */
-    protected static final String CACHE_OFFHEAP = "cache_offheap";
-
-    /** Offheap tiered cache name. */
-    protected static final String CACHE_OFFHEAP_TIERED = "cache_offheap_tiered";
-
     /** Key 1. */
     protected static final Integer KEY_1 = 1;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
index de0b532..b70ca6a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
@@ -59,7 +59,6 @@ import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteTransactions;
 import org.apache.ignite.cache.CacheEntry;
 import org.apache.ignite.cache.CacheEntryProcessor;
-import org.apache.ignite.cache.CacheMemoryMode;
 import org.apache.ignite.cache.CachePeekMode;
 import org.apache.ignite.cache.affinity.Affinity;
 import org.apache.ignite.cache.query.QueryCursor;
@@ -108,14 +107,10 @@ import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.CLOCK;
 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.OFFHEAP_VALUES;
-import static org.apache.ignite.cache.CacheMemoryMode.ONHEAP_TIERED;
 import static org.apache.ignite.cache.CacheMode.LOCAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
 import static org.apache.ignite.cache.CachePeekMode.ALL;
-import static org.apache.ignite.cache.CachePeekMode.BACKUP;
 import static org.apache.ignite.cache.CachePeekMode.OFFHEAP;
 import static org.apache.ignite.cache.CachePeekMode.ONHEAP;
 import static org.apache.ignite.cache.CachePeekMode.PRIMARY;
@@ -203,13 +198,6 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
         return true;
     }
 
-    /**
-     * @return {@code True} if values should be stored off-heap.
-     */
-    protected CacheMemoryMode memoryMode() {
-        return ONHEAP_TIERED;
-    }
-
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
@@ -237,18 +225,6 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
     }
 
     /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration(String igniteInstanceName) throws Exception {
-        CacheConfiguration ccfg = super.cacheConfiguration(igniteInstanceName);
-
-        if (memoryMode() == OFFHEAP_TIERED || memoryMode() == OFFHEAP_VALUES) {
-            ccfg.setMemoryMode(memoryMode());
-            ccfg.setOffHeapMaxMemory(0);
-        }
-
-        return ccfg;
-    }
-
-    /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
         initStoreStrategy();
 
@@ -3133,8 +3109,8 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      *
      * @throws Exception If failed.
      */
-    public void _testDeletedEntriesFlag() throws Exception {
-        if (cacheMode() != LOCAL && cacheMode() != REPLICATED && memoryMode() != OFFHEAP_TIERED) {
+    public void testDeletedEntriesFlag() throws Exception {
+        if (cacheMode() != LOCAL && cacheMode() != REPLICATED) {
             final int cnt = 3;
 
             IgniteCache<String, Integer> cache = jcache();
@@ -4060,7 +4036,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      * TODO GG-11133.
      * @throws Exception In case of error.
      */
-    public void _testEvictExpired() throws Exception {
+    public void testEvictExpired() throws Exception {
         final IgniteCache<String, Integer> cache = jcache();
 
         final String key = primaryKeysForCache(cache, 1).get(0);
@@ -4115,7 +4091,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      *
      * @throws Exception If failed.
      */
-    public void _testPeekExpired() throws Exception {
+    public void testPeekExpired() throws Exception {
         final IgniteCache<String, Integer> c = jcache();
 
         final String key = primaryKeysForCache(c, 1).get(0);
@@ -4150,7 +4126,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      *
      * @throws Exception If failed.
      */
-    public void _testPeekExpiredTx() throws Exception {
+    public void testPeekExpiredTx() throws Exception {
         if (txShouldBeUsed()) {
             final IgniteCache<String, Integer> c = jcache();
 
@@ -4209,9 +4185,6 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
         if (true)
             return;
 
-        if (memoryMode() == OFFHEAP_TIERED)
-            return;
-
         int ttl = 1000;
 
         final ExpiryPolicy expiry = new TouchedExpiryPolicy(new Duration(MILLISECONDS, ttl));
@@ -4493,144 +4466,6 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
     }
 
     /**
-     * @throws Exception If failed.
-     *
-     * TODO: GG-11148 check if test makes sense.
-     */
-    public void _testUnswap() throws Exception {
-        IgniteCache<String, Integer> cache = grid(0).cache(null);
-
-        List<String> keys = primaryKeysForCache(jcache(), 3);
-
-        String k1 = keys.get(0);
-        String k2 = keys.get(1);
-        String k3 = keys.get(2);
-
-        cache.getAndPut(k1, 1);
-        cache.getAndPut(k2, 2);
-        cache.getAndPut(k3, 3);
-
-        final AtomicInteger swapEvts = new AtomicInteger(0);
-        final AtomicInteger unswapEvts = new AtomicInteger(0);
-
-        Collection<String> locKeys = new HashSet<>();
-
-        if (grid(0).context().cache().cache(null).context().affinityNode()) {
-            Iterable<Cache.Entry<String, Integer>> entries = cache.localEntries(PRIMARY, BACKUP);
-
-            for (Cache.Entry<String, Integer> entry : entries)
-                locKeys.add(entry.getKey());
-
-            info("Local keys (primary + backup): " + locKeys);
-        }
-
-        for (int i = 0; i < gridCount(); i++)
-            grid(i).events().localListen(
-                new SwapEvtsLocalListener(swapEvts, unswapEvts), EVT_CACHE_OBJECT_SWAPPED, EVT_CACHE_OBJECT_UNSWAPPED);
-
-        cache.localEvict(F.asList(k2, k3));
-
-        if (memoryMode() == OFFHEAP_TIERED) {
-            assertNotNull(cache.localPeek(k1, ONHEAP, OFFHEAP));
-            assertNotNull(cache.localPeek(k2, ONHEAP, OFFHEAP));
-            assertNotNull(cache.localPeek(k3, ONHEAP, OFFHEAP));
-        }
-        else {
-            assertNotNull(cache.localPeek(k1, ONHEAP, OFFHEAP));
-            assertNull(cache.localPeek(k2, ONHEAP, OFFHEAP));
-            assertNull(cache.localPeek(k3, ONHEAP, OFFHEAP));
-        }
-
-        int cnt = 0;
-
-        if (locKeys.contains(k2)) {
-            assertNull(cache.localPeek(k2, ONHEAP_PEEK_MODES));
-
-            cache.localPromote(Collections.singleton(k2));
-
-            assertEquals((Integer)2, cache.localPeek(k2, ONHEAP_PEEK_MODES));
-
-            cnt++;
-        }
-        else {
-            cache.localPromote(Collections.singleton(k2));
-
-            assertNull(cache.localPeek(k2, ONHEAP_PEEK_MODES));
-        }
-
-        if (locKeys.contains(k3)) {
-            assertNull(cache.localPeek(k3, ONHEAP_PEEK_MODES));
-
-            cache.localPromote(Collections.singleton(k3));
-
-            assertEquals((Integer)3, cache.localPeek(k3, ONHEAP_PEEK_MODES));
-
-            cnt++;
-        }
-        else {
-            cache.localPromote(Collections.singleton(k3));
-
-            assertNull(cache.localPeek(k3, ONHEAP_PEEK_MODES));
-        }
-
-        if (memoryMode() != OFFHEAP_TIERED) {
-            assertEquals(cnt, swapEvts.get());
-            assertEquals(cnt, unswapEvts.get());
-        }
-
-        cache.localEvict(Collections.singleton(k1));
-
-        assertEquals((Integer)1, cache.get(k1));
-
-        if (locKeys.contains(k1))
-            cnt++;
-
-        if (memoryMode() != OFFHEAP_TIERED) {
-            assertEquals(cnt, swapEvts.get());
-            assertEquals(cnt, unswapEvts.get());
-        }
-
-        cache.clear();
-
-        // Check with multiple arguments.
-        cache.getAndPut(k1, 1);
-        cache.getAndPut(k2, 2);
-        cache.getAndPut(k3, 3);
-
-        swapEvts.set(0);
-        unswapEvts.set(0);
-
-        cache.localEvict(Collections.singleton(k2));
-        cache.localEvict(Collections.singleton(k3));
-
-        if (memoryMode() == OFFHEAP_TIERED) {
-            assertNotNull(cache.localPeek(k1, ONHEAP, OFFHEAP));
-            assertNotNull(cache.localPeek(k2, ONHEAP, OFFHEAP));
-            assertNotNull(cache.localPeek(k3, ONHEAP, OFFHEAP));
-        }
-        else {
-            assertNotNull(cache.localPeek(k1, ONHEAP, OFFHEAP));
-            assertNull(cache.localPeek(k2, ONHEAP, OFFHEAP));
-            assertNull(cache.localPeek(k3, ONHEAP, OFFHEAP));
-        }
-
-        cache.localPromote(F.asSet(k2, k3));
-
-        cnt = 0;
-
-        if (locKeys.contains(k2))
-            cnt++;
-
-        if (locKeys.contains(k3))
-            cnt++;
-
-        if (memoryMode() != OFFHEAP_TIERED) {
-            assertEquals(cnt, swapEvts.get());
-            assertEquals(cnt, unswapEvts.get());
-        }
-    }
-
-    /**
      * JUnit.
      */
     public void testCacheProxy() {
@@ -4644,7 +4479,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      *
      * @throws Exception If failed.
      */
-    public void _testCompactExpired() throws Exception {
+    public void testCompactExpired() throws Exception {
         final IgniteCache<String, Integer> cache = jcache();
 
         final String key = F.first(primaryKeysForCache(cache, 1));
@@ -4835,9 +4670,6 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
     public void testToMap() throws Exception {
         IgniteCache<String, Integer> cache = jcache();
 
-        if (offheapTiered(cache))
-            return;
-
         cache.put("key1", 1);
         cache.put("key2", 2);
 
@@ -4858,9 +4690,6 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      * @throws Exception If failed.
      */
     protected void checkSize(final Collection<String> keys) throws Exception {
-        if (memoryMode() == OFFHEAP_TIERED)
-            return;
-
         if (nearEnabled())
             assertEquals(keys.size(), jcache().localSize(CachePeekMode.ALL));
         else {

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractLocalStoreSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractLocalStoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractLocalStoreSelfTest.java
index a294196..64c60fe 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractLocalStoreSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractLocalStoreSelfTest.java
@@ -37,7 +37,6 @@ import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.Ignition;
 import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMemoryMode;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CachePeekMode;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
@@ -64,7 +63,6 @@ import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_LOCAL_STORE_KEEPS_PRIMARY_ONLY;
 import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.PRIMARY;
-import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_TIERED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
 import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
@@ -194,10 +192,6 @@ public abstract class GridCacheAbstractLocalStoreSelfTest extends GridCommonAbst
         cacheCfg.setWriteThrough(true);
         cacheCfg.setReadThrough(true);
         cacheCfg.setBackups(backups);
-        cacheCfg.setOffHeapMaxMemory(0);
-
-        if (isOffHeapTieredMode())
-            cacheCfg.setMemoryMode(OFFHEAP_TIERED);
 
         return cacheCfg;
     }
@@ -219,13 +213,6 @@ public abstract class GridCacheAbstractLocalStoreSelfTest extends GridCommonAbst
      */
     protected abstract CacheMode getCacheMode();
 
-    /**
-     * @return {@code True} if {@link CacheMemoryMode#OFFHEAP_TIERED} memory mode should be used.
-     */
-    protected boolean isOffHeapTieredMode() {
-        return false;
-    }
-
     /** {@inheritDoc} */
     @Override protected void afterTestsStopped() throws Exception {
         stopAllGrids();

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java
index bda0606..8606f7c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java
@@ -164,7 +164,7 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCommonAbstr
      * @throws Exception If failed.
      */
     public void testPutAndRemove() throws Exception {
-        putAndRemove(DUR, null, null, GridTestUtils.TestMemoryMode.HEAP);
+        putAndRemove(DUR, null, null);
     }
 
     /**
@@ -174,7 +174,7 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCommonAbstr
         if (atomicityMode() != CacheAtomicityMode.TRANSACTIONAL)
             return;
 
-        putAndRemove(30_000, PESSIMISTIC, REPEATABLE_READ, GridTestUtils.TestMemoryMode.HEAP);
+        putAndRemove(30_000, PESSIMISTIC, REPEATABLE_READ);
     }
 
     /**
@@ -184,34 +184,18 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCommonAbstr
         if (atomicityMode() != CacheAtomicityMode.TRANSACTIONAL)
             return;
 
-        putAndRemove(30_000, OPTIMISTIC, SERIALIZABLE, GridTestUtils.TestMemoryMode.HEAP);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPutAndRemoveOffheapEvict() throws Exception {
-        putAndRemove(30_000, null, null, GridTestUtils.TestMemoryMode.OFFHEAP_EVICT);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPutAndRemoveOffheapEvictSwap() throws Exception {
-        putAndRemove(30_000, null, null, GridTestUtils.TestMemoryMode.OFFHEAP_EVICT_SWAP);
+        putAndRemove(30_000, OPTIMISTIC, SERIALIZABLE);
     }
 
     /**
      * @param duration Test duration.
      * @param txConcurrency Transaction concurrency if test explicit transaction.
      * @param txIsolation Transaction isolation if test explicit transaction.
-     * @param memMode Memory mode.
      * @throws Exception If failed.
      */
     private void putAndRemove(long duration,
         final TransactionConcurrency txConcurrency,
-        final TransactionIsolation txIsolation,
-        GridTestUtils.TestMemoryMode memMode) throws Exception {
+        final TransactionIsolation txIsolation) throws Exception {
         assertEquals(testClientNode(), (boolean) grid(0).configuration().isClientMode());
 
         grid(0).destroyCache(null);
@@ -229,8 +213,6 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCommonAbstr
         ccfg.setAtomicWriteOrderMode(atomicWriteOrderMode());
         ccfg.setNearConfiguration(nearCache());
 
-        GridTestUtils.setMemoryMode(null, ccfg, memMode, 100, 1024);
-
         final IgniteCache<Integer, Integer> sndCache0 = grid(0).createCache(ccfg);
 
         final AtomicBoolean stop = new AtomicBoolean();

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
index c758ec1..95f8bb8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
@@ -58,7 +58,6 @@ import org.apache.ignite.transactions.Transaction;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
-import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_TIERED;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 
@@ -143,7 +142,7 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest {
 
                     assertTrue(
                         "Cache is not empty: " + " localSize = " + jcache(fi).localSize(CachePeekMode.ALL)
-                        + ", local entries " + entrySet(jcache(fi).localEntries()),
+                            + ", local entries " + entrySet(jcache(fi).localEntries()),
                         GridTestUtils.waitForCondition(
                             // Preloading may happen as nodes leave, so we need to wait.
                             new GridAbsPredicateX() {
@@ -251,6 +250,7 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest {
         cfg.setAtomicityMode(atomicityMode());
         cfg.setWriteSynchronizationMode(writeSynchronization());
         cfg.setNearConfiguration(nearConfiguration());
+        cfg.setOnheapCacheEnabled(onheapCacheEnabled());
 
         Class<?>[] idxTypes = indexedTypes();
 
@@ -335,6 +335,13 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @return {@code True} if on-heap cache is enabled.
+     */
+    protected boolean onheapCacheEnabled() {
+        return false;
+    }
+
+    /**
      * @return {@code True} for partitioned caches.
      */
     protected final boolean partitionedMode() {
@@ -387,14 +394,6 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     * @param cache Cache.
-     * @return {@code True} if cache has OFFHEAP_TIERED memory mode.
-     */
-    protected <K, V> boolean offheapTiered(IgniteCache<K, V> cache) {
-        return cache.getConfiguration(CacheConfiguration.class).getMemoryMode() == OFFHEAP_TIERED;
-    }
-
-    /**
      * Executes regular peek or peek from swap.
      *
      * @param cache Cache projection.
@@ -413,7 +412,7 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest {
      */
     @SuppressWarnings("unchecked")
     protected boolean containsKey(IgniteCache cache, Object key) throws Exception {
-        return offheapTiered(cache) ? cache.localPeek(key, CachePeekMode.OFFHEAP) != null : cache.containsKey(key);
+        return cache.containsKey(key);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheClearSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheClearSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheClearSelfTest.java
index 796268d..707e275 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheClearSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheClearSelfTest.java
@@ -23,7 +23,6 @@ import java.util.UUID;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.Ignition;
-import org.apache.ignite.cache.CacheMemoryMode;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CachePeekMode;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -80,177 +79,92 @@ public class GridCacheClearSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testClearPartitioned() throws Exception {
-        testClear(CacheMode.PARTITIONED, CacheMemoryMode.ONHEAP_TIERED, false, null);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClearPartitionedOffHeap() throws Exception {
-        testClear(CacheMode.PARTITIONED, CacheMemoryMode.OFFHEAP_TIERED, false, null);
+        testClear(CacheMode.PARTITIONED, false, null);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testClearPartitionedNear() throws Exception {
-        testClear(CacheMode.PARTITIONED, CacheMemoryMode.ONHEAP_TIERED, true, null);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClearPartitionedOffHeapNear() throws Exception {
-        testClear(CacheMode.PARTITIONED, CacheMemoryMode.OFFHEAP_TIERED, true, null);
+        testClear(CacheMode.PARTITIONED, true, null);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testClearReplicated() throws Exception {
-        testClear(CacheMode.REPLICATED, CacheMemoryMode.ONHEAP_TIERED, false, null);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClearReplicatedOffHeap() throws Exception {
-        testClear(CacheMode.REPLICATED, CacheMemoryMode.OFFHEAP_TIERED, false, null);
+        testClear(CacheMode.REPLICATED, false, null);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testClearReplicatedNear() throws Exception {
-        testClear(CacheMode.REPLICATED, CacheMemoryMode.ONHEAP_TIERED, true, null);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClearReplicatedOffHeapNear() throws Exception {
-        testClear(CacheMode.REPLICATED, CacheMemoryMode.OFFHEAP_TIERED, true, null);
+        testClear(CacheMode.REPLICATED, true, null);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testClearKeyPartitioned() throws Exception {
-        testClear(CacheMode.PARTITIONED, CacheMemoryMode.ONHEAP_TIERED, false, Collections.singleton(3));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClearKeyPartitionedOffHeap() throws Exception {
-        testClear(CacheMode.PARTITIONED, CacheMemoryMode.OFFHEAP_TIERED, false, Collections.singleton(3));
+        testClear(CacheMode.PARTITIONED, false, Collections.singleton(3));
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testClearKeyPartitionedNear() throws Exception {
-        testClear(CacheMode.PARTITIONED, CacheMemoryMode.ONHEAP_TIERED, true, Collections.singleton(3));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClearKeyPartitionedOffHeapNear() throws Exception {
-        testClear(CacheMode.PARTITIONED, CacheMemoryMode.OFFHEAP_TIERED, true, Collections.singleton(3));
+        testClear(CacheMode.PARTITIONED, true, Collections.singleton(3));
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testClearKeyReplicated() throws Exception {
-        testClear(CacheMode.REPLICATED, CacheMemoryMode.ONHEAP_TIERED, false, Collections.singleton(3));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClearKeyReplicatedOffHeap() throws Exception {
-        testClear(CacheMode.REPLICATED, CacheMemoryMode.OFFHEAP_TIERED, false, Collections.singleton(3));
+        testClear(CacheMode.REPLICATED, false, Collections.singleton(3));
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testClearKeyReplicatedNear() throws Exception {
-        testClear(CacheMode.REPLICATED, CacheMemoryMode.ONHEAP_TIERED, true, Collections.singleton(3));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClearKeyReplicatedOffHeapNear() throws Exception {
-        testClear(CacheMode.REPLICATED, CacheMemoryMode.OFFHEAP_TIERED, true, Collections.singleton(3));
+        testClear(CacheMode.REPLICATED, true, Collections.singleton(3));
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testClearKeysPartitioned() throws Exception {
-        testClear(CacheMode.PARTITIONED, CacheMemoryMode.ONHEAP_TIERED, false, F.asSet(2, 6, 9));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClearKeysPartitionedOffHeap() throws Exception {
-        testClear(CacheMode.PARTITIONED, CacheMemoryMode.OFFHEAP_TIERED, false, F.asSet(2, 6, 9));
+        testClear(CacheMode.PARTITIONED, false, F.asSet(2, 6, 9));
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testClearKeysPartitionedNear() throws Exception {
-        testClear(CacheMode.PARTITIONED, CacheMemoryMode.ONHEAP_TIERED, true, F.asSet(2, 6, 9));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClearKeysPartitionedOffHeapNear() throws Exception {
-        testClear(CacheMode.PARTITIONED, CacheMemoryMode.OFFHEAP_TIERED, true, F.asSet(2, 6, 9));
+        testClear(CacheMode.PARTITIONED, true, F.asSet(2, 6, 9));
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testClearKeysReplicated() throws Exception {
-        testClear(CacheMode.REPLICATED, CacheMemoryMode.ONHEAP_TIERED, false, F.asSet(2, 6, 9));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClearKeysReplicatedOffHeap() throws Exception {
-        testClear(CacheMode.REPLICATED, CacheMemoryMode.OFFHEAP_TIERED, false, F.asSet(2, 6, 9));
+        testClear(CacheMode.REPLICATED, false, F.asSet(2, 6, 9));
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testClearKeysReplicatedNear() throws Exception {
-        testClear(CacheMode.REPLICATED, CacheMemoryMode.ONHEAP_TIERED, true, F.asSet(2, 6, 9));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClearKeysReplicatedOffHeapNear() throws Exception {
-        testClear(CacheMode.REPLICATED, CacheMemoryMode.OFFHEAP_TIERED, true, F.asSet(2, 6, 9));
+        testClear(CacheMode.REPLICATED, true, F.asSet(2, 6, 9));
     }
 
     /**
      * @param cacheMode Cache mode.
-     * @param memMode Memory mode.
      * @param near Near cache flag.
      * @param keys Keys to clear.
      */
-    private void testClear(CacheMode cacheMode, CacheMemoryMode memMode, boolean near, @Nullable Set<Integer> keys) {
+    private void testClear(CacheMode cacheMode, boolean near, @Nullable Set<Integer> keys) {
         Ignite client1 = client1();
         Ignite client2 = client2();
 
@@ -261,7 +175,6 @@ public class GridCacheClearSelfTest extends GridCommonAbstractTest {
             CacheConfiguration<Integer, Integer> cfg = new CacheConfiguration<>(cacheName);
 
             cfg.setCacheMode(cacheMode);
-            cfg.setMemoryMode(memMode);
 
             IgniteCache<Integer, Integer> cache1 = near ?
                 client1.createCache(cfg, new NearCacheConfiguration<Integer, Integer>()) :

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java
index 491abff..4776456 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java
@@ -144,7 +144,7 @@ public class GridCacheConcurrentTxMultiNodeTest extends GridCommonAbstractTest {
             plc.setMaxSize(1000);
 
             cc.setEvictionPolicy(plc);
-            cc.setEvictSynchronized(false);
+            cc.setOnheapCacheEnabled(true);
             cc.setWriteSynchronizationMode(FULL_SYNC);
             cc.setRebalanceMode(NONE);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java
index 74c5348..59a5ebb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java
@@ -362,6 +362,7 @@ public class GridCacheConfigurationConsistencySelfTest extends GridCommonAbstrac
                 /** {@inheritDoc} */
                 @Override public Void apply(CacheConfiguration cfg) {
                     cfg.setEvictionPolicy(new FifoEvictionPolicy());
+                    cfg.setOnheapCacheEnabled(true);
                     return null;
                 }
             },
@@ -383,6 +384,7 @@ public class GridCacheConfigurationConsistencySelfTest extends GridCommonAbstrac
                 /** {@inheritDoc} */
                 @Override public Void apply(CacheConfiguration cfg) {
                     cfg.setEvictionPolicy(new SortedEvictionPolicy());
+                    cfg.setOnheapCacheEnabled(true);
                     return null;
                 }
             },
@@ -390,6 +392,7 @@ public class GridCacheConfigurationConsistencySelfTest extends GridCommonAbstrac
                 /** {@inheritDoc} */
                 @Override public Void apply(CacheConfiguration cfg) {
                     cfg.setEvictionPolicy(new FifoEvictionPolicy());
+                    cfg.setOnheapCacheEnabled(true);
                     return null;
                 }
             }
@@ -443,42 +446,6 @@ public class GridCacheConfigurationConsistencySelfTest extends GridCommonAbstrac
     /**
      * @throws Exception If failed.
      */
-    public void testDifferentEvictSynchronized() throws Exception {
-        cacheMode = PARTITIONED;
-
-        checkSecondGridStartFails(
-            new C1<CacheConfiguration, Void>() {
-                /** {@inheritDoc} */
-                @Override public Void apply(CacheConfiguration cfg) {
-                    cfg.setEvictSynchronized(true);
-
-                    FifoEvictionPolicy plc = new FifoEvictionPolicy();
-
-                    plc.setMaxSize(100);
-
-                    cfg.setEvictionPolicy(plc);
-                    return null;
-                }
-            },
-            new C1<CacheConfiguration, Void>() {
-                /** {@inheritDoc} */
-                @Override public Void apply(CacheConfiguration cfg) {
-                    cfg.setEvictSynchronized(false);
-
-                    FifoEvictionPolicy plc = new FifoEvictionPolicy();
-
-                    plc.setMaxSize(100);
-
-                    cfg.setEvictionPolicy(plc);
-                    return null;
-                }
-            }
-        );
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
     public void testDifferentAtomicity() throws Exception {
         cacheMode = PARTITIONED;
 
@@ -672,6 +639,7 @@ public class GridCacheConfigurationConsistencySelfTest extends GridCommonAbstrac
                 cfg.setAffinity(new TestRendezvousAffinityFunction());
 
                 cfg.setEvictionPolicy(new FifoEvictionPolicy());
+                cfg.setOnheapCacheEnabled(true);
 
                 cfg.setCacheStoreFactory(new IgniteCacheAbstractTest.TestStoreFactory());
                 cfg.setReadThrough(true);
@@ -690,6 +658,7 @@ public class GridCacheConfigurationConsistencySelfTest extends GridCommonAbstrac
                 cfg.setAffinity(new RendezvousAffinityFunction());
 
                 cfg.setEvictionPolicy(new LruEvictionPolicy());
+                cfg.setOnheapCacheEnabled(true);
 
                 cfg.setCacheStoreFactory(null);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentOffHeapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentOffHeapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentOffHeapSelfTest.java
deleted file mode 100644
index ed58c3e..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentOffHeapSelfTest.java
+++ /dev/null
@@ -1,41 +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;
-
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.NearCacheConfiguration;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
-import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_TIERED;
-
-/**
- * Tests deployment with off-heap storage.
- */
-public class GridCacheDeploymentOffHeapSelfTest extends GridCacheDeploymentSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration() throws Exception {
-        CacheConfiguration cacheCfg = super.cacheConfiguration();
-
-        cacheCfg.setMemoryMode(OFFHEAP_TIERED);
-        cacheCfg.setOffHeapMaxMemory(0);
-        cacheCfg.setAtomicityMode(TRANSACTIONAL);
-        cacheCfg.setNearConfiguration(new NearCacheConfiguration());
-
-        return cacheCfg;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentOffHeapValuesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentOffHeapValuesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentOffHeapValuesSelfTest.java
deleted file mode 100644
index 73d5183..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentOffHeapValuesSelfTest.java
+++ /dev/null
@@ -1,41 +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;
-
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.NearCacheConfiguration;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
-import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_VALUES;
-
-/**
- * Tests deployment with off-heap storage.
- */
-public class GridCacheDeploymentOffHeapValuesSelfTest extends GridCacheDeploymentSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration() throws Exception {
-        CacheConfiguration cacheCfg = super.cacheConfiguration();
-
-        cacheCfg.setMemoryMode(OFFHEAP_VALUES);
-        cacheCfg.setOffHeapMaxMemory(0);
-        cacheCfg.setAtomicityMode(TRANSACTIONAL);
-        cacheCfg.setNearConfiguration(new NearCacheConfiguration());
-
-        return cacheCfg;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionEventAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionEventAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionEventAbstractTest.java
index 40de38c..eae8dad 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionEventAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionEventAbstractTest.java
@@ -25,6 +25,7 @@ import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicy;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.events.CacheEvent;
@@ -69,6 +70,8 @@ public abstract class GridCacheEvictionEventAbstractTest extends GridCommonAbstr
 
         cc.setCacheMode(cacheMode());
         cc.setAtomicityMode(atomicityMode());
+        cc.setEvictionPolicy(new FifoEvictionPolicy());
+        cc.setOnheapCacheEnabled(true);
 
         c.setCacheConfiguration(cc);
 
@@ -116,7 +119,5 @@ public abstract class GridCacheEvictionEventAbstractTest extends GridCommonAbstr
         c.localEvict(Collections.singleton("1"));
 
         assertTrue("Failed to wait for eviction event", latch.await(10, TimeUnit.SECONDS));
-
-        assertNotNull(oldVal.get());
     }
 }
\ 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/GridCacheInterceptorAtomicOffheapRebalanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorAtomicOffheapRebalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorAtomicOffheapRebalanceTest.java
deleted file mode 100644
index 103322f..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorAtomicOffheapRebalanceTest.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache;
-
-import org.apache.ignite.cache.CacheMemoryMode;
-
-/**
- *
- */
-public class GridCacheInterceptorAtomicOffheapRebalanceTest extends GridCacheInterceptorAtomicRebalanceTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMemoryMode memoryMode() {
-        return CacheMemoryMode.OFFHEAP_TIERED;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorAtomicRebalanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorAtomicRebalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorAtomicRebalanceTest.java
index aaeda4b..909810f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorAtomicRebalanceTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorAtomicRebalanceTest.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMemoryMode;
 
 /**
  *
@@ -28,9 +27,4 @@ public class GridCacheInterceptorAtomicRebalanceTest extends GridAbstractCacheIn
     @Override protected CacheAtomicityMode atomicityMode() {
         return CacheAtomicityMode.ATOMIC;
     }
-
-    /** {@inheritDoc} */
-    @Override protected CacheMemoryMode memoryMode() {
-        return CacheMemoryMode.ONHEAP_TIERED;
-    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorTransactionalOffheapRebalanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorTransactionalOffheapRebalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorTransactionalOffheapRebalanceTest.java
deleted file mode 100644
index bb90062..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorTransactionalOffheapRebalanceTest.java
+++ /dev/null
@@ -1,35 +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;
-
-import org.apache.ignite.cache.CacheMemoryMode;
-
-/**
- *
- */
-public class GridCacheInterceptorTransactionalOffheapRebalanceTest extends GridCacheInterceptorTransactionalRebalanceTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMemoryMode memoryMode() {
-        return CacheMemoryMode.OFFHEAP_TIERED;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected long getTestTimeout() {
-        return 10 * 60_000;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorTransactionalRebalanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorTransactionalRebalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorTransactionalRebalanceTest.java
index bace87c..87c3f32 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorTransactionalRebalanceTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorTransactionalRebalanceTest.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMemoryMode;
 
 /**
  *
@@ -28,9 +27,4 @@ public class GridCacheInterceptorTransactionalRebalanceTest extends GridAbstract
     @Override protected CacheAtomicityMode atomicityMode() {
         return CacheAtomicityMode.TRANSACTIONAL;
     }
-
-    /** {@inheritDoc} */
-    @Override protected CacheMemoryMode memoryMode() {
-        return CacheMemoryMode.ONHEAP_TIERED;
-    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheLifecycleAwareSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheLifecycleAwareSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheLifecycleAwareSelfTest.java
index 089e278..22d94fb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheLifecycleAwareSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheLifecycleAwareSelfTest.java
@@ -318,6 +318,7 @@ public class GridCacheLifecycleAwareSelfTest extends GridAbstractLifecycleAwareS
         TestEvictionPolicy evictionPlc = new TestEvictionPolicy();
 
         ccfg.setEvictionPolicy(evictionPlc);
+        ccfg.setOnheapCacheEnabled(true);
 
         lifecycleAwares.add(evictionPlc);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMemoryModeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMemoryModeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMemoryModeSelfTest.java
deleted file mode 100644
index 6768a0e..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMemoryModeSelfTest.java
+++ /dev/null
@@ -1,285 +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;
-
-import java.util.HashMap;
-import java.util.Map;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMemoryMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.cache.CachePeekMode;
-import org.apache.ignite.cache.eviction.lru.LruEvictionPolicy;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.util.typedef.CIX1;
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.junit.Assert;
-
-import static java.lang.String.valueOf;
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
-
-/**
- * Memory model self test.
- */
-@SuppressWarnings("deprecation")
-public class GridCacheMemoryModeSelfTest extends GridCommonAbstractTest {
-    /** */
-    private TcpDiscoveryIpFinder ipFinder;
-
-    /** */
-    private boolean swapEnabled;
-
-    /** */
-    private CacheMode mode;
-
-    /** */
-    private CacheMemoryMode memoryMode;
-
-    /** */
-    private int maxOnheapSize;
-
-    /** */
-    private long offheapSize;
-
-    /** */
-    private CacheAtomicityMode atomicity;
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
-
-        TcpDiscoverySpi disco = new TcpDiscoverySpi();
-
-        disco.setIpFinder(ipFinder);
-
-        cfg.setDiscoverySpi(disco);
-
-        cfg.setNetworkTimeout(2000);
-
-        cfg.setCacheConfiguration(cacheConfiguration());
-
-        return cfg;
-    }
-
-    /**
-     * Returns cache configuration.
-     *
-     * @return cache configuration.
-     */
-    protected CacheConfiguration cacheConfiguration() {
-        CacheConfiguration cacheCfg = defaultCacheConfiguration();
-
-        cacheCfg.setWriteSynchronizationMode(FULL_SYNC);
-
-        cacheCfg.setCacheMode(mode);
-        cacheCfg.setMemoryMode(memoryMode);
-
-        LruEvictionPolicy plc = null;
-
-        if (maxOnheapSize != Integer.MAX_VALUE) {
-            plc = new LruEvictionPolicy();
-            plc.setMaxSize(maxOnheapSize);
-        }
-
-        cacheCfg.setEvictionPolicy(plc);
-
-        cacheCfg.setAtomicityMode(atomicity);
-        cacheCfg.setOffHeapMaxMemory(offheapSize);
-
-        return cacheCfg;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testOnheap() throws Exception {
-        mode = CacheMode.LOCAL;
-        memoryMode = CacheMemoryMode.ONHEAP_TIERED;
-        maxOnheapSize = Integer.MAX_VALUE;
-        swapEnabled = false;
-        atomicity = CacheAtomicityMode.ATOMIC;
-        offheapSize = -1;
-
-        doTestPutAndPutAll(1000, 0, true, true);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testOnheapSwap() throws Exception {
-        mode = CacheMode.LOCAL;
-        memoryMode = CacheMemoryMode.ONHEAP_TIERED;
-        maxOnheapSize = 330;
-        swapEnabled = true;
-        atomicity = CacheAtomicityMode.ATOMIC;
-        offheapSize = -1;
-
-        doTestPutAndPutAll(330, 670, true, false);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testOffheap() throws Exception {
-        mode = CacheMode.LOCAL;
-        memoryMode = CacheMemoryMode.OFFHEAP_TIERED;
-        maxOnheapSize = Integer.MAX_VALUE;
-        swapEnabled = false;
-        atomicity = CacheAtomicityMode.ATOMIC;
-        offheapSize = -1; // Must be fixed in config validation.
-
-        doTestPutAndPutAll(0, 1000, false, true);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testOffheapSwap() throws Exception {
-        mode = CacheMode.LOCAL;
-        memoryMode = CacheMemoryMode.OFFHEAP_TIERED;
-        maxOnheapSize = Integer.MAX_VALUE;
-        swapEnabled = true;
-        atomicity = CacheAtomicityMode.ATOMIC;
-        offheapSize = 1000; // Small for evictions from offheap to swap.
-
-        doTestPutAndPutAll(0, 1000, false, false);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testTiered() throws Exception {
-        mode = CacheMode.LOCAL;
-        memoryMode = CacheMemoryMode.ONHEAP_TIERED;
-        maxOnheapSize = 24;
-        swapEnabled = true;
-        atomicity = CacheAtomicityMode.ATOMIC;
-        offheapSize = 1000; // Small for evictions from offheap to swap.
-
-        doTestPutAndPutAll(24, 976, false, false);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testOffheapValuesConfigFixBackward() throws Exception {
-        mode = CacheMode.LOCAL;
-        memoryMode = CacheMemoryMode.OFFHEAP_VALUES;
-        maxOnheapSize = 24;
-        swapEnabled = true;
-        atomicity = CacheAtomicityMode.ATOMIC;
-        offheapSize = -1;
-
-        Ignite g = startGrid();
-
-        CacheConfiguration cfg = g.cache(null).getConfiguration(CacheConfiguration.class);
-
-        assertEquals(memoryMode, cfg.getMemoryMode());
-        assertEquals(0, cfg.getOffHeapMaxMemory());
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        super.afterTest();
-
-        stopAllGrids();
-    }
-
-    /**
-     * @param cache In cache.
-     * @param offheapSwap In swap and offheap.
-     * @param offheapEmpty Offheap is empty.
-     * @param swapEmpty Swap is empty.
-     * @throws Exception If failed.
-     */
-    private void doTestPutAndPutAll(int cache, int offheapSwap, boolean offheapEmpty, boolean swapEmpty)
-        throws Exception {
-        final int all = cache + offheapSwap;
-
-        // put
-        doTest(cache, offheapSwap, offheapEmpty, swapEmpty, new CIX1<IgniteCache<String, Integer>>() {
-            @Override public void applyx(IgniteCache<String, Integer> c) throws IgniteCheckedException {
-                for (int i = 0; i < all; i++)
-                    c.put(valueOf(i), i);
-            }
-        });
-
-        //putAll
-        doTest(cache, offheapSwap, offheapEmpty, swapEmpty, new CIX1<IgniteCache<String, Integer>>() {
-            @Override public void applyx(IgniteCache<String, Integer> c) throws IgniteCheckedException {
-                putAll(c, 0, all / 2);
-
-                putAll(c, all / 2 + 1, all - 1);
-            }
-
-            private void putAll(IgniteCache<String, Integer> c, int k1, int k2) {
-                Map<String, Integer> m = new HashMap<>();
-
-                for (int i = k1; i <= k2; i++)
-                    m.put(valueOf(i), i);
-
-                c.putAll(m);
-            }
-        });
-    }
-
-    /**
-     * @param cache Cache size.
-     * @param offheapSwap Offheap + swap size.
-     * @param offheapEmpty Offheap is empty.
-     * @param swapEmpty Swap is empty.
-     * @param x Cache modifier.
-     * @throws IgniteCheckedException If failed.
-     */
-    void doTest(int cache, int offheapSwap, boolean offheapEmpty, boolean swapEmpty,
-        CIX1<IgniteCache<String, Integer>> x) throws Exception {
-        ipFinder = new TcpDiscoveryVmIpFinder(true);
-
-        startGrid();
-
-        final IgniteCache<String, Integer> c = jcache();
-
-        x.applyx(c);
-
-        assertEquals(cache, c.size(CachePeekMode.ONHEAP));
-        assertEquals(offheapSwap, c.localSize(CachePeekMode.OFFHEAP) + c.localSize(CachePeekMode.SWAP));
-
-        info("size: " + c.size());
-        info("heap: " + c.localSize(CachePeekMode.ONHEAP));
-        info("offheap: " + c.localSize(CachePeekMode.OFFHEAP));
-        info("swap: " + c.localSize(CachePeekMode.SWAP));
-
-        if (offheapEmpty)
-            Assert.assertEquals(0, c.localSize(CachePeekMode.OFFHEAP));
-        else
-            Assert.assertNotEquals(0, c.localSize(CachePeekMode.OFFHEAP));
-
-        if (swapEmpty)
-            Assert.assertEquals(0, c.localSize(CachePeekMode.SWAP));
-        else
-            Assert.assertNotEquals(0, c.localSize(CachePeekMode.SWAP));
-
-        stopAllGrids();
-    }
-}
\ 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/GridCacheObjectToStringSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheObjectToStringSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheObjectToStringSelfTest.java
index 98d5682..54a3aa9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheObjectToStringSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheObjectToStringSelfTest.java
@@ -68,6 +68,7 @@ public class GridCacheObjectToStringSelfTest extends GridCommonAbstractTest {
 
         cacheCfg.setCacheMode(cacheMode);
         cacheCfg.setEvictionPolicy(evictionPlc);
+        cacheCfg.setOnheapCacheEnabled(true);
         cacheCfg.setNearConfiguration(nearEnabled ? new NearCacheConfiguration() : null);
         cacheCfg.setAtomicityMode(TRANSACTIONAL);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapMultiThreadedUpdateAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapMultiThreadedUpdateAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapMultiThreadedUpdateAbstractSelfTest.java
index e83ec3a..a806b7f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapMultiThreadedUpdateAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapMultiThreadedUpdateAbstractSelfTest.java
@@ -30,7 +30,6 @@ import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.testframework.GridTestUtils;
 
 import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.PRIMARY;
-import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_TIERED;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 
@@ -53,8 +52,6 @@ public abstract class GridCacheOffHeapMultiThreadedUpdateAbstractSelfTest extend
         ccfg.setAtomicityMode(atomicityMode());
         ccfg.setCacheMode(PARTITIONED);
         ccfg.setBackups(1);
-        ccfg.setMemoryMode(OFFHEAP_TIERED);
-        ccfg.setOffHeapMaxMemory(1024 * 1024);
         ccfg.setWriteSynchronizationMode(FULL_SYNC);
         ccfg.setAtomicWriteOrderMode(PRIMARY);