You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2018/11/29 12:16:20 UTC

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

IGNITE-10002: MVCC: Create "Cache 2" test suite for MVCC mode. This closes #5198.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/7577c877
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/7577c877
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/7577c877

Branch: refs/heads/ignite-10044
Commit: 7577c8770eaf29e21f1d46f650af1d8fbd891c42
Parents: c63a60a
Author: Andrey V. Mashenkov <an...@gmail.com>
Authored: Tue Nov 27 11:45:12 2018 +0300
Committer: Igor Seliverstov <gv...@gmail.com>
Committed: Tue Nov 27 11:45:26 2018 +0300

----------------------------------------------------------------------
 .../apache/ignite/IgniteSystemProperties.java   |   3 +
 .../cache/mvcc/MvccProcessorImpl.java           |  11 +
 .../cache/CacheConcurrentReadThroughTest.java   |   8 +
 .../cache/CacheEnumOperationsAbstractTest.java  |  18 +-
 .../cache/CrossCacheTxRandomOperationsTest.java |  23 +-
 .../cache/GridCacheBasicApiAbstractTest.java    |  27 ++
 .../cache/GridCacheBasicStoreAbstractTest.java  |   8 +
 .../GridCacheEvictionEventAbstractTest.java     |   9 +
 .../GridCacheFinishPartitionsSelfTest.java      |   8 +
 ...idCacheGetAndTransformStoreAbstractTest.java |   8 +
 .../cache/GridCacheOffheapUpdateSelfTest.java   |  17 +-
 .../cache/GridCachePartitionedGetSelfTest.java  |   6 +
 .../GridCacheVariableTopologySelfTest.java      |   9 +
 .../IgniteCacheEntryProcessorNodeJoinTest.java  | 111 ++++---
 .../cache/IgniteCacheIncrementTxTest.java       |   7 +
 .../cache/IgniteCacheNoSyncForGetTest.java      |   8 +
 .../IgniteClientCacheStartFailoverTest.java     |  37 +++
 .../IgniteMvccTxMultiThreadedAbstractTest.java  | 121 +++++++
 .../IgniteMvccTxSingleThreadedAbstractTest.java |  50 +++
 .../cache/IgniteNearClientCacheCloseTest.java   |  22 ++
 .../processors/cache/IgniteTxAbstractTest.java  |  16 +-
 .../IgniteTxMultiThreadedAbstractTest.java      |   1 -
 ...CacheLoadingConcurrentGridStartSelfTest.java |   8 +
 .../CacheLockReleaseNodeLeaveTest.java          |   8 +
 .../GridCacheAbstractPrimarySyncSelfTest.java   |   4 +
 .../GridCacheBasicOpAbstractTest.java           |   8 +
 .../distributed/GridCacheLockAbstractTest.java  |   8 +
 .../GridCacheMultiNodeAbstractTest.java         |   5 +
 .../GridCacheMultiNodeLockAbstractTest.java     |   9 +
 .../GridCacheNodeFailureAbstractTest.java       |   8 +
 ...ridCachePartitionNotLoadedEventSelfTest.java |  13 +-
 ...NearDisabledMvccTxMultiThreadedSelfTest.java |  31 ++
 ...GridCachePreloadRestartAbstractSelfTest.java |  12 +
 .../GridCacheTransformEventSelfTest.java        |  64 +++-
 ...niteCacheClientNodeChangingTopologyTest.java |   8 +
 .../IgniteCacheTxIteratorSelfTest.java          |  27 +-
 .../IgniteMvccTxTimeoutAbstractTest.java        | 146 +++++++++
 .../IgniteTxTimeoutAbstractTest.java            |  39 ++-
 ...heColocatedMvccTxSingleThreadedSelfTest.java |  85 +++++
 .../dht/GridCacheDhtEntrySelfTest.java          |   5 +
 .../dht/GridCacheDhtMappingSelfTest.java        |   8 +
 .../GridCacheDhtPreloadDisabledSelfTest.java    |   8 +
 ...ridCacheDhtPreloadMultiThreadedSelfTest.java |   3 +
 .../dht/GridCacheDhtPreloadPutGetSelfTest.java  |   7 +
 .../dht/GridCacheDhtPreloadSelfTest.java        |   2 -
 .../IgniteCacheClearDuringRebalanceTest.java    |   9 +
 ...artitionedBackupNodeFailureRecoveryTest.java |   2 +-
 .../near/GridCacheNearClientHitTest.java        |   5 +-
 .../near/GridCacheNearMultiGetSelfTest.java     |  73 +++--
 .../near/GridCacheNearMultiNodeSelfTest.java    |  10 +
 .../near/GridCacheNearOneNodeSelfTest.java      |  11 +-
 .../GridCacheNearPartitionedClearSelfTest.java  |   8 +
 .../GridCacheNearReaderPreloadSelfTest.java     |   8 +
 .../near/GridCacheNearReadersSelfTest.java      |  10 +-
 ...ePartitionedBasicStoreMultiNodeSelfTest.java |   6 +
 .../near/GridCachePartitionedEventSelfTest.java |   8 +
 ...titionedExplicitLockNodeFailureSelfTest.java |   8 +
 .../GridCachePartitionedLoadCacheSelfTest.java  |   8 +
 ...ePartitionedMvccTxMultiThreadedSelfTest.java | 101 ++++++
 ...PartitionedMvccTxSingleThreadedSelfTest.java |  84 +++++
 ...idCachePartitionedMvccTxTimeoutSelfTest.java |  47 +++
 ...achePartitionedPreloadLifecycleSelfTest.java |   1 -
 .../GridCachePartitionedTxTimeoutSelfTest.java  |  21 --
 .../near/GridNearCacheStoreUpdateTest.java      |   9 +
 .../near/NearCacheSyncUpdateTest.java           |  17 +
 .../near/NoneRebalanceModeSelfTest.java         |   9 +
 .../GridCacheReplicatedTxTimeoutSelfTest.java   |  18 --
 .../local/GridCacheLocalBasicApiSelfTest.java   |   8 +
 .../local/GridCacheLocalBasicStoreSelfTest.java |   8 +
 .../local/GridCacheLocalEventSelfTest.java      |   9 +
 .../GridCacheLocalEvictionEventSelfTest.java    |   8 +
 ...dCacheLocalGetAndTransformStoreSelfTest.java |   8 +
 .../GridCacheLocalIsolatedNodesSelfTest.java    |   8 +
 .../local/GridCacheLocalLoadAllSelfTest.java    |   8 +
 .../cache/local/GridCacheLocalLockSelfTest.java |   8 +
 .../GridCacheLocalMultithreadedSelfTest.java    |   8 +
 .../GridCacheLocalTxMultiThreadedSelfTest.java  |   9 +
 .../GridCacheLocalTxSingleThreadedSelfTest.java |   8 +
 .../local/GridCacheLocalTxTimeoutSelfTest.java  |   8 +
 .../testframework/MvccFeatureChecker.java       | 137 ++++++++
 .../testsuites/IgniteCacheMvccTestSuite2.java   | 197 ++++++++++++
 .../testsuites/IgniteCacheTestSuite2.java       | 321 ++++++++++---------
 .../testsuites/IgniteCacheTestSuite6.java       |   3 -
 83 files changed, 1949 insertions(+), 324 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index 2d27840..3c59ea1 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -1039,6 +1039,9 @@ public final class IgniteSystemProperties {
      */
     public static final String IGNITE_ALLOW_START_CACHES_IN_PARALLEL = "IGNITE_ALLOW_START_CACHES_IN_PARALLEL";
 
+    /** For test purposes only. Force Mvcc mode. */
+    public static final String IGNITE_FORCE_MVCC_MODE_IN_TESTS = "IGNITE_FORCE_MVCC_MODE_IN_TESTS";
+
     /**
      * Allows to log additional information about all restored partitions after binary and logical recovery phases.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java
index c6515c2..3ea8f4c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java
@@ -34,6 +34,7 @@ import java.util.stream.Collectors;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.DataRegionConfiguration;
@@ -106,6 +107,7 @@ import org.apache.ignite.thread.IgniteThread;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
 import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
@@ -134,6 +136,10 @@ import static org.apache.ignite.internal.processors.cache.persistence.CacheDataR
 @SuppressWarnings("serial")
 public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProcessor, DatabaseLifecycleListener {
     /** */
+    private static final boolean FORCE_MVCC =
+        IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_FORCE_MVCC_MODE_IN_TESTS, false);
+
+    /** */
     private static final IgniteProductVersion MVCC_SUPPORTED_SINCE = IgniteProductVersion.fromString("2.7.0");
 
     /** */
