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

[39/50] [abbrv] ignite git commit: IGNITE-9750: MVCC: do not allow near cache for TRANSACTIONAL_SNAPSHOT atomicity mode. This closes #4891.

IGNITE-9750: MVCC: do not allow near cache for TRANSACTIONAL_SNAPSHOT atomicity mode. This closes #4891.


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

Branch: refs/heads/ignite-5797
Commit: 72b5c05ae94b068cfbefaa47901bce95ad8020d9
Parents: dab050a
Author: devozerov <vo...@gridgain.com>
Authored: Wed Oct 3 16:09:52 2018 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed Oct 3 16:09:52 2018 +0300

----------------------------------------------------------------------
 .../JdbcThinConnectionMvccEnabledSelfTest.java  |   8 +-
 .../jdbc/thin/JdbcThinTransactionsSelfTest.java |   4 +-
 ...ThinTransactionsWithMvccEnabledSelfTest.java |  15 +-
 .../processors/cache/GridCacheProcessor.java    |  33 +++
 .../cache/mvcc/MvccProcessorImpl.java           |  31 +-
 .../IgniteCacheTxIteratorSelfTest.java          |   2 +-
 .../CacheMvccConfigurationValidationTest.java   | 287 ++++++++-----------
 .../cache/mvcc/CacheMvccTransactionsTest.java   |   4 -
 .../DataStreamProcessorMvccSelfTest.java        |  40 +--
 9 files changed, 174 insertions(+), 250 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/72b5c05a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionMvccEnabledSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionMvccEnabledSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionMvccEnabledSelfTest.java
index 0196cb2..a26f547 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionMvccEnabledSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionMvccEnabledSelfTest.java
@@ -52,11 +52,11 @@ public class JdbcThinConnectionMvccEnabledSelfTest extends JdbcThinAbstractSelfT
     private static final String URL = "jdbc:ignite:thin://127.0.0.1";
 
     /** {@inheritDoc} */
-    @SuppressWarnings("deprecation")
+    @SuppressWarnings({"deprecation", "unchecked"})
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
 
-        cfg.setCacheConfiguration(cacheConfiguration(DEFAULT_CACHE_NAME));
+        cfg.setCacheConfiguration(cacheConfiguration(DEFAULT_CACHE_NAME).setNearConfiguration(null));
 
         TcpDiscoverySpi disco = new TcpDiscoverySpi();
 
@@ -65,7 +65,6 @@ public class JdbcThinConnectionMvccEnabledSelfTest extends JdbcThinAbstractSelfT
         cfg.setDiscoverySpi(disco);
 
         cfg.setMarshaller(new BinaryMarshaller());
-
         cfg.setGridLogger(new GridStringLogger());
 
         return cfg;
@@ -74,9 +73,8 @@ public class JdbcThinConnectionMvccEnabledSelfTest extends JdbcThinAbstractSelfT
     /**
      * @param name Cache name.
      * @return Cache configuration.
-     * @throws Exception In case of error.
      */
