You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by gv...@apache.org on 2019/02/01 14:10:04 UTC

[ignite] branch master updated: IGNITE-10421: MVCC: Assertion in checkpointer thread fixed. This closes #5696.

This is an automated email from the ASF dual-hosted git repository.

gvvinblade pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/master by this push:
     new ef04782  IGNITE-10421: MVCC: Assertion in checkpointer thread fixed. This closes #5696.
ef04782 is described below

commit ef04782f2543ad3a744fd33b3cfe963cc1fc88c4
Author: Andrey V. Mashenkov <an...@gmail.com>
AuthorDate: Fri Feb 1 17:09:52 2019 +0300

    IGNITE-10421: MVCC: Assertion in checkpointer thread fixed. This closes #5696.
    
    Signed-off-by: rkondakov <ko...@mail.ru>
---
 .../processors/cache/GridCacheProcessor.java       |   8 +-
 .../processors/cache/mvcc/MvccProcessor.java       |  12 ++
 .../processors/cache/mvcc/MvccProcessorImpl.java   |  32 +++-
 .../GridCacheDatabaseSharedManager.java            |  21 ++-
 .../processors/cache/CacheMetricsManageTest.java   |   2 +-
 .../cache/IgniteClusterActivateDeactivateTest.java |   6 -
 ...teDynamicCacheStartFailWithPersistenceTest.java |   7 -
 .../cache/WalModeChangeAdvancedSelfTest.java       |   4 -
 .../CacheDataLossOnPartitionMoveTest.java          |   4 -
 ...dCacheRebalancingWithAsyncClearingMvccTest.java |   6 +-
 .../cache/mvcc/CacheMvccAbstractTest.java          |   6 +
 .../processors/cache/mvcc/CacheMvccVacuumTest.java | 184 ++++++++++++++++++++-
 .../IgnitePdsPartitionFilesDestroyTest.java        |   4 -
 ...gniteRebalanceScheduleResendPartitionsTest.java |   4 -
 ...ocalWalModeChangeDuringRebalancingSelfTest.java |  13 --
 ...NoChangeDuringRebalanceOnNonNodeAssignTest.java |   6 +-
 ...itePdsPageEvictionDuringPartitionClearTest.java |   4 -
 ...gnitePdsRebalancingOnNotStableTopologyTest.java |   4 -
 .../wal/IgniteNodeStoppedDuringDisableWALTest.java |   9 +-
 .../database/IgniteDbDynamicCacheSelfTest.java     |   4 -
 20 files changed, 257 insertions(+), 83 deletions(-)

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 6082bd1..4fc4f60 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
@@ -1390,6 +1390,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             ctx.kernalContext().cache().context().snapshot().onCacheStop(ctx);
 
+            ctx.kernalContext().coordinators().onCacheStop(ctx);
+
             ctx.group().stopCache(ctx, destroy);
 
             U.stopLifecycleAware(log, lifecycleAwares(ctx.group(), cache.configuration(), ctx.store().configuredStore()));
@@ -1545,7 +1547,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         pluginMgr.validate();
 
-        if (!recoveryMode && cfg.getAtomicityMode() == TRANSACTIONAL_SNAPSHOT)
+        if (!recoveryMode && cfg.getAtomicityMode() == TRANSACTIONAL_SNAPSHOT && grp.affinityNode())
             sharedCtx.coordinators().ensureStarted();
 
         sharedCtx.jta().registerCache(cfg);
@@ -2380,7 +2382,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         cacheContext.finishRecovery(cacheStartVer, updatedDescriptor);
 
-        if (cacheContext.config().getAtomicityMode() == TRANSACTIONAL_SNAPSHOT)
+        if (cacheContext.config().getAtomicityMode() == TRANSACTIONAL_SNAPSHOT && groupContext.affinityNode())
             sharedCtx.coordinators().ensureStarted();
 
         onKernalStart(cacheContext.cache());
@@ -5636,6 +5638,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             stopCaches(true);
 
+            sharedCtx.coordinators().stopTxLog();
+
             sharedCtx.database().cleanupRestoredCaches();
         }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessor.java