@@ -262,6 +268,11 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
 
     /** {@inheritDoc} */
     @Override public void preProcessCacheConfiguration(CacheConfiguration ccfg) {
+        if (FORCE_MVCC && ccfg.getAtomicityMode() == TRANSACTIONAL && !CU.isSystemCache(ccfg.getName())) {
+            ccfg.setAtomicityMode(TRANSACTIONAL_SNAPSHOT);
+            ccfg.setNearConfiguration(null);
+        }
+
         if (ccfg.getAtomicityMode() == TRANSACTIONAL_SNAPSHOT) {
             if (!mvccSupported)
                 throw new IgniteException("Cannot start MVCC transactional cache. " +

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheConcurrentReadThroughTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheConcurrentReadThroughTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheConcurrentReadThroughTest.java
index 26f54a2..1e8ed4d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheConcurrentReadThroughTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheConcurrentReadThroughTest.java
@@ -34,6 +34,7 @@ import org.apache.ignite.resources.IgniteInstanceResource;
 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;
 
 /**
@@ -50,6 +51,13 @@ public class CacheConcurrentReadThroughTest extends GridCommonAbstractTest {
     private boolean client;
 
     /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.CACHE_STORE);
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         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/CacheEnumOperationsAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsAbstractTest.java
index 148b60e..480d959 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsAbstractTest.java
@@ -32,11 +32,15 @@ import org.apache.ignite.marshaller.Marshaller;
 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.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;
+import static org.junit.Assert.fail;
 
 /**
  *
@@ -92,7 +96,19 @@ public abstract class CacheEnumOperationsAbstractTest extends GridCommonAbstract
      * @throws Exception If failed.
      */
     public void testTx() throws Exception {
-        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 1, ATOMIC);
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 1, TRANSACTIONAL);
+
+        enumOperations(ccfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMvccTx() throws Exception {
+        if (!singleNode())
+            fail("https://issues.apache.org/jira/browse/IGNITE-7187");
+
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 1, TRANSACTIONAL_SNAPSHOT);
 
         enumOperations(ccfg);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java
index cc9823b..cba3771 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java
@@ -38,6 +38,7 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 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.apache.ignite.transactions.TransactionConcurrency;
@@ -92,6 +93,9 @@ public class CrossCacheTxRandomOperationsTest extends GridCommonAbstractTest {
 
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
+        if (nearCacheEnabled())
+            MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.NEAR_CACHE);
+
         super.beforeTestsStarted();
 
         startGridsMultiThreaded(GRID_CNT - 1);
@@ -194,6 +198,13 @@ public class CrossCacheTxRandomOperationsTest extends GridCommonAbstractTest {
     private void txOperations(CacheMode cacheMode,
         CacheWriteSynchronizationMode writeSync,
         boolean crossCacheTx) throws Exception {
+        if (MvccFeatureChecker.forcedMvcc()) {
+            assert !nearCacheEnabled();
+
+            if(writeSync != CacheWriteSynchronizationMode.FULL_SYNC)
+                return;
+        }
+
         Ignite ignite = ignite(0);
 
         try {
@@ -203,12 +214,14 @@ public class CrossCacheTxRandomOperationsTest extends GridCommonAbstractTest {
             txOperations(PESSIMISTIC, REPEATABLE_READ, crossCacheTx, false);
             txOperations(PESSIMISTIC, REPEATABLE_READ, crossCacheTx, true);
 
-            txOperations(OPTIMISTIC, REPEATABLE_READ, crossCacheTx, false);
-            txOperations(OPTIMISTIC, REPEATABLE_READ, crossCacheTx, true);
+            if(!MvccFeatureChecker.forcedMvcc()) {
+                txOperations(OPTIMISTIC, REPEATABLE_READ, crossCacheTx, false);
+                txOperations(OPTIMISTIC, REPEATABLE_READ, crossCacheTx, true);
 
-            if (writeSync == FULL_SYNC) {
-                txOperations(OPTIMISTIC, SERIALIZABLE, crossCacheTx, false);
-                txOperations(OPTIMISTIC, SERIALIZABLE, crossCacheTx, true);
+                if (writeSync == FULL_SYNC) {
+                    txOperations(OPTIMISTIC, SERIALIZABLE, crossCacheTx, false);
+                    txOperations(OPTIMISTIC, SERIALIZABLE, crossCacheTx, true);
+                }
             }
         }
         finally {

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicApiAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicApiAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicApiAbstractTest.java
index f766d01..0b7de6e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicApiAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicApiAbstractTest.java
@@ -38,6 +38,7 @@ import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestThread;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jetbrains.annotations.Nullable;
 
@@ -89,6 +90,8 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
      * @throws Exception If test failed.
      */
     public void testBasicLock() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+
         IgniteCache<Integer, String> cache = ignite.cache(DEFAULT_CACHE_NAME);
 
         Lock lock = cache.lock(1);
@@ -106,6 +109,8 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
      * @throws IgniteCheckedException If test failed.
      */
     public void testSingleLockReentry() throws IgniteCheckedException {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+
         IgniteCache<Integer, String> cache = ignite.cache(DEFAULT_CACHE_NAME);
 
         Lock lock = cache.lock(1);
@@ -134,6 +139,8 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
      * @throws Exception If test failed.
      */
     public void testReentry() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+
         IgniteCache<Integer, String> cache = ignite.cache(DEFAULT_CACHE_NAME);
 
         Lock lock = cache.lock(1);
@@ -173,6 +180,8 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
      *
      */
     public void testInterruptLock() throws InterruptedException {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+
         final IgniteCache<Integer, String> cache = ignite.cache(DEFAULT_CACHE_NAME);
 
         final Lock lock = cache.lock(1);
@@ -217,6 +226,8 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
      *
      */
     public void testInterruptLockWithTimeout() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+
         final IgniteCache<Integer, String> cache = ignite.cache(DEFAULT_CACHE_NAME);
 
         startGrid(1);
@@ -276,6 +287,8 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
      * @throws IgniteCheckedException If test failed.
      */
     public void testManyLockReentries() throws IgniteCheckedException {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+
         IgniteCache<Integer, String> cache = ignite.cache(DEFAULT_CACHE_NAME);
 
         Integer key = 1;
@@ -319,6 +332,8 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
      * @throws Exception If test failed.
      */
     public void testLockMultithreaded() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+
         final IgniteCache<Integer, String> cache = ignite.cache(DEFAULT_CACHE_NAME);
 
         final CountDownLatch l1 = new CountDownLatch(1);
@@ -437,6 +452,9 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
      * @throws Exception If error occur.
      */
     public void testBasicOps() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.CACHE_EVENTS);
+
         IgniteCache<Integer, String> cache = ignite.cache(DEFAULT_CACHE_NAME);
 
         CountDownLatch latch = new CountDownLatch(1);
@@ -498,6 +516,9 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
      * @throws Exception If error occur.
      */
     public void testBasicOpsWithReentry() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.CACHE_EVENTS);
+
         IgniteCache<Integer, String> cache = ignite.cache(DEFAULT_CACHE_NAME);
 
         int key = (int)System.currentTimeMillis();
@@ -570,6 +591,8 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
      * @throws Exception If test failed.
      */
     public void testMultiLocks() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+
         IgniteCache<Integer, String> cache = ignite.cache(DEFAULT_CACHE_NAME);
 
         Collection<Integer> keys = Arrays.asList(1, 2, 3);
@@ -625,6 +648,10 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
      * @throws Exception In case of error.
      */
     public void testPutWithExpiration() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.EXPIRATION);
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.CACHE_EVENTS);
+
         IgniteCache<Integer, String> cache = ignite.cache(DEFAULT_CACHE_NAME);
 
         CacheEventListener lsnr = new CacheEventListener(new CountDownLatch(1));

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicStoreAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicStoreAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicStoreAbstractTest.java
index 1ae8be2..3ebb398 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicStoreAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicStoreAbstractTest.java
@@ -33,6 +33,7 @@ import org.apache.ignite.internal.util.typedef.P2;
 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;
@@ -53,6 +54,13 @@ public abstract class GridCacheBasicStoreAbstractTest extends GridCommonAbstract
     /** Cache store. */
     private static final GridCacheTestStore store = new GridCacheTestStore();
 
+    /** {@inheritDoc} */
+    @Override public void setUp() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.CACHE_STORE);
+
+        super.setUp();
+    }
+
     /**
      *
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionEventAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionEventAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionEventAbstractTest.java
index 554a7a9..b2c87a9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionEventAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionEventAbstractTest.java
@@ -35,6 +35,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.events.EventType.EVT_CACHE_ENTRY_EVICTED;
@@ -49,6 +50,14 @@ public abstract class GridCacheEvictionEventAbstractTest extends GridCommonAbstr
     /** */
     private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
+    /** {@inheritDoc} */
+    @Override public void setUp() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.EVICTION);
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.CACHE_EVENTS);
+
+        super.setUp();
+    }
+
     /**
      *
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFinishPartitionsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFinishPartitionsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFinishPartitionsSelfTest.java
index 9732272..d60e4c6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFinishPartitionsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFinishPartitionsSelfTest.java
@@ -34,6 +34,7 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.util.typedef.CI1;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.transactions.Transaction;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
@@ -57,6 +58,13 @@ public class GridCacheFinishPartitionsSelfTest extends GridCacheAbstractSelfTest
     }
 
     /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
         grid = (IgniteKernal)grid(0);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheGetAndTransformStoreAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheGetAndTransformStoreAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheGetAndTransformStoreAbstractTest.java
index f140945..af9751f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheGetAndTransformStoreAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheGetAndTransformStoreAbstractTest.java
@@ -32,6 +32,7 @@ import org.apache.ignite.internal.IgniteInternalFuture;
 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;
@@ -48,6 +49,13 @@ public abstract class GridCacheGetAndTransformStoreAbstractTest extends GridComm
     /** Cache store. */
     private static final GridCacheTestStore store = new GridCacheTestStore();
 
+    /** {@inheritDoc} */
+    @Override public void setUp() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.CACHE_STORE);
+
+        super.setUp();
+    }
+
     /**
      *
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapUpdateSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapUpdateSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapUpdateSelfTest.java
index b8f6858..cf16417 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapUpdateSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapUpdateSelfTest.java
@@ -127,12 +127,19 @@ public class GridCacheOffheapUpdateSelfTest extends GridCommonAbstractTest {
 
             assertEquals(10, cache.get(key));
 
-            try (Transaction ignored = grid.transactions().txStart(OPTIMISTIC, REPEATABLE_READ)) {
-                assertEquals(10, cache.get(key));
+            if(((IgniteCacheProxy)cache).context().config().getAtomicityMode() != CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT) {
+                try (Transaction ignored = grid.transactions().txStart(OPTIMISTIC, REPEATABLE_READ)) {
+                    assertEquals(10, cache.get(key));
+                }
+
+                try (Transaction ignored = grid.transactions().txStart(PESSIMISTIC, READ_COMMITTED)) {
+                    assertEquals(10, cache.get(key));
+                }
             }
-
-            try (Transaction ignored = grid.transactions().txStart(PESSIMISTIC, READ_COMMITTED)) {
-                assertEquals(10, cache.get(key));
+            else {
+                try (Transaction ignored = grid.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                    assertEquals(10, cache.get(key));
+                }
             }
         }
         finally {

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedGetSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedGetSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedGetSelfTest.java
index 1ed5a12..f4c32ee 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedGetSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedGetSelfTest.java
@@ -33,6 +33,7 @@ import org.apache.ignite.spi.discovery.DiscoverySpi;
 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.CacheMode.PARTITIONED;
@@ -129,6 +130,11 @@ public class GridCachePartitionedGetSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testGetFromBackupNode() throws Exception {
+        if (MvccFeatureChecker.forcedMvcc())
+            fail("https://issues.apache.org/jira/browse/IGNITE-10274");
+
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.EVICTION);
+
         for (int i = 0; i < GRID_CNT; i++) {
             IgniteCache<String, Integer> c = grid(i).cache(DEFAULT_CACHE_NAME);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheVariableTopologySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheVariableTopologySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheVariableTopologySelfTest.java
index 25817a1..b9f7a35 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheVariableTopologySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheVariableTopologySelfTest.java
@@ -35,6 +35,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.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionRollbackException;
@@ -52,6 +53,14 @@ public class GridCacheVariableTopologySelfTest extends GridCommonAbstractTest {
     /** */
     private TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
