You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by se...@apache.org on 2021/06/03 08:12:27 UTC

[ignite] branch master updated: IGNITE-14739 Permanent logging of tx states to ensure correct tx states recovery after node crash - Fixes #9112.

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

sergeychugunov 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 d785ee7  IGNITE-14739 Permanent logging of tx states to ensure correct tx states recovery after node crash - Fixes #9112.
d785ee7 is described below

commit d785ee76e552a6c74578c34b30838c14c277c406
Author: zstan <st...@gmail.com>
AuthorDate: Thu Jun 3 11:03:55 2021 +0300

    IGNITE-14739 Permanent logging of tx states to ensure correct tx states recovery after node crash - Fixes #9112.
    
    Signed-off-by: Sergey Chugunov <se...@gmail.com>
---
 .../org/apache/ignite/IgniteSystemProperties.java   | 10 ----------
 .../distributed/GridDistributedTxRemoteAdapter.java |  4 ++--
 .../distributed/dht/GridDhtTxPrepareFuture.java     | 12 ++++++++++--
 .../cache/transactions/IgniteTxLocalAdapter.java    |  2 +-
 .../cache/transactions/IgniteTxManager.java         | 14 --------------
 .../distributed/CacheBaselineTopologyTest.java      | 11 +++--------
 .../db/IgniteLogicalRecoveryWithParamsTest.java     |  9 +++------
 .../db/wal/reader/IgniteWalReaderTest.java          | 10 +++-------
 .../TxRollbackAsyncWithPersistenceTest.java         | 14 --------------
 .../persistence/db/wal/IgniteWalRecoveryTest.java   | 21 +++++++++------------
 ...scoverySegmentationAndConnectionRestoreTest.java |  3 ---
 11 files changed, 31 insertions(+), 79 deletions(-)

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 64d4ac7..176b28fa 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -1203,16 +1203,6 @@ public final class IgniteSystemProperties {
         defaults = "" + DFLT_USE_ASYNC_FILE_IO_FACTORY)
     public static final String IGNITE_USE_ASYNC_FILE_IO_FACTORY = "IGNITE_USE_ASYNC_FILE_IO_FACTORY";
 
-    /**
-     * If the property is set {@link org.apache.ignite.internal.pagemem.wal.record.TxRecord} records
-     * will be logged to WAL.
-     *
-     * Default value is {@code false}.
-     */
-    @SystemProperty("If the property is set org.apache.ignite.internal.pagemem.wal.record.TxRecord records " +
-        "will be logged to WAL")
-    public static final String IGNITE_WAL_LOG_TX_RECORDS = "IGNITE_WAL_LOG_TX_RECORDS";
-
     /** Max amount of remembered errors for {@link GridLogThrottle}. */
     @SystemProperty(value = "Max amount of remembered errors for GridLogThrottle", type = Integer.class,
         defaults = "" + DFLT_LOG_THROTTLE_CAPACITY)
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
index 173888b..1f1e700 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
@@ -423,7 +423,7 @@ public abstract class GridDistributedTxRemoteAdapter extends IgniteTxAdapter
             if (pessimistic() || isSystemInvalidate())
                 state(PREPARED);
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteCheckedException | IgniteException e) {
             setRollbackOnly();
 
             throw e;
@@ -811,7 +811,7 @@ public abstract class GridDistributedTxRemoteAdapter extends IgniteTxAdapter
                             ptr = cctx.wal().log(new DataRecord(entriesWithCounters));
                         }
 