-    private CacheConfiguration cacheConfiguration(@NotNull String name) throws Exception {
+    private CacheConfiguration cacheConfiguration(@NotNull String name) {
         CacheConfiguration cfg = defaultCacheConfiguration();
 
         cfg.setName(name);

http://git-wip-us.apache.org/repos/asf/ignite/blob/72b5c05a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsSelfTest.java
index a8fa47b..1d68712 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsSelfTest.java
@@ -55,11 +55,11 @@ public class JdbcThinTransactionsSelfTest extends JdbcThinAbstractSelfTest {
     private GridStringLogger log;
 
     /** {@inheritDoc} */
-    @SuppressWarnings("deprecation")
+    @SuppressWarnings({"deprecation", "unchecked"})
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
 
-        cfg.setCacheConfiguration(cacheConfiguration(DEFAULT_CACHE_NAME));
+        cfg.setCacheConfiguration(cacheConfiguration(DEFAULT_CACHE_NAME).setNearConfiguration(null));
 
         TcpDiscoverySpi disco = new TcpDiscoverySpi();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/72b5c05a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsWithMvccEnabledSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsWithMvccEnabledSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsWithMvccEnabledSelfTest.java
index e01a53d..3ee003a 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsWithMvccEnabledSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinTransactionsWithMvccEnabledSelfTest.java
@@ -55,11 +55,11 @@ public class JdbcThinTransactionsWithMvccEnabledSelfTest extends JdbcThinAbstrac
     private GridStringLogger log;
 
     /** {@inheritDoc} */
-    @SuppressWarnings("deprecation")
+    @SuppressWarnings({"deprecation", "unchecked"})
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
 
-        cfg.setCacheConfiguration(cacheConfiguration(DEFAULT_CACHE_NAME));
+        cfg.setCacheConfiguration(cacheConfiguration(DEFAULT_CACHE_NAME).setNearConfiguration(null));
 
         TcpDiscoverySpi disco = new TcpDiscoverySpi();
 
@@ -77,9 +77,8 @@ public class JdbcThinTransactionsWithMvccEnabledSelfTest extends JdbcThinAbstrac
     /**
      * @param name Cache name.
      * @return Cache configuration.
-     * @throws Exception In case of error.
      */
-    private CacheConfiguration cacheConfiguration(@NotNull String name) throws Exception {
+    private CacheConfiguration cacheConfiguration(@NotNull String name) {
         CacheConfiguration cfg = defaultCacheConfiguration();
 
         cfg.setName(name);
@@ -211,7 +210,7 @@ public class JdbcThinTransactionsWithMvccEnabledSelfTest extends JdbcThinAbstrac
      *
      */
     @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
-    public void testErrorNestedTxAutocommitOff() throws SQLException {
+    public void testErrorNestedTxAutocommitOff() {
         GridTestUtils.assertThrows(null, new Callable<Void>() {
             @Override public Void call() throws Exception {
                 try (Connection c = c(false, NestedTxMode.ERROR)) {
@@ -251,7 +250,7 @@ public class JdbcThinTransactionsWithMvccEnabledSelfTest extends JdbcThinAbstrac
      *
      */
     @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
-    public void testErrorNestedTxAutocommitOn() throws SQLException {
+    public void testErrorNestedTxAutocommitOn() {
         GridTestUtils.assertThrows(null, new Callable<Void>() {
             @Override public Void call() throws Exception {
                 try (Connection c = c(true, NestedTxMode.ERROR)) {
@@ -291,7 +290,7 @@ public class JdbcThinTransactionsWithMvccEnabledSelfTest extends JdbcThinAbstrac
      *
      */
     @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
-    public void testErrorNestedTxAutocommitOffBatched() throws SQLException {
+    public void testErrorNestedTxAutocommitOffBatched() {
         GridTestUtils.assertThrows(null, new Callable<Void>() {
             @Override public Void call() throws Exception {
                 try (Connection c = c(false, NestedTxMode.ERROR)) {
@@ -331,7 +330,7 @@ public class JdbcThinTransactionsWithMvccEnabledSelfTest extends JdbcThinAbstrac
      *
      */
     @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
-    public void testErrorNestedTxAutocommitOnBatched() throws SQLException {
+    public void testErrorNestedTxAutocommitOnBatched() {
         GridTestUtils.assertThrows(null, new Callable<Void>() {
             @Override public Void call() throws Exception {
                 try (Connection c = c(true, NestedTxMode.ERROR)) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/72b5c05a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 2b05d96..16e1799 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -35,6 +35,9 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.CountDownLatch;
 import java.util.stream.Collectors;
+import javax.cache.configuration.FactoryBuilder;
+import javax.cache.expiry.EternalExpiryPolicy;
+import javax.cache.expiry.ExpiryPolicy;
 import javax.management.MBeanServer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
@@ -487,6 +490,36 @@ public class GridCacheProcessor extends GridProcessorAdapter implements Metastor
                 CacheConfiguration.MAX_PARTITIONS_COUNT + " partitions [actual=" + cc.getAffinity().partitions() +
                 ", affFunction=" + cc.getAffinity() + ", cacheName=" + cc.getName() + ']');
 
+        if (cc.getAtomicityMode() == TRANSACTIONAL_SNAPSHOT) {
+            assertParameter(cc.getCacheMode() != LOCAL,
+                "LOCAL cache mode cannot be used with TRANSACTIONAL_SNAPSHOT atomicity mode");
+
+            assertParameter(cc.getNearConfiguration() == null,
+                "near cache cannot be used with TRANSACTIONAL_SNAPSHOT atomicity mode");
+
+            assertParameter(!cc.isReadThrough(),
+                "readThrough cannot be used with TRANSACTIONAL_SNAPSHOT atomicity mode");
+
+            assertParameter(!cc.isWriteThrough(),
+                "writeThrough cannot be used with TRANSACTIONAL_SNAPSHOT atomicity mode");
+
+            assertParameter(!cc.isWriteBehindEnabled(),
+                "writeBehindEnabled cannot be used with TRANSACTIONAL_SNAPSHOT atomicity mode");
+
+            ExpiryPolicy expPlc = null;
+
+            if (cc.getExpiryPolicyFactory() instanceof FactoryBuilder.SingletonFactory)
+                expPlc = (ExpiryPolicy)cc.getExpiryPolicyFactory().create();
+
+            if (!(expPlc instanceof EternalExpiryPolicy)) {
+                assertParameter(cc.getExpiryPolicyFactory() == null,
+                    "expiry policy cannot be used with TRANSACTIONAL_SNAPSHOT atomicity mode");
+            }
+
+            assertParameter(cc.getInterceptor() == null,
+                "interceptor cannot be used with TRANSACTIONAL_SNAPSHOT atomicity mode");
+        }
+
         if (cc.isWriteBehindEnabled()) {
             if (cfgStore == null)
                 throw new IgniteCheckedException("Cannot enable write-behind (writer or store is not provided) " +

http://git-wip-us.apache.org/repos/asf/ignite/blob/72b5c05a/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 ca77bf9..3c45963 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
@@ -28,11 +28,9 @@ import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.atomic.AtomicInteger;
-import javax.cache.expiry.EternalExpiryPolicy;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.DataRegionConfiguration;
@@ -79,7 +77,6 @@ import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxState;
 import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.persistence.DatabaseLifecycleListener;
 import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager;
-import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccDataRow;
 import org.apache.ignite.internal.processors.cache.tree.mvcc.search.MvccLinkAwareSearchRow;
@@ -259,32 +256,12 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
     }
 
     /** {@inheritDoc} */
-    @Override public void validateCacheConfiguration(CacheConfiguration ccfg) throws IgniteCheckedException {
+    @Override public void validateCacheConfiguration(CacheConfiguration ccfg) {
         if (ccfg.getAtomicityMode() == TRANSACTIONAL_SNAPSHOT) {
             if (!mvccSupported)
                 throw new IgniteException("Cannot start MVCC transactional cache. " +
                     "MVCC is unsupported by the cluster.");
 
-            if (ccfg.getCacheStoreFactory() != null) {
-                throw new IgniteCheckedException("Transactional cache may not have a third party cache store when " +
-                    "MVCC is enabled.");
-            }
-
-            if (ccfg.getExpiryPolicyFactory() != null && !(ccfg.getExpiryPolicyFactory().create() instanceof
-                EternalExpiryPolicy)) {
-                throw new IgniteCheckedException("Transactional cache may not have expiry policy when " +
-                    "MVCC is enabled.");
-            }
-
-            if (ccfg.getInterceptor() != null) {
-                throw new IgniteCheckedException("Transactional cache may not have an interceptor when " +
-                    "MVCC is enabled.");
-            }
-
-            if (ccfg.getCacheMode() == CacheMode.LOCAL)
-                throw new IgniteCheckedException("Local caches are not supported by MVCC engine. Use " +
-                    "CacheAtomicityMode.TRANSACTIONAL for local caches.");
-
             mvccEnabled = true;
         }
     }
@@ -333,7 +310,7 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
     }
 
     /** {@inheritDoc} */
-    @Override public void afterInitialise(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException {
+    @Override public void afterInitialise(IgniteCacheDatabaseSharedManager mgr) {
         // No-op.
     }
 
@@ -348,7 +325,7 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
     }
 
     /** {@inheritDoc} */
-    @Override public void afterMemoryRestore(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException {
+    @Override public void afterMemoryRestore(IgniteCacheDatabaseSharedManager mgr) {
         // No-op.
     }
 
@@ -2134,8 +2111,6 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
                         prevKey = row.key();
 
                     if (cctx == null) {
-                        assert shared;
-
                         cctx = part.group().shared().cacheContext(curCacheId = row.cacheId());
 
                         if (cctx == null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/72b5c05a/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 6a00ea4..d5212e5 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
@@ -107,7 +107,7 @@ public class IgniteCacheTxIteratorSelfTest extends GridCommonAbstractTest {
         try {
             for (CacheMode mode : CacheMode.values()) {
                 for (CacheAtomicityMode atomMode : CacheAtomicityMode.values()) {
-                    if (mode == CacheMode.PARTITIONED) {
+                    if (mode == CacheMode.PARTITIONED && atomMode != CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT) {
                         // Near cache makes sense only for partitioned cache.
                         checkTxCache(CacheMode.PARTITIONED, atomMode, true, false);
                     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/72b5c05a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccConfigurationValidationTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccConfigurationValidationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccConfigurationValidationTest.java
index 8b70468..c677656 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccConfigurationValidationTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccConfigurationValidationTest.java
@@ -17,36 +17,37 @@
 
 package org.apache.ignite.internal.processors.cache.mvcc;
 
+import java.io.Serializable;
+import java.util.UUID;
 import java.util.concurrent.Callable;
-import java.util.concurrent.TimeUnit;
+import javax.cache.Cache;
 import javax.cache.CacheException;
 import javax.cache.configuration.Factory;
-import javax.cache.configuration.FactoryBuilder;
-import javax.cache.expiry.CreatedExpiryPolicy;
-import javax.cache.expiry.Duration;
 import javax.cache.expiry.ExpiryPolicy;
+
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.cache.CacheInterceptorAdapter;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheInterceptor;
 import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.cache.store.CacheStore;
-import org.apache.ignite.cache.store.CacheStoreReadFromBackupTest;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.configuration.NearCacheConfiguration;
+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.GridTestUtils;
-import org.apache.ignite.testframework.configvariations.ConfigVariations;
 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.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
+import static org.apache.ignite.cache.CacheMode.LOCAL;
 
 /**
  *
@@ -75,14 +76,16 @@ public class CacheMvccConfigurationValidationTest extends GridCommonAbstractTest
     /**
      * @throws Exception If failed.
      */
+    @SuppressWarnings("ThrowableNotThrown")
     public void testMvccModeMismatchForGroup1() throws Exception {
         final Ignite node = startGrid(0);
 
         node.createCache(new CacheConfiguration("cache1").setGroupName("grp1").setAtomicityMode(ATOMIC));
 
         GridTestUtils.assertThrows(log, new Callable<Void>() {
-            @Override public Void call() throws Exception {
-                node.createCache(new CacheConfiguration("cache2").setGroupName("grp1").setAtomicityMode(TRANSACTIONAL_SNAPSHOT));
+            @Override public Void call() {
+                node.createCache(
+                    new CacheConfiguration("cache2").setGroupName("grp1").setAtomicityMode(TRANSACTIONAL_SNAPSHOT));
 
                 return null;
             }
@@ -94,25 +97,29 @@ public class CacheMvccConfigurationValidationTest extends GridCommonAbstractTest
     /**
      * @throws Exception If failed.
      */
+    @SuppressWarnings("ThrowableNotThrown")
     public void testMvccModeMismatchForGroup2() throws Exception {
         final Ignite node = startGrid(0);
 
-        node.createCache(new CacheConfiguration("cache1").setGroupName("grp1").setAtomicityMode(TRANSACTIONAL_SNAPSHOT));
+        node.createCache(
+            new CacheConfiguration("cache1").setGroupName("grp1").setAtomicityMode(TRANSACTIONAL_SNAPSHOT));
 
         GridTestUtils.assertThrows(log, new Callable<Void>() {
-            @Override public Void call() throws Exception {
+            @Override public Void call() {
                 node.createCache(new CacheConfiguration("cache2").setGroupName("grp1").setAtomicityMode(ATOMIC));
 
                 return null;
             }
         }, CacheException.class, null);
 
-        node.createCache(new CacheConfiguration("cache2").setGroupName("grp1").setAtomicityMode(TRANSACTIONAL_SNAPSHOT));
+        node.createCache(
+            new CacheConfiguration("cache2").setGroupName("grp1").setAtomicityMode(TRANSACTIONAL_SNAPSHOT));
     }
 
     /**
      * @throws Exception If failed.
      */
+    @SuppressWarnings("ThrowableNotThrown")
     public void testMvccLocalCacheDisabled() throws Exception {
         final Ignite node1 = startGrid(1);
         final Ignite node2 = startGrid(2);
@@ -125,7 +132,7 @@ public class CacheMvccConfigurationValidationTest extends GridCommonAbstractTest
         cache1.put(2,2);
 
         GridTestUtils.assertThrows(log, new Callable<Void>() {
-            @Override public Void call() throws Exception {
+            @Override public Void call() {
                 node1.createCache(new CacheConfiguration("cache2").setCacheMode(CacheMode.LOCAL)
                     .setAtomicityMode(TRANSACTIONAL_SNAPSHOT));
 
@@ -144,108 +151,7 @@ public class CacheMvccConfigurationValidationTest extends GridCommonAbstractTest
     /**
      * @throws Exception If failed.
      */
-    public void testMvccExpiredPolicyCacheDisabled() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-8640");
-
-        final Ignite node1 = startGrid(1);
-        final Ignite node2 = startGrid(2);
-
-        IgniteCache cache1 = node1.createCache(new CacheConfiguration("cache1")
-            .setAtomicityMode(TRANSACTIONAL_SNAPSHOT));
-
-        cache1.put(1,1);
-        cache1.put(2,2);
-        cache1.put(2,2);
-
-        GridTestUtils.assertThrows(log, new Callable<Void>() {
-            @Override public Void call() throws Exception {
-                node1.createCache(new CacheConfiguration("cache2")
-                    .setExpiryPolicyFactory(CreatedExpiryPolicy.factoryOf(new Duration(TimeUnit.MINUTES, 1)))
-                    .setAtomicityMode(TRANSACTIONAL_SNAPSHOT));
-
-                return null;
-            }
-        }, CacheException.class, null);
-
-        IgniteCache cache3 = node2.createCache(new CacheConfiguration("cache3")
-            .setAtomicityMode(TRANSACTIONAL_SNAPSHOT));
-
-        cache3.put(1, 1);
-        cache3.put(2, 2);
-        cache3.put(3, 3);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMvccThirdPartyStoreCacheDisabled() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-8640");
-
-        final Ignite node1 = startGrid(1);
-        final Ignite node2 = startGrid(2);
-
-        IgniteCache cache1 = node1.createCache(new CacheConfiguration("cache1")
-            .setAtomicityMode(TRANSACTIONAL_SNAPSHOT));
-
-        cache1.put(1,1);
-        cache1.put(2,2);
-        cache1.put(2,2);
-
-        GridTestUtils.assertThrows(log, new Callable<Void>() {
-            @Override public Void call() throws Exception {
-                node1.createCache(new CacheConfiguration("cache2")
-                    .setCacheStoreFactory(FactoryBuilder.factoryOf(CacheStoreReadFromBackupTest.TestStore.class))
-                    .setAtomicityMode(TRANSACTIONAL_SNAPSHOT));
-
-                return null;
-            }
-        }, CacheException.class, null);
-
-        IgniteCache cache3 = node2.createCache(new CacheConfiguration("cache3")
-            .setAtomicityMode(TRANSACTIONAL_SNAPSHOT));
-
-        cache3.put(1, 1);
-        cache3.put(2, 2);
-        cache3.put(3, 3);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMvccInterceptorCacheDisabled() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-8640");
-
-        final Ignite node1 = startGrid(1);
-        final Ignite node2 = startGrid(2);
-
-        IgniteCache cache1 = node1.createCache(new CacheConfiguration("cache1")
-            .setAtomicityMode(TRANSACTIONAL_SNAPSHOT));
-
-        cache1.put(1,1);
-        cache1.put(2,2);
-        cache1.put(2,2);
-
-        GridTestUtils.assertThrows(log, new Callable<Void>() {
-            @Override public Void call() throws Exception {
-                node1.createCache(new CacheConfiguration("cache2")
-                    .setInterceptor(new ConfigVariations.NoopInterceptor())
-                    .setAtomicityMode(TRANSACTIONAL_SNAPSHOT));
-
-                return null;
-            }
-        }, CacheException.class, null);
-
-        IgniteCache cache3 = node2.createCache(new CacheConfiguration("cache3")
-            .setAtomicityMode(TRANSACTIONAL_SNAPSHOT));
-
-        cache3.put(1, 1);
-        cache3.put(2, 2);
-        cache3.put(3, 3);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
+    @SuppressWarnings("ThrowableNotThrown")
     public void testNodeRestartWithCacheModeChangedTxToMvcc() throws Exception {
         cleanPersistenceDir();
 
@@ -292,6 +198,7 @@ public class CacheMvccConfigurationValidationTest extends GridCommonAbstractTest
     /**
      * @throws Exception If failed.
      */
+    @SuppressWarnings("ThrowableNotThrown")
     public void testNodeRestartWithCacheModeChangedMvccToTx() throws Exception {
         cleanPersistenceDir();
 
@@ -336,89 +243,123 @@ public class CacheMvccConfigurationValidationTest extends GridCommonAbstractTest
     }
 
     /**
+     * Test TRANSACTIONAL_SNAPSHOT and near cache.
+     *
      * @throws Exception If failed.
      */
-    public void testTxCacheWithCacheStore() throws Exception {
-        checkTransactionalModeConflict("cacheStoreFactory", new TestFactory(),
-            "Transactional cache may not have a third party cache store when MVCC is enabled.");
+    @SuppressWarnings("unchecked")
+    public void testTransactionalSnapshotLimitations() throws Exception {
+        assertCannotStart(
+            mvccCacheConfig().setCacheMode(LOCAL),
+            "LOCAL cache mode cannot be used with TRANSACTIONAL_SNAPSHOT atomicity mode"
+        );
+
+        assertCannotStart(
+            mvccCacheConfig().setNearConfiguration(new NearCacheConfiguration<>()),
+            "near cache cannot be used with TRANSACTIONAL_SNAPSHOT atomicity mode"
+        );
+
+        assertCannotStart(
+            mvccCacheConfig().setReadThrough(true),
+            "readThrough cannot be used with TRANSACTIONAL_SNAPSHOT atomicity mode"
+        );
+
+        assertCannotStart(
+            mvccCacheConfig().setWriteThrough(true),
+            "writeThrough cannot be used with TRANSACTIONAL_SNAPSHOT atomicity mode"
+        );
+
+        assertCannotStart(
+            mvccCacheConfig().setWriteBehindEnabled(true),
+            "writeBehindEnabled cannot be used with TRANSACTIONAL_SNAPSHOT atomicity mode"
+        );
+
+        assertCannotStart(
+            mvccCacheConfig().setExpiryPolicyFactory(new TestExpiryPolicyFactory()),
+            "expiry policy cannot be used with TRANSACTIONAL_SNAPSHOT atomicity mode"
+        );
+
+        assertCannotStart(
+            mvccCacheConfig().setInterceptor(new TestCacheInterceptor()),
+            "interceptor cannot be used with TRANSACTIONAL_SNAPSHOT atomicity mode"
+        );
     }
 
     /**
+     * Make sure cache cannot be started with the given configuration.
+     *
+     * @param ccfg Cache configuration.
+     * @param msg Message.
      * @throws Exception If failed.
      */
-    public void testTxCacheWithExpiryPolicy() throws Exception {
-        checkTransactionalModeConflict("expiryPolicyFactory0", CreatedExpiryPolicy.factoryOf(Duration.FIVE_MINUTES),
-            "Transactional cache may not have expiry policy when MVCC is enabled.");
+    @SuppressWarnings("unchecked")
+    private void assertCannotStart(CacheConfiguration ccfg, String msg) throws Exception {
+        Ignite node = startGrid(0);
+
+        try {
+            try {
+                node.getOrCreateCache(ccfg);
+
+                fail("Cache should not start.");
+            }
+            catch (Exception e) {
+                if (msg != null) {
+                    assert e.getMessage() != null : "Error message is null";
+                    assert e.getMessage().contains(msg) : "Wrong error message: " + e.getMessage();
+                }
+            }
+        }
+        finally {
+            stopAllGrids();
+        }
     }
 
     /**
-     * @throws Exception If failed.
+     * @return MVCC-enabled cache configuration.
      */
-    public void testTxCacheWithInterceptor() throws Exception {
-        checkTransactionalModeConflict("interceptor", new CacheInterceptorAdapter(),
-            "Transactional cache may not have an interceptor when MVCC is enabled.");
+    private static CacheConfiguration mvccCacheConfig() {
+        return new CacheConfiguration().setName(DEFAULT_CACHE_NAME + UUID.randomUUID())
+            .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT);
     }
 
     /**
-     * Check that setting specified property conflicts with transactional cache atomicity mode.
-     * @param propName Property name.
-     * @param obj Property value.
-     * @param errMsg Expected error message.
-     * @throws IgniteCheckedException if failed.
+     * Test expiry policy.
      */
-    @SuppressWarnings("ThrowableNotThrown")
-    private void checkTransactionalModeConflict(String propName, Object obj, String errMsg)
-        throws Exception {
-        final String setterName = "set" + propName.substring(0, 1).toUpperCase() + propName.substring(1);
-
-        try (final Ignite node = startGrid(0)) {
-            final CacheConfiguration cfg = new TestConfiguration("cache");
-
-            cfg.setAtomicityMode(TRANSACTIONAL_SNAPSHOT);
-
-            U.invoke(TestConfiguration.class, cfg, setterName, obj);
-
-            GridTestUtils.assertThrows(log, new Callable<Void>() {
-                @SuppressWarnings("unchecked")
-                @Override public Void call() {
-                    node.getOrCreateCache(cfg);
-
-                    return null;
-                }
-            }, IgniteCheckedException.class, errMsg);
+    private static class TestExpiryPolicyFactory implements Factory<ExpiryPolicy>, Serializable {
+        /** {@inheritDoc} */
+        @Override public ExpiryPolicy create() {
+            return null;
         }
     }
 
     /**
-     * Dummy class to overcome ambiguous method name "setExpiryPolicyFactory".
+     * Test cache interceptor.
      */
-    private final static class TestConfiguration extends CacheConfiguration {
-        /**
-         *
-         */
-        TestConfiguration(String cacheName) {
-            super(cacheName);
+    private static class TestCacheInterceptor implements CacheInterceptor, Serializable {
+        /** {@inheritDoc} */
+        @Nullable
+        @Override public Object onGet(Object key, @Nullable Object val) {
+            return null;
         }
 
-        /**
-         *
-         */
-        @SuppressWarnings("unused")
-        public void setExpiryPolicyFactory0(Factory<ExpiryPolicy> plcFactory) {
-            super.setExpiryPolicyFactory(plcFactory);
+        /** {@inheritDoc} */
+        @Nullable @Override public Object onBeforePut(Cache.Entry entry, Object newVal) {
+            return null;
         }
-    }
 
-    /**
-     *
-     */
-    private static class TestFactory implements Factory<CacheStore> {
-        /** Serial version uid. */
-        private static final long serialVersionUID = 0L;
+        /** {@inheritDoc} */
+        @Override public void onAfterPut(Cache.Entry entry) {
+            // No-op.
+        }
 
         /** {@inheritDoc} */
-        @Override public CacheStore create() {
+        @Nullable @Override public IgniteBiTuple onBeforeRemove(Cache.Entry entry) {
             return null;
         }
+
+        /** {@inheritDoc} */
+        @Override public void onAfterRemove(Cache.Entry entry) {
+            // No-op.
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/72b5c05a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccTransactionsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccTransactionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccTransactionsTest.java
index 4d1145c..66555a0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccTransactionsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccTransactionsTest.java
@@ -2229,8 +2229,6 @@ public class CacheMvccTransactionsTest extends CacheMvccAbstractTest {
      * @throws Exception If failed.
      */
     public void testRebalanceSimple() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-9451");
-
         Ignite srv0 = startGrid(0);
 
         IgniteCache<Integer, Integer> cache = (IgniteCache)srv0.createCache(
@@ -2310,8 +2308,6 @@ public class CacheMvccTransactionsTest extends CacheMvccAbstractTest {
      * @throws Exception If failed.
      */
     public void testRebalanceWithRemovedValuesSimple() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-9451");
-
         Ignite node = startGrid(0);
 
         IgniteTransactions txs = node.transactions();

http://git-wip-us.apache.org/repos/asf/ignite/blob/72b5c05a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorMvccSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorMvccSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorMvccSelfTest.java
index 381d9a9..ca8be09 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorMvccSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorMvccSelfTest.java
@@ -28,13 +28,19 @@ import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
  */
 public class DataStreamProcessorMvccSelfTest extends DataStreamProcessorSelfTest {
     /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration igniteConfiguration = super.getConfiguration(igniteInstanceName);
 
-        CacheConfiguration[] cacheConfigurations = igniteConfiguration.getCacheConfiguration();
+        CacheConfiguration[] ccfgs = igniteConfiguration.getCacheConfiguration();
 
-        assert cacheConfigurations == null || cacheConfigurations.length == 0
-                || (cacheConfigurations.length == 1 && cacheConfigurations[0].getAtomicityMode() == TRANSACTIONAL_SNAPSHOT);
+        if (ccfgs != null) {
+            for (CacheConfiguration ccfg : ccfgs)
+                ccfg.setNearConfiguration(null);
+        }
+
+        assert ccfgs == null || ccfgs.length == 0 ||
+            (ccfgs.length == 1 && ccfgs[0].getAtomicityMode() == TRANSACTIONAL_SNAPSHOT);
 
         return igniteConfiguration;
     }
@@ -45,30 +51,6 @@ public class DataStreamProcessorMvccSelfTest extends DataStreamProcessorSelfTest
     }
 
     /** {@inheritDoc} */
-    @Override public void testPartitioned() throws Exception {
-        // test uses batchedSorted StreamReceiver which depends on Cache.putAll, Cache.removeAll
-        fail("https://issues.apache.org/jira/browse/IGNITE-9451");
-
-        super.testPartitioned();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void testColocated() throws Exception {
-        // test uses batchedSorted StreamReceiver which depends on Cache.putAll, Cache.removeAll
-        fail("https://issues.apache.org/jira/browse/IGNITE-9451");
-
-        super.testColocated();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void testReplicated() throws Exception {
-        // test uses batchedSorted StreamReceiver which depends on Cache.putAll, Cache.removeAll
-        fail("https://issues.apache.org/jira/browse/IGNITE-9451");
-
-        super.testReplicated();
-    }
-
-    /** {@inheritDoc} */
     @Override public void testUpdateStore() throws Exception {
         fail("https://issues.apache.org/jira/browse/IGNITE-8582");
 
@@ -76,12 +58,12 @@ public class DataStreamProcessorMvccSelfTest extends DataStreamProcessorSelfTest
     }
 
     /** {@inheritDoc} */
-    @Override public void testFlushTimeout() throws Exception {
+    @Override public void testFlushTimeout() {
         fail("https://issues.apache.org/jira/browse/IGNITE-9321");
     }
 
     /** {@inheritDoc} */
-    @Override public void testLocal() throws Exception {
+    @Override public void testLocal() {
         // Do not check local caches with MVCC enabled.
     }
 }