+    /** {@inheritDoc} */
+    @Override public void setUp() throws Exception {
+        if (MvccFeatureChecker.forcedMvcc())
+            fail("https://issues.apache.org/jira/browse/IGNITE-7388");
+
+        super.setUp();
+    }
+
     /** Constructs test. */
     public GridCacheVariableTopologySelfTest() {
         super(/* don't start grid */ false);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorNodeJoinTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorNodeJoinTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorNodeJoinTest.java
index d6afb49..5f99b55 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorNodeJoinTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorNodeJoinTest.java
@@ -43,9 +43,11 @@ 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.GridTestUtils.SF;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jetbrains.annotations.Nullable;
 
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
@@ -115,7 +117,7 @@ public class IgniteCacheEntryProcessorNodeJoinTest extends GridCommonAbstractTes
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        startGrids(GRID_CNT);
+        startGridsMultiThreaded(GRID_CNT, true);
     }
 
     /** {@inheritDoc} */
@@ -141,10 +143,13 @@ public class IgniteCacheEntryProcessorNodeJoinTest extends GridCommonAbstractTes
      * @throws Exception If failed.
      */
     public void testEntryProcessorNodeLeave() throws Exception {
+        if (MvccFeatureChecker.forcedMvcc())
+            fail("https://issues.apache.org/jira/browse/IGNITE-10254");
+
         startGrid(GRID_CNT);
 
         // TODO: IGNITE-1525 (test fails with one-phase commit).
-        boolean createCache = atomicityMode() == TRANSACTIONAL;
+        boolean createCache = atomicityMode() != ATOMIC;
 
         String cacheName = DEFAULT_CACHE_NAME;
 
@@ -212,32 +217,34 @@ public class IgniteCacheEntryProcessorNodeJoinTest extends GridCommonAbstractTes
      * @throws Exception If failed.
      */
     private void checkEntryProcessorNodeJoin(boolean invokeAll) throws Exception {
+        if (MvccFeatureChecker.forcedMvcc())
+            fail("https://issues.apache.org/jira/browse/IGNITE-10391");
+
         final AtomicBoolean stop = new AtomicBoolean();
         final AtomicReference<Throwable> error = new AtomicReference<>();
         final int started = 6;
 
-        try {
-            IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
-                @Override public void run() {
-                    try {
-                        for (int i = 0; i < started; i++) {
-                            U.sleep(1_000);
+        IgniteInternalFuture fut = GridTestUtils.runAsync(new Runnable() {
+            @Override public void run() {
+                try {
+                    for (int i = 0; i < started && !stop.get(); i++) {
+                        U.sleep(1_000);
 
+                        if (!stop.get())
                             startGrid(GRID_CNT + i);
-                        }
-                    }
-                    catch (Exception e) {
-                        error.compareAndSet(null, e);
                     }
                 }
-            }, 1, "starter");
+                catch (Exception e) {
+                    error.compareAndSet(null, e);
+                }
+            }
+        }, "starter");
 
+        try {
             try {
                 checkIncrement(DEFAULT_CACHE_NAME, invokeAll, null, null);
             }
             finally {
-                stop.set(true);
-
                 fut.get(getTestTimeout());
             }
 
@@ -251,8 +258,10 @@ public class IgniteCacheEntryProcessorNodeJoinTest extends GridCommonAbstractTes
             }
         }
         finally {
-            for (int i = 0; i < started; i++)
-                stopGrid(GRID_CNT + i);
+            stop.set(true);
+
+            if (!fut.isDone())
+                fut.cancel();
         }
     }
 
@@ -331,51 +340,51 @@ public class IgniteCacheEntryProcessorNodeJoinTest extends GridCommonAbstractTes
         final AtomicReference<Throwable> error = new AtomicReference<>();
         final int started = 6;
 
-        try {
-            int keys = 100;
+        int keys = 100;
 
-            final AtomicBoolean done = new AtomicBoolean(false);
+        final AtomicBoolean stop = new AtomicBoolean(false);
 
-            for (int i = 0; i < keys; i++)
-                ignite(0).cache(DEFAULT_CACHE_NAME).put(i, 0);
+        for (int i = 0; i < keys; i++)
+            ignite(0).cache(DEFAULT_CACHE_NAME).put(i, 0);
 
-            IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
-                @Override public void run() {
-                    try {
-                        for (int i = 0; i < started; i++) {
-                            U.sleep(1_000);
+        IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
+            @Override public void run() {
+                try {
+                    for (int i = 0; i < started && !stop.get(); i++) {
+                        U.sleep(1_000);
 
-                            IgniteEx grid = startGrid(GRID_CNT + i);
+                        if (stop.get())
+                            continue;
 
-                            info("Test started grid [idx=" + (GRID_CNT + i) + ", nodeId=" + grid.localNode().id() + ']');
-                        }
-                    }
-                    catch (Exception e) {
-                        error.compareAndSet(null, e);
-                    }
-                    finally {
-                        done.set(true);
+                        IgniteEx grid = startGrid(GRID_CNT + i);
+
+                        info("Test started grid [idx=" + (GRID_CNT + i) + ", nodeId=" + grid.localNode().id() + ']');
                     }
                 }
-            }, 1, "starter");
+                catch (Exception e) {
+                    error.compareAndSet(null, e);
+                }
+                finally {
+                    stop.set(true);
+                }
+            }
+        }, 1, "starter");
 