-                        if (ptr != null && !cctx.tm().logTxRecords())
+                        if (ptr != null)
                             cctx.wal().flush(ptr, false);
                     }
                     catch (Throwable ex) {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
index 8607458..d04baea 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
@@ -34,6 +34,7 @@ import javax.cache.expiry.Duration;
 import javax.cache.expiry.ExpiryPolicy;
 import javax.cache.processor.EntryProcessor;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteInterruptedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cluster.ClusterNode;
@@ -727,7 +728,7 @@ public final class GridDhtTxPrepareFuture extends GridCacheCompoundFuture<Ignite
                 try {
                     prepare0();
                 }
-                catch (IgniteTxRollbackCheckedException e) {
+                catch (IgniteTxRollbackCheckedException | IgniteException e) {
                     onError(e);
                 }
             else {
@@ -1040,7 +1041,14 @@ public final class GridDhtTxPrepareFuture extends GridCacheCompoundFuture<Ignite
             tx.setRollbackOnly();
         }
         else if (!tx.onePhaseCommit() && ((last || tx.isSystemInvalidate()) && !(tx.near() && tx.local())))
-            tx.state(PREPARED);
+            try {
+                tx.state(PREPARED);
+            }
+            catch (IgniteException e) {
+                tx.setRollbackOnly();
+
+                res.error(e);
+            }
 
         if (super.onDone(res, res == null ? err : null)) {
             // Don't forget to clean up.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index 50d07d4..9a3da6e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -926,7 +926,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
 
                 cctx.mvccCaching().onTxFinished(this, true);
 
-                if (ptr != null && !cctx.tm().logTxRecords())
+                if (ptr != null)
                     cctx.wal().flush(ptr, false);
             }
             catch (Throwable ex) {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
index 3f25582..862c216 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
@@ -137,7 +137,6 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_TRANSACTION_TIME_D
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_TRANSACTION_TIME_DUMP_SAMPLES_PER_SECOND_LIMIT;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_TX_DEADLOCK_DETECTION_MAX_ITERS;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_TX_OWNER_DUMP_REQUESTS_ALLOWED;
-import static org.apache.ignite.IgniteSystemProperties.IGNITE_WAL_LOG_TX_RECORDS;
 import static org.apache.ignite.IgniteSystemProperties.getLong;
 import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
@@ -338,9 +337,6 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
     /** TxDeadlock detection. */
     private TxDeadlockDetection txDeadlockDetection;
 
-    /** Flag indicates that {@link TxRecord} records will be logged to WAL. */
-    private boolean logTxRecords;
-
     /**
      * Indicates whether {@code suspend()} and {@code resume()} operations are supported for pessimistic transactions
      * cluster wide.
@@ -435,8 +431,6 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
 
         cctx.gridIO().addMessageListener(TOPIC_TX, new DeadlockDetectionListener());
 
-        this.logTxRecords = IgniteSystemProperties.getBoolean(IGNITE_WAL_LOG_TX_RECORDS, false);
-
         cctx.txMetrics().onTxManagerStarted();
 
         cctx.kernalContext().systemView().registerView(TXS_MON_LIST, TXS_MON_LIST_DESC,
@@ -2853,13 +2847,6 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
     }
 
     /**
-     * @return True if {@link TxRecord} records should be logged to WAL.
-     */
-    public boolean logTxRecords() {
-        return logTxRecords;
-    }
-
-    /**
      * Sets MVCC state.
      *
      * @param tx Transaction.
@@ -2917,7 +2904,6 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
 
         // Log tx state change to WAL.
         if (cctx.wal() == null
-            || (!logTxRecords && !tx.txState().mvccEnabled())
             || (baselineTop = cctx.kernalContext().state().clusterState().baselineTopology()) == null
             || !baselineTop.consistentIds().contains(cctx.localNode().consistentId()))
             return null;
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBaselineTopologyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBaselineTopologyTest.java
index cd94e68..956270e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBaselineTopologyTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBaselineTopologyTest.java
@@ -33,7 +33,6 @@ import java.util.stream.Collectors;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteException;
-import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CachePeekMode;
@@ -42,6 +41,7 @@ import org.apache.ignite.cache.affinity.AffinityFunctionContext;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.cluster.BaselineNode;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
@@ -61,7 +61,6 @@ import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.junit.Test;
 
-import static org.apache.ignite.IgniteSystemProperties.IGNITE_WAL_LOG_TX_RECORDS;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 import static org.apache.ignite.cache.PartitionLossPolicy.READ_ONLY_SAFE;
@@ -101,8 +100,6 @@ public class CacheBaselineTopologyTest extends GridCommonAbstractTest {
         cleanPersistenceDir();
 
         disableAutoActivation = false;
-
-        System.clearProperty(IGNITE_WAL_LOG_TX_RECORDS);
     }
 
     /** {@inheritDoc} */
@@ -216,7 +213,7 @@ public class CacheBaselineTopologyTest extends GridCommonAbstractTest {
 
     /** */
     private static class DataNodeFilter implements IgnitePredicate<ClusterNode> {
-
+        /** {@inheritDoc} */
         @Override public boolean apply(ClusterNode clusterNode) {
             return clusterNode.attribute(DATA_NODE);
         }
@@ -875,13 +872,11 @@ public class CacheBaselineTopologyTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void checkMapTxNodes(boolean primary, boolean near) throws Exception {
-        System.setProperty(IgniteSystemProperties.IGNITE_WAL_LOG_TX_RECORDS, "true");
-
         int bltNodesCnt = 3;
 
         Ignite ig = startGrids(bltNodesCnt);
 
-        ig.cluster().active(true);
+        ig.cluster().state(ClusterState.ACTIVE);
 
         ig.createCache(new CacheConfiguration<>()
             .setName(CACHE_NAME)
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgniteLogicalRecoveryWithParamsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgniteLogicalRecoveryWithParamsTest.java
index aac00dc..59e25cd 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgniteLogicalRecoveryWithParamsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgniteLogicalRecoveryWithParamsTest.java
@@ -62,7 +62,6 @@ import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
-import static org.apache.ignite.IgniteSystemProperties.IGNITE_WAL_LOG_TX_RECORDS;
 import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.IGNITE_PDS_SKIP_CHECKPOINT_ON_NODE_STOP;
 import static org.apache.ignite.testframework.GridTestUtils.DFLT_BUSYWAIT_SLEEP_INTERVAL;
 import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
@@ -137,7 +136,7 @@ public class IgniteLogicalRecoveryWithParamsTest extends GridCommonAbstractTest
             {1, false, 1},
             {2, true, 0},
             {2, true, 1},
-            {2, false, 0},
+            //{2, false, 0}, such case is not fixed by now
             {2, false, 1},
         });
     }
@@ -145,15 +144,13 @@ public class IgniteLogicalRecoveryWithParamsTest extends GridCommonAbstractTest
     /**Tests partially commited transactions with further recovery. */
     @Test
     @WithSystemProperty(key = IGNITE_PDS_SKIP_CHECKPOINT_ON_NODE_STOP, value = "true")
-    @WithSystemProperty(key = IGNITE_WAL_LOG_TX_RECORDS, value = "true")
-    public void testPartiallyCommitedTx_TwoNode_WithCpOnNodeStop_MultiNodeTx_OneBackup() throws Exception {
+    public void testPartiallyCommitedTx_WithCpOnNodeStop() throws Exception {
         testPartiallyCommitedTx();
     }
 
     /**Tests partially commited transactions with further recovery. */
     @Test
-    @WithSystemProperty(key = IGNITE_WAL_LOG_TX_RECORDS, value = "true")
-    public void testPartiallyCommitedTx_TwoNode_WithoutCpOnNodeStop_SingleNodeTx() throws Exception {
+    public void testPartiallyCommitedTx_WithoutCpOnNodeStop() throws Exception {
         testPartiallyCommitedTx();
     }
 
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
index 77e25c3..efa2924 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
@@ -45,11 +45,11 @@ import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.IgniteEvents;
-import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheRebalanceMode;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
@@ -186,8 +186,6 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
         stopAllGrids();
 
         cleanPersistenceDir();
-
-        System.clearProperty(IgniteSystemProperties.IGNITE_WAL_LOG_TX_RECORDS);
     }
 
     /**
@@ -1105,11 +1103,9 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
      */
     @Test
     public void testTxRecordsReadWoBinaryMeta() throws Exception {
-        System.setProperty(IgniteSystemProperties.IGNITE_WAL_LOG_TX_RECORDS, "true");
-
         Ignite ignite = startGrid("node0");
 
-        ignite.cluster().active(true);
+        ignite.cluster().state(ClusterState.ACTIVE);
 
         Map<Object, IndexedObject> map = new TreeMap<>();
 
@@ -1118,7 +1114,7 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
 
         ignite.cache(CACHE_NAME).putAll(map);
 
-        ignite.cluster().active(false);
+        ignite.cluster().state(ClusterState.INACTIVE);
 
         String workDir = U.defaultWorkDirectory();
 
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRollbackAsyncWithPersistenceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRollbackAsyncWithPersistenceTest.java
index a11dca7..60b811e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRollbackAsyncWithPersistenceTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRollbackAsyncWithPersistenceTest.java
@@ -17,8 +17,6 @@
 
 package org.apache.ignite.internal.processors.cache.transactions;
 
-import static org.apache.ignite.IgniteSystemProperties.IGNITE_WAL_LOG_TX_RECORDS;
-
 /**
  * Tests an ability to rollback near transactions.
  */
@@ -29,18 +27,6 @@ public class TxRollbackAsyncWithPersistenceTest extends TxRollbackAsyncTest {
     }
 
     /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        System.setProperty(IGNITE_WAL_LOG_TX_RECORDS, "true");
-
-        super.beforeTestsStarted();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        System.clearProperty(IGNITE_WAL_LOG_TX_RECORDS);
-    }
-
-    /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
         cleanPersistenceDir();
 
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java
index ea130c9..b0ae418 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java
@@ -53,6 +53,7 @@ import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.cache.query.annotations.QuerySqlField;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.DataRegionConfiguration;
@@ -131,7 +132,6 @@ import org.junit.Assert;
 import org.junit.Test;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_DISABLE_WAL_DURING_REBALANCING;
-import static org.apache.ignite.IgniteSystemProperties.IGNITE_WAL_LOG_TX_RECORDS;
 import static org.apache.ignite.configuration.DataStorageConfiguration.DFLT_CHECKPOINT_FREQ;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IGNITE_INSTANCE_NAME;
 import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
@@ -1565,8 +1565,8 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest {
      */
     @Test
     public void testRecoveryOnTransactionalAndPartitionedCache() throws Exception {
-        IgniteEx ignite = (IgniteEx)startGrids(3);
-        ignite.cluster().active(true);
+        IgniteEx ignite = startGrids(3);
+        ignite.cluster().state(ClusterState.ACTIVE);
 
         final String cacheName = "transactional";
 
@@ -1618,8 +1618,8 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest {
 
         stopAllGrids();
 
-        ignite = (IgniteEx)startGrids(3);
-        ignite.cluster().active(true);
+        ignite = startGrids(3);
+        ignite.cluster().state(ClusterState.ACTIVE);
 
         cache = ignite.cache(cacheName);
 
@@ -1636,10 +1636,9 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest {
      * @throws Exception If any fail.
      */
     @Test
-    @WithSystemProperty(key = IGNITE_WAL_LOG_TX_RECORDS, value = "true")
     public void testTxRecordsConsistency() throws Exception {
-        IgniteEx ignite = (IgniteEx)startGrids(3);
-        ignite.cluster().active(true);
+        IgniteEx ignite = startGrids(3);
+        ignite.cluster().state(ClusterState.ACTIVE);
 
         final String cacheName = "transactional";
 
@@ -1682,12 +1681,10 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest {
                 cache.put(key, value);
             }
 
-            if (random.nextBoolean()) {
+            if (random.nextBoolean())
                 tx.commit();
-            }
-            else {
+            else
                 tx.rollback();
-            }
 
             if (t % 50 == 0)
                 log.info("Finished transaction " + t);
diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySegmentationAndConnectionRestoreTest.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySegmentationAndConnectionRestoreTest.java
index d33932b..9a1cf48 100644
--- a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySegmentationAndConnectionRestoreTest.java
+++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySegmentationAndConnectionRestoreTest.java
@@ -41,7 +41,6 @@ import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.spi.discovery.zk.ZookeeperDiscoverySpi;
 import org.apache.ignite.spi.discovery.zk.ZookeeperDiscoverySpiTestUtil;
 import org.apache.ignite.testframework.GridTestUtils;
-import org.apache.ignite.testframework.junits.WithSystemProperty;
 import org.apache.ignite.transactions.Transaction;
 import org.apache.zookeeper.ZkTestClientCnxnSocketNIO;
 import org.apache.zookeeper.ZooKeeper;
@@ -49,7 +48,6 @@ import org.apache.zookeeper.server.quorum.QuorumPeer;
 import org.junit.Ignore;
 import org.junit.Test;
 
-import static org.apache.ignite.IgniteSystemProperties.IGNITE_WAL_LOG_TX_RECORDS;
 import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC;
 import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
 import static org.apache.ignite.transactions.TransactionIsolation.READ_COMMITTED;
@@ -67,7 +65,6 @@ public class ZookeeperDiscoverySegmentationAndConnectionRestoreTest extends Zook
      * @see <a href="https://issues.apache.org/jira/browse/IGNITE-9040">IGNITE-9040</a> ticket for more context of the test.
      */
     @Test
-    @WithSystemProperty(key = IGNITE_WAL_LOG_TX_RECORDS, value = "true")
     public void testStopNodeOnSegmentaion() throws Exception {
         sesTimeout = 2000;
         testSockNio = true;