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/29 14:08:15 UTC

ignite git commit: IGNITE-8902 Fixed rollback of GridDhtTxRemote for one-phase commit - Fixes #5089.

Repository: ignite
Updated Branches:
  refs/heads/master 8bd957578 -> 996a1d5ed


IGNITE-8902 Fixed rollback of GridDhtTxRemote for one-phase commit - Fixes #5089.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


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

Branch: refs/heads/master
Commit: 996a1d5ed8af0034dd1541b206bda0ac9467f981
Parents: 8bd9575
Author: ascherbakoff <al...@gmail.com>
Authored: Mon Oct 29 16:39:44 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Oct 29 17:08:00 2018 +0300

----------------------------------------------------------------------
 .../cache/GridCacheSharedContext.java           |   1 -
 .../distributed/dht/GridDhtTxPrepareFuture.java | 140 ++++++------
 .../cache/transactions/IgniteTxHandler.java     |   1 +
 .../cache/transactions/IgniteTxManager.java     |   3 +-
 .../internal/TestRecordingCommunicationSpi.java |  20 ++
 .../TxRollbackOnTimeoutOnePhaseCommitTest.java  | 215 +++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite6.java       |   2 +
 7 files changed, 302 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/996a1d5e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
index b5cd82b..6c72169 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
@@ -1149,5 +1149,4 @@ public class GridCacheSharedContext<K, V> {
     public void setTxManager(IgniteTxManager txMgr) {
         this.txMgr = txMgr;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/996a1d5e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
----------------------------------------------------------------------
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 8d35a47..3232305 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
@@ -26,6 +26,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
@@ -216,6 +217,9 @@ public final class GridDhtTxPrepareFuture extends GridCacheCompoundFuture<Ignite
     /** Timeout object. */
     private final PrepareTimeoutObject timeoutObj;
 
+    /** */
+    private CountDownLatch timeoutAddedLatch;
+
     /**
      * @param cctx Context.
      * @param tx Transaction.
@@ -259,6 +263,9 @@ public final class GridDhtTxPrepareFuture extends GridCacheCompoundFuture<Ignite
         assert nearMap != null;
 
         timeoutObj = timeout > 0 ? new PrepareTimeoutObject(timeout) : null;
+
+        if (tx.onePhaseCommit())
+            timeoutAddedLatch = new CountDownLatch(1);
     }
 
     /** {@inheritDoc} */
@@ -691,6 +698,14 @@ public final class GridDhtTxPrepareFuture extends GridCacheCompoundFuture<Ignite
             if (!MAPPED_UPD.compareAndSet(this, 0, 1))
                 return false;
 
+            if (timeoutObj != null && tx.onePhaseCommit()) {
+                U.awaitQuiet(timeoutAddedLatch);
+
+                // Disable timeouts after all locks are acquired for one-phase commit or partition desync will occur.
+                if (!cctx.time().removeTimeoutObject(timeoutObj))
+                    return true; // Should not proceed with prepare if tx is already timed out.
+            }
+
             if (forceKeysFut == null || (forceKeysFut.isDone() && forceKeysFut.error() == null))
                 prepare0();
             else {
@@ -729,7 +744,7 @@ public final class GridDhtTxPrepareFuture extends GridCacheCompoundFuture<Ignite
             tx.clearPrepareFuture(this);
 
         // Do not commit one-phase commit transaction if originating node has near cache enabled.
-        if (tx.onePhaseCommit() && tx.commitOnPrepare()) {
+        if (tx.commitOnPrepare()) {
             assert last;
 
             Throwable prepErr = this.err;
@@ -739,61 +754,55 @@ public final class GridDhtTxPrepareFuture extends GridCacheCompoundFuture<Ignite
 
             onComplete(res);
 
-            if (tx.commitOnPrepare()) {
-                if (tx.markFinalizing(IgniteInternalTx.FinalizationStatus.USER_FINISH)) {
-                    CIX1<IgniteInternalFuture<IgniteInternalTx>> resClo =
-                        new CIX1<IgniteInternalFuture<IgniteInternalTx>>() {
-                            @Override public void applyx(IgniteInternalFuture<IgniteInternalTx> fut) {
-                                if(res.error() == null && fut.error() != null)
-                                    res.error(fut.error());
+            if (tx.markFinalizing(IgniteInternalTx.FinalizationStatus.USER_FINISH)) {
+                CIX1<IgniteInternalFuture<IgniteInternalTx>> resClo =
+                    new CIX1<IgniteInternalFuture<IgniteInternalTx>>() {
+                        @Override public void applyx(IgniteInternalFuture<IgniteInternalTx> fut) {
+                            if (res.error() == null && fut.error() != null)
+                                res.error(fut.error());
 
-                                if (REPLIED_UPD.compareAndSet(GridDhtTxPrepareFuture.this, 0, 1))
-                                    sendPrepareResponse(res);
-                            }
-                        };
-
-                    try {
-                        if (prepErr == null) {
-                            try {
-                                tx.commitAsync().listen(resClo);
-                            }
-                            catch (Throwable e) {
-                                res.error(e);
+                            if (REPLIED_UPD.compareAndSet(GridDhtTxPrepareFuture.this, 0, 1))
+                                sendPrepareResponse(res);
+                        }
+                    };
 
-                                tx.systemInvalidate(true);
+                try {
+                    if (prepErr == null) {
+                        try {
+                            tx.commitAsync().listen(resClo);
+                        }
+                        catch (Throwable e) {
+                            res.error(e);
 
-                                try {
-                                    tx.rollbackAsync().listen(resClo);
-                                }
-                                catch (Throwable e1) {
-                                    e.addSuppressed(e1);
-                                }
+                            tx.systemInvalidate(true);
 
-                                throw e;
-                            }
-                        }
-                        else if (!cctx.kernalContext().isStopping()) {
                             try {
                                 tx.rollbackAsync().listen(resClo);
                             }
-                            catch (Throwable e) {
-                                if (err != null)
-                                    err.addSuppressed(e);
-
-                                throw err;
+                            catch (Throwable e1) {
+                                e.addSuppressed(e1);
                             }
+
+                            throw e;
                         }
                     }
-                    catch (Throwable e){
-                        tx.logTxFinishErrorSafe(log, true, e);
+                    else if (!cctx.kernalContext().isStopping()) {
+                        try {
+                            tx.rollbackAsync().listen(resClo);
+                        }
+                        catch (Throwable e) {
+                            if (err != null)
+                                err.addSuppressed(e);
 
-                        cctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e));
+                            throw err;
+                        }
                     }
                 }
-            }
-            else {
-                if (REPLIED_UPD.compareAndSet(this, 0, 1))
-                    sendPrepareResponse(res);
+                catch (Throwable e) {
+                    tx.logTxFinishErrorSafe(log, true, e);
+
+                    cctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e));
+                }
             }
 
             return true;
@@ -1062,9 +1071,14 @@ public final class GridDhtTxPrepareFuture extends GridCacheCompoundFuture<Ignite
 
         readyLocks();
 
-        if (timeoutObj != null && !isDone()) {
-            // Start timeout tracking after 'readyLocks' to avoid race with timeout processing.
+        // Start timeout tracking after 'readyLocks' to avoid race with timeout processing.
+        if (timeoutObj != null) {
             cctx.time().addTimeoutObject(timeoutObj);
+
+            // Fix race with add/remove timeout object if locks are mapped from another
+            // thread before timeout object is enqueued.
+            if (tx.onePhaseCommit())
+                timeoutAddedLatch.countDown();
         }
 
         mapIfLocked();
@@ -1216,8 +1230,6 @@ public final class GridDhtTxPrepareFuture extends GridCacheCompoundFuture<Ignite
      *
      */
     private void prepare0() {
-        boolean skipInit = false;
-
         try {
             if (tx.serializable() && tx.optimistic()) {
                 IgniteCheckedException err0;
@@ -1252,8 +1264,6 @@ public final class GridDhtTxPrepareFuture extends GridCacheCompoundFuture<Ignite
                 }
             }
 
-            IgniteInternalFuture<MvccSnapshot> waitCrdCntrFut = null;
-
             if (req.requestMvccCounter()) {
                 assert last;
 
@@ -1294,37 +1304,11 @@ public final class GridDhtTxPrepareFuture extends GridCacheCompoundFuture<Ignite
             if (isDone())
                 return;
 
-            if (last) {
-                if (waitCrdCntrFut != null) {
-                    skipInit = true;
-
-                    waitCrdCntrFut.listen(new IgniteInClosure<IgniteInternalFuture<MvccSnapshot>>() {
-                        @Override public void apply(IgniteInternalFuture<MvccSnapshot> fut) {
-                            try {
-                                fut.get();
-
-                                sendPrepareRequests();
-
-                                markInitialized();
-                            }
-                            catch (Throwable e) {
-                                U.error(log, "Failed to get mvcc version for tx [txId=" + tx.nearXidVersion() +
-                                    ", err=" + e + ']', e);
-
-                                GridNearTxPrepareResponse res = createPrepareResponse(e);
-
-                                onDone(res, res.error());
-                            }
-                        }
-                    });
-                }
-                else
-                    sendPrepareRequests();
-            }
+            if (last)
+                sendPrepareRequests();
         }
         finally {
-            if (!skipInit)
-                markInitialized();
+            markInitialized();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/996a1d5e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
index 75e2087..0c51e15 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
@@ -1685,6 +1685,7 @@ public class IgniteTxHandler {
                     single,
                     req.storeWriteThrough());
 
+                tx.onePhaseCommit(req.onePhaseCommit());
                 tx.writeVersion(req.writeVersion());
 
                 tx = ctx.tm().onCreated(null, tx);

http://git-wip-us.apache.org/repos/asf/ignite/blob/996a1d5e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
----------------------------------------------------------------------
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 0c2ca34..032df22 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
@@ -891,7 +891,8 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
             throw new IgniteCheckedException("Transaction is marked for rollback: " + tx);
         }
 
-        if (tx.remainingTime() == -1) {
+        // One-phase commit tx cannot timeout on prepare because it is expected to be committed.
+        if (tx.remainingTime() == -1 && !tx.onePhaseCommit()) {
             tx.setRollbackOnly();
 
             throw new IgniteTxTimeoutCheckedException("Transaction timed out: " + this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/996a1d5e/modules/core/src/test/java/org/apache/ignite/internal/TestRecordingCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/TestRecordingCommunicationSpi.java b/modules/core/src/test/java/org/apache/ignite/internal/TestRecordingCommunicationSpi.java
index 7b68a6b..988395f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/TestRecordingCommunicationSpi.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/TestRecordingCommunicationSpi.java
@@ -218,6 +218,26 @@ public class TestRecordingCommunicationSpi extends TcpCommunicationSpi {
     }
 
     /**
+     * @param size Size
+     * @param timeout Timeout.
+     * @throws InterruptedException
+     */
+    public boolean waitForBlocked(int size, long timeout) throws InterruptedException {
+        long t0 = U.currentTimeMillis() + timeout;
+
+        synchronized (this) {
+            while (blockedMsgs.size() < size) {
+                wait(1000);
+
+                if (U.currentTimeMillis() >= t0)
+                    return false;
+            }
+        }
+
+        return true;
+    }
+
+    /**
      * @throws InterruptedException If interrupted.
      */
     public void waitForRecorded() throws InterruptedException {

http://git-wip-us.apache.org/repos/asf/ignite/blob/996a1d5e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRollbackOnTimeoutOnePhaseCommitTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRollbackOnTimeoutOnePhaseCommitTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRollbackOnTimeoutOnePhaseCommitTest.java
new file mode 100644
index 0000000..2375d9e
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRollbackOnTimeoutOnePhaseCommitTest.java
@@ -0,0 +1,215 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.transactions;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareResponse;
+import org.apache.ignite.internal.processors.cache.verify.IdleVerifyResultV2;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionTimeoutException;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC;
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+
+/**
+ * Tests rollback on timeout scenarios for one-phase commit protocol.
+ */
+public class TxRollbackOnTimeoutOnePhaseCommitTest extends GridCommonAbstractTest {
+    /** IP finder. */
+    private static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final int GRID_CNT = 2;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        boolean client = igniteInstanceName.startsWith("client");
+
+        cfg.setClientMode(client);
+
+        if (!client) {
+            CacheConfiguration ccfg = new CacheConfiguration(DEFAULT_CACHE_NAME);
+
+            ccfg.setAtomicityMode(TRANSACTIONAL);
+            ccfg.setBackups(1);
+            ccfg.setWriteSynchronizationMode(FULL_SYNC);
+            ccfg.setOnheapCacheEnabled(false);
+
+            cfg.setCacheConfiguration(ccfg);
+        }
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        startGridsMultiThreaded(GRID_CNT);
+
+        startGrid("client");
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /** */
+    public void testRollbackOnTimeoutPartitionDesyncPessimistic() throws Exception {
+        doTestRollbackOnTimeoutPartitionDesync(PESSIMISTIC);
+    }
+
+    /** */
+    public void testRollbackOnTimeoutPartitionDesyncOptimistic() throws Exception {
+        doTestRollbackOnTimeoutPartitionDesync(OPTIMISTIC);
+    }
+
+    /** */
+    public void testUnlockOptimistic() throws IgniteCheckedException {
+        IgniteEx client = grid("client");
+
+        assertNotNull(client.cache(DEFAULT_CACHE_NAME));
+
+        int key = 0;
+
+        CountDownLatch lock = new CountDownLatch(1);
+        CountDownLatch finish = new CountDownLatch(1);
+
+        IgniteInternalFuture fut = runAsync(() -> {
+            try (Transaction tx = client.transactions().txStart(PESSIMISTIC, REPEATABLE_READ, 0, 1)) {
+                client.cache(DEFAULT_CACHE_NAME).put(key, key + 1);
+
+                lock.countDown();
+
+                try {
+                    assertTrue(U.await(finish, 30, TimeUnit.SECONDS));
+                }
+                catch (IgniteInterruptedCheckedException e) {
+                    fail();
+                }
+
+                tx.commit();
+            }
+        });
+
+        try (Transaction tx = client.transactions().txStart(OPTIMISTIC, REPEATABLE_READ, 200, 1)) {
+            try {
+                assertTrue(U.await(lock, 30, TimeUnit.SECONDS));
+            }
+            catch (IgniteInterruptedCheckedException e) {
+                fail();
+            }
+
+            client.cache(DEFAULT_CACHE_NAME).put(key, key);
+
+            tx.commit();
+
+            // fail(); // TODO IGNITE-10027 throw timeout exception for optimistic timeout.
+        }
+        catch (Exception e) {
+            assertTrue(e.getClass().getName(), X.hasCause(e, TransactionTimeoutException.class));
+        }
+
+        assertNull(client.cache(DEFAULT_CACHE_NAME).get(key));
+
+        finish.countDown();
+
+        fut.get();
+
+        assertEquals(1, client.cache(DEFAULT_CACHE_NAME).get(key));
+    }
+
+    /** */
+    private void doTestRollbackOnTimeoutPartitionDesync(TransactionConcurrency concurrency) throws Exception {
+        IgniteEx client = grid("client");
+
+        assertNotNull(client.cache(DEFAULT_CACHE_NAME));
+
+        int key = 0;
+
+        Ignite primary = primaryNode(key, DEFAULT_CACHE_NAME);
+        Ignite backup = backupNode(key, DEFAULT_CACHE_NAME);
+
+        TestRecordingCommunicationSpi backupSpi = TestRecordingCommunicationSpi.spi(backup);
+        backupSpi.blockMessages(GridDhtTxPrepareResponse.class, primary.name());
+
+        IgniteInternalFuture fut = runAsync(() -> {
+            try {
+                backupSpi.waitForBlocked(1, 5000);
+            }
+            catch (InterruptedException e) {
+                fail();
+            }
+
+            doSleep(500);
+
+            backupSpi.stopBlock();
+        });
+
+        try (Transaction tx = client.transactions().txStart(concurrency, REPEATABLE_READ, 500, 1)) {
+            client.cache(DEFAULT_CACHE_NAME).put(key, key);
+
+            tx.commit();
+        }
+        catch (Exception e) {
+            assertTrue(e.getClass().getName(), X.hasCause(e, TransactionTimeoutException.class));
+        }
+
+        fut.get();
+
+        IdleVerifyResultV2 res = idleVerify(client, DEFAULT_CACHE_NAME);
+
+        if (res.hasConflicts()) {
+            StringBuilder b = new StringBuilder();
+
+            res.print(b::append);
+
+            fail(b.toString());
+        }
+
+        checkFutures();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/996a1d5e/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite6.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite6.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite6.java
index b69ebe4..535797c 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite6.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite6.java
@@ -45,6 +45,7 @@ import org.apache.ignite.internal.processors.cache.transactions.TxRollbackAsyncT
 import org.apache.ignite.internal.processors.cache.transactions.TxRollbackOnIncorrectParamsTest;
 import org.apache.ignite.internal.processors.cache.transactions.TxRollbackOnTimeoutNearCacheTest;
 import org.apache.ignite.internal.processors.cache.transactions.TxRollbackOnTimeoutNoDeadlockDetectionTest;
+import org.apache.ignite.internal.processors.cache.transactions.TxRollbackOnTimeoutOnePhaseCommitTest;
 import org.apache.ignite.internal.processors.cache.transactions.TxRollbackOnTimeoutTest;
 import org.apache.ignite.internal.processors.cache.transactions.TxRollbackOnTopologyChangeTest;
 import org.apache.ignite.internal.processors.cache.transactions.TxStateChangeEventTest;
@@ -79,6 +80,7 @@ public class IgniteCacheTestSuite6 extends TestSuite {
         suite.addTestSuite(TxRollbackAsyncTest.class);
         suite.addTestSuite(TxRollbackAsyncNearCacheTest.class);
         suite.addTestSuite(TxRollbackOnTopologyChangeTest.class);
+        suite.addTestSuite(TxRollbackOnTimeoutOnePhaseCommitTest.class);
 
         suite.addTestSuite(TxOptimisticPrepareOnUnstableTopologyTest.class);