You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by gv...@apache.org on 2018/11/27 08:45:34 UTC

[2/3] ignite git commit: IGNITE-10002: MVCC: Create "Cache 2" test suite for MVCC mode. This closes #5198.

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
index ce8749c..dd6e36e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
@@ -86,6 +86,7 @@ import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode;
 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.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionConcurrency;
@@ -140,6 +141,13 @@ public class IgniteCacheClientNodeChangingTopologyTest extends GridCommonAbstrac
     }
 
     /** {@inheritDoc} */
+    @Override public void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         super.afterTest();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheTxIteratorSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheTxIteratorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheTxIteratorSelfTest.java
index c211590..e5bb583 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheTxIteratorSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheTxIteratorSelfTest.java
@@ -27,8 +27,9 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.configuration.TransactionConfiguration;
-import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
 import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.testframework.MvccFeatureChecker;
+import org.apache.ignite.testframework.MvccFeatureChecker.Feature;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionConcurrency;
@@ -105,14 +106,14 @@ public class IgniteCacheTxIteratorSelfTest extends GridCommonAbstractTest {
         try {
             for (CacheMode mode : CacheMode.values()) {
                 for (CacheAtomicityMode atomMode : CacheAtomicityMode.values()) {
-                    if (mode == CacheMode.PARTITIONED && atomMode != CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT) {
+                    if (mode == CacheMode.PARTITIONED) {
                         // Near cache makes sense only for partitioned cache.
                         checkTxCache(CacheMode.PARTITIONED, atomMode, true, false);
                     }
 
-                    checkTxCache(CacheMode.PARTITIONED, atomMode, false, true);
+                    checkTxCache(mode, atomMode, false, true);
 
-                    checkTxCache(CacheMode.PARTITIONED, atomMode, false, false);
+                    checkTxCache(mode, atomMode, false, false);
                 }
             }
         }
@@ -130,6 +131,13 @@ public class IgniteCacheTxIteratorSelfTest extends GridCommonAbstractTest {
         boolean nearEnabled,
         boolean useEvicPlc
     ) throws Exception {
+        if (atomMode == CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT) {
+            if (!MvccFeatureChecker.isSupported(mode) ||
+                (nearEnabled && !MvccFeatureChecker.isSupported(Feature.NEAR_CACHE)) ||
+                (useEvicPlc && !MvccFeatureChecker.isSupported(Feature.EVICTION)))
+                return; // Nothing to do. Mode is not supported.
+        }
+
         final Ignite ignite = grid(0);
 
         final CacheConfiguration<String, TestClass> ccfg = cacheConfiguration(
@@ -154,14 +162,11 @@ public class IgniteCacheTxIteratorSelfTest extends GridCommonAbstractTest {
 
                 for (TransactionIsolation iso : TransactionIsolation.values()) {
                     for (TransactionConcurrency con : TransactionConcurrency.values()) {
-                        try (Transaction transaction = ignite.transactions().txStart(con, iso)) {
-                            //TODO: IGNITE-7187: Fix when ticket will be implemented. (Near cache)
-                            //TODO: IGNITE-7956: Fix when ticket will be implemented. (Eviction)
-                            if (((IgniteCacheProxy)cache).context().mvccEnabled() &&
-                                ((iso != TransactionIsolation.REPEATABLE_READ && con != TransactionConcurrency.PESSIMISTIC)
-                                    || nearEnabled || useEvicPlc))
-                                return; // Nothing to do. Mode is not supported.
+                        if (atomMode == CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT &&
+                            !MvccFeatureChecker.isSupported(con, iso))
+                            continue; // Mode not supported.
 
+                        try (Transaction transaction = ignite.transactions().txStart(con, iso)) {
                             assertEquals(val, cache.get(key));
 
                             transaction.commit();

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteMvccTxTimeoutAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteMvccTxTimeoutAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteMvccTxTimeoutAbstractTest.java
new file mode 100644
index 0000000..15d23f4
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteMvccTxTimeoutAbstractTest.java
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed;
+
+import java.util.Random;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.TransactionConfiguration;
+import org.apache.ignite.internal.util.typedef.X;
+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.apache.ignite.transactions.Transaction;
+import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionIsolation;
+import org.apache.ignite.transactions.TransactionTimeoutException;
+
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+
+/**
+ * Simple cache test.
+ */
+public class IgniteMvccTxTimeoutAbstractTest extends GridCommonAbstractTest {
+    /** Random number generator. */
+    private static final Random RAND = new Random();
+
+    /** Grid count. */
+    private static final int GRID_COUNT = 2;
+
+    /** Transaction timeout. */
+    private static final long TIMEOUT = 50;
+
+    /** */
+    private TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Override protected void beforeTestsStarted() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-7388");
+
+        startGridsMultiThreaded(GRID_COUNT, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration c = super.getConfiguration(igniteInstanceName);
+
+        TransactionConfiguration txCfg = c.getTransactionConfiguration();
+
+        txCfg.setDefaultTxTimeout(TIMEOUT);
+
+        TcpDiscoverySpi spi = new TcpDiscoverySpi();
+
+        spi.setIpFinder(ipFinder);
+
+        c.setDiscoverySpi(spi);
+
+        return c;
+    }
+
+    /**
+     * @param i Grid index.
+     * @return Cache.
+     */
+    @Override protected <K, V> IgniteCache<K, V> jcache(int i) {
+        return grid(i).cache(DEFAULT_CACHE_NAME);
+    }
+
+    /**
+     * @throws IgniteCheckedException If test failed.
+     */
+    public void testPessimisticRepeatableRead() throws Exception {
+        checkTransactionTimeout(PESSIMISTIC, REPEATABLE_READ);
+    }
+
+    /**
+     * @param concurrency Concurrency.
+     * @param isolation Isolation.
+     * @throws IgniteCheckedException If test failed.
+     */
+    private void checkTransactionTimeout(TransactionConcurrency concurrency,
+        TransactionIsolation isolation) throws Exception {
+        int idx = RAND.nextInt(GRID_COUNT);
+
+        IgniteCache<Integer, String> cache = jcache(idx);
+
+        Transaction tx = ignite(idx).transactions().txStart(concurrency, isolation, TIMEOUT, 0);
+
+        try {
+            info("Storing value in cache [key=1, val=1]");
+
+            cache.put(1, "1");
+
+            long sleep = TIMEOUT * 2;
+
+            info("Going to sleep for (ms): " + sleep);
+
+            Thread.sleep(sleep);
+
+            info("Storing value in cache [key=1, val=2]");
+
+            cache.put(1, "2");
+
+            info("Committing transaction: " + tx);
+
+            tx.commit();
+
+            assert false : "Timeout never happened for transaction: " + tx;
+        }
+        catch (Exception e) {
+            if (!(X.hasCause(e, TransactionTimeoutException.class)))
+                throw e;
+
+            info("Received expected timeout exception [msg=" + e.getMessage() + ", tx=" + tx + ']');
+        }
+        finally {
+            tx.close();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxTimeoutAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxTimeoutAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxTimeoutAbstractTest.java
index a0ec70a..4cd64e3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxTimeoutAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxTimeoutAbstractTest.java
@@ -17,13 +17,15 @@
 
 package org.apache.ignite.internal.processors.cache.distributed;
 
-import java.util.ArrayList;
-import java.util.List;
 import java.util.Random;
-import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.TransactionConfiguration;
 import org.apache.ignite.internal.util.typedef.X;
+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.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionConcurrency;
@@ -46,25 +48,41 @@ public class IgniteTxTimeoutAbstractTest extends GridCommonAbstractTest {
     /** Grid count. */
     private static final int GRID_COUNT = 2;
 
-    /** Grid instances. */
-    private static final List<Ignite> IGNITEs = new ArrayList<>();
-
     /** Transaction timeout. */
     private static final long TIMEOUT = 50;
 
+    /** */
+    private TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
     /**
      * @throws Exception If failed.
      */
     @Override protected void beforeTestsStarted() throws Exception {
-        for (int i = 0; i < GRID_COUNT; i++)
-            IGNITEs.add(startGrid(i));
+        startGridsMultiThreaded(GRID_COUNT);
     }
 
     /**
      * @throws Exception If failed.
      */
     @Override protected void afterTestsStopped() throws Exception {
-        IGNITEs.clear();
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration c = super.getConfiguration(igniteInstanceName);
+
+        TransactionConfiguration txCfg = c.getTransactionConfiguration();
+
+        txCfg.setDefaultTxTimeout(TIMEOUT);
+
+        TcpDiscoverySpi spi = new TcpDiscoverySpi();
+
+        spi.setIpFinder(ipFinder);
+
+        c.setDiscoverySpi(spi);
+
+        return c;
     }
 
     /**
@@ -72,7 +90,7 @@ public class IgniteTxTimeoutAbstractTest extends GridCommonAbstractTest {
      * @return Cache.
      */
     @Override protected <K, V> IgniteCache<K, V> jcache(int i) {
-        return IGNITEs.get(i).cache(DEFAULT_CACHE_NAME);
+        return grid(i).cache(DEFAULT_CACHE_NAME);
     }
 
     /**
@@ -124,7 +142,6 @@ public class IgniteTxTimeoutAbstractTest extends GridCommonAbstractTest {
      */
     private void checkTransactionTimeout(TransactionConcurrency concurrency,
         TransactionIsolation isolation) throws Exception {
-
         int idx = RAND.nextInt(GRID_COUNT);
 
         IgniteCache<Integer, String> cache = jcache(idx);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedMvccTxSingleThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedMvccTxSingleThreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedMvccTxSingleThreadedSelfTest.java
new file mode 100644
index 0000000..717d771
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedMvccTxSingleThreadedSelfTest.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed.dht;
+
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.processors.cache.IgniteMvccTxSingleThreadedAbstractTest;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheRebalanceMode.NONE;
+
+/**
+ * Test Mvcc txs in single-threaded mode for colocated cache.
+ */
+public class GridCacheColocatedMvccTxSingleThreadedSelfTest extends IgniteMvccTxSingleThreadedAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<?, ?> ccfg = defaultCacheConfiguration();
+
+        ccfg.setCacheMode(PARTITIONED);
+        ccfg.setBackups(1);
+        ccfg.setNearConfiguration(null);
+        ccfg.setAtomicityMode(TRANSACTIONAL_SNAPSHOT);
+
+        ccfg.setEvictionPolicy(null);
+
+        ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_ASYNC);
+
+        ccfg.setRebalanceMode(NONE);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 4;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int keyCount() {
+        return 3;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int maxKeyValue() {
+        return 3;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int iterations() {
+        return 3000;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean isTestDebug() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean printMemoryStats() {
+        return true;
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEntrySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEntrySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEntrySelfTest.java
index 5aa6510..853763e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEntrySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEntrySelfTest.java
@@ -37,6 +37,7 @@ import org.apache.ignite.lang.IgniteBiTuple;
 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.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 
@@ -78,6 +79,10 @@ public class GridCacheDhtEntrySelfTest extends GridCommonAbstractTest {
 
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.NEAR_CACHE);
+
+        super.beforeTestsStarted();
+
         startGridsMultiThreaded(GRID_CNT);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtMappingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtMappingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtMappingSelfTest.java
index bd0af34..5456e4c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtMappingSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtMappingSelfTest.java
@@ -27,6 +27,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCach
 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.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
@@ -44,6 +45,13 @@ public class GridCacheDhtMappingSelfTest extends GridCommonAbstractTest {
     private TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
     /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.NEAR_CACHE);
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDisabledSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDisabledSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDisabledSelfTest.java
index 66efc18..d8bc44a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDisabledSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDisabledSelfTest.java
@@ -40,6 +40,7 @@ import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
@@ -73,6 +74,13 @@ public class GridCacheDhtPreloadDisabledSelfTest extends GridCommonAbstractTest
     /** IP finder. */
     private TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.NEAR_CACHE);
+
+        super.beforeTestsStarted();
+    }
+
     /**
      *
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java
index 6c32a67..5f7be7a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java
@@ -33,6 +33,7 @@ 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.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jetbrains.annotations.Nullable;
 
@@ -137,6 +138,8 @@ public class GridCacheDhtPreloadMultiThreadedSelfTest extends GridCommonAbstract
      * @throws Exception If failed.
      */
     public void testConcurrentNodesStartStop() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.LOCAL_CACHE);
+
         try {
             multithreadedAsync(
                 new Callable<Object>() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadPutGetSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadPutGetSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadPutGetSelfTest.java
index 71911e8..a62984a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadPutGetSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadPutGetSelfTest.java
@@ -33,6 +33,7 @@ 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.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jetbrains.annotations.Nullable;
 
@@ -163,6 +164,9 @@ public class GridCacheDhtPreloadPutGetSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testPutGetNone1() throws Exception {
+        if (MvccFeatureChecker.forcedMvcc())
+            fail("https://issues.apache.org/jira/browse/IGNITE-10261");
+
         preloadMode = NONE;
         backups = 1;
 
@@ -173,6 +177,9 @@ public class GridCacheDhtPreloadPutGetSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testPutGetNone2() throws Exception {
+        if (MvccFeatureChecker.forcedMvcc())
+            fail("https://issues.apache.org/jira/browse/IGNITE-10261");
+
         preloadMode = NONE;
         backups = 2;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
index e47d775..76b1424 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
@@ -212,7 +212,6 @@ public class GridCacheDhtPreloadSelfTest extends GridCommonAbstractTest {
      */
     private void checkActivePartitionTransfer(int keyCnt, int nodeCnt, boolean sameCoord, boolean shuffle)
         throws Exception {
-
         try {
             Ignite ignite1 = startGrid(0);
 
@@ -456,7 +455,6 @@ public class GridCacheDhtPreloadSelfTest extends GridCommonAbstractTest {
      */
     private void checkNodes(int keyCnt, int nodeCnt, boolean sameCoord, boolean shuffle)
         throws Exception {
-
         try {
             Ignite ignite1 = startGrid(0);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheClearDuringRebalanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheClearDuringRebalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheClearDuringRebalanceTest.java
index 8561c5c..5194020 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheClearDuringRebalanceTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheClearDuringRebalanceTest.java
@@ -28,6 +28,7 @@ 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.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
@@ -43,6 +44,14 @@ public class IgniteCacheClearDuringRebalanceTest extends GridCommonAbstractTest
     private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
     /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        if(MvccFeatureChecker.forcedMvcc())
+            fail("https://issues.apache.org/jira/browse/IGNITE-7952");
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         super.afterTest();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePartitionedBackupNodeFailureRecoveryTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePartitionedBackupNodeFailureRecoveryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePartitionedBackupNodeFailureRecoveryTest.java
index 98520ab..6cecce1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePartitionedBackupNodeFailureRecoveryTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePartitionedBackupNodeFailureRecoveryTest.java
@@ -44,7 +44,7 @@ import static org.apache.ignite.testframework.GridTestUtils.waitForCondition;
 /**
  */
 public class IgniteCachePartitionedBackupNodeFailureRecoveryTest extends IgniteCacheAbstractTest {
-    /** {@inheritDoc}*/
+  /** {@inheritDoc}*/
     @Override protected int gridCount() {
         return 3;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearClientHitTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearClientHitTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearClientHitTest.java
index fd378b7..6663a67 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearClientHitTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearClientHitTest.java
@@ -71,14 +71,11 @@ public class GridCacheNearClientHitTest extends GridCommonAbstractTest {
         CacheConfiguration<Object, Object> cfg = new CacheConfiguration<>();
 
         cfg.setAtomicityMode(CacheAtomicityMode.ATOMIC);
-
         cfg.setCacheMode(CacheMode.PARTITIONED);
-
         cfg.setBackups(1);
-
         cfg.setCopyOnRead(false);
-
         cfg.setName(CACHE_NAME);
+        cfg.setNearConfiguration(new NearCacheConfiguration<>());
 
         return cfg;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMultiGetSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMultiGetSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMultiGetSelfTest.java
index 3e13ccd..4633161 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMultiGetSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMultiGetSelfTest.java
@@ -20,12 +20,13 @@ package org.apache.ignite.internal.processors.cache.distributed.near;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cache.affinity.Affinity;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.IgniteEx;
-import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
@@ -35,9 +36,9 @@ import org.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionConcurrency;
 import org.apache.ignite.transactions.TransactionIsolation;
 import org.apache.ignite.transactions.TransactionOptimisticException;
-import org.apache.log4j.Level;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheRebalanceMode.NONE;
 import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC;
@@ -50,9 +51,6 @@ import static org.apache.ignite.transactions.TransactionIsolation.SERIALIZABLE;
  * Test getting the same value twice within the same transaction.
  */
 public class GridCacheNearMultiGetSelfTest extends GridCommonAbstractTest {
-    /** Cache debug flag. */
-    private static final boolean CACHE_DEBUG = false;
-
     /** Number of gets. */
     private static final int GET_CNT = 5;
 
@@ -62,40 +60,47 @@ public class GridCacheNearMultiGetSelfTest extends GridCommonAbstractTest {
     /** */
     private TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
+    /** */
+    private CacheAtomicityMode atomicityMode;
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration c = super.getConfiguration(igniteInstanceName);
 
-        c.getTransactionConfiguration().setTxSerializableEnabled(true);
-
-        CacheConfiguration cc = defaultCacheConfiguration();
-
-        cc.setCacheMode(PARTITIONED);
-        cc.setBackups(1);
-        cc.setAtomicityMode(TRANSACTIONAL);
-
-        cc.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-
-        cc.setRebalanceMode(NONE);
-
         TcpDiscoverySpi spi = new TcpDiscoverySpi();
 
         spi.setIpFinder(ipFinder);
 
         c.setDiscoverySpi(spi);
 
-        c.setCacheConfiguration(cc);
+        return c;
+    }
 
-        if (CACHE_DEBUG)
-            resetLog4j(Level.DEBUG, false, GridCacheProcessor.class.getPackage().getName());
+    /**
+     * @return Cache configuration.
+     */
+    private CacheConfiguration cacheConfiguration() {
+        CacheConfiguration cc = defaultCacheConfiguration();
 
-        return c;
+        cc.setCacheMode(PARTITIONED);
+        cc.setBackups(1);
+        cc.setAtomicityMode(atomicityMode);
+        cc.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        cc.setRebalanceMode(NONE);
+        cc.setNearConfiguration(new NearCacheConfiguration());
+
+        return cc;
     }
 
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
-        for (int i = 0; i < GRID_CNT; i++)
-            startGrid(i);
+        startGridsMultiThreaded(GRID_CNT);
+    }
+
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        atomicityMode = TRANSACTIONAL;
     }
 
     /** {@inheritDoc} */
@@ -110,6 +115,8 @@ public class GridCacheNearMultiGetSelfTest extends GridCommonAbstractTest {
             assertEquals("Cache size mismatch for grid [igniteInstanceName=" + g.name() +
                     ", entrySet=" + entrySet(c) + ']', 0, c.size());
         }
+
+        grid(0).destroyCache(DEFAULT_CACHE_NAME);
     }
 
     /** @return {@code True} if debug enabled. */
@@ -213,6 +220,24 @@ public class GridCacheNearMultiGetSelfTest extends GridCommonAbstractTest {
         checkDoubleGet(PESSIMISTIC, SERIALIZABLE, true);
     }
 
+    /** @throws Exception If failed. */
+    public void testMvccPessimisticRepeatableReadNoPut() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-7187");
+
+        atomicityMode = TRANSACTIONAL_SNAPSHOT;
+
+        checkDoubleGet(PESSIMISTIC, REPEATABLE_READ, false);
+    }
+
+    /** @throws Exception If failed. */
+    public void testMvccPessimisticRepeatableReadWithPut() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-7187");
+
+        atomicityMode = TRANSACTIONAL_SNAPSHOT;
+
+        checkDoubleGet(PESSIMISTIC, REPEATABLE_READ, true);
+    }
+
     /**
      * @param concurrency Concurrency.
      * @param isolation Isolation.
@@ -222,7 +247,7 @@ public class GridCacheNearMultiGetSelfTest extends GridCommonAbstractTest {
     private void checkDoubleGet(TransactionConcurrency concurrency, TransactionIsolation isolation, boolean put)
         throws Exception {
         IgniteEx ignite = grid(0);
-        IgniteCache<Integer, String> cache = ignite.cache(DEFAULT_CACHE_NAME);
+        IgniteCache<Integer, String> cache = ignite.getOrCreateCache(cacheConfiguration());
 
         Integer key = 1;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMultiNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMultiNodeSelfTest.java
index e83b14a..875284d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMultiNodeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMultiNodeSelfTest.java
@@ -56,6 +56,7 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 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.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 import org.jetbrains.annotations.Nullable;
@@ -79,6 +80,15 @@ public class GridCacheNearMultiNodeSelfTest extends GridCommonAbstractTest {
     /** Cache store. */
     private static TestStore store = new TestStore();
 
+    /** {@inheritDoc} */
+    @Override public void setUp() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.NEAR_CACHE);
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.CACHE_STORE);
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+
+        super.setUp();
+    }
+
     /** */
     private TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOneNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOneNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOneNodeSelfTest.java
index 1595350..2b4b60c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOneNodeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOneNodeSelfTest.java
@@ -26,10 +26,12 @@ import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cache.store.CacheStoreAdapter;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 
@@ -46,6 +48,13 @@ public class GridCacheNearOneNodeSelfTest extends GridCommonAbstractTest {
     /** Cache store. */
     private static TestStore store = new TestStore();
 
+    /** {@inheritDoc} */
+    @Override public void setUp() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.NEAR_CACHE);
+
+        super.setUp();
+    }
+
     /**
      *
      */
@@ -80,8 +89,8 @@ public class GridCacheNearOneNodeSelfTest extends GridCommonAbstractTest {
         cacheCfg.setCacheMode(PARTITIONED);
         cacheCfg.setBackups(1);
         cacheCfg.setAtomicityMode(TRANSACTIONAL);
-
         cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        cacheCfg.setNearConfiguration(new NearCacheConfiguration());
 
         cacheCfg.setCacheStoreFactory(singletonFactory(store));
         cacheCfg.setReadThrough(true);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearPartitionedClearSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearPartitionedClearSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearPartitionedClearSelfTest.java
index cbdc855..90a7406 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearPartitionedClearSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearPartitionedClearSelfTest.java
@@ -29,6 +29,7 @@ import org.apache.ignite.internal.util.typedef.G;
 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.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
@@ -57,6 +58,13 @@ public class GridCacheNearPartitionedClearSelfTest extends GridCommonAbstractTes
     private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
 
     /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.NEAR_CACHE);
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
         startGrids(GRID_CNT);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReaderPreloadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReaderPreloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReaderPreloadSelfTest.java
index e143260..316cc6e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReaderPreloadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReaderPreloadSelfTest.java
@@ -30,6 +30,7 @@ import org.apache.ignite.internal.util.typedef.G;
 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.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
@@ -62,6 +63,13 @@ public class GridCacheNearReaderPreloadSelfTest extends GridCommonAbstractTest {
     private IgniteCache<Integer, Integer> cache3;
 
     /** {@inheritDoc} */
+    @Override public void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.NEAR_CACHE);
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         cache1 = null;
         cache2 = null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java
index e30106c..88a757d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java
@@ -48,6 +48,7 @@ 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.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
@@ -67,12 +68,19 @@ public class GridCacheNearReadersSelfTest extends GridCommonAbstractTest {
     private TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
     /** Grid counter. */
-    private static AtomicInteger cntr = new AtomicInteger(0);
+    private AtomicInteger cntr = new AtomicInteger(0);
 
     /** Test cache affinity. */
     private GridCacheModuloAffinityFunction aff = new GridCacheModuloAffinityFunction();
 
     /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.NEAR_CACHE);
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedBasicStoreMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedBasicStoreMultiNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedBasicStoreMultiNodeSelfTest.java
index 91c9b22..73d5b5f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedBasicStoreMultiNodeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedBasicStoreMultiNodeSelfTest.java
@@ -33,6 +33,7 @@ import org.apache.ignite.internal.util.typedef.F;
 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.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 
@@ -72,6 +73,11 @@ public class GridCachePartitionedBasicStoreMultiNodeSelfTest extends GridCommonA
 
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.CACHE_STORE);
+
+        if (nearCacheConfiguration() != null)
+            MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.NEAR_CACHE);
+
         super.beforeTestsStarted();
 
         stores = Collections.synchronizedList(new ArrayList<GridCacheTestStore>());

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEventSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEventSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEventSelfTest.java
index 1c79db7..54eb32d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEventSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEventSelfTest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.processors.cache.distributed.GridCacheEventAbstractTest;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
@@ -31,6 +32,13 @@ import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
  */
 public class GridCachePartitionedEventSelfTest extends GridCacheEventAbstractTest {
     /** {@inheritDoc} */
+    @Override public void setUp() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.CACHE_EVENTS);
+
+        super.setUp();
+    }
+
+    /** {@inheritDoc} */
     @Override protected CacheConfiguration cacheConfiguration(String igniteInstanceName) throws Exception {
         CacheConfiguration cfg = defaultCacheConfiguration();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java
index 96fb8f6..28afce1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java
@@ -33,6 +33,7 @@ import org.apache.ignite.lang.IgniteFuture;
 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.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
@@ -51,6 +52,13 @@ public class GridCachePartitionedExplicitLockNodeFailureSelfTest extends GridCom
     public static final int GRID_CNT = 4;
 
     /** {@inheritDoc} */
+    @Override public void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration c = super.getConfiguration(igniteInstanceName);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedLoadCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedLoadCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedLoadCacheSelfTest.java
index 9474950..2ad8b50 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedLoadCacheSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedLoadCacheSelfTest.java
@@ -29,6 +29,7 @@ import org.apache.ignite.lang.IgniteBiInClosure;
 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.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jetbrains.annotations.Nullable;
 
@@ -49,6 +50,13 @@ public class GridCachePartitionedLoadCacheSelfTest extends GridCommonAbstractTes
     private static final int PUT_CNT = 100;
 
     /** {@inheritDoc} */
+    @Override public void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.CACHE_STORE);
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMvccTxMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMvccTxMultiThreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMvccTxMultiThreadedSelfTest.java
new file mode 100644
index 0000000..184132e
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMvccTxMultiThreadedSelfTest.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed.near;
+
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.internal.processors.cache.IgniteMvccTxMultiThreadedAbstractTest;
+import org.apache.ignite.testframework.MvccFeatureChecker;
+
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+
+/**
+ * Tests for partitioned cache transactions.
+ */
+public class GridCachePartitionedMvccTxMultiThreadedSelfTest extends IgniteMvccTxMultiThreadedAbstractTest {
+    @Override protected void beforeTestsStarted() throws Exception {
+        if (nearEnabled())
+            MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.NEAR_CACHE);
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<?, ?> ccfg = defaultCacheConfiguration();
+
+        ccfg.setCacheMode(PARTITIONED);
+        ccfg.setBackups(1);
+
+        ccfg.setEvictionPolicy(null);
+
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+
+        ccfg.setNearConfiguration(nearEnabled() ? new NearCacheConfiguration() : null);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /**
+     * @return {@code True} if near cache is enabled.
+     */
+    protected boolean nearEnabled() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 3;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int keyCount() {
+        return 3;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int maxKeyValue() {
+        return 3;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int threadCount() {
+        return 5;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int iterations() {
+        return 1000;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean isTestDebug() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean printMemoryStats() {
+        return true;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMvccTxSingleThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMvccTxSingleThreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMvccTxSingleThreadedSelfTest.java
new file mode 100644
index 0000000..1404839
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMvccTxSingleThreadedSelfTest.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed.near;
+
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.processors.cache.IgniteMvccTxSingleThreadedAbstractTest;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheRebalanceMode.NONE;
+
+/**
+ * Tests for partitioned cache transactions.
+ */
+public class GridCachePartitionedMvccTxSingleThreadedSelfTest extends IgniteMvccTxSingleThreadedAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<?, ?> ccfg = defaultCacheConfiguration();
+
+        ccfg.setCacheMode(PARTITIONED);
+        ccfg.setBackups(1);
+        ccfg.setAtomicityMode(TRANSACTIONAL_SNAPSHOT);
+
+        ccfg.setNearConfiguration(null);
+        ccfg.setEvictionPolicy(null);
+
+        ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_ASYNC);
+
+        ccfg.setRebalanceMode(NONE);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 4;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int keyCount() {
+        return 3;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int maxKeyValue() {
+        return 3;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int iterations() {
+        return 3000;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean isTestDebug() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean printMemoryStats() {
+        return true;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMvccTxTimeoutSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMvccTxTimeoutSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMvccTxTimeoutSelfTest.java
new file mode 100644
index 0000000..aad95aa
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMvccTxTimeoutSelfTest.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed.near;
+
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.processors.cache.distributed.IgniteMvccTxTimeoutAbstractTest;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+
+/**
+ * Simple cache test.
+ */
+public class GridCachePartitionedMvccTxTimeoutSelfTest extends IgniteMvccTxTimeoutAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration c = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration cc = defaultCacheConfiguration();
+
+        cc.setCacheMode(PARTITIONED);
+        cc.setBackups(1);
+        cc.setAtomicityMode(TRANSACTIONAL);
+
+        //cacheCfg.setPreloadMode(NONE);
+
+        c.setCacheConfiguration(cc);
+
+        return c;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedPreloadLifecycleSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedPreloadLifecycleSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedPreloadLifecycleSelfTest.java
index da802e3..7f25cba 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedPreloadLifecycleSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedPreloadLifecycleSelfTest.java
@@ -56,7 +56,6 @@ public class GridCachePartitionedPreloadLifecycleSelfTest extends GridCachePrelo
         cc1.setRebalanceMode(preloadMode);
         cc1.setEvictionPolicy(null);
         cc1.setCacheStoreFactory(null);
-        cc1.setEvictionPolicy(null);
 
         // Identical configuration.
         CacheConfiguration cc2 = new CacheConfiguration(cc1);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxTimeoutSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxTimeoutSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxTimeoutSelfTest.java
index 202476e..1a88ae0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxTimeoutSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxTimeoutSelfTest.java
@@ -19,11 +19,7 @@ package org.apache.ignite.internal.processors.cache.distributed.near;
 
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.configuration.TransactionConfiguration;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteTxTimeoutAbstractTest;
-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 static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
@@ -32,27 +28,10 @@ import static org.apache.ignite.cache.CacheMode.PARTITIONED;
  * Simple cache test.
  */
 public class GridCachePartitionedTxTimeoutSelfTest extends IgniteTxTimeoutAbstractTest {
-    /** Transaction timeout. */
-    private static final long TIMEOUT = 50;
-
-    /** */
-    private TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
-
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration c = super.getConfiguration(igniteInstanceName);
 
-        TransactionConfiguration txCfg = c.getTransactionConfiguration();
-
-        txCfg.setTxSerializableEnabled(true);
-        txCfg.setDefaultTxTimeout(TIMEOUT);
-
-        TcpDiscoverySpi spi = new TcpDiscoverySpi();
-
-        spi.setIpFinder(ipFinder);
-
-        c.setDiscoverySpi(spi);
-
         CacheConfiguration cc = defaultCacheConfiguration();
 
         cc.setCacheMode(PARTITIONED);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheStoreUpdateTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheStoreUpdateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheStoreUpdateTest.java
index bf8ad78..02a9223 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheStoreUpdateTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheStoreUpdateTest.java
@@ -39,6 +39,7 @@ import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.TransactionConcurrency;
 import org.apache.ignite.transactions.TransactionIsolation;
@@ -60,6 +61,14 @@ public class GridNearCacheStoreUpdateTest extends GridCommonAbstractTest {
     private IgniteCache<String, String> cache;
 
     /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.NEAR_CACHE);
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.CACHE_STORE);
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(final String gridName) throws Exception {
         final IgniteConfiguration cfg = super.getConfiguration(gridName);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/NearCacheSyncUpdateTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/NearCacheSyncUpdateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/NearCacheSyncUpdateTest.java
index d253013..e1ca01f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/NearCacheSyncUpdateTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/NearCacheSyncUpdateTest.java
@@ -34,6 +34,7 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 
@@ -60,6 +61,13 @@ public class NearCacheSyncUpdateTest extends GridCommonAbstractTest {
         startGridsMultiThreaded(3);
     }
 
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
     /**
      * @throws Exception If failed.
      */
@@ -75,6 +83,15 @@ public class NearCacheSyncUpdateTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testNearCacheSyncUpdateMvccTx() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-7187");
+
+        nearCacheSyncUpdateTx(TRANSACTIONAL_SNAPSHOT);
+    }
+
+    /**
      * @param atomicityMode Atomicity mode.
      * @throws Exception If failed.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/NoneRebalanceModeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/NoneRebalanceModeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/NoneRebalanceModeSelfTest.java
index c1123cb..109dbff 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/NoneRebalanceModeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/NoneRebalanceModeSelfTest.java
@@ -45,9 +45,18 @@ public class NoneRebalanceModeSelfTest extends GridCommonAbstractTest {
 
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
         startGrid(0);
     }
 
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
     /**
      * @throws Exception If failed.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedTxTimeoutSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedTxTimeoutSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedTxTimeoutSelfTest.java
index a63a302..9a748ba 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedTxTimeoutSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedTxTimeoutSelfTest.java
@@ -20,9 +20,6 @@ package org.apache.ignite.internal.processors.cache.distributed.replicated;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteTxTimeoutAbstractTest;
-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 static org.apache.ignite.cache.CacheMode.REPLICATED;
 
@@ -30,31 +27,16 @@ import static org.apache.ignite.cache.CacheMode.REPLICATED;
  * Simple cache test.
  */
 public class GridCacheReplicatedTxTimeoutSelfTest extends IgniteTxTimeoutAbstractTest {
-    /** Transaction timeout. */
-    private static final long TIMEOUT = 50;
-
-    /** */
-    private TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
-
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration c = super.getConfiguration(igniteInstanceName);
 
-        c.getTransactionConfiguration().setDefaultTxTimeout(TIMEOUT);
-        c.getTransactionConfiguration().setTxSerializableEnabled(true);
-
         CacheConfiguration cc = defaultCacheConfiguration();
 
         cc.setCacheMode(REPLICATED);
 
         c.setCacheConfiguration(cc);
 
-        TcpDiscoverySpi spi = new TcpDiscoverySpi();
-
-        spi.setIpFinder(ipFinder);
-
-        c.setDiscoverySpi(spi);
-
         return c;
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalBasicApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalBasicApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalBasicApiSelfTest.java
index d35590f..91df599 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalBasicApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalBasicApiSelfTest.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.local;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheBasicApiAbstractTest;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 
 import static org.apache.ignite.cache.CacheMode.LOCAL;
 
@@ -28,6 +29,13 @@ import static org.apache.ignite.cache.CacheMode.LOCAL;
  */
 public class GridCacheLocalBasicApiSelfTest extends GridCacheBasicApiAbstractTest {
     /** {@inheritDoc} */
+    @Override public void setUp() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.LOCAL_CACHE);
+
+        super.setUp();
+    }
+
+    /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalBasicStoreSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalBasicStoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalBasicStoreSelfTest.java
index 183b456..5d6492b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalBasicStoreSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalBasicStoreSelfTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.local;
 
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.internal.processors.cache.GridCacheBasicStoreAbstractTest;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 
 import static org.apache.ignite.cache.CacheMode.LOCAL;
 
@@ -27,6 +28,13 @@ import static org.apache.ignite.cache.CacheMode.LOCAL;
  */
 public class GridCacheLocalBasicStoreSelfTest extends GridCacheBasicStoreAbstractTest {
     /** {@inheritDoc} */
+    @Override public void setUp() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.LOCAL_CACHE);
+
+        super.setUp();
+    }
+
+    /** {@inheritDoc} */
     @Override protected CacheMode cacheMode() {
         return LOCAL;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalEventSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalEventSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalEventSelfTest.java
index 0bf01c2..c0b6071 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalEventSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalEventSelfTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.local;
 
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.internal.processors.cache.distributed.GridCacheEventAbstractTest;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 
 import static org.apache.ignite.cache.CacheMode.LOCAL;
 
@@ -27,6 +28,14 @@ import static org.apache.ignite.cache.CacheMode.LOCAL;
  */
 public class GridCacheLocalEventSelfTest extends GridCacheEventAbstractTest {
     /** {@inheritDoc} */
+    @Override public void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.LOCAL_CACHE);
+
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
     @Override protected CacheMode cacheMode() {
         return LOCAL;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalEvictionEventSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalEvictionEventSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalEvictionEventSelfTest.java
index b2d98c6..7406689 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalEvictionEventSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalEvictionEventSelfTest.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.local;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.internal.processors.cache.GridCacheEvictionEventAbstractTest;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 
@@ -28,6 +29,13 @@ import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
  */
 public class GridCacheLocalEvictionEventSelfTest extends GridCacheEvictionEventAbstractTest {
     /** {@inheritDoc} */
+    @Override public void setUp() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.LOCAL_CACHE);
+
+        super.setUp();
+    }
+
+    /** {@inheritDoc} */
     @Override protected CacheMode cacheMode() {
         return CacheMode.LOCAL;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalGetAndTransformStoreSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalGetAndTransformStoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalGetAndTransformStoreSelfTest.java
index 19b4b61..9752b7e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalGetAndTransformStoreSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalGetAndTransformStoreSelfTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.local;
 
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.internal.processors.cache.GridCacheGetAndTransformStoreAbstractTest;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 
 import static org.apache.ignite.cache.CacheMode.LOCAL;
 
@@ -27,6 +28,13 @@ import static org.apache.ignite.cache.CacheMode.LOCAL;
  */
 public class GridCacheLocalGetAndTransformStoreSelfTest extends GridCacheGetAndTransformStoreAbstractTest {
     /** {@inheritDoc} */
+    @Override public void setUp() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.LOCAL_CACHE);
+
+        super.setUp();
+    }
+
+    /** {@inheritDoc} */
     @Override protected CacheMode cacheMode() {
         return LOCAL;
     }