index 2910eed..05fd9cf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessor.java
@@ -228,4 +228,16 @@ public interface MvccProcessor extends GridProcessor {
      * @param e Exception reflecting failure reason.
      */
     void failWaiter(MvccVersion mvccVer, Exception e);
+
+    /**
+     * Cache stop callback.
+     * @param cctx Cache context.
+     *
+     */
+    void onCacheStop(GridCacheContext cctx);
+
+    /**
+     * Force txLog stop.
+     */
+    void stopTxLog();
 }
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 3ed686d..0fa8226 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
@@ -307,10 +307,31 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
     }
 
     /** {@inheritDoc} */
+    @Override public void onCacheStop(final GridCacheContext cctx) {
+        if (cctx.mvccEnabled() && txLog != null) {
+            assert mvccEnabled && mvccSupported;
+
+            boolean hasMvccCaches = ctx.cache().cacheDescriptors().values().stream()
+                .anyMatch(c -> c.cacheConfiguration().getAtomicityMode() == TRANSACTIONAL_SNAPSHOT);
+
+            if (!hasMvccCaches)
+                stopTxLog();
+        }
+    }
+
+
+    /** {@inheritDoc} */
     @Override public void beforeStop(IgniteCacheDatabaseSharedManager mgr) {
+        stopTxLog();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stopTxLog() {
         stopVacuumWorkers();
 
         txLog = null;
+
+        mvccEnabled = false;
     }
 
     /** {@inheritDoc} */
@@ -357,7 +378,7 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
         assert CU.isPersistenceEnabled(ctx.config());
 
         //noinspection ConstantConditions
-        ctx.cache().context().pageStore().initialize(TX_LOG_CACHE_ID, 1,
+        ctx.cache().context().pageStore().initialize(TX_LOG_CACHE_ID, 0,
             TX_LOG_CACHE_NAME, mgr.dataRegion(TX_LOG_CACHE_NAME).memoryMetrics());
     }
 
@@ -595,7 +616,7 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
 
     /** {@inheritDoc} */
     @Override public byte state(long crdVer, long cntr) {
-        assert txLog != null && mvccEnabled;
+        assert txLog != null && mvccEnabled : mvccEnabled;
 
         try {
             return txLog.get(crdVer, cntr);
@@ -614,7 +635,12 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
 
     /** {@inheritDoc} */
     @Override public void updateState(MvccVersion ver, byte state, boolean primary) {
-        assert txLog != null && mvccEnabled;
+        assert mvccEnabled;
+        assert txLog != null || waitMap.isEmpty();
+
+        // txLog may not exist if node is non-affinity for any mvcc-cache.
+        if (txLog == null)
+            return;
 
         try {
             txLog.put(new TxKey(ver.coordinatorVersion(), ver.counter()), state, primary);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
index 0863360..9f126a6 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
@@ -182,6 +182,7 @@ import static java.nio.file.StandardOpenOption.READ;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_CHECKPOINT_READ_LOCK_TIMEOUT;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_WAL_REBALANCE_THRESHOLD;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_RECOVERY_SEMAPHORE_PERMITS;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
 import static org.apache.ignite.failure.FailureType.CRITICAL_ERROR;
 import static org.apache.ignite.failure.FailureType.SYSTEM_CRITICAL_OPERATION_TIMEOUT;
 import static org.apache.ignite.failure.FailureType.SYSTEM_WORKER_TERMINATION;
@@ -560,7 +561,11 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         if (dataRegionMap.isEmpty())
             return;
 
+        boolean hasMvccCache = false;
+
         for (CacheGroupDescriptor grpDesc : cctx.cache().cacheGroupDescriptors().values()) {
+            hasMvccCache |= grpDesc.config().getAtomicityMode() == TRANSACTIONAL_SNAPSHOT;
+
             String regionName = grpDesc.config().getDataRegionName();
 
             DataRegion region = regionName != null ? dataRegionMap.get(regionName) : dfltDataRegion;
@@ -583,10 +588,20 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
             }
         }
 
+        if (!hasMvccCache && dataRegionMap.containsKey(TxLog.TX_LOG_CACHE_NAME)) {
+            PageMemory memory = dataRegionMap.get(TxLog.TX_LOG_CACHE_NAME).pageMemory();
+
+            if (memory instanceof PageMemoryEx)
+                ((PageMemoryEx)memory).invalidate(TxLog.TX_LOG_CACHE_ID, PageIdAllocator.INDEX_PARTITION);
+        }
+
+        final boolean hasMvccCache0 = hasMvccCache;
+
         storeMgr.cleanupPageStoreIfMatch(
             new Predicate<Integer>() {
                 @Override public boolean test(Integer grpId) {
-                    return MetaStorage.METASTORAGE_CACHE_ID != grpId;
+                    return MetaStorage.METASTORAGE_CACHE_ID != grpId &&
+                        (TxLog.TX_LOG_CACHE_ID != grpId || !hasMvccCache0);
                 }
             },
             true);
@@ -2539,7 +2554,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
                             byte txState = convertToTxState(txRecord.state());
 
-                            cctx.coordinators().updateState(txRecord.mvccVersion(), txState, false);
+                            cctx.coordinators().updateState(txRecord.mvccVersion(), txState, true);
                         }
                         finally {
                             checkpointReadUnlock();
@@ -2634,7 +2649,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
                         byte txState = convertToTxState(txRecord.state());
 
-                        cctx.coordinators().updateState(txRecord.mvccVersion(), txState, false);
+                        cctx.coordinators().updateState(txRecord.mvccVersion(), txState, true);
 
                         break;
 
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMetricsManageTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMetricsManageTest.java
index 8def752..423b703 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMetricsManageTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMetricsManageTest.java
@@ -88,7 +88,7 @@ public class CacheMetricsManageTest extends GridCommonAbstractTest {
      */
     @Test
     public void testJmxPdsStatisticsEnable() throws Exception {
-        Assume.assumeFalse("https://issues.apache.org/jira/browse/IGNITE-10421", MvccFeatureChecker.forcedMvcc());
+        Assume.assumeFalse("https://issues.apache.org/jira/browse/IGNITE-9224", MvccFeatureChecker.forcedMvcc());
 
         testJmxStatisticsEnable(true);
     }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java
index 358d982..80262cd 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java
@@ -51,10 +51,8 @@ import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.testframework.GridTestUtils;
-import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.junit.Assert;
-import org.junit.Assume;
 import org.junit.Test;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
@@ -347,8 +345,6 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
      */
     @Test
     public void testJoinWhileActivate1_Server() throws Exception {
-        Assume.assumeFalse("https://issues.apache.org/jira/browse/IGNITE-10421", MvccFeatureChecker.forcedMvcc());
-
         joinWhileActivate1(false, false);
     }
 
@@ -357,8 +353,6 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
      */
     @Test
     public void testJoinWhileActivate1_WithCache_Server() throws Exception {
-        Assume.assumeFalse("https://issues.apache.org/jira/browse/IGNITE-10421", MvccFeatureChecker.forcedMvcc());
-
         joinWhileActivate1(false, true);
     }
 
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartFailWithPersistenceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartFailWithPersistenceTest.java
index 771705b..8643b66 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartFailWithPersistenceTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartFailWithPersistenceTest.java
@@ -24,8 +24,6 @@ import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.WALMode;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-import org.apache.ignite.testframework.MvccFeatureChecker;
-import org.junit.Assume;
 
 /**
  * Tests the recovery after a dynamic cache start failure, with enabled persistence.
@@ -69,11 +67,6 @@ public class IgniteDynamicCacheStartFailWithPersistenceTest extends IgniteAbstra
     }
 
     /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        Assume.assumeFalse("https://issues.apache.org/jira/browse/IGNITE-10421", MvccFeatureChecker.forcedMvcc());
-    }
-
-    /** {@inheritDoc} */
     @Override protected void afterTestsStopped() throws Exception {
         stopAllGrids();
 
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/WalModeChangeAdvancedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/WalModeChangeAdvancedSelfTest.java
index 23a254a..ba85c9f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/WalModeChangeAdvancedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/WalModeChangeAdvancedSelfTest.java
@@ -30,10 +30,8 @@ import org.apache.ignite.internal.IgniteClientReconnectAbstractTest;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.GridTestUtils.SF;
-import org.junit.Assume;
 import org.junit.Ignore;
 import org.junit.Test;
-import org.apache.ignite.testframework.MvccFeatureChecker;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
@@ -140,8 +138,6 @@ public class WalModeChangeAdvancedSelfTest extends WalModeChangeCommonAbstractSe
      */
     @Test
     public void testJoin() throws Exception {
-        Assume.assumeFalse("https://issues.apache.org/jira/browse/IGNITE-10421", MvccFeatureChecker.forcedMvcc());
-
         checkJoin(false);
     }
 
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheDataLossOnPartitionMoveTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheDataLossOnPartitionMoveTest.java
index 2611c10..8b28834 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheDataLossOnPartitionMoveTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheDataLossOnPartitionMoveTest.java
@@ -43,10 +43,8 @@ import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.testframework.GridTestNode;
-import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jetbrains.annotations.Nullable;
-import org.junit.Assume;
 import org.junit.Test;
 
 import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.EVICTED;
@@ -123,8 +121,6 @@ public class CacheDataLossOnPartitionMoveTest extends GridCommonAbstractTest {
      */
     @Test
     public void testDataLossOnPartitionMove() throws Exception {
-        Assume.assumeFalse("https://issues.apache.org/jira/browse/IGNITE-10421", MvccFeatureChecker.forcedMvcc());
-
         try {
             Ignite ignite = startGridsMultiThreaded(GRIDS_CNT / 2, false);
 
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingWithAsyncClearingMvccTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingWithAsyncClearingMvccTest.java
index 2a14a36..a8a9b4a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingWithAsyncClearingMvccTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingWithAsyncClearingMvccTest.java
@@ -17,10 +17,10 @@
 package org.apache.ignite.internal.processors.cache.distributed.rebalancing;
 
 import org.apache.ignite.cache.CacheAtomicityMode;
-import org.junit.Ignore;
 
-/** */
-@Ignore("https://issues.apache.org/jira/browse/IGNITE-10421")
+/**
+ *
+ */
 public class GridCacheRebalancingWithAsyncClearingMvccTest extends GridCacheRebalancingWithAsyncClearingTest {
     /** {@inheritDoc} */
     @Override protected CacheAtomicityMode atomicityMode() {
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractTest.java
index 47804cc..5c11e46 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractTest.java
@@ -61,6 +61,7 @@ import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.TransactionConfiguration;
 import org.apache.ignite.configuration.WALMode;
+import org.apache.ignite.failure.StopNodeFailureHandler;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteFutureCancelledCheckedException;
@@ -165,6 +166,8 @@ public abstract class CacheMvccAbstractTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
         if (disableScheduledVacuum)
             cfg.setMvccVacuumFrequency(Integer.MAX_VALUE);
 
@@ -218,6 +221,9 @@ public abstract class CacheMvccAbstractTest extends GridCommonAbstractTest {
     @Override protected void beforeTest() throws Exception {
         super.beforeTest();
 
+        ccfg = null;
+        ccfgs = null;
+
         MvccProcessorImpl.coordinatorAssignClosure(null);
 
         cleanPersistenceDir();
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccVacuumTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccVacuumTest.java
index ede79dc..3f952d8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccVacuumTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccVacuumTest.java
@@ -17,14 +17,22 @@
 
 package org.apache.ignite.internal.processors.cache.mvcc;
 
+import java.util.Collections;
 import java.util.List;
+import java.util.UUID;
 import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxLog;
 import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Ignore;
 import org.junit.Test;
 
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
@@ -88,14 +96,17 @@ public class CacheMvccVacuumTest extends CacheMvccAbstractTest {
         ensureNoVacuum(node1);
 
         node1.createCache(new CacheConfiguration<>("test1")
-                .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL));
+            .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL));
 
         ensureNoVacuum(node0);
         ensureNoVacuum(node1);
 
-        node1.createCache(new CacheConfiguration<>("test2")
+        IgniteCache<Object, Object> cache = node1.createCache(new CacheConfiguration<>("test2")
             .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT));
 
+        cache.put(primaryKey(cache), 0);
+        cache.put(primaryKey(node0.cache("test2")), 0);
+
         ensureVacuum(node0);
         ensureVacuum(node1);
 
@@ -122,9 +133,6 @@ public class CacheMvccVacuumTest extends CacheMvccAbstractTest {
         node0 = startGrid(0);
         node1 = startGrid(1);
 
-        ensureNoVacuum(node0);
-        ensureNoVacuum(node1);
-
         node1.cluster().active(true);
 
         ensureVacuum(node0);
@@ -141,6 +149,140 @@ public class CacheMvccVacuumTest extends CacheMvccAbstractTest {
         Ignite node = startGrid(cfg);
 
         ensureNoVacuum(node);
+
+        IgniteCache<Object, Object> cache = node.createCache(
+            cacheConfiguration(PARTITIONED, CacheWriteSynchronizationMode.FULL_SYNC, 1, 16));
+
+        ensureVacuum(node);
+
+        cache.put(0, 0);
+
+        cache.destroy();
+
+        ensureNoVacuum(node);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Ignore("https://issues.apache.org/jira/browse/IGNITE-8414")
+    @Test
+    public void testVacuumNotStartedOnNonBaselineNode() throws Exception {
+        persistence = true;
+
+        Ignite node0 = startGrid(0);
+
+        ensureNoVacuum(node0);
+
+        node0.cluster().active(true);
+
+        IgniteCache<Object, Object> cache = node0.createCache(
+            cacheConfiguration(PARTITIONED, CacheWriteSynchronizationMode.FULL_SYNC, 1, 16));
+
+        cache.put(1, 0);
+
+        ensureVacuum(node0);
+
+        Ignite node1 = startGrid(1);
+
+        //TODO IGNITE-8414: Test fails here due to cache context initializes on node join unless IGNITE-8414 is fixed.
+        ensureNoVacuum(node1);
+
+        node0.cluster().setBaselineTopology(node0.cluster().topologyVersion());
+
+        ensureVacuum(node0);
+        ensureVacuum(node1);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Ignore("https://issues.apache.org/jira/browse/IGNITE-8717")
+    @Test
+    public void testVacuumNotStartedOnNonBaselineNode2() throws Exception {
+        persistence = true;
+
+        Ignite node0 = startGrid(0);
+        Ignite node1 = startGrid(1);
+
+        node0.cluster().active(true);
+
+        IgniteCache<Object, Object> cache = node0.createCache(
+            cacheConfiguration(PARTITIONED, CacheWriteSynchronizationMode.FULL_SYNC, 1, 16));
+
+        cache.put(primaryKey(cache), 0);
+        cache.put(primaryKey(node1.cache(DEFAULT_CACHE_NAME)), 0);
+
+        stopGrid(1);
+
+        node0.cluster().setBaselineTopology(Collections.singleton(node0.cluster().node()));
+
+        //TODO IGNITE-8717: Rejoin node after cache destroy leads critical error unless IGNITE-8717 fixed.
+        node0.cache(DEFAULT_CACHE_NAME).destroy();
+
+        ensureNoVacuum(node0);
+
+        node1 = startGrid(1);
+
+        ensureNoVacuum(node0);
+        ensureNoVacuum(node1);
+
+        node0.cluster().setBaselineTopology(node0.cluster().topologyVersion());
+
+        ensureNoVacuum(node0);
+        ensureNoVacuum(node1);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testVacuumNotStartedOnNonAffinityNode() throws Exception {
+        persistence = true;
+
+        Ignite node0 = startGrid(0);
+        Ignite node1 = startGrid(1);
+
+        ensureNoVacuum(node0);
+        ensureNoVacuum(node1);
+
+        node0.cluster().active(true);
+
+        IgniteCache<Object, Object> cache = node0.createCache(
+            cacheConfiguration(PARTITIONED, CacheWriteSynchronizationMode.FULL_SYNC, 1, 16)
+                .setNodeFilter(new NodeFilter(node0.cluster().node().id())));
+
+        cache.put(1, 0);
+
+        ensureVacuum(node0);
+        ensureNoVacuum(node1);
+
+        node0.cluster().active(false);
+
+        ensureNoVacuum(node0);
+        ensureNoVacuum(node1);
+
+        stopGrid(1);
+
+        ensureNoVacuum(node0);
+        ensureNoVacuum(node1);
+
+        node0.cluster().active(true);
+        node0.cluster().setBaselineTopology(Collections.singleton(node0.cluster().node()));
+
+        ensureVacuum(node0);
+        ensureNoVacuum(node1);
+
+        // Check non-baseline node.
+        node1 = startGrid(1);
+
+        ensureVacuum(node0);
+        ensureNoVacuum(node1);
+
+        node0.cluster().setBaselineTopology(node0.cluster().topologyVersion());
+
+        ensureVacuum(node0);
+        ensureNoVacuum(node1);
     }
 
     /**
@@ -171,10 +313,14 @@ public class CacheMvccVacuumTest extends CacheMvccAbstractTest {
 
         assertNotNull(crd);
 
+        TxLog txLog = GridTestUtils.getFieldValue(crd, "txLog");
+
+        assertNotNull("TxLog wasn't initialized.", txLog);
+
         List<GridWorker> vacuumWorkers = GridTestUtils.getFieldValue(crd, "vacuumWorkers");
 
-        assertNotNull(vacuumWorkers);
-        assertFalse(vacuumWorkers.isEmpty());
+        assertNotNull("No vacuum workers was initialized.", vacuumWorkers);
+        assertFalse("No vacuum workers was initialized.", vacuumWorkers.isEmpty());
 
         for (GridWorker w : vacuumWorkers) {
             assertFalse(w.isCancelled());
@@ -190,6 +336,28 @@ public class CacheMvccVacuumTest extends CacheMvccAbstractTest {
     private void ensureNoVacuum(Ignite node) {
         MvccProcessorImpl crd = mvccProcessor(node);
 
-        assertNull(GridTestUtils.<List<GridWorker>>getFieldValue(crd, "vacuumWorkers"));
+        assertNull("Vacuums workers shouldn't be started.", GridTestUtils.<List<GridWorker>>getFieldValue(crd, "vacuumWorkers"));
+
+        assertNull("TxLog shouldn't exists.", GridTestUtils.getFieldValue(crd, "txLog"));
+    }
+
+    /**
+     * Filter specifying on which node the cache should be started.
+     */
+    public static class NodeFilter implements IgnitePredicate<ClusterNode> {
+        /** Cache should be created node with certain UUID. */
+        public UUID uuid;
+
+        /**
+         * @param uuid node ID.
+         */
+        public NodeFilter(UUID uuid) {
+            this.uuid = uuid;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean apply(ClusterNode clusterNode) {
+            return clusterNode.id().equals(uuid);
+        }
     }
 }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPartitionFilesDestroyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPartitionFilesDestroyTest.java
index 3c4060a..d09f346 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPartitionFilesDestroyTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPartitionFilesDestroyTest.java
@@ -43,10 +43,8 @@ import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactor
 import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.junit.Assert;
-import org.junit.Assume;
 import org.junit.Test;
 
 import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
@@ -93,8 +91,6 @@ public class IgnitePdsPartitionFilesDestroyTest extends GridCommonAbstractTest {
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        Assume.assumeFalse("https://issues.apache.org/jira/browse/IGNITE-10421", MvccFeatureChecker.forcedMvcc());
-
         stopAllGrids();
 
         cleanPersistenceDir();
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteRebalanceScheduleResendPartitionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteRebalanceScheduleResendPartitionsTest.java
index 3fafbd6..de9992d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteRebalanceScheduleResendPartitionsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteRebalanceScheduleResendPartitionsTest.java
@@ -45,10 +45,8 @@ import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.resources.LoggerResource;
 import org.apache.ignite.spi.IgniteSpiException;
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
-import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.junit.Assert;
-import org.junit.Assume;
 import org.junit.Test;
 
 import static org.apache.ignite.testframework.GridTestUtils.runAsync;
@@ -109,8 +107,6 @@ public class IgniteRebalanceScheduleResendPartitionsTest extends GridCommonAbstr
      */
     @Test
     public void test() throws Exception {
-        Assume.assumeFalse("https://issues.apache.org/jira/browse/IGNITE-10421", MvccFeatureChecker.forcedMvcc());
-
         Ignite ig0 = startGrids(3);
 
         ig0.cluster().active(true);
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java
index eddc763..8a505cb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java
@@ -57,7 +57,6 @@ import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.junit.Assert;
-import org.junit.Assume;
 import org.junit.Test;
 
 import static org.apache.ignite.testframework.GridTestUtils.waitForCondition;
@@ -227,8 +226,6 @@ public class LocalWalModeChangeDuringRebalancingSelfTest extends GridCommonAbstr
      */
     @Test
     public void testWalDisabledDuringRebalancing() throws Exception {
-        Assume.assumeFalse("https://issues.apache.org/jira/browse/IGNITE-10421", MvccFeatureChecker.forcedMvcc());
-
         doTestSimple();
     }
 
@@ -237,8 +234,6 @@ public class LocalWalModeChangeDuringRebalancingSelfTest extends GridCommonAbstr
      */
     @Test
     public void testWalNotDisabledIfParameterSetToFalse() throws Exception {
-        Assume.assumeFalse("https://issues.apache.org/jira/browse/IGNITE-10421", MvccFeatureChecker.forcedMvcc());
-
         disableWalDuringRebalancing = false;
 
         doTestSimple();
@@ -378,8 +373,6 @@ public class LocalWalModeChangeDuringRebalancingSelfTest extends GridCommonAbstr
      */
     @Test
     public void testLocalAndGlobalWalStateInterdependence() throws Exception {
-        Assume.assumeFalse("https://issues.apache.org/jira/browse/IGNITE-10421", MvccFeatureChecker.forcedMvcc());
-
         Ignite ignite = startGrids(3);
 
         ignite.cluster().active(true);
@@ -474,8 +467,6 @@ public class LocalWalModeChangeDuringRebalancingSelfTest extends GridCommonAbstr
      */
     @Test
     public void testParallelExchangeDuringRebalance() throws Exception {
-        Assume.assumeFalse("https://issues.apache.org/jira/browse/IGNITE-10421", MvccFeatureChecker.forcedMvcc());
-
         doTestParallelExchange(supplyMessageLatch);
     }
 
@@ -484,8 +475,6 @@ public class LocalWalModeChangeDuringRebalancingSelfTest extends GridCommonAbstr
      */
     @Test
     public void testParallelExchangeDuringCheckpoint() throws Exception {
-        Assume.assumeFalse("https://issues.apache.org/jira/browse/IGNITE-10421", MvccFeatureChecker.forcedMvcc());
-
         doTestParallelExchange(fileIOLatch);
     }
 
@@ -542,8 +531,6 @@ public class LocalWalModeChangeDuringRebalancingSelfTest extends GridCommonAbstr
      */
     @Test
     public void testDataClearedAfterRestartWithDisabledWal() throws Exception {
-        Assume.assumeFalse("https://issues.apache.org/jira/browse/IGNITE-10421", MvccFeatureChecker.forcedMvcc());
-
         Ignite ignite = startGrid(0);
 
         ignite.cluster().active(true);
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeNoChangeDuringRebalanceOnNonNodeAssignTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeNoChangeDuringRebalanceOnNonNodeAssignTest.java
index 786b3bd..94dbf55 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeNoChangeDuringRebalanceOnNonNodeAssignTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeNoChangeDuringRebalanceOnNonNodeAssignTest.java
@@ -35,7 +35,6 @@ import org.apache.ignite.internal.processors.cache.persistence.wal.reader.Ignite
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.junit.Ignore;
 import org.junit.Test;
 
 import static java.lang.String.valueOf;
@@ -82,6 +81,8 @@ public class LocalWalModeNoChangeDuringRebalanceOnNonNodeAssignTest extends Grid
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-10652");
+
         super.beforeTest();
 
         cleanPersistenceDir();
@@ -101,7 +102,6 @@ public class LocalWalModeNoChangeDuringRebalanceOnNonNodeAssignTest extends Grid
     /**
      * @throws Exception If failed.
      */
-    @Ignore("https://issues.apache.org/jira/browse/IGNITE-10652")
     @Test
     public void testAtomic() throws Exception {
         atomicityMode = CacheAtomicityMode.ATOMIC;
@@ -112,7 +112,6 @@ public class LocalWalModeNoChangeDuringRebalanceOnNonNodeAssignTest extends Grid
     /**
      * @throws Exception If failed.
      */
-    @Ignore("https://issues.apache.org/jira/browse/IGNITE-10652")
     @Test
     public void testTx() throws Exception {
         atomicityMode = CacheAtomicityMode.TRANSACTIONAL;
@@ -123,7 +122,6 @@ public class LocalWalModeNoChangeDuringRebalanceOnNonNodeAssignTest extends Grid
     /**
      * @throws Exception If failed.
      */
-    @Ignore("https://issues.apache.org/jira/browse/IGNITE-10421")
     @Test
     public void testMvcc() throws Exception {
         atomicityMode = CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsPageEvictionDuringPartitionClearTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsPageEvictionDuringPartitionClearTest.java
index 10886e0..ad3bf51 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsPageEvictionDuringPartitionClearTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsPageEvictionDuringPartitionClearTest.java
@@ -34,9 +34,7 @@ import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.GridTestUtils;
-import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.junit.Assume;
 import org.junit.Test;
 
 /**
@@ -90,8 +88,6 @@ public class IgnitePdsPageEvictionDuringPartitionClearTest extends GridCommonAbs
      */
     @Test
     public void testPageEvictionOnNodeStart() throws Exception {
-        Assume.assumeFalse("https://issues.apache.org/jira/browse/IGNITE-10421", MvccFeatureChecker.forcedMvcc());
-
         for (int r = 0; r < 3; r++) {
             cleanPersistenceDir();
 
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsRebalancingOnNotStableTopologyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsRebalancingOnNotStableTopologyTest.java
index 880439b..231bf61 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsRebalancingOnNotStableTopologyTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsRebalancingOnNotStableTopologyTest.java
@@ -33,10 +33,8 @@ import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.WALMode;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.testframework.junits.multijvm.IgniteProcessProxy;
-import org.junit.Assume;
 import org.junit.Test;
 
 /**
@@ -57,8 +55,6 @@ public class IgnitePdsRebalancingOnNotStableTopologyTest extends GridCommonAbstr
      */
     @Test
     public void test() throws Exception {
-        Assume.assumeFalse("https://issues.apache.org/jira/browse/IGNITE-10421", MvccFeatureChecker.forcedMvcc());
-
         Ignite ex = startGrid(0);
 
         ex.active(true);
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteNodeStoppedDuringDisableWALTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteNodeStoppedDuringDisableWALTest.java
index 17ebe64..6293b52 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteNodeStoppedDuringDisableWALTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteNodeStoppedDuringDisableWALTest.java
@@ -34,13 +34,12 @@ import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
 import org.apache.ignite.internal.pagemem.wal.WALIterator;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.WalStateManager.WALDisableContext;
+import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxLog;
 import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
 import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
 import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFoldersResolver;
-import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.junit.Assert;
-import org.junit.Assume;
 import org.junit.Test;
 
 import static java.nio.file.FileVisitResult.CONTINUE;
@@ -90,8 +89,6 @@ public class IgniteNodeStoppedDuringDisableWALTest extends GridCommonAbstractTes
      */
     @Test
     public void test() throws Exception {
-        Assume.assumeFalse("https://issues.apache.org/jira/browse/IGNITE-10421", MvccFeatureChecker.forcedMvcc());
-
         for (NodeStopPoint nodeStopPoint : NodeStopPoint.values()) {
             testStopNodeWithDisableWAL(nodeStopPoint);
 
@@ -200,7 +197,9 @@ public class IgniteNodeStoppedDuringDisableWALTest extends GridCommonAbstractTes
 
                     String filePath = path.toString();
 
-                    if (path.toFile().getParentFile().getName().equals(META_STORAGE_NAME))
+                    String parentDirName = path.toFile().getParentFile().getName();
+
+                    if (parentDirName.equals(META_STORAGE_NAME) || parentDirName.equals(TxLog.TX_LOG_CACHE_NAME))
                         return CONTINUE;
 
                     if (WAL_NAME_PATTERN.matcher(name).matches() || WAL_TEMP_NAME_PATTERN.matcher(name).matches())
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbDynamicCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbDynamicCacheSelfTest.java
index 9a4a74e..1f09cb2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbDynamicCacheSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbDynamicCacheSelfTest.java
@@ -29,9 +29,7 @@ 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.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.junit.Assume;
 import org.junit.Test;
 
 /**
@@ -118,8 +116,6 @@ public class IgniteDbDynamicCacheSelfTest extends GridCommonAbstractTest {
      */
     @Test
     public void testMultipleDynamicCaches() throws Exception {
-        Assume.assumeFalse("https://issues.apache.org/jira/browse/IGNITE-10421", MvccFeatureChecker.forcedMvcc());
-
         int caches = 10;
 
         int entries = 10;