+        try {
             int updVal = 0;
 
-            try {
-                while (!done.get()) {
-                    info("Will put: " + (updVal + 1));
+            while (!stop.get()) {
+                info("Will put: " + (updVal + 1));
 
-                    for (int i = 0; i < keys; i++)
-                        assertTrue("Failed [key=" + i + ", oldVal=" + updVal + ']',
-                            ignite(0).cache(DEFAULT_CACHE_NAME).replace(i, updVal, updVal + 1));
+                for (int i = 0; i < keys; i++)
+                    assertTrue("Failed [key=" + i + ", oldVal=" + updVal + ']',
+                        ignite(0).cache(DEFAULT_CACHE_NAME).replace(i, updVal, updVal + 1));
 
-                    updVal++;
-                }
-            }
-            finally {
-                fut.get(getTestTimeout());
+                updVal++;
             }
 
+            fut.get(getTestTimeout());
+
             for (int i = 0; i < keys; i++) {
                 for (int g = 0; g < GRID_CNT + started; g++) {
                     Integer val = ignite(g).<Integer, Integer>cache(DEFAULT_CACHE_NAME).get(i);
@@ -390,8 +399,10 @@ public class IgniteCacheEntryProcessorNodeJoinTest extends GridCommonAbstractTes
             }
         }
         finally {
-            for (int i = 0; i < started; i++)
-                stopGrid(GRID_CNT + i);
+            stop.set(true);
+
+            if (!fut.isDone())
+                fut.cancel();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheIncrementTxTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheIncrementTxTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheIncrementTxTest.java
index b3a5055..918d655 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheIncrementTxTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheIncrementTxTest.java
@@ -36,6 +36,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.apache.ignite.transactions.Transaction;
 
@@ -88,6 +89,9 @@ public class IgniteCacheIncrementTxTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testIncrementTxTopologyChange1() throws Exception {
+        if (MvccFeatureChecker.forcedMvcc())
+            fail("https://issues.apache.org/jira/browse/IGNITE-10264");
+
         nodeJoin(cacheConfiguration(1));
     }
 
@@ -95,6 +99,9 @@ public class IgniteCacheIncrementTxTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testIncrementTxTopologyChange2() throws Exception {
+        if (MvccFeatureChecker.forcedMvcc())
+            fail("https://issues.apache.org/jira/browse/IGNITE-10264");
+
         nodeJoin(cacheConfiguration(2));
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNoSyncForGetTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNoSyncForGetTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNoSyncForGetTest.java
index b14fecc..6324df4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNoSyncForGetTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNoSyncForGetTest.java
@@ -47,6 +47,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.CacheWriteSynchronizationMode.FULL_SYNC;
 
 /**
@@ -103,6 +104,13 @@ public class IgniteCacheNoSyncForGetTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testMvccTxGet() throws Exception {
+        getTest(TRANSACTIONAL_SNAPSHOT);
+    }
+
+    /**
      * @param atomicityMode Cache 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/IgniteClientCacheStartFailoverTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheStartFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheStartFailoverTest.java
index a2d9da7..fa7d0a8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheStartFailoverTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheStartFailoverTest.java
@@ -56,6 +56,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.CacheWriteSynchronizationMode.FULL_SYNC;
 
 /**
@@ -82,6 +83,13 @@ public class IgniteClientCacheStartFailoverTest extends GridCommonAbstractTest {
     }
 
     /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        client = false;
+    }
+
+    /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         stopAllGrids();
 
@@ -103,6 +111,13 @@ public class IgniteClientCacheStartFailoverTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testClientStartCoordinatorFailsMvccTx() throws Exception {
+        clientStartCoordinatorFails(TRANSACTIONAL_SNAPSHOT);
+    }
+
+    /**
      * @param atomicityMode Cache atomicity mode.
      * @throws Exception If failed.
      */
@@ -164,6 +179,16 @@ public class IgniteClientCacheStartFailoverTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testClientStartLastServerFailsMvccTx() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-10262");
+
+        clientStartLastServerFails(TRANSACTIONAL_SNAPSHOT);
+    }
+
+
+    /**
      * @param atomicityMode Cache atomicity mode.
      * @throws Exception If failed.
      */
@@ -547,6 +572,18 @@ public class IgniteClientCacheStartFailoverTest extends GridCommonAbstractTest {
             cache.putAll(map);
         }
 
+        //TODO: uncomment TRANSACTIONAL_SNAPSHOT cache creation when IGNITE-9470 will be fixed.
+       /* for (int i = 0; i < 3; i++) {
+            CacheConfiguration<Object, Object> ccfg = cacheConfiguration("mvcc-" + i, TRANSACTIONAL_SNAPSHOT, i);
+
+            IgniteCache<Object, Object> cache = node.createCache(ccfg);
+
+            cacheNames.add(ccfg.getName());
+
+            cache.putAll(map);
+        }*/
+
+
         return cacheNames;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMvccTxMultiThreadedAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMvccTxMultiThreadedAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMvccTxMultiThreadedAbstractTest.java
new file mode 100644
index 0000000..9e4609b
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMvccTxMultiThreadedAbstractTest.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionIsolation;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+
+/**
+ * Tests for local transactions.
+ */
+public abstract class IgniteMvccTxMultiThreadedAbstractTest extends IgniteTxAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9470");
+
+        super.beforeTestsStarted();
+    }
+
+    /**
+     * @return Thread count.
+     */
+    protected abstract int threadCount();
+
+    /**
+     /**
+     * @throws IgniteCheckedException If test failed.
+     */
+    public void testPessimisticRepeatableReadCommitMultithreaded() throws Exception {
+        checkCommitMultithreaded(PESSIMISTIC, REPEATABLE_READ);
+
+        finalChecks();
+    }
+
+    /**
+     * @throws IgniteCheckedException If test failed.
+     */
+    public void testPessimisticRepeatableReadRollbackMultithreaded() throws Exception {
+        checkRollbackMultithreaded(PESSIMISTIC, REPEATABLE_READ);
+
+        finalChecks();
+    }
+
+    /**
+     * @param concurrency Concurrency.
+     * @param isolation Isolation.
+     * @throws Exception If check failed.
+     */
+    protected void checkCommitMultithreaded(final TransactionConcurrency concurrency,
+        final TransactionIsolation isolation) throws Exception {
+        GridTestUtils.runMultiThreaded(new Callable<Object>() {
+            @Nullable @Override public Object call() throws Exception {
+                Thread t = Thread.currentThread();
+
+                t.setName(t.getName() + "-id-" + t.getId());
+
+                info("Starting commit thread: " + Thread.currentThread().getName());
+
+                try {
+                    checkCommit(concurrency, isolation);
+                }
+                finally {
+                    info("Finished commit thread: " + Thread.currentThread().getName());
+                }
+
+                return null;
+            }
+        }, threadCount(), concurrency + "-" + isolation);
+    }
+
+    /**
+     * @param concurrency Concurrency.
+     * @param isolation Isolation.
+     * @throws Exception If check failed.
+     */
+    protected void checkRollbackMultithreaded(final TransactionConcurrency concurrency,
+        final TransactionIsolation isolation) throws Exception {
+        final ConcurrentMap<Integer, String> map = new ConcurrentHashMap<>();
+        GridTestUtils.runMultiThreaded(new Callable<Object>() {
+            @Nullable @Override public Object call() throws Exception {
+                Thread t = Thread.currentThread();
+
+                t.setName(t.getName() + "-id-" + t.getId());
+
+                info("Starting rollback thread: " + Thread.currentThread().getName());
+
+                try {
+                    checkRollback(map, concurrency, isolation);
+
+                    return null;
+                }
+                finally {
+                    info("Finished rollback thread: " + Thread.currentThread().getName());
+                }
+            }
+        }, threadCount(), concurrency + "-" + isolation);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMvccTxSingleThreadedAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMvccTxSingleThreadedAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMvccTxSingleThreadedAbstractTest.java
new file mode 100644
index 0000000..b9e7cee
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMvccTxSingleThreadedAbstractTest.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.IgniteCheckedException;
+
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+
+/**
+ * Tests for local transactions.
+ */
+public abstract class IgniteMvccTxSingleThreadedAbstractTest extends IgniteTxAbstractTest {
+    /**
+     * @throws IgniteCheckedException If test failed.
+     */
+    public void testPessimisticRepeatableReadCommit() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-10261");
+
+        checkCommit(PESSIMISTIC, REPEATABLE_READ);
+
+        finalChecks();
+    }
+
+    /**
+     * @throws IgniteCheckedException If test failed.
+     */
+    public void testPessimisticRepeatableReadRollback() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-10261");
+
+        checkRollback(PESSIMISTIC, REPEATABLE_READ);
+
+        finalChecks();
+    }
+}
\ 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/IgniteNearClientCacheCloseTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteNearClientCacheCloseTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteNearClientCacheCloseTest.java
index e7ab805..0ca0f3e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteNearClientCacheCloseTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteNearClientCacheCloseTest.java
@@ -37,10 +37,12 @@ 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;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 
 /**
@@ -108,6 +110,26 @@ public class IgniteNearClientCacheCloseTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testNearCacheCloseMvccTx1() throws Exception {
+        nearCacheClose(1, false, TRANSACTIONAL_SNAPSHOT);
+
+        if (MvccFeatureChecker.isSupported(MvccFeatureChecker.Feature.NEAR_CACHE))
+            nearCacheClose(1, true, TRANSACTIONAL_SNAPSHOT);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNearCacheCloseMvccTx2() throws Exception {
+        nearCacheClose(4, false, TRANSACTIONAL_SNAPSHOT);
+
+        if (MvccFeatureChecker.isSupported(MvccFeatureChecker.Feature.NEAR_CACHE))
+            nearCacheClose(4, true, TRANSACTIONAL_SNAPSHOT);
+    }
+
+    /**
      * @param srvs Number of server nodes.
      * @param srvNearCache {@code True} to enable near cache on server nodes.
      * @param atomicityMode Cache atomicity mode.

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxAbstractTest.java
index 1830db0..3ae4224 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxAbstractTest.java
@@ -57,7 +57,7 @@ abstract class IgniteTxAbstractTest extends GridCommonAbstractTest {
     private static final AtomicInteger cntr = new AtomicInteger();
 
     /** */
-    private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+    private final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
     /**
      * Start grid by default.
@@ -115,12 +115,16 @@ abstract class IgniteTxAbstractTest extends GridCommonAbstractTest {
             info(msg);
     }
 
-    /**
-     * @throws Exception If failed.
-     */
+    /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
-        for (int i = 0; i < gridCount(); i++)
-            startGrid(i);
+        startGridsMultiThreaded(gridCount());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxMultiThreadedAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxMultiThreadedAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxMultiThreadedAbstractTest.java
index 9a385a3..65dc2e7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxMultiThreadedAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxMultiThreadedAbstractTest.java
@@ -85,7 +85,6 @@ public abstract class IgniteTxMultiThreadedAbstractTest extends IgniteTxAbstract
     protected void checkRollbackMultithreaded(final TransactionConcurrency concurrency,
         final TransactionIsolation isolation) throws Exception {
         final ConcurrentMap<Integer, String> map = new ConcurrentHashMap<>();
-
         GridTestUtils.runMultiThreaded(new Callable<Object>() {
             @Nullable @Override public Object call() throws Exception {
                 Thread t = Thread.currentThread();

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java
index 5cb206f..10d04bd 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java
@@ -49,6 +49,7 @@ import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 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;
 
@@ -80,6 +81,13 @@ public class CacheLoadingConcurrentGridStartSelfTest extends GridCommonAbstractT
     protected volatile boolean restarts;
 
     /** {@inheritDoc} */
+    @Override protected 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/CacheLockReleaseNodeLeaveTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLockReleaseNodeLeaveTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLockReleaseNodeLeaveTest.java
index 852e855..844a430 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLockReleaseNodeLeaveTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLockReleaseNodeLeaveTest.java
@@ -36,6 +36,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.apache.ignite.transactions.Transaction;
 
@@ -56,6 +57,13 @@ public class CacheLockReleaseNodeLeaveTest extends GridCommonAbstractTest {
     private static final String REPLICATED_TEST_CACHE = "REPLICATED_TEST_CACHE";
 
     /** {@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 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/GridCacheAbstractPrimarySyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractPrimarySyncSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractPrimarySyncSelfTest.java
index 625cb18..f6b9302 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractPrimarySyncSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractPrimarySyncSelfTest.java
@@ -24,6 +24,7 @@ import org.apache.ignite.configuration.NearCacheConfiguration;
 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;
 
@@ -70,6 +71,9 @@ public abstract class GridCacheAbstractPrimarySyncSelfTest extends GridCommonAbs
 
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
+        if (nearConfiguration() != null)
+            MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.NEAR_CACHE);
+
         assert GRID_CNT > 1;
 
         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/GridCacheBasicOpAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheBasicOpAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheBasicOpAbstractTest.java
index a55ff2d..0398813 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheBasicOpAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheBasicOpAbstractTest.java
@@ -32,6 +32,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 org.apache.ignite.transactions.Transaction;
 
@@ -73,6 +74,11 @@ public abstract class GridCacheBasicOpAbstractTest extends GridCommonAbstractTes
 
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.CACHE_EVENTS);
+
+        if (MvccFeatureChecker.forcedMvcc())
+            fail("https://issues.apache.org/jira/browse/IGNITE-7952");
+
         startGridsMultiThreaded(3);
 
         ignite1 = grid(0);
@@ -327,6 +333,8 @@ public abstract class GridCacheBasicOpAbstractTest extends GridCommonAbstractTes
      * @throws Exception In case of error.
      */
     public void testPutWithExpiration() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.EXPIRATION);
+
         IgniteCache<String, String> cache1 = ignite1.cache(DEFAULT_CACHE_NAME);
         IgniteCache<String, String> cache2 = ignite2.cache(DEFAULT_CACHE_NAME);
         IgniteCache<String, String> cache3 = ignite3.cache(DEFAULT_CACHE_NAME);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheLockAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheLockAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheLockAbstractTest.java
index 48b9a20..5909213 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheLockAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheLockAbstractTest.java
@@ -38,6 +38,7 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestThread;
 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;
 
@@ -65,6 +66,13 @@ public abstract class GridCacheLockAbstractTest extends GridCommonAbstractTest {
     /** Ip-finder. */
     private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
+    /** {@inheritDoc} */
+    @Override public void setUp() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+
+        super.setUp();
+    }
+
     /**
      *
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultiNodeAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultiNodeAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultiNodeAbstractTest.java
index 1d56ab6..912aece 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultiNodeAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultiNodeAbstractTest.java
@@ -35,6 +35,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 java.util.concurrent.TimeUnit.MILLISECONDS;
@@ -87,6 +88,8 @@ public abstract class GridCacheMultiNodeAbstractTest extends GridCommonAbstractT
 
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.CACHE_EVENTS);
+
         ignite1 = startGrid(1);
         ignite2 = startGrid(2);
         ignite3 = startGrid(3);
@@ -228,6 +231,8 @@ public abstract class GridCacheMultiNodeAbstractTest extends GridCommonAbstractT
      * @throws Exception If test failed.
      */
     public void testLockUnlock() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+
         CacheEventListener lockLsnr1 = new CacheEventListener(ignite1, new CountDownLatch(1), EVT_CACHE_OBJECT_LOCKED);
 
         addListener(ignite1, lockLsnr1, EVT_CACHE_OBJECT_LOCKED);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultiNodeLockAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultiNodeLockAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultiNodeLockAbstractTest.java
index cc3b894..6636196 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultiNodeLockAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultiNodeLockAbstractTest.java
@@ -38,6 +38,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.GridTestThread;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jetbrains.annotations.Nullable;
 
@@ -63,6 +64,14 @@ public abstract class GridCacheMultiNodeLockAbstractTest extends GridCommonAbstr
     /** Listeners. */
     private static Collection<IgnitePredicate<Event>> lsnrs = new ArrayList<>();
 
+    /** {@inheritDoc} */
+    @Override public void setUp() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.CACHE_EVENTS);
+
+        super.setUp();
+    }
+
     /**
      *
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheNodeFailureAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheNodeFailureAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheNodeFailureAbstractTest.java
index 8de2d79..6375e3e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheNodeFailureAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheNodeFailureAbstractTest.java
@@ -37,6 +37,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 org.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionConcurrency;
@@ -75,6 +76,13 @@ public abstract class GridCacheNodeFailureAbstractTest extends GridCommonAbstrac
     /** Grid instances. */
     private static final List<Ignite> IGNITEs = new ArrayList<>();
 
+    /** {@inheritDoc} */
+    @Override public void setUp() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+
+        super.setUp();
+    }
+
     /**
      * Start grid by default.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionNotLoadedEventSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionNotLoadedEventSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionNotLoadedEventSelfTest.java
index 334608d..f5695ca 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionNotLoadedEventSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionNotLoadedEventSelfTest.java
@@ -40,6 +40,7 @@ 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.GridTestUtils.SF;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.util.TestTcpCommunicationSpi;
 
@@ -57,6 +58,13 @@ public class GridCachePartitionNotLoadedEventSelfTest extends GridCommonAbstract
     private int backupCnt;
 
     /** {@inheritDoc} */
+    @Override public void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.CACHE_EVENTS);
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
 
@@ -101,10 +109,7 @@ public class GridCachePartitionNotLoadedEventSelfTest extends GridCommonAbstract
     public void testPrimaryAndBackupDead() throws Exception {
         backupCnt = 1;
 
-        startGrid(0);
-        startGrid(1);
-        startGrid(2);
-        startGrid(3);
+        startGridsMultiThreaded(4);
 
         awaitPartitionMapExchange();
 

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePreloadRestartAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePreloadRestartAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePreloadRestartAbstractSelfTest.java
index 2ea1ed8..8ef5fb8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePreloadRestartAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePreloadRestartAbstractSelfTest.java
@@ -27,6 +27,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 static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
@@ -97,6 +98,14 @@ public abstract class GridCachePreloadRestartAbstractSelfTest extends GridCommon
     private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
     /** {@inheritDoc} */
+    @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 c = super.getConfiguration(igniteInstanceName);
 
@@ -197,6 +206,9 @@ public abstract class GridCachePreloadRestartAbstractSelfTest extends GridCommon
      * @throws Exception If failed.
      */
     public void testDisabledPreloadRestart() throws Exception {
+        if (MvccFeatureChecker.forcedMvcc())
+            fail("https://issues.apache.org/jira/browse/IGNITE-10261");
+
         preloadMode = NONE;
 
         checkRestart();

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTransformEventSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTransformEventSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTransformEventSelfTest.java
index 913321e..8cb0885 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTransformEventSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTransformEventSelfTest.java
@@ -49,6 +49,7 @@ import org.apache.ignite.transactions.TransactionIsolation;
 
 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.LOCAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
@@ -74,6 +75,9 @@ public class GridCacheTransformEventSelfTest extends GridCommonAbstractTest {
     /** Cache name. */
     private static final String CACHE_NAME = "cache";
 
+    /** IP finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
     /** Key 1. */
     private Integer key1;
 
@@ -83,9 +87,6 @@ public class GridCacheTransformEventSelfTest extends GridCommonAbstractTest {
     /** Two keys in form of a set. */
     private Set<Integer> keys;
 
-    /** IP finder. */
-    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
-
     /** Nodes. */
     private Ignite[] ignites;
 
@@ -328,6 +329,17 @@ public class GridCacheTransformEventSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * Test TRANSACTIONAL_SNAPSHOT LOCAL cache with PESSIMISTIC/REPEATABLE_READ transaction.
+     *
+     * @throws Exception If failed.
+     */
+    public void testMvccTxLocalPessimisticRepeatableRead() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9530");
+
+        checkMvccTx(LOCAL, PESSIMISTIC, REPEATABLE_READ);
+    }
+
+    /**
      * Test TRANSACTIONAL PARTITIONED cache with OPTIMISTIC/REPEATABLE_READ transaction.
      *
      * @throws Exception If failed.
@@ -382,6 +394,18 @@ public class GridCacheTransformEventSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * Test TRANSACTIONAL_SNAPSHOT PARTITIONED cache with PESSIMISTIC/REPEATABLE_READ transaction.
+     *
+     * @throws Exception If failed.
+     */
+    public void testMvccTxPartitionedPessimisticRepeatableRead() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9321");
+
+        checkMvccTx(PARTITIONED, PESSIMISTIC, REPEATABLE_READ);
+    }
+
+
+    /**
      * Test TRANSACTIONAL REPLICATED cache with OPTIMISTIC/REPEATABLE_READ transaction.
      *
      * @throws Exception If failed.
@@ -436,6 +460,17 @@ public class GridCacheTransformEventSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * Test TRANSACTIONAL_SNAPSHOT REPLICATED cache with PESSIMISTIC/REPEATABLE_READ transaction.
+     *
+     * @throws Exception If failed.
+     */
+    public void testMvccTxReplicatedPessimisticRepeatableRead() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9321");
+
+        checkMvccTx(REPLICATED, PESSIMISTIC, REPEATABLE_READ);
+    }
+
+    /**
      * Test ATOMIC LOCAL cache.
      *
      * @throws Exception If failed.
@@ -495,6 +530,21 @@ public class GridCacheTransformEventSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * Check TRANSACTIONAL_SNAPSHOT cache.
+     *
+     * @param cacheMode Cache mode.
+     * @param txConcurrency TX concurrency.
+     * @param txIsolation TX isolation.
+     * @throws Exception If failed.
+     */
+    private void checkMvccTx(CacheMode cacheMode, TransactionConcurrency txConcurrency,
+        TransactionIsolation txIsolation) throws Exception {
+        initialize(cacheMode, TRANSACTIONAL_SNAPSHOT, txConcurrency, txIsolation);
+
+        checkTx0();
+    }
+
+    /**
      * Check TRANSACTIONAL cache.
      *
      * @param cacheMode Cache mode.
@@ -506,6 +556,14 @@ public class GridCacheTransformEventSelfTest extends GridCommonAbstractTest {
         TransactionIsolation txIsolation) throws Exception {
         initialize(cacheMode, TRANSACTIONAL, txConcurrency, txIsolation);
 
+        checkTx0();
+    }
+
+    /**
+     * Check TX cache.
+     */
+    private void checkTx0()  {
+
         System.out.println("BEFORE: " + evts.size());
 
         caches[0].invoke(key1, new Transformer());