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 2015/09/17 04:04:35 UTC

[29/55] [abbrv] ignite git commit: ignite-971 Fixed offheap to swap eviction, added failover tests with swap/offheap, added retries for tx 'check backup' rollback.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePutAllFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePutAllFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePutAllFailoverSelfTest.java
index b401907..0c0d9c1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePutAllFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePutAllFailoverSelfTest.java
@@ -60,6 +60,7 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.spi.failover.FailoverContext;
 import org.apache.ignite.spi.failover.always.AlwaysFailoverSpi;
+import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
@@ -101,6 +102,9 @@ public class GridCachePutAllFailoverSelfTest extends GridCommonAbstractTest {
     /** Backups count. */
     private int backups;
 
+    /** */
+    private GridTestUtils.TestMemoryMode memMode = GridTestUtils.TestMemoryMode.HEAP;
+
     /** Filter to include only worker nodes. */
     private static final IgnitePredicate<ClusterNode> workerNodesFilter = new PN() {
         @SuppressWarnings("unchecked")
@@ -202,6 +206,60 @@ public class GridCachePutAllFailoverSelfTest extends GridCommonAbstractTest {
         checkPutAllFailoverColocated(false, 5, 2);
     }
 
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAllFailoverColocatedNearEnabledTwoBackupsSwap() throws Exception {
+        memMode = GridTestUtils.TestMemoryMode.SWAP;
+
+        checkPutAllFailoverColocated(true, 5, 2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAllFailoverColocatedTwoBackupsSwap() throws Exception {
+        memMode = GridTestUtils.TestMemoryMode.SWAP;
+
+        checkPutAllFailoverColocated(false, 5, 2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAllFailoverColocatedNearEnabledTwoBackupsOffheapTiered() throws Exception {
+        memMode = GridTestUtils.TestMemoryMode.OFFHEAP_TIERED;
+
+        checkPutAllFailoverColocated(true, 5, 2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAllFailoverColocatedNearEnabledTwoBackupsOffheapTieredSwap() throws Exception {
+        memMode = GridTestUtils.TestMemoryMode.OFFHEAP_TIERED_SWAP;
+
+        checkPutAllFailoverColocated(true, 5, 2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAllFailoverColocatedNearEnabledTwoBackupsOffheapEvict() throws Exception {
+        memMode = GridTestUtils.TestMemoryMode.OFFHEAP_EVICT;
+
+        checkPutAllFailoverColocated(true, 5, 2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAllFailoverColocatedNearEnabledTwoBackupsOffheapEvictSwap() throws Exception {
+        memMode = GridTestUtils.TestMemoryMode.OFFHEAP_EVICT_SWAP;
+
+        checkPutAllFailoverColocated(true, 5, 2);
+    }
+
     /** {@inheritDoc} */
     @Override protected long getTestTimeout() {
         return super.getTestTimeout() * 5;
@@ -696,6 +754,8 @@ public class GridCachePutAllFailoverSelfTest extends GridCommonAbstractTest {
 
             cacheCfg.setWriteSynchronizationMode(FULL_SYNC);
 
+            GridTestUtils.setMemoryMode(cfg, cacheCfg, memMode, 1000, 10 * 1024);
+
             cfg.setCacheConfiguration(cacheCfg);
         }
         else

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
index d9510e6..1fef4d5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
@@ -803,6 +803,12 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
     }
 
     /** {@inheritDoc} */
+    @Override public boolean offheapSwapEvict(byte[] vb, GridCacheVersion evictVer, GridCacheVersion obsoleteVer)
+        throws IgniteCheckedException, GridCacheEntryRemovedException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Override public CacheObject unswap(boolean needVal) throws IgniteCheckedException {
         return null;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/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 757dce8..6f0565b 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,11 +36,13 @@ 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.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 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.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionConcurrency;
@@ -110,6 +112,9 @@ 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} */
@@ -129,11 +134,23 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
 
         c.setDiscoverySpi(disco);
 
+        CacheConfiguration ccfg = cacheConfiguration();
+
+        GridTestUtils.setMemoryMode(c, ccfg, memMode, 100, 1024);
+
+        c.setCacheConfiguration(ccfg);
+
         return c;
     }
 
+    /**
+     * @return Cache configuration.
+     */
+    protected abstract CacheConfiguration cacheConfiguration();
+
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
+        // No-op.
     }
 
     /** {@inheritDoc} */
@@ -280,7 +297,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
 
         long duration = 30000;
 
-        checkRestartWithTx(duration, 1, 1);
+        checkRestartWithTx(duration, 1, 1, 3);
     }
 
     /**
@@ -310,7 +327,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
 
         long duration = 30000;
 
-        checkRestartWithTx(duration, 1, 1);
+        checkRestartWithTx(duration, 1, 1, 3);
     }
 
     /**
@@ -340,7 +357,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
 
         long duration = 60000;
 
-        checkRestartWithTx(duration, 2, 2);
+        checkRestartWithTx(duration, 2, 2, 3);
     }
 
     /**
@@ -361,6 +378,59 @@ 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;
+        partitions = 29;
+        rebalancMode = ASYNC;
+        this.memMode = memMode;
+
+        long duration = 30_000;
+
+        checkRestartWithPut(duration, 2, 2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testRestartWithTxFourNodesOneBackups() throws Throwable {
         backups = 1;
         nodeCnt = 4;
@@ -370,7 +440,59 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
 
         long duration = 60000;
 
-        checkRestartWithTx(duration, 2, 2);
+        checkRestartWithTx(duration, 2, 2, 3);
+    }
+
+    /**
+     * @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;
+
+        long duration = 30_000;
+
+        checkRestartWithTx(duration, 2, 2, 100);
     }
 
     /**
@@ -400,7 +522,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
 
         long duration = 90000;
 
-        checkRestartWithTx(duration, 3, 3);
+        checkRestartWithTx(duration, 3, 3, 3);
     }
 
     /**
@@ -430,7 +552,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
 
         long duration = 90000;
 
-        checkRestartWithTx(duration, 4, 4);
+        checkRestartWithTx(duration, 4, 4, 3);
     }
 
     /**
@@ -460,7 +582,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
 
         long duration = 90000;
 
-        checkRestartWithTx(duration, 5, 5);
+        checkRestartWithTx(duration, 5, 5, 3);
     }
 
     /**
@@ -499,7 +621,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
      * @param restartThreads Restart threads count.
      * @throws Exception If failed.
      */
-    public void checkRestartWithPut(long duration, int putThreads, int restartThreads) throws Throwable {
+    private void checkRestartWithPut(long duration, int putThreads, int restartThreads) throws Throwable {
         final long endTime = System.currentTimeMillis() + duration;
 
         final AtomicReference<Throwable> err = new AtomicReference<>();
@@ -613,9 +735,13 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
      * @param duration Test duration.
      * @param putThreads Put threads count.
      * @param restartThreads Restart threads count.
+     * @param txKeys Keys per transaction.
      * @throws Exception If failed.
      */
-    public void checkRestartWithTx(long duration, int putThreads, int restartThreads) throws Throwable {
+    private void checkRestartWithTx(long duration,
+        int putThreads,
+        int restartThreads,
+        final int txKeys) throws Throwable {
         if (atomicityMode() == ATOMIC)
             return;
 
@@ -634,8 +760,6 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
 
             final CyclicBarrier barrier = new CyclicBarrier(putThreads + restartThreads);
 
-            final int txKeys = 3;
-
             for (int i = 0; i < putThreads; i++) {
                 final int gridIdx = i;
 
@@ -699,6 +823,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
 
                                 if (c % logFreq == 0) {
                                     info(">>> Tx iteration finished [cnt=" + c +
+                                        ", cacheSize=" + cache.localSize() +
                                         ", keys=" + keys +
                                         ", locNodeId=" + locNodeId + ']');
                                 }
@@ -773,7 +898,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
      * @param restartThreads Restart threads count.
      * @throws Exception If failed.
      */
-    public void checkRestartWithTxPutAll(long duration, int putThreads, int restartThreads) throws Throwable {
+    private void checkRestartWithTxPutAll(long duration, int putThreads, int restartThreads) throws Throwable {
         if (atomicityMode() == ATOMIC)
             return;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtAtomicRemoveFailureTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtAtomicRemoveFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtAtomicRemoveFailureTest.java
index b7f52ba..bee94a2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtAtomicRemoveFailureTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtAtomicRemoveFailureTest.java
@@ -17,11 +17,13 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.dht;
 
+import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheAbstractRemoveFailureTest;
 
+import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.CLOCK;
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 
@@ -40,12 +42,12 @@ public class GridCacheDhtAtomicRemoveFailureTest extends GridCacheAbstractRemove
     }
 
     /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
-        CacheConfiguration cfg = super.cacheConfiguration(gridName);
-
-        cfg.setNearConfiguration(null);
-        cfg.setBackups(1);
+    @Override protected CacheAtomicWriteOrderMode atomicWriteOrderMode() {
+        return CLOCK;
+    }
 
-        return cfg;
+    /** {@inheritDoc} */
+    @Override protected NearCacheConfiguration nearCache() {
+        return null;
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtRemoveFailureTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtRemoveFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtRemoveFailureTest.java
index 1c351ff..a13ba30 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtRemoveFailureTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtRemoveFailureTest.java
@@ -17,10 +17,12 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.dht;
 
+import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheAbstractRemoveFailureTest;
 
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 
 /**
@@ -33,12 +35,12 @@ public class GridCacheDhtRemoveFailureTest extends GridCacheAbstractRemoveFailur
     }
 
     /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
-        CacheConfiguration cfg = super.cacheConfiguration(gridName);
-
-        cfg.setNearConfiguration(null);
-        cfg.setBackups(1);
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return TRANSACTIONAL;
+    }
 
-        return cfg;
+    /** {@inheritDoc} */
+    @Override protected NearCacheConfiguration nearCache() {
+        return null;
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheTxNodeFailureSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheTxNodeFailureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheTxNodeFailureSelfTest.java
index dc78003..4302486 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheTxNodeFailureSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheTxNodeFailureSelfTest.java
@@ -199,7 +199,7 @@ public class GridCacheTxNodeFailureSelfTest extends GridCommonAbstractTest {
         try {
             final Ignite ignite = ignite(0);
 
-            final IgniteCache<Object, Object> cache = ignite.cache(null);
+            final IgniteCache<Object, Object> cache = ignite.cache(null).withNoRetries();
 
             final int key = generateKey(ignite, backup);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/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 4c52e6f..7425e23 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
@@ -40,7 +40,7 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 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.spi.swapspace.file.FileSwapSpaceSpi;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionConcurrency;
@@ -51,6 +51,9 @@ 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;
@@ -86,6 +89,8 @@ public class IgniteCacheCrossCacheTxFailoverTest extends GridCommonAbstractTest
         if (gridName.equals(getTestGridName(GRID_CNT - 1)))
             cfg.setClientMode(true);
 
+        cfg.setSwapSpaceSpi(new FileSwapSpaceSpi());
+
         return cfg;
     }
 
@@ -107,9 +112,13 @@ 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) {
+    private CacheConfiguration cacheConfiguration(String name,
+        CacheMode cacheMode,
+        int parts,
+        TestMemoryMode memMode) {
         CacheConfiguration ccfg = new CacheConfiguration();
 
         ccfg.setName(name);
@@ -122,6 +131,8 @@ public class IgniteCacheCrossCacheTxFailoverTest extends GridCommonAbstractTest
 
         ccfg.setAffinity(new RendezvousAffinityFunction(false, parts));
 
+        setMemoryMode(null, ccfg, memMode, 100, 1024);
+
         return ccfg;
     }
 
@@ -134,49 +145,63 @@ public class IgniteCacheCrossCacheTxFailoverTest extends GridCommonAbstractTest
      * @throws Exception If failed.
      */
     public void testCrossCachePessimisticTxFailover() throws Exception {
-        crossCacheTxFailover(PARTITIONED, true, PESSIMISTIC, REPEATABLE_READ);
+        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);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testCrossCachePessimisticTxFailoverDifferentAffinity() throws Exception {
-        crossCacheTxFailover(PARTITIONED, false, PESSIMISTIC, REPEATABLE_READ);
+        crossCacheTxFailover(PARTITIONED, false, PESSIMISTIC, REPEATABLE_READ, TestMemoryMode.HEAP);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testCrossCacheOptimisticTxFailover() throws Exception {
-        crossCacheTxFailover(PARTITIONED, true, OPTIMISTIC, REPEATABLE_READ);
+        crossCacheTxFailover(PARTITIONED, true, OPTIMISTIC, REPEATABLE_READ, TestMemoryMode.HEAP);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCrossCacheOptimisticTxFailoverOffheapSwap() throws Exception {
+        crossCacheTxFailover(PARTITIONED, true, OPTIMISTIC, REPEATABLE_READ, TestMemoryMode.OFFHEAP_EVICT_SWAP);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testCrossCacheOptimisticTxFailoverDifferentAffinity() throws Exception {
-        crossCacheTxFailover(PARTITIONED, false, OPTIMISTIC, REPEATABLE_READ);
+        crossCacheTxFailover(PARTITIONED, false, OPTIMISTIC, REPEATABLE_READ, TestMemoryMode.HEAP);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testCrossCachePessimisticTxFailoverReplicated() throws Exception {
-        crossCacheTxFailover(REPLICATED, true, PESSIMISTIC, REPEATABLE_READ);
+        crossCacheTxFailover(REPLICATED, true, PESSIMISTIC, REPEATABLE_READ, TestMemoryMode.HEAP);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testCrossCacheOptimisticTxFailoverReplicated() throws Exception {
-        crossCacheTxFailover(REPLICATED, true, OPTIMISTIC, REPEATABLE_READ);
+        crossCacheTxFailover(REPLICATED, true, OPTIMISTIC, REPEATABLE_READ, TestMemoryMode.HEAP);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testCrossCachePessimisticTxFailoverDifferentAffinityReplicated() throws Exception {
-        crossCacheTxFailover(PARTITIONED, false, PESSIMISTIC, REPEATABLE_READ);
+        crossCacheTxFailover(PARTITIONED, false, PESSIMISTIC, REPEATABLE_READ, TestMemoryMode.HEAP);
     }
 
     /**
@@ -184,23 +209,25 @@ 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) throws Exception {
+        final TransactionIsolation isolation,
+        TestMemoryMode memMode) throws Exception {
         IgniteKernal ignite0 = (IgniteKernal)ignite(0);
 
         final AtomicBoolean stop = new AtomicBoolean();
 
         try {
-            ignite0.createCache(cacheConfiguration(CACHE1, cacheMode, 256));
-            ignite0.createCache(cacheConfiguration(CACHE2, cacheMode, sameAff ? 256 : 128));
+            ignite0.createCache(cacheConfiguration(CACHE1, cacheMode, 256, memMode));
+            ignite0.createCache(cacheConfiguration(CACHE2, cacheMode, sameAff ? 256 : 128, memMode));
 
             final AtomicInteger threadIdx = new AtomicInteger();
 
-            IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Void>() {
+            IgniteInternalFuture<?> fut = runMultiThreadedAsync(new Callable<Void>() {
                 @Override public Void call() throws Exception {
                     int idx = threadIdx.getAndIncrement();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
index 8a72bb9..943caeb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
@@ -24,35 +24,54 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.atomic.AtomicBoolean;
+import javax.cache.Cache;
+import javax.cache.configuration.Factory;
+import javax.cache.integration.CacheLoaderException;
+import javax.cache.integration.CacheWriterException;
 import javax.cache.processor.EntryProcessorResult;
 import javax.cache.processor.MutableEntry;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
+import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheEntryProcessor;
 import org.apache.ignite.cache.CachePartialUpdateException;
+import org.apache.ignite.cache.store.CacheStore;
+import org.apache.ignite.cache.store.CacheStoreAdapter;
 import org.apache.ignite.configuration.AtomicConfiguration;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
-import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+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.spi.swapspace.file.FileSwapSpaceSpi;
 import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.CLOCK;
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 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.runAsync;
 
 /**
  *
  */
-public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCacheAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 4;
-    }
+public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCommonAbstractTest {
+    /** IP finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final long DURATION = 60_000;
+
+    /** */
+    protected static final int GRID_CNT = 4;
 
     /**
      * @return Keys count for the test.
@@ -61,14 +80,29 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCacheAbstr
         return 10_000;
     }
 
-    /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
-        CacheConfiguration cfg = super.cacheConfiguration(gridName);
+    /**
+     * @param memMode Memory mode.
+     * @param store If {@code true} adds cache store.
+     * @return Cache configuration.
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("unchecked")
+    protected CacheConfiguration cacheConfiguration(TestMemoryMode memMode, boolean store) throws Exception {
+        CacheConfiguration cfg = new CacheConfiguration();
 
+        cfg.setAtomicityMode(atomicityMode());
+        cfg.setWriteSynchronizationMode(FULL_SYNC);
         cfg.setAtomicWriteOrderMode(writeOrderMode());
         cfg.setBackups(1);
         cfg.setRebalanceMode(SYNC);
 
+        if (store) {
+            cfg.setCacheStoreFactory(new TestStoreFactory());
+            cfg.setWriteThrough(true);
+        }
+
+        GridTestUtils.setMemoryMode(null, cfg, memMode, 100, 1024);
+
         return cfg;
     }
 
@@ -76,15 +110,47 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCacheAbstr
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
+        ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1);
+
         AtomicConfiguration acfg = new AtomicConfiguration();
 
         acfg.setBackups(1);
 
         cfg.setAtomicConfiguration(acfg);
 
+        cfg.setSwapSpaceSpi(new FileSwapSpaceSpi());
+
         return cfg;
     }
 
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGridsMultiThreaded(GRID_CNT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        ignite(0).destroyCache(null);
+    }
+
+    /**
+     * @return Cache atomicity mode.
+     */
+    protected abstract CacheAtomicityMode atomicityMode();
+
     /**
      * @return Write order mode.
      */
@@ -96,47 +162,79 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCacheAbstr
      * @throws Exception If failed.
      */
     public void testPut() throws Exception {
-        checkRetry(Test.PUT);
+        checkRetry(Test.PUT, TestMemoryMode.HEAP, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutStoreEnabled() throws Exception {
+        checkRetry(Test.PUT, TestMemoryMode.HEAP, true);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testPutAll() throws Exception {
-        checkRetry(Test.PUT_ALL);
+        checkRetry(Test.PUT_ALL, TestMemoryMode.HEAP, false);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testPutAsync() throws Exception {
-        checkRetry(Test.PUT_ASYNC);
+        checkRetry(Test.PUT_ASYNC, TestMemoryMode.HEAP, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAsyncStoreEnabled() throws Exception {
+        checkRetry(Test.PUT_ASYNC, TestMemoryMode.HEAP, true);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testInvoke() throws Exception {
-        checkRetry(Test.INVOKE);
+        checkRetry(Test.INVOKE, TestMemoryMode.HEAP, false);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testInvokeAll() throws Exception {
-        checkRetry(Test.INVOKE_ALL);
+        checkRetry(Test.INVOKE_ALL, TestMemoryMode.HEAP, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeAllOffheapSwap() throws Exception {
+        checkRetry(Test.INVOKE_ALL, TestMemoryMode.OFFHEAP_EVICT_SWAP, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeAllOffheapTiered() throws Exception {
+        checkRetry(Test.INVOKE_ALL, TestMemoryMode.OFFHEAP_TIERED, false);
     }
 
     /**
      * @param test Test type.
+     * @param memMode Memory mode.
+     * @param store If {@code true} uses cache with store.
      * @throws Exception If failed.
      */
-    private void checkRetry(Test test) throws Exception {
+    private void checkRetry(Test test, TestMemoryMode memMode, boolean store) throws Exception {
+        ignite(0).createCache(cacheConfiguration(memMode, store));
+
         final AtomicBoolean finished = new AtomicBoolean();
 
         int keysCnt = keysCount();
 
-        IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
+        IgniteInternalFuture<Object> fut = runAsync(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 while (!finished.get()) {
                     stopGrid(3);
@@ -155,10 +253,12 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCacheAbstr
         int iter = 0;
 
         try {
-            if (atomicityMode() == ATOMIC)
-                assertEquals(writeOrderMode(), cache.getConfiguration(CacheConfiguration.class).getAtomicWriteOrderMode());
+            if (atomicityMode() == ATOMIC) {
+                assertEquals(writeOrderMode(),
+                    cache.getConfiguration(CacheConfiguration.class).getAtomicWriteOrderMode());
+            }
 
-            long stopTime = System.currentTimeMillis() + 60_000;
+            long stopTime = System.currentTimeMillis() + DURATION;
 
             switch (test) {
                 case PUT: {
@@ -288,7 +388,7 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCacheAbstr
         for (int i = 0; i < keysCnt; i++)
             assertEquals((Integer)iter, cache.get(i));
 
-        for (int i = 0; i < gridCount(); i++) {
+        for (int i = 0; i < GRID_CNT; i++) {
             IgniteKernal ignite = (IgniteKernal)grid(i);
 
             Collection<?> futs = ignite.context().cache().context().mvcc().atomicFutures();
@@ -316,9 +416,11 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCacheAbstr
      * @throws Exception If failed.
      */
     private void checkFailsWithNoRetries(boolean async) throws Exception {
+        ignite(0).createCache(cacheConfiguration(TestMemoryMode.HEAP, false));
+
         final AtomicBoolean finished = new AtomicBoolean();
 
-        IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
+        IgniteInternalFuture<Object> fut = runAsync(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 while (!finished.get()) {
                     stopGrid(3);
@@ -428,4 +530,26 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCacheAbstr
             return old == null ? 0 : old;
         }
     }
+
+    /**
+     *
+     */
+    private static class TestStoreFactory implements Factory<CacheStore> {
+        /** {@inheritDoc} */
+        @Override public CacheStore create() {
+            return new CacheStoreAdapter() {
+                @Override public Object load(Object key) throws CacheLoaderException {
+                    return null;
+                }
+
+                @Override public void write(Cache.Entry entry) throws CacheWriterException {
+                    // No-op.
+                }
+
+                @Override public void delete(Object key) throws CacheWriterException {
+                    // No-op.
+                }
+            };
+        }
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAtomicSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAtomicSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAtomicSelfTest.java
index 9d84609..3d7c7d7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAtomicSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAtomicSelfTest.java
@@ -47,6 +47,8 @@ public class IgniteCachePutRetryAtomicSelfTest extends IgniteCachePutRetryAbstra
      * @throws Exception If failed.
      */
     public void testPutInsideTransaction() throws Exception {
+        ignite(0).createCache(cacheConfiguration(GridTestUtils.TestMemoryMode.HEAP, false));
+
         CacheConfiguration<Integer, Integer> ccfg = new CacheConfiguration<>();
 
         ccfg.setName("tx-cache");

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
index 7c66efc..f61faf2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
@@ -31,12 +31,13 @@ import org.apache.ignite.IgniteAtomicLong;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheEntryProcessor;
-import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.testframework.GridTestUtils;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.testframework.GridTestUtils.TestMemoryMode;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+import static org.apache.ignite.testframework.GridTestUtils.runMultiThreadedAsync;
 
 /**
  *
@@ -50,11 +51,6 @@ public class IgniteCachePutRetryTransactionalSelfTest extends IgniteCachePutRetr
         return TRANSACTIONAL;
     }
 
-    /** {@inheritDoc} */
-    @Override protected NearCacheConfiguration nearConfiguration() {
-        return null;
-    }
-
     /**
      * @throws Exception If failed.
      */
@@ -63,7 +59,7 @@ public class IgniteCachePutRetryTransactionalSelfTest extends IgniteCachePutRetr
 
         IgniteAtomicLong atomic = ignite(0).atomicLong("TestAtomic", 0, true);
 
-        IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
+        IgniteInternalFuture<Object> fut = runAsync(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 while (!finished.get()) {
                     stopGrid(3);
@@ -92,15 +88,42 @@ public class IgniteCachePutRetryTransactionalSelfTest extends IgniteCachePutRetr
         }
     }
 
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
+    /**
+     * @throws Exception If failed.
+     */
     public void testExplicitTransactionRetries() throws Exception {
+        explicitTransactionRetries(TestMemoryMode.HEAP, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testExplicitTransactionRetriesStoreEnabled() throws Exception {
+        explicitTransactionRetries(TestMemoryMode.HEAP, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testExplicitTransactionRetriesOffheapSwap() throws Exception {
+        explicitTransactionRetries(TestMemoryMode.OFFHEAP_EVICT_SWAP, false);
+    }
+
+    /**
+     * @param memMode Memory mode.
+     * @param store If {@code true} uses cache with store.
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("unchecked")
+    public void explicitTransactionRetries(TestMemoryMode memMode, boolean store) throws Exception {
+        ignite(0).createCache(cacheConfiguration(memMode, store));
+
         final AtomicInteger idx = new AtomicInteger();
         int threads = 8;
 
         final AtomicReferenceArray<Exception> err = new AtomicReferenceArray<>(threads);
 
-        IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+        IgniteInternalFuture<Long> fut = runMultiThreadedAsync(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 int th = idx.getAndIncrement();
                 int base = th * FACTOR;
@@ -115,8 +138,7 @@ public class IgniteCachePutRetryTransactionalSelfTest extends IgniteCachePutRetr
                         if (i > 0 && i % 500 == 0)
                             info("Done: " + i);
                     }
-                }
-                catch (Exception e) {
+                } catch (Exception e) {
                     err.set(th, e);
                 }
 
@@ -142,7 +164,7 @@ public class IgniteCachePutRetryTransactionalSelfTest extends IgniteCachePutRetr
         }
 
         // Verify contents of the cache.
-        for (int g = 0; g < gridCount(); g++) {
+        for (int g = 0; g < GRID_CNT; g++) {
             IgniteCache<Object, Object> cache = ignite(g).cache(null);
 
             for (int th = 0; th < threads; th++) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java
index c598e97..3913957 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java
@@ -43,6 +43,7 @@ import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.managers.communication.GridIoMessage;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtInvalidPartitionException;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.typedef.internal.CU;
@@ -65,6 +66,7 @@ import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_ASYNC;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.PRIMARY_SYNC;
+import static org.apache.ignite.testframework.GridTestUtils.TestMemoryMode;
 
 /**
  * Test GridDhtInvalidPartitionException handling in ATOMIC cache during restarts.
@@ -83,19 +85,26 @@ public class GridCacheAtomicInvalidPartitionHandlingSelfTest extends GridCommonA
     /** Write sync. */
     private CacheWriteSynchronizationMode writeSync;
 
+    /** Memory mode. */
+    private TestMemoryMode memMode;
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
         cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER).setForceServerMode(true));
 
-        cfg.setCacheConfiguration(cacheConfiguration());
+        CacheConfiguration ccfg = cacheConfiguration();
+
+        cfg.setCacheConfiguration(ccfg);
 
         cfg.setCommunicationSpi(new DelayCommunicationSpi());
 
         if (testClientNode() && getTestGridName(0).equals(gridName))
             cfg.setClientMode(true);
 
+        GridTestUtils.setMemoryMode(cfg, ccfg, memMode, 100, 1024);
+
         return cfg;
     }
 
@@ -137,53 +146,99 @@ public class GridCacheAtomicInvalidPartitionHandlingSelfTest extends GridCommonA
      * @throws Exception If failed.
      */
     public void testClockFullSync() throws Exception {
-        checkRestarts(CLOCK, FULL_SYNC);
+        checkRestarts(CLOCK, FULL_SYNC, TestMemoryMode.HEAP);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClockFullSyncSwap() throws Exception {
+        checkRestarts(CLOCK, FULL_SYNC, TestMemoryMode.SWAP);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClockFullSyncOffheapTiered() throws Exception {
+        checkRestarts(CLOCK, FULL_SYNC, TestMemoryMode.OFFHEAP_TIERED);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClockFullSyncOffheapSwap() throws Exception {
+        checkRestarts(CLOCK, FULL_SYNC, TestMemoryMode.OFFHEAP_EVICT_SWAP);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testClockPrimarySync() throws Exception {
-        checkRestarts(CLOCK, PRIMARY_SYNC);
+        checkRestarts(CLOCK, PRIMARY_SYNC, TestMemoryMode.HEAP);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testClockFullAsync() throws Exception {
-        checkRestarts(CLOCK, FULL_ASYNC);
+        checkRestarts(CLOCK, FULL_ASYNC, TestMemoryMode.HEAP);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testPrimaryFullSync() throws Exception {
-        checkRestarts(PRIMARY, FULL_SYNC);
+        checkRestarts(PRIMARY, FULL_SYNC, TestMemoryMode.HEAP);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPrimaryFullSyncSwap() throws Exception {
+        checkRestarts(PRIMARY, FULL_SYNC, TestMemoryMode.SWAP);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPrimaryFullSyncOffheapTiered() throws Exception {
+        checkRestarts(PRIMARY, FULL_SYNC, TestMemoryMode.OFFHEAP_TIERED);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPrimaryFullSyncOffheapSwap() throws Exception {
+        checkRestarts(PRIMARY, FULL_SYNC, TestMemoryMode.OFFHEAP_EVICT_SWAP);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testPrimaryPrimarySync() throws Exception {
-        checkRestarts(PRIMARY, PRIMARY_SYNC);
+        checkRestarts(PRIMARY, PRIMARY_SYNC, TestMemoryMode.HEAP);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testPrimaryFullAsync() throws Exception {
-        checkRestarts(PRIMARY, FULL_ASYNC);
+        checkRestarts(PRIMARY, FULL_ASYNC, TestMemoryMode.HEAP);
     }
 
     /**
      * @param writeOrder Write order to check.
      * @param writeSync Write synchronization mode to check.
+     * @param memMode Memory mode.
      * @throws Exception If failed.
      */
-    private void checkRestarts(CacheAtomicWriteOrderMode writeOrder, CacheWriteSynchronizationMode writeSync)
+    private void checkRestarts(CacheAtomicWriteOrderMode writeOrder,
+        CacheWriteSynchronizationMode writeSync,
+        TestMemoryMode memMode)
         throws Exception {
         this.writeOrder = writeOrder;
         this.writeSync = writeSync;
+        this.memMode = memMode;
 
         final int gridCnt = 6;
 
@@ -227,16 +282,16 @@ public class GridCacheAtomicInvalidPartitionHandlingSelfTest extends GridCommonA
                         for (int i = 0; i < gridCnt; i++) {
                             ClusterNode locNode = grid(i).localNode();
 
-                            GridCacheAdapter<Object, Object> c = ((IgniteKernal)grid(i)).internalCache();
+                            IgniteCache<Object, Object> cache = grid(i).cache(null);
 
-                            GridCacheEntryEx entry = c.peekEx(key);
+                            Object val = cache.localPeek(key);
 
                             if (affNodes.contains(locNode)) {
-                                if (entry == null)
+                                if (val == null)
                                     return false;
                             }
                             else
-                                assertNull(entry);
+                                assertNull(val);
                         }
 
                         it.remove();
@@ -323,7 +378,20 @@ public class GridCacheAtomicInvalidPartitionHandlingSelfTest extends GridCommonA
 
                     GridCacheAdapter<Object, Object> c = ((IgniteKernal)grid(i)).internalCache();
 
-                    GridCacheEntryEx entry = c.peekEx(k);
+                    GridCacheEntryEx entry = null;
+
+                    if (memMode == TestMemoryMode.HEAP)
+                        entry = c.peekEx(k);
+                    else {
+                        try {
+                            entry = c.entryEx(k);
+
+                            entry.unswap();
+                        }
+                        catch (GridDhtInvalidPartitionException e) {
+                            // Skip key.
+                        }
+                    }
 
                     for (int r = 0; r < 10; r++) {
                         try {
@@ -383,7 +451,7 @@ public class GridCacheAtomicInvalidPartitionHandlingSelfTest extends GridCommonA
      */
     private static class DelayCommunicationSpi extends TcpCommunicationSpi {
         /** {@inheritDoc} */
-        @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure<IgniteException> ackClosure)
+        @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure<IgniteException> ackC)
             throws IgniteSpiException {
             try {
                 if (delayMessage((GridIoMessage)msg))
@@ -393,7 +461,7 @@ public class GridCacheAtomicInvalidPartitionHandlingSelfTest extends GridCommonA
                 throw new IgniteSpiException(e);
             }
 
-            super.sendMessage(node, msg, ackClosure);
+            super.sendMessage(node, msg, ackC);
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicPrimaryWriteOrderRemoveFailureTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicPrimaryWriteOrderRemoveFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicPrimaryWriteOrderRemoveFailureTest.java
index 844bde3..2fd8a82 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicPrimaryWriteOrderRemoveFailureTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicPrimaryWriteOrderRemoveFailureTest.java
@@ -17,9 +17,9 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.dht.atomic;
 
+import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheAbstractRemoveFailureTest;
 
@@ -42,17 +42,12 @@ public class GridCacheAtomicPrimaryWriteOrderRemoveFailureTest extends GridCache
     }
 
     /** {@inheritDoc} */
-    @Override protected NearCacheConfiguration nearConfiguration() {
-        return null;
+    @Override protected CacheAtomicWriteOrderMode atomicWriteOrderMode() {
+        return PRIMARY;
     }
 
     /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
-        CacheConfiguration cfg = super.cacheConfiguration(gridName);
-
-        cfg.setBackups(1);
-        cfg.setAtomicWriteOrderMode(PRIMARY);
-
-        return cfg;
+    @Override protected NearCacheConfiguration nearCache() {
+        return null;
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicRemoveFailureTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicRemoveFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicRemoveFailureTest.java
index 9966e18..a21cd35 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicRemoveFailureTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicRemoveFailureTest.java
@@ -17,9 +17,9 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.dht.atomic;
 
+import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheAbstractRemoveFailureTest;
 
@@ -42,17 +42,12 @@ public class GridCacheAtomicRemoveFailureTest extends GridCacheAbstractRemoveFai
     }
 
     /** {@inheritDoc} */
-    @Override protected NearCacheConfiguration nearConfiguration() {
-        return null;
+    @Override protected CacheAtomicWriteOrderMode atomicWriteOrderMode() {
+        return CLOCK;
     }
 
     /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
-        CacheConfiguration cfg = super.cacheConfiguration(gridName);
-
-        cfg.setBackups(1);
-        cfg.setAtomicWriteOrderMode(CLOCK);
-
-        return cfg;
+    @Override protected NearCacheConfiguration nearCache() {
+        return null;
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearRemoveFailureTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearRemoveFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearRemoveFailureTest.java
index a6fc2b9..d9c6e01 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearRemoveFailureTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearRemoveFailureTest.java
@@ -17,9 +17,9 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.near;
 
+import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheAbstractRemoveFailureTest;
 
@@ -41,13 +41,12 @@ public class GridCacheAtomicNearRemoveFailureTest extends GridCacheAbstractRemov
     }
 
     /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
-        CacheConfiguration cfg = super.cacheConfiguration(gridName);
-
-        cfg.setNearConfiguration(new NearCacheConfiguration());
-        cfg.setBackups(1);
-        cfg.setAtomicWriteOrderMode(CLOCK);
+    @Override protected CacheAtomicWriteOrderMode atomicWriteOrderMode() {
+        return CLOCK;
+    }
 
-        return cfg;
+    /** {@inheritDoc} */
+    @Override protected NearCacheConfiguration nearCache() {
+        return new NearCacheConfiguration();
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicPrimaryWriteOrderNearRemoveFailureTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicPrimaryWriteOrderNearRemoveFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicPrimaryWriteOrderNearRemoveFailureTest.java
index 6362bc8..f6e7e2f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicPrimaryWriteOrderNearRemoveFailureTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicPrimaryWriteOrderNearRemoveFailureTest.java
@@ -17,9 +17,9 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.near;
 
+import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheAbstractRemoveFailureTest;
 
@@ -41,13 +41,12 @@ public class GridCacheAtomicPrimaryWriteOrderNearRemoveFailureTest extends GridC
     }
 
     /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
-        CacheConfiguration cfg = super.cacheConfiguration(gridName);
-
-        cfg.setNearConfiguration(new NearCacheConfiguration());
-        cfg.setBackups(1);
-        cfg.setAtomicWriteOrderMode(PRIMARY);
+    @Override protected CacheAtomicWriteOrderMode atomicWriteOrderMode() {
+        return PRIMARY;
+    }
 
-        return cfg;
+    /** {@inheritDoc} */
+    @Override protected NearCacheConfiguration nearCache() {
+        return new NearCacheConfiguration();
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearRemoveFailureTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearRemoveFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearRemoveFailureTest.java
index 1cf3da1..a7e8eb9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearRemoveFailureTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearRemoveFailureTest.java
@@ -17,11 +17,12 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.near;
 
+import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheAbstractRemoveFailureTest;
 
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 
 /**
@@ -34,12 +35,12 @@ public class GridCacheNearRemoveFailureTest extends GridCacheAbstractRemoveFailu
     }
 
     /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
-        CacheConfiguration cfg = super.cacheConfiguration(gridName);
-
-        cfg.setNearConfiguration(new NearCacheConfiguration());
-        cfg.setBackups(1);
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return TRANSACTIONAL;
+    }
 
-        return cfg;
+    /** {@inheritDoc} */
+    @Override protected NearCacheConfiguration nearCache() {
+        return new NearCacheConfiguration();
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNodeRestartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNodeRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNodeRestartTest.java
index 265c185..684d6e4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNodeRestartTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNodeRestartTest.java
@@ -36,6 +36,11 @@ public class GridCachePartitionedNodeRestartTest extends GridCacheAbstractNodeRe
 
         c.getTransactionConfiguration().setDefaultTxConcurrency(PESSIMISTIC);
 
+        return c;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheConfiguration cacheConfiguration() {
         CacheConfiguration cc = defaultCacheConfiguration();
 
         cc.setName(CACHE_NAME);
@@ -49,9 +54,7 @@ public class GridCachePartitionedNodeRestartTest extends GridCacheAbstractNodeRe
         cc.setAffinity(new RendezvousAffinityFunction(false, partitions));
         cc.setBackups(backups);
 
-        c.setCacheConfiguration(cc);
-
-        return c;
+        return cc;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOptimisticTxNodeRestartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOptimisticTxNodeRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOptimisticTxNodeRestartTest.java
index 0e81828..a458aa7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOptimisticTxNodeRestartTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOptimisticTxNodeRestartTest.java
@@ -44,6 +44,11 @@ public class GridCachePartitionedOptimisticTxNodeRestartTest extends GridCacheAb
 
         c.getTransactionConfiguration().setDefaultTxConcurrency(OPTIMISTIC);
 
+        return c;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheConfiguration cacheConfiguration() {
         CacheConfiguration cc = defaultCacheConfiguration();
 
         cc.setName(CACHE_NAME);
@@ -57,9 +62,7 @@ public class GridCachePartitionedOptimisticTxNodeRestartTest extends GridCacheAb
 
         cc.setNearConfiguration(nearEnabled() ? new NearCacheConfiguration() : null);
 
-        c.setCacheConfiguration(cc);
-
-        return c;
+        return cc;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedNodeRestartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedNodeRestartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedNodeRestartSelfTest.java
index 5be228d..7e3e7e5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedNodeRestartSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedNodeRestartSelfTest.java
@@ -30,9 +30,7 @@ import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
  */
 public class GridCacheReplicatedNodeRestartSelfTest extends GridCacheAbstractNodeRestartSelfTest {
     /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration c = super.getConfiguration(gridName);
-
+    @Override protected CacheConfiguration cacheConfiguration() {
         CacheConfiguration cc = defaultCacheConfiguration();
 
         cc.setNearConfiguration(null);
@@ -51,9 +49,7 @@ public class GridCacheReplicatedNodeRestartSelfTest extends GridCacheAbstractNod
 
         cc.setRebalanceBatchSize(20);
 
-        c.setCacheConfiguration(cc);
-
-        return c;
+        return cc;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapAbstractSelfTest.java
index 4bda4f4..4cda0c2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapAbstractSelfTest.java
@@ -42,7 +42,7 @@ public abstract class GridOffHeapMapAbstractSelfTest extends GridCommonAbstractT
     private static final Random RAND = new Random();
 
     /** Unsafe map. */
-    private GridOffHeapMap<String> map;
+    private GridOffHeapMap map;
 
     /** */
     protected float load = 0.75f;
@@ -86,7 +86,7 @@ public abstract class GridOffHeapMapAbstractSelfTest extends GridCommonAbstractT
     /**
      * @return New map.
      */
-    protected abstract <K> GridOffHeapMap<K> newMap();
+    protected abstract GridOffHeapMap newMap();
 
     /**
      *
@@ -551,6 +551,10 @@ public abstract class GridOffHeapMapAbstractSelfTest extends GridCommonAbstractT
 
                 evictCnt.incrementAndGet();
             }
+
+            @Override public boolean removeEvicted() {
+                return true;
+            }
         };
 
         map = newMap();
@@ -587,6 +591,10 @@ public abstract class GridOffHeapMapAbstractSelfTest extends GridCommonAbstractT
             @Override public void onEvict(int part, int hash, byte[] k, byte[] v) {
                 evictCnt.incrementAndGet();
             }
+
+            @Override public boolean removeEvicted() {
+                return true;
+            }
         };
 
         map = newMap();
@@ -622,6 +630,10 @@ public abstract class GridOffHeapMapAbstractSelfTest extends GridCommonAbstractT
             @Override public void onEvict(int part, int hash, byte[] k, byte[] v) {
                 evictCnt.incrementAndGet();
             }
+
+            @Override public boolean removeEvicted() {
+                return true;
+            }
         };
 
         map = newMap();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapPerformanceAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapPerformanceAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapPerformanceAbstractTest.java
index d1a1b20..f7388e8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapPerformanceAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapPerformanceAbstractTest.java
@@ -39,7 +39,7 @@ public abstract class GridOffHeapMapPerformanceAbstractTest extends GridCommonAb
         new HashMap<>(LOAD_CNT);
 
     /** Unsafe map. */
-    private GridOffHeapMap<String> map;
+    private GridOffHeapMap map;
 
     /** */
     protected float load = 0.75f;
@@ -91,7 +91,7 @@ public abstract class GridOffHeapMapPerformanceAbstractTest extends GridCommonAb
     /**
      * @return New map.
      */
-    protected abstract <K> GridOffHeapMap<K> newMap();
+    protected abstract GridOffHeapMap newMap();
 
     /**
      * @param key Key.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapAbstractSelfTest.java
index 03fcd4a..c40c10f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapAbstractSelfTest.java
@@ -882,6 +882,10 @@ public abstract class GridOffHeapPartitionedMapAbstractSelfTest extends GridComm
 
                 evictCnt.incrementAndGet();
             }
+
+            @Override public boolean removeEvicted() {
+                return true;
+            }
         };
 
         map = newMap();
@@ -921,6 +925,10 @@ public abstract class GridOffHeapPartitionedMapAbstractSelfTest extends GridComm
                 @Override public void onEvict(int part, int hash, byte[] k, byte[] v) {
                     evictCnt.incrementAndGet();
                 }
+
+                @Override public boolean removeEvicted() {
+                    return true;
+                }
             };
 
             map = newMap();
@@ -957,6 +965,10 @@ public abstract class GridOffHeapPartitionedMapAbstractSelfTest extends GridComm
             @Override public void onEvict(int part, int hash, byte[] k, byte[] v) {
                 evictCnt.incrementAndGet();
             }
+
+            @Override public boolean removeEvicted() {
+                return true;
+            }
         };
 
         map = newMap();
@@ -1009,6 +1021,10 @@ public abstract class GridOffHeapPartitionedMapAbstractSelfTest extends GridComm
 
                 evicted.set(key);
             }
+
+            @Override public boolean removeEvicted() {
+                return true;
+            }
         };
 
         map = newMap();
@@ -1072,6 +1088,10 @@ public abstract class GridOffHeapPartitionedMapAbstractSelfTest extends GridComm
 
                 evicted.set(key);
             }
+
+            @Override public boolean removeEvicted() {
+                return true;
+            }
         };
 
         map = newMap();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapPerformanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapPerformanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapPerformanceTest.java
index e758246..58ad494 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapPerformanceTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapPerformanceTest.java
@@ -26,7 +26,7 @@ import org.apache.ignite.internal.util.offheap.GridOffHeapMapPerformanceAbstract
  */
 public class GridUnsafeMapPerformanceTest extends GridOffHeapMapPerformanceAbstractTest {
     /** {@inheritDoc} */
-    @Override protected <K> GridOffHeapMap<K> newMap() {
+    @Override protected GridOffHeapMap newMap() {
         return GridOffHeapMapFactory.unsafeMap(concurrency, load, initCap, mem, lruStripes, evictClo);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapSelfTest.java
index 43fdb34..0e36f3d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapSelfTest.java
@@ -26,7 +26,7 @@ import org.apache.ignite.internal.util.offheap.GridOffHeapMapFactory;
  */
 public class GridUnsafeMapSelfTest extends GridOffHeapMapAbstractSelfTest {
     /** {@inheritDoc} */
-    @Override protected <K> GridOffHeapMap<K> newMap() {
+    @Override protected GridOffHeapMap newMap() {
         return GridOffHeapMapFactory.unsafeMap(concurrency, load, initCap, mem, lruStripes, evictLsnr);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/GridOffHeapMapPerformanceAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/GridOffHeapMapPerformanceAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/GridOffHeapMapPerformanceAbstractTest.java
index ed37306..4064482 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/GridOffHeapMapPerformanceAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/GridOffHeapMapPerformanceAbstractTest.java
@@ -42,7 +42,7 @@ public abstract class GridOffHeapMapPerformanceAbstractTest extends GridCommonAb
         new HashMap<>(LOAD_CNT);
 
     /** Unsafe map. */
-    private GridOffHeapMap<String> map;
+    private GridOffHeapMap map;
 
     /** */
     protected float load = 0.75f;
@@ -94,7 +94,7 @@ public abstract class GridOffHeapMapPerformanceAbstractTest extends GridCommonAb
     /**
      * @return New map.
      */
-    protected abstract <K> GridOffHeapMap<K> newMap();
+    protected abstract GridOffHeapMap newMap();
 
     /**
      * @param key Key.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/unsafe/GridUnsafeMapPerformanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/unsafe/GridUnsafeMapPerformanceTest.java b/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/unsafe/GridUnsafeMapPerformanceTest.java
index 1486a9c..af691b4 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/unsafe/GridUnsafeMapPerformanceTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/unsafe/GridUnsafeMapPerformanceTest.java
@@ -26,7 +26,7 @@ import org.apache.ignite.internal.util.offheap.GridOffHeapMapPerformanceAbstract
  */
 public class GridUnsafeMapPerformanceTest extends GridOffHeapMapPerformanceAbstractTest {
     /** {@inheritDoc} */
-    @Override protected <K> GridOffHeapMap<K> newMap() {
+    @Override protected GridOffHeapMap newMap() {
         return GridOffHeapMapFactory.unsafeMap(concurrency, load, initCap, mem, lruStripes, evictClo);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
index b585a8f..edf7c52 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
@@ -59,7 +59,11 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.Ignition;
+import org.apache.ignite.cache.CacheMemoryMode;
+import org.apache.ignite.cache.eviction.lru.LruEvictionPolicy;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
@@ -83,6 +87,7 @@ import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.LT;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi;
 import org.apache.ignite.ssl.SslContextFactory;
 import org.apache.ignite.testframework.config.GridTestProperties;
 import org.jetbrains.annotations.NotNull;
@@ -1582,4 +1587,116 @@ public final class GridTestUtils {
 
         suite.addTestSuite(test);
     }
+
+    /**
+     * Sets cache configuration parameters according to test memory mode.
+     *
+     * @param cfg Ignite configuration.
+     * @param ccfg Cache configuration.
+     * @param testMode Test memory mode.
+     * @param maxHeapCnt Maximum number of entries in heap (used if test mode involves eviction from heap).
+     * @param maxOffheapSize Maximum offheap memory size (used if test mode involves eviction from offheap to swap).
+     */
+    public static void setMemoryMode(IgniteConfiguration cfg, CacheConfiguration ccfg,
+        TestMemoryMode testMode,
+        int maxHeapCnt,
+        long maxOffheapSize) {
+        assert testMode != null;
+        assert ccfg != null;
+
+        CacheMemoryMode memMode;
+        boolean swap = false;
+        boolean evictionPlc = false;
+        long offheapMaxMem = -1L;
+
+        switch (testMode) {
+            case HEAP: {
+                memMode = CacheMemoryMode.ONHEAP_TIERED;
+                swap = false;
+
+                break;
+            }
+
+            case SWAP: {
+                memMode = CacheMemoryMode.ONHEAP_TIERED;
+                evictionPlc = true;
+                swap = true;
+
+                break;
+            }
+
+            case OFFHEAP_TIERED: {
+                memMode = CacheMemoryMode.OFFHEAP_TIERED;
+                offheapMaxMem = 0;
+
+                break;
+            }
+
+            case OFFHEAP_TIERED_SWAP: {
+                assert maxOffheapSize > 0 : maxOffheapSize;
+
+                memMode = CacheMemoryMode.OFFHEAP_TIERED;
+                offheapMaxMem = maxOffheapSize;
+                swap = true;
+
+                break;
+            }
+
+            case OFFHEAP_EVICT: {
+                memMode = CacheMemoryMode.ONHEAP_TIERED;
+                evictionPlc = true;
+                offheapMaxMem = 0;
+
+                break;
+            }
+
+            case OFFHEAP_EVICT_SWAP: {
+                assert maxOffheapSize > 0 : maxOffheapSize;
+
+                memMode = CacheMemoryMode.ONHEAP_TIERED;
+                swap = true;
+                evictionPlc = true;
+                offheapMaxMem = maxOffheapSize;
+
+                break;
+            }
+
+            default:
+                throw new IllegalArgumentException("Invalid mode: " + testMode);
+        }
+
+        ccfg.setMemoryMode(memMode);
+        ccfg.setSwapEnabled(swap);
+
+        if (swap && cfg != null)
+            cfg.setSwapSpaceSpi(new FileSwapSpaceSpi());
+
+        if (evictionPlc) {
+            LruEvictionPolicy plc = new LruEvictionPolicy();
+
+            plc.setMaxSize(maxHeapCnt);
+
+            ccfg.setEvictionPolicy(plc);
+        }
+
+        ccfg.setOffHeapMaxMemory(offheapMaxMem);
+    }
+
+    /**
+     *
+     */
+    public enum TestMemoryMode {
+        /** Heap only. */
+        HEAP,
+        /** Evict from heap to swap with eviction policy. */
+        SWAP,
+        /** Always evict to offheap, no swap. */
+        OFFHEAP_TIERED,
+        /** Always evict to offheap + evict from offheap to swap when max offheap memory limit is reached. */
+        OFFHEAP_TIERED_SWAP,
+        /** Evict to offheap with eviction policy, no swap. */
+        OFFHEAP_EVICT,
+        /** Evict to offheap with eviction policy + evict from offheap to swap when max offheap memory limit is reached. */
+        OFFHEAP_EVICT_SWAP,
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a7490a6e/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java
index eaeb7b3..abc8765 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java
@@ -91,12 +91,6 @@ public class IgniteCacheFailoverTestSuite extends TestSuite {
         suite.addTestSuite(IgniteCacheTxNearDisabledPutGetRestartTest.class);
         suite.addTestSuite(IgniteCacheTxNearDisabledFairAffinityPutGetRestartTest.class);
 
-        suite.addTestSuite(IgniteCachePutRetryAtomicSelfTest.class);
-        suite.addTestSuite(IgniteCachePutRetryAtomicPrimaryWriteOrderSelfTest.class);
-        suite.addTestSuite(IgniteCachePutRetryTransactionalSelfTest.class);
-
-        suite.addTestSuite(IgniteCacheSslStartStopSelfTest.class);
-
         suite.addTestSuite(IgniteCacheSizeFailoverTest.class);
 
         suite.addTestSuite(IgniteCacheTopologySafeGetSelfTest.class);