You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2022/08/02 18:08:04 UTC

[GitHub] [ignite] skorotkov opened a new pull request, #10178: Fix cluster lock after tx recovery

skorotkov opened a new pull request, #10178:
URL: https://github.com/apache/ignite/pull/10178

   Thank you for submitting the pull request to the Apache Ignite.
   
   In order to streamline the review of the contribution 
   we ask you to ensure the following steps have been taken:
   
   ### The Contribution Checklist
   - [ ] There is a single JIRA ticket related to the pull request. 
   - [ ] The web-link to the pull request is attached to the JIRA ticket.
   - [ ] The JIRA ticket has the _Patch Available_ state.
   - [ ] The pull request body describes changes that have been made. 
   The description explains _WHAT_ and _WHY_ was made instead of _HOW_.
   - [ ] The pull request title is treated as the final commit message. 
   The following pattern must be used: `IGNITE-XXXX Change summary` where `XXXX` - number of JIRA issue.
   - [ ] A reviewer has been mentioned through the JIRA comments 
   (see [the Maintainers list](https://cwiki.apache.org/confluence/display/IGNITE/How+to+Contribute#HowtoContribute-ReviewProcessandMaintainers)) 
   - [ ] The pull request has been checked by the Teamcity Bot and 
   the `green visa` attached to the JIRA ticket (see [TC.Bot: Check PR](https://mtcga.gridgain.com/prs.html))
   
   ### Notes
   - [How to Contribute](https://cwiki.apache.org/confluence/display/IGNITE/How+to+Contribute)
   - [Coding abbreviation rules](https://cwiki.apache.org/confluence/display/IGNITE/Abbreviation+Rules)
   - [Coding Guidelines](https://cwiki.apache.org/confluence/display/IGNITE/Coding+Guidelines)
   - [Apache Ignite Teamcity Bot](https://cwiki.apache.org/confluence/display/IGNITE/Apache+Ignite+Teamcity+Bot)
   
   If you need any help, please email dev@ignite.apache.org or ask anу advice on http://asf.slack.com _#ignite_ channel.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r938099732


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +259,110 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <ul>
+     *  <li>Start 3 nodes (g0, g1, g2) and cache with 2 backups.</li>
+     *  <li>Prepare a transaction with g2 as a primary node.</li>
+     *  <li>Kill g2.</li>
+     *  <li>Enforce the concurrent processing of transaction in tx recovery on g1.</li>
+     * </ul>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished both on g0 and g1
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 75; iter++) {
+            stopAllGrids();
+
+            log.info("iteration=" + iter);
+            final IgniteEx grid0 = startGrid("g0");
+            final IgniteEx grid1 = startGrid("g1",
+                    cfg -> cfg.setSystemThreadPoolSize(1).setStripedPoolSize(1));
+            final IgniteEx grid2 = startGrid("g2");
+
+            grid0.cluster().state(ACTIVE);
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Integer g2Key = primaryKeys(grid2.cache(DEFAULT_CACHE_NAME), 1, 0).get(0);
+
+            List<IgniteInternalTx> txs0 = null;
+            List<IgniteInternalTx> txs1 = null;
+
+            CountDownLatch grid1BlockLatch = new CountDownLatch(1);
+
+            try (final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                cache.put(g2Key, Boolean.TRUE);
+                TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;
+                p.tx().prepare(true);
+
+                txs0 = txs(grid0);
+                txs1 = txs(grid1);
+                List<IgniteInternalTx> txs2 = txs(grid2);
+
+                assertTrue(txs0.size() == 1);
+                assertTrue(txs1.size() == 1);
+                assertTrue(txs2.size() == 1);
+
+                // Prevent tx recovery request to be sent from grid0 to grid1.
+                spi(grid0).blockMessages(GridCacheTxRecoveryRequest.class, grid1.name());
+
+                // Block recovery procedure processing on grid1
+                grid1.context().pools().getSystemExecutorService().execute(() -> U.awaitQuiet(grid1BlockLatch));
+                // Block stripe tx recovery request processing on grid1
+                int stripe = U.safeAbs(p.tx().xidVersion().hashCode());
+                grid1.context().pools().getStripedExecutorService().execute(stripe, () -> U.awaitQuiet(grid1BlockLatch));
+
+                // Prevent finish request processing on grid0 and grid1.
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid0.name());
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid1.name());
+
+                runAsync(() -> {
+                    grid2.close();
+                    return null;
+                });
+            }
+            catch (Exception ignored) {
+                // Expected.
+            }
+
+            // Wait grid0 is ready to send the tx recovery request to grid1
+            spi(grid0).waitForBlocked();
+            // Let grid0 send the tx recovery request to grid1
+            log.info("unblock grid0");
+            spi(grid0).stopBlock();
+            // Give grid1 some time to receive the tx recovery request (processing is still blocked in grid1).
+            log.info("sleep in grid0");
+            doSleep(100);

Review Comment:
   Get rid of sleep



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] anton-vinogradov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
anton-vinogradov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r947027468


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentTest.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.Collection;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ThreadPoolExecutor;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME)
+            .setCacheMode(PARTITIONED)
+            .setBackups(2)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setAffinity(new RendezvousAffinityFunction(false, 1)));
+
+        return cfg;
+    }
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to near and primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Blocking is used to force concurrent processing on one of the backup nodes. Another
+     * backup works unconstrained to provide the right environment for the blocked one.
+     * In particular, it should send a tx recovery request to the blocked backup.
+     * <p>
+     * Use several attempts to reproduce the race condition in the blocked backup node.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnNearAndPrimaryFail() throws Exception {
+        startGrids(2);
+
+        final int key = 0;
+
+        for (int iter = 0; iter < 100; iter++) {
+            startGrid(iter + 2);
+
+            awaitPartitionMapExchange();
+
+            final Ignite primary = primaryNode(key, DEFAULT_CACHE_NAME);
+            final List<Ignite> backups = backupNodes(key, DEFAULT_CACHE_NAME);
+
+            final IgniteCache<Object, Object> cache = primary.cache(DEFAULT_CACHE_NAME);
+
+            final TransactionProxyImpl<?, ?> tx = (TransactionProxyImpl<?, ?>)primary.transactions().txStart();
+
+            cache.put(key, key);
+
+            tx.tx().prepare(true);
+
+            for (Ignite grid : G.allGrids())
+                assertTrue(((IgniteEx)grid).context().cache().context().tm().activeTransactions().size() == 1);
+
+            final Collection<IgniteInternalTx> backupTransactions = new LinkedList<>();
+
+            for (Ignite backup : backups)
+                backupTransactions.addAll(((IgniteEx)backup).context().cache().context().tm().activeTransactions());
+
+            assertTrue(backupTransactions.size() == 2);
+
+            final CountDownLatch ensureBothPoolsAreBlockedLatch = new CountDownLatch(2);
+            final CountDownLatch unblockBothPoolsLatch = new CountDownLatch(1);
+
+            final Runnable poolBlockerTask = () -> {
+                try {
+                    ensureBothPoolsAreBlockedLatch.countDown();
+                    unblockBothPoolsLatch.await();
+                }
+                catch (InterruptedException ignored) {
+                    // No-op.
+                }
+            };
+
+            final IgniteEx blockedBackup = (IgniteEx)backups.get(0);
+
+            blockedBackup.context().pools().getSystemExecutorService().execute(poolBlockerTask);
+
+            blockedBackup.context().pools().getStripedExecutorService().execute(0, poolBlockerTask);
+
+            ensureBothPoolsAreBlockedLatch.await();
+
+            runAsync(primary::close);
+
+            waitForTxRecoveryRequestEnqueuedOn(blockedBackup);
+
+            waitForTxRecoveryTaskEnqueuedOn(blockedBackup);
+
+            // Unblock processing in blocked backup node. Simultaneously in striped and system pools to start recovery
+            // procedure and the tx recovery request processing at the "same" moment (for the same transaction). This
+            // should increase chances for race condition occur in the IgniteTxAdapter#markFinalizing.
+            unblockBothPoolsLatch.countDown();
+
+            waitForTopology(2);
+
+            awaitPartitionMapExchange();

Review Comment:
   We must check tx recovered correctly here to gain some explicit achievement.
   Possible solution is just to perform read from both nodes (read from backup should be enabled to make sure both contain same value).
   Feel free to check some other way.



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentTest.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.Collection;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ThreadPoolExecutor;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME)
+            .setCacheMode(PARTITIONED)
+            .setBackups(2)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setAffinity(new RendezvousAffinityFunction(false, 1)));
+
+        return cfg;
+    }
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to near and primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Blocking is used to force concurrent processing on one of the backup nodes. Another
+     * backup works unconstrained to provide the right environment for the blocked one.
+     * In particular, it should send a tx recovery request to the blocked backup.
+     * <p>
+     * Use several attempts to reproduce the race condition in the blocked backup node.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnNearAndPrimaryFail() throws Exception {
+        startGrids(2);
+
+        final int key = 0;
+
+        for (int iter = 0; iter < 100; iter++) {
+            startGrid(iter + 2);
+
+            awaitPartitionMapExchange();
+
+            final Ignite primary = primaryNode(key, DEFAULT_CACHE_NAME);
+            final List<Ignite> backups = backupNodes(key, DEFAULT_CACHE_NAME);
+
+            final IgniteCache<Object, Object> cache = primary.cache(DEFAULT_CACHE_NAME);
+
+            final TransactionProxyImpl<?, ?> tx = (TransactionProxyImpl<?, ?>)primary.transactions().txStart();
+
+            cache.put(key, key);
+
+            tx.tx().prepare(true);
+
+            for (Ignite grid : G.allGrids())
+                assertTrue(((IgniteEx)grid).context().cache().context().tm().activeTransactions().size() == 1);
+
+            final Collection<IgniteInternalTx> backupTransactions = new LinkedList<>();
+
+            for (Ignite backup : backups)
+                backupTransactions.addAll(((IgniteEx)backup).context().cache().context().tm().activeTransactions());
+
+            assertTrue(backupTransactions.size() == 2);
+
+            final CountDownLatch ensureBothPoolsAreBlockedLatch = new CountDownLatch(2);
+            final CountDownLatch unblockBothPoolsLatch = new CountDownLatch(1);
+
+            final Runnable poolBlockerTask = () -> {
+                try {
+                    ensureBothPoolsAreBlockedLatch.countDown();
+                    unblockBothPoolsLatch.await();
+                }
+                catch (InterruptedException ignored) {
+                    // No-op.
+                }
+            };
+
+            final IgniteEx blockedBackup = (IgniteEx)backups.get(0);
+
+            blockedBackup.context().pools().getSystemExecutorService().execute(poolBlockerTask);
+
+            blockedBackup.context().pools().getStripedExecutorService().execute(0, poolBlockerTask);
+
+            ensureBothPoolsAreBlockedLatch.await();
+
+            runAsync(primary::close);
+
+            waitForTxRecoveryRequestEnqueuedOn(blockedBackup);
+
+            waitForTxRecoveryTaskEnqueuedOn(blockedBackup);

Review Comment:
   no newline needed I think



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentTest.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.Collection;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ThreadPoolExecutor;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME)
+            .setCacheMode(PARTITIONED)
+            .setBackups(2)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setAffinity(new RendezvousAffinityFunction(false, 1)));
+
+        return cfg;
+    }
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to near and primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Blocking is used to force concurrent processing on one of the backup nodes. Another
+     * backup works unconstrained to provide the right environment for the blocked one.
+     * In particular, it should send a tx recovery request to the blocked backup.
+     * <p>
+     * Use several attempts to reproduce the race condition in the blocked backup node.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnNearAndPrimaryFail() throws Exception {
+        startGrids(2);
+
+        final int key = 0;
+
+        for (int iter = 0; iter < 100; iter++) {
+            startGrid(iter + 2);
+
+            awaitPartitionMapExchange();
+
+            final Ignite primary = primaryNode(key, DEFAULT_CACHE_NAME);
+            final List<Ignite> backups = backupNodes(key, DEFAULT_CACHE_NAME);
+
+            final IgniteCache<Object, Object> cache = primary.cache(DEFAULT_CACHE_NAME);
+
+            final TransactionProxyImpl<?, ?> tx = (TransactionProxyImpl<?, ?>)primary.transactions().txStart();
+
+            cache.put(key, key);
+
+            tx.tx().prepare(true);
+
+            for (Ignite grid : G.allGrids())
+                assertTrue(((IgniteEx)grid).context().cache().context().tm().activeTransactions().size() == 1);
+
+            final Collection<IgniteInternalTx> backupTransactions = new LinkedList<>();
+
+            for (Ignite backup : backups)
+                backupTransactions.addAll(((IgniteEx)backup).context().cache().context().tm().activeTransactions());
+
+            assertTrue(backupTransactions.size() == 2);
+
+            final CountDownLatch ensureBothPoolsAreBlockedLatch = new CountDownLatch(2);
+            final CountDownLatch unblockBothPoolsLatch = new CountDownLatch(1);
+
+            final Runnable poolBlockerTask = () -> {
+                try {
+                    ensureBothPoolsAreBlockedLatch.countDown();
+                    unblockBothPoolsLatch.await();
+                }
+                catch (InterruptedException ignored) {
+                    // No-op.
+                }
+            };
+
+            final IgniteEx blockedBackup = (IgniteEx)backups.get(0);
+
+            blockedBackup.context().pools().getSystemExecutorService().execute(poolBlockerTask);
+
+            blockedBackup.context().pools().getStripedExecutorService().execute(0, poolBlockerTask);

Review Comment:
   no newline needed I think



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r944405331


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentOnPrimaryFailTest.java:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.Collection;
+import java.util.LinkedList;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentOnPrimaryFailTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+
+        cfg.setStripedPoolSize(1);

Review Comment:
   fixed



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentOnPrimaryFailTest.java:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.Collection;
+import java.util.LinkedList;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentOnPrimaryFailTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME).setCacheMode(PARTITIONED)
+            .setBackups(2).setAtomicityMode(TRANSACTIONAL).setAffinity(new RendezvousAffinityFunction(false, 1)));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        final IgniteEx backup = startGrids(2);
+
+        final CyclicBarrier backupBlockerBarrier = new CyclicBarrier(3);
+
+        final Runnable backupBlockerTask = () -> {
+            try {
+                backupBlockerBarrier.await();
+            }
+            catch (InterruptedException | BrokenBarrierException ignored) {
+                // Just supress.
+            }
+        };
+
+        for (int iter = 0; iter < 100; iter++) {
+            final IgniteEx primary = startGrid(2);
+
+            awaitPartitionMapExchange();
+
+            final IgniteCache<Object, Object> cache = primary.cache(DEFAULT_CACHE_NAME);
+
+            final TransactionProxyImpl<?, ?> tx = (TransactionProxyImpl<?, ?>)primary.transactions().txStart();
+
+            final Integer key = primaryKeys(cache, 1, 0).get(0);
+
+            cache.put(key, key);
+
+            tx.tx().prepare(true);
+
+            for (Ignite grid : G.allGrids())
+                assertTrue(((IgniteEx)grid).context().cache().context().tm().activeTransactions().size() == 1);
+
+            Collection<IgniteInternalTx> backupTransactions = new LinkedList<>();
+
+            for (Ignite grid : G.allGrids()) {
+                if (grid != primary)

Review Comment:
   fixed



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentOnPrimaryFailTest.java:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.Collection;
+import java.util.LinkedList;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentOnPrimaryFailTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME).setCacheMode(PARTITIONED)
+            .setBackups(2).setAtomicityMode(TRANSACTIONAL).setAffinity(new RendezvousAffinityFunction(false, 1)));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        final IgniteEx backup = startGrids(2);
+
+        final CyclicBarrier backupBlockerBarrier = new CyclicBarrier(3);
+
+        final Runnable backupBlockerTask = () -> {
+            try {
+                backupBlockerBarrier.await();
+            }
+            catch (InterruptedException | BrokenBarrierException ignored) {
+                // Just supress.
+            }
+        };
+
+        for (int iter = 0; iter < 100; iter++) {
+            final IgniteEx primary = startGrid(2);
+
+            awaitPartitionMapExchange();
+
+            final IgniteCache<Object, Object> cache = primary.cache(DEFAULT_CACHE_NAME);
+
+            final TransactionProxyImpl<?, ?> tx = (TransactionProxyImpl<?, ?>)primary.transactions().txStart();
+
+            final Integer key = primaryKeys(cache, 1, 0).get(0);
+
+            cache.put(key, key);
+
+            tx.tx().prepare(true);
+
+            for (Ignite grid : G.allGrids())
+                assertTrue(((IgniteEx)grid).context().cache().context().tm().activeTransactions().size() == 1);
+
+            Collection<IgniteInternalTx> backupTransactions = new LinkedList<>();
+
+            for (Ignite grid : G.allGrids()) {
+                if (grid != primary)
+                    backupTransactions.addAll(((IgniteEx)grid).context().cache().context().tm().activeTransactions());
+            }
+
+            assertTrue(backupTransactions.size() == 2);
+
+            // Block recovery procedure processing on one of the backup nodes.
+            backup.context().pools().getSystemExecutorService().execute(backupBlockerTask);
+
+            // Block stripe tx recovery request processing on one of the backup nodes (note that the only stripe is
+            // configured in the executor).
+            backup.context().pools().getStripedExecutorService().execute(0, backupBlockerTask);
+
+            // Prevent tx finish request processing on both backup nodes.
+            for (Ignite grid : G.allGrids()) {
+                if (grid != primary)

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] anton-vinogradov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
anton-vinogradov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r947691323


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentTest.java:
##########
@@ -103,7 +105,7 @@ public void testRecoveryNotDeadLockOnNearAndPrimaryFail() throws Exception {
 
             final TransactionProxyImpl<?, ?> tx = (TransactionProxyImpl<?, ?>)primary.transactions().txStart();
 
-            cache.put(key, key);
+            cache.put(key, key + iter);

Review Comment:
   why not just (key, iter) ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r938095310


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +259,110 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <ul>
+     *  <li>Start 3 nodes (g0, g1, g2) and cache with 2 backups.</li>
+     *  <li>Prepare a transaction with g2 as a primary node.</li>
+     *  <li>Kill g2.</li>
+     *  <li>Enforce the concurrent processing of transaction in tx recovery on g1.</li>
+     * </ul>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished both on g0 and g1
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 75; iter++) {
+            stopAllGrids();
+
+            log.info("iteration=" + iter);
+            final IgniteEx grid0 = startGrid("g0");
+            final IgniteEx grid1 = startGrid("g1",
+                    cfg -> cfg.setSystemThreadPoolSize(1).setStripedPoolSize(1));
+            final IgniteEx grid2 = startGrid("g2");
+
+            grid0.cluster().state(ACTIVE);
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Integer g2Key = primaryKeys(grid2.cache(DEFAULT_CACHE_NAME), 1, 0).get(0);
+
+            List<IgniteInternalTx> txs0 = null;
+            List<IgniteInternalTx> txs1 = null;
+
+            CountDownLatch grid1BlockLatch = new CountDownLatch(1);
+
+            try (final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                cache.put(g2Key, Boolean.TRUE);
+                TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;
+                p.tx().prepare(true);
+
+                txs0 = txs(grid0);
+                txs1 = txs(grid1);
+                List<IgniteInternalTx> txs2 = txs(grid2);
+
+                assertTrue(txs0.size() == 1);
+                assertTrue(txs1.size() == 1);
+                assertTrue(txs2.size() == 1);
+
+                // Prevent tx recovery request to be sent from grid0 to grid1.
+                spi(grid0).blockMessages(GridCacheTxRecoveryRequest.class, grid1.name());
+
+                // Block recovery procedure processing on grid1
+                grid1.context().pools().getSystemExecutorService().execute(() -> U.awaitQuiet(grid1BlockLatch));
+                // Block stripe tx recovery request processing on grid1
+                int stripe = U.safeAbs(p.tx().xidVersion().hashCode());
+                grid1.context().pools().getStripedExecutorService().execute(stripe, () -> U.awaitQuiet(grid1BlockLatch));
+
+                // Prevent finish request processing on grid0 and grid1.
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid0.name());
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid1.name());
+
+                runAsync(() -> {
+                    grid2.close();
+                    return null;
+                });
+            }
+            catch (Exception ignored) {
+                // Expected.
+            }
+
+            // Wait grid0 is ready to send the tx recovery request to grid1
+            spi(grid0).waitForBlocked();
+            // Let grid0 send the tx recovery request to grid1
+            log.info("unblock grid0");
+            spi(grid0).stopBlock();
+            // Give grid1 some time to receive the tx recovery request (processing is still blocked in grid1).
+            log.info("sleep in grid0");
+            doSleep(100);
+
+            // Unblock processing in grid1. Simultaneously in striped and system pools to start
+            // recovery procedure and the tx recovery request processing at the "same" moment
+            // (for the same transaction). This should increase chances for race condition occur in
+            // the IgniteTxAdapter::markFinalizing.
+            log.info("unblock grid1");

Review Comment:
   Removed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r937578120


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +259,110 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <ul>
+     *  <li>Start 3 nodes (g0, g1, g2) and cache with 2 backups.</li>
+     *  <li>Prepare a transaction with g2 as a primary node.</li>
+     *  <li>Kill g2.</li>
+     *  <li>Enforce the concurrent processing of transaction in tx recovery on g1.</li>
+     * </ul>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished both on g0 and g1
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 75; iter++) {

Review Comment:
   I used 100 initially. But  it appears that in a teamcity environment it takes more than 60 seconds and tc bot complains that test is too long.  So reduced to 75.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] anton-vinogradov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
anton-vinogradov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r943297407


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentOnPrimaryFailTest.java:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.Collection;
+import java.util.LinkedList;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentOnPrimaryFailTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME).setCacheMode(PARTITIONED)
+            .setBackups(2).setAtomicityMode(TRANSACTIONAL).setAffinity(new RendezvousAffinityFunction(false, 1)));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        final IgniteEx backup = startGrids(2);
+
+        final CyclicBarrier backupBlockerBarrier = new CyclicBarrier(3);
+
+        final Runnable backupBlockerTask = () -> {
+            try {
+                backupBlockerBarrier.await();
+            }
+            catch (InterruptedException | BrokenBarrierException ignored) {
+                // Just supress.
+            }
+        };
+
+        for (int iter = 0; iter < 100; iter++) {
+            final IgniteEx primary = startGrid(2);
+
+            awaitPartitionMapExchange();
+
+            final IgniteCache<Object, Object> cache = primary.cache(DEFAULT_CACHE_NAME);
+
+            final TransactionProxyImpl<?, ?> tx = (TransactionProxyImpl<?, ?>)primary.transactions().txStart();
+
+            final Integer key = primaryKeys(cache, 1, 0).get(0);
+
+            cache.put(key, key);
+
+            tx.tx().prepare(true);
+
+            for (Ignite grid : G.allGrids())
+                assertTrue(((IgniteEx)grid).context().cache().context().tm().activeTransactions().size() == 1);
+
+            Collection<IgniteInternalTx> backupTransactions = new LinkedList<>();
+
+            for (Ignite grid : G.allGrids()) {
+                if (grid != primary)

Review Comment:
   Better case is to iterate over backups explicitly



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentOnPrimaryFailTest.java:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.Collection;
+import java.util.LinkedList;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentOnPrimaryFailTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME).setCacheMode(PARTITIONED)
+            .setBackups(2).setAtomicityMode(TRANSACTIONAL).setAffinity(new RendezvousAffinityFunction(false, 1)));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        final IgniteEx backup = startGrids(2);
+
+        final CyclicBarrier backupBlockerBarrier = new CyclicBarrier(3);
+
+        final Runnable backupBlockerTask = () -> {
+            try {
+                backupBlockerBarrier.await();
+            }
+            catch (InterruptedException | BrokenBarrierException ignored) {
+                // Just supress.
+            }
+        };
+
+        for (int iter = 0; iter < 100; iter++) {
+            final IgniteEx primary = startGrid(2);
+
+            awaitPartitionMapExchange();
+
+            final IgniteCache<Object, Object> cache = primary.cache(DEFAULT_CACHE_NAME);
+
+            final TransactionProxyImpl<?, ?> tx = (TransactionProxyImpl<?, ?>)primary.transactions().txStart();
+
+            final Integer key = primaryKeys(cache, 1, 0).get(0);
+
+            cache.put(key, key);
+
+            tx.tx().prepare(true);
+
+            for (Ignite grid : G.allGrids())
+                assertTrue(((IgniteEx)grid).context().cache().context().tm().activeTransactions().size() == 1);
+
+            Collection<IgniteInternalTx> backupTransactions = new LinkedList<>();
+
+            for (Ignite grid : G.allGrids()) {
+                if (grid != primary)
+                    backupTransactions.addAll(((IgniteEx)grid).context().cache().context().tm().activeTransactions());
+            }
+
+            assertTrue(backupTransactions.size() == 2);
+
+            // Block recovery procedure processing on one of the backup nodes.
+            backup.context().pools().getSystemExecutorService().execute(backupBlockerTask);
+
+            // Block stripe tx recovery request processing on one of the backup nodes (note that the only stripe is
+            // configured in the executor).
+            backup.context().pools().getStripedExecutorService().execute(0, backupBlockerTask);
+
+            // Prevent tx finish request processing on both backup nodes.
+            for (Ignite grid : G.allGrids()) {
+                if (grid != primary)

Review Comment:
   Better case is to iterate over backups explicitly



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentOnPrimaryFailTest.java:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.Collection;
+import java.util.LinkedList;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentOnPrimaryFailTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+
+        cfg.setStripedPoolSize(1);

Review Comment:
   You may group such things (besause they do the same) to help readers.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r939661796


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +264,129 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there will be 2 backup nodes
+     * to execute the tx recovery in parallel if primary one would fail. These backup nodes will send the
+     * tx recovery requests to each other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 100; iter++) {
+            stopAllGrids();
+
+            log.info("iteration=" + iter);
+
+            final IgniteEx grid0 = startGrid(0);
+
+            final IgniteEx grid1 = startGrid(1, (UnaryOperator<IgniteConfiguration>)cfg ->
+                    cfg.setSystemThreadPoolSize(1).setStripedPoolSize(1));
+
+            final IgniteEx grid2 = startGrid(2);
+
+            grid0.cluster().state(ACTIVE);
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Integer g2Key = primaryKeys(cache, 1, 0).get(0);
+
+            List<IgniteInternalTx> txs0 = null;
+            List<IgniteInternalTx> txs1 = null;
+            List<IgniteInternalTx> txs2;
+
+            final CountDownLatch grid1BlockLatch = new CountDownLatch(1);
+
+            final CountDownLatch grid1NodeLeftEventLatch = new CountDownLatch(1);
+
+            int[] stripeHolder = new int[1];
+
+            try (final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                cache.put(g2Key, Boolean.TRUE);
+
+                TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;
+
+                p.tx().prepare(true);
+
+                txs0 = txs(grid0);
+                txs1 = txs(grid1);
+                txs2 = txs(grid2);
+
+                assertTrue(txs0.size() == 1);
+                assertTrue(txs1.size() == 1);
+                assertTrue(txs2.size() == 1);
+
+                grid1.events().localListen(new PE() {
+                    @Override public boolean apply(Event evt) {
+                        grid1NodeLeftEventLatch.countDown();
+
+                        return true;
+                    }
+                }, EventType.EVT_NODE_LEFT);
+
+                // Block recovery procedure processing on grid1
+                grid1.context().pools().getSystemExecutorService().execute(() -> U.awaitQuiet(grid1BlockLatch));
+
+                int stripe = U.safeAbs(p.tx().xidVersion().hashCode());
+
+                stripeHolder[0] = stripe;
+
+                // Block stripe tx recovery request processing on grid1
+                grid1.context().pools().getStripedExecutorService().execute(stripe, () -> U.awaitQuiet(grid1BlockLatch));
+
+                // Prevent finish request processing on grid0
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid0.name());
+
+                // Prevent finish request processing on grid1
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid1.name());
+
+                runAsync(() -> {
+                    grid2.close();
+                    return null;
+                });
+            }
+            catch (Exception ignored) {
+                // Expected.

Review Comment:
   Narrowed scope of try.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r946480928


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentTest.java:
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.Collection;
+import java.util.LinkedList;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ThreadPoolExecutor;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME).setCacheMode(PARTITIONED)
+            .setBackups(2).setAtomicityMode(TRANSACTIONAL).setAffinity(new RendezvousAffinityFunction(false, 10)));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }

Review Comment:
   Removed afterTest altogether



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentTest.java:
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.Collection;
+import java.util.LinkedList;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ThreadPoolExecutor;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME).setCacheMode(PARTITIONED)
+            .setBackups(2).setAtomicityMode(TRANSACTIONAL).setAffinity(new RendezvousAffinityFunction(false, 10)));

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r946807984


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentTest.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.Collection;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ThreadPoolExecutor;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME)
+            .setCacheMode(PARTITIONED)
+            .setBackups(2)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setAffinity(new RendezvousAffinityFunction(false, 1)));
+
+        return cfg;
+    }
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to near and primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Blocking is used to force concurrent processing on one of the backup nodes. Another
+     * backup works unconstrained to provide the right environment for the blocked one.
+     * In particular, it should send a tx recovery request to the blocked backup.
+     * <p>
+     * Use several attempts to reproduce the race condition in the blocked backup node.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnNearAndPrimaryFail() throws Exception {
+        startGrids(2);
+
+        final CyclicBarrier ensurePoolsAreBlockedBarrier = new CyclicBarrier(3);
+
+        final CyclicBarrier unblockPoolsBarrier = new CyclicBarrier(3);

Review Comment:
   fixed



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentTest.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.Collection;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ThreadPoolExecutor;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME)
+            .setCacheMode(PARTITIONED)
+            .setBackups(2)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setAffinity(new RendezvousAffinityFunction(false, 1)));
+
+        return cfg;
+    }
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to near and primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Blocking is used to force concurrent processing on one of the backup nodes. Another
+     * backup works unconstrained to provide the right environment for the blocked one.
+     * In particular, it should send a tx recovery request to the blocked backup.
+     * <p>
+     * Use several attempts to reproduce the race condition in the blocked backup node.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnNearAndPrimaryFail() throws Exception {
+        startGrids(2);
+
+        final CyclicBarrier ensurePoolsAreBlockedBarrier = new CyclicBarrier(3);
+
+        final CyclicBarrier unblockPoolsBarrier = new CyclicBarrier(3);
+
+        final Runnable backupBlockerTask = () -> {
+            try {
+                ensurePoolsAreBlockedBarrier.await();
+
+                unblockPoolsBarrier.await();
+            }
+            catch (InterruptedException | BrokenBarrierException ignored) {
+                // Just supress.

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] anton-vinogradov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
anton-vinogradov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r945703012


##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java:
##########
@@ -603,7 +603,13 @@ protected void uncommit() {
             case RECOVERY_FINISH:
                 FinalizationStatus old = finalizing;
 
-                res = old != FinalizationStatus.USER_FINISH && FINALIZING_UPD.compareAndSet(this, old, status);
+                if (old == FinalizationStatus.USER_FINISH)
+                    res = false;
+                else if (old == FinalizationStatus.RECOVERY_FINISH)
+                    res = true;
+                else
+                    res = FINALIZING_UPD.compareAndSet(this, old, status) ||
+                        finalizing == FinalizationStatus.RECOVERY_FINISH;

Review Comment:
   Could this be simplified with 
   ```
   case USER_FINISH:
       res = FINALIZING_UPD.compareAndSet(this, FinalizationStatus.NONE, status);
   
       break;
   
   case RECOVERY_FINISH:
       res = FINALIZING_UPD.compareAndSet(this, FinalizationStatus.NONE, status) || finalizing == status;
   ```



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentTest.java:
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.Collection;
+import java.util.LinkedList;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ThreadPoolExecutor;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME).setCacheMode(PARTITIONED)
+            .setBackups(2).setAtomicityMode(TRANSACTIONAL).setAffinity(new RendezvousAffinityFunction(false, 10)));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }

Review Comment:
   You have a single test, but `stopAlGrids()` already happen at `afterLastTest`;



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentTest.java:
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.Collection;
+import java.util.LinkedList;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ThreadPoolExecutor;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME).setCacheMode(PARTITIONED)
+            .setBackups(2).setAtomicityMode(TRANSACTIONAL).setAffinity(new RendezvousAffinityFunction(false, 10)));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to near and primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnNearAndPrimaryFail() throws Exception {
+        final IgniteEx backup = startGrids(2);
+
+        final CyclicBarrier backupBlockerBarrier = new CyclicBarrier(3);
+
+        final Runnable backupBlockerTask = () -> {
+            try {
+                backupBlockerBarrier.await();
+                backupBlockerBarrier.await();
+            }
+            catch (InterruptedException | BrokenBarrierException ignored) {
+                // Just supress.
+            }
+        };
+
+        for (int iter = 0; iter < 100; iter++) {
+            final IgniteEx primary = startGrid(2);
+
+            awaitPartitionMapExchange();
+
+            final IgniteCache<Object, Object> cache = primary.cache(DEFAULT_CACHE_NAME);
+
+            final TransactionProxyImpl<?, ?> tx = (TransactionProxyImpl<?, ?>)primary.transactions().txStart();
+
+            final Integer key = primaryKeys(cache, 1, 0).get(0);
+
+            cache.put(key, key);
+
+            tx.tx().prepare(true);
+
+            for (Ignite grid : G.allGrids())
+                assertTrue(((IgniteEx)grid).context().cache().context().tm().activeTransactions().size() == 1);
+
+            Collection<IgniteInternalTx> backupTransactions = new LinkedList<>();
+
+            for (Ignite grid : backupNodes(key, DEFAULT_CACHE_NAME))
+                backupTransactions.addAll(((IgniteEx)grid).context().cache().context().tm().activeTransactions());
+
+            assertTrue(backupTransactions.size() == 2);
+
+            // Block recovery procedure processing on one of the backup nodes.
+            backup.context().pools().getSystemExecutorService().execute(backupBlockerTask);
+
+            // Block stripe tx recovery request processing on one of the backup nodes (note that the only stripe is
+            // configured in the executor).
+            backup.context().pools().getStripedExecutorService().execute(0, backupBlockerTask);
+
+            // Prevent tx finish request processing on both backup nodes.
+            for (Ignite grid : backupNodes(key, DEFAULT_CACHE_NAME))

Review Comment:
   Any reason to recalculate `backupNodes()`?



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentTest.java:
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.Collection;
+import java.util.LinkedList;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ThreadPoolExecutor;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME).setCacheMode(PARTITIONED)
+            .setBackups(2).setAtomicityMode(TRANSACTIONAL).setAffinity(new RendezvousAffinityFunction(false, 10)));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to near and primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnNearAndPrimaryFail() throws Exception {
+        final IgniteEx backup = startGrids(2);

Review Comment:
   Just start nodes here.
   You'll calculate primary and backups later.
   



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentTest.java:
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.Collection;
+import java.util.LinkedList;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ThreadPoolExecutor;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME).setCacheMode(PARTITIONED)
+            .setBackups(2).setAtomicityMode(TRANSACTIONAL).setAffinity(new RendezvousAffinityFunction(false, 10)));

Review Comment:
   Please start each .setXXX() from newline to make this more readable.



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentTest.java:
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.Collection;
+import java.util.LinkedList;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ThreadPoolExecutor;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME).setCacheMode(PARTITIONED)
+            .setBackups(2).setAtomicityMode(TRANSACTIONAL).setAffinity(new RendezvousAffinityFunction(false, 10)));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to near and primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnNearAndPrimaryFail() throws Exception {
+        final IgniteEx backup = startGrids(2);
+
+        final CyclicBarrier backupBlockerBarrier = new CyclicBarrier(3);
+
+        final Runnable backupBlockerTask = () -> {
+            try {
+                backupBlockerBarrier.await();
+                backupBlockerBarrier.await();

Review Comment:
   Looks strage, type a typo duplication.
   Looks like you should refactor this.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r947740521


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentTest.java:
##########
@@ -103,7 +105,7 @@ public void testRecoveryNotDeadLockOnNearAndPrimaryFail() throws Exception {
 
             final TransactionProxyImpl<?, ?> tx = (TransactionProxyImpl<?, ?>)primary.transactions().txStart();
 
-            cache.put(key, key);
+            cache.put(key, key + iter);

Review Comment:
   Hm, by no reason in fact.  May be to imphasize the fact that it's something that is changed on each iteration.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r947452359


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentTest.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.Collection;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ThreadPoolExecutor;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME)
+            .setCacheMode(PARTITIONED)
+            .setBackups(2)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setAffinity(new RendezvousAffinityFunction(false, 1)));
+
+        return cfg;
+    }
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to near and primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Blocking is used to force concurrent processing on one of the backup nodes. Another
+     * backup works unconstrained to provide the right environment for the blocked one.
+     * In particular, it should send a tx recovery request to the blocked backup.
+     * <p>
+     * Use several attempts to reproduce the race condition in the blocked backup node.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnNearAndPrimaryFail() throws Exception {
+        startGrids(2);
+
+        final int key = 0;
+
+        for (int iter = 0; iter < 100; iter++) {
+            startGrid(iter + 2);
+
+            awaitPartitionMapExchange();
+
+            final Ignite primary = primaryNode(key, DEFAULT_CACHE_NAME);
+            final List<Ignite> backups = backupNodes(key, DEFAULT_CACHE_NAME);
+
+            final IgniteCache<Object, Object> cache = primary.cache(DEFAULT_CACHE_NAME);
+
+            final TransactionProxyImpl<?, ?> tx = (TransactionProxyImpl<?, ?>)primary.transactions().txStart();
+
+            cache.put(key, key);
+
+            tx.tx().prepare(true);
+
+            for (Ignite grid : G.allGrids())
+                assertTrue(((IgniteEx)grid).context().cache().context().tm().activeTransactions().size() == 1);
+
+            final Collection<IgniteInternalTx> backupTransactions = new LinkedList<>();
+
+            for (Ignite backup : backups)
+                backupTransactions.addAll(((IgniteEx)backup).context().cache().context().tm().activeTransactions());
+
+            assertTrue(backupTransactions.size() == 2);
+
+            final CountDownLatch ensureBothPoolsAreBlockedLatch = new CountDownLatch(2);
+            final CountDownLatch unblockBothPoolsLatch = new CountDownLatch(1);
+
+            final Runnable poolBlockerTask = () -> {
+                try {
+                    ensureBothPoolsAreBlockedLatch.countDown();
+                    unblockBothPoolsLatch.await();
+                }
+                catch (InterruptedException ignored) {
+                    // No-op.
+                }
+            };
+
+            final IgniteEx blockedBackup = (IgniteEx)backups.get(0);
+
+            blockedBackup.context().pools().getSystemExecutorService().execute(poolBlockerTask);
+
+            blockedBackup.context().pools().getStripedExecutorService().execute(0, poolBlockerTask);

Review Comment:
   fixed



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentTest.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.Collection;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ThreadPoolExecutor;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME)
+            .setCacheMode(PARTITIONED)
+            .setBackups(2)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setAffinity(new RendezvousAffinityFunction(false, 1)));
+
+        return cfg;
+    }
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to near and primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Blocking is used to force concurrent processing on one of the backup nodes. Another
+     * backup works unconstrained to provide the right environment for the blocked one.
+     * In particular, it should send a tx recovery request to the blocked backup.
+     * <p>
+     * Use several attempts to reproduce the race condition in the blocked backup node.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnNearAndPrimaryFail() throws Exception {
+        startGrids(2);
+
+        final int key = 0;
+
+        for (int iter = 0; iter < 100; iter++) {
+            startGrid(iter + 2);
+
+            awaitPartitionMapExchange();
+
+            final Ignite primary = primaryNode(key, DEFAULT_CACHE_NAME);
+            final List<Ignite> backups = backupNodes(key, DEFAULT_CACHE_NAME);
+
+            final IgniteCache<Object, Object> cache = primary.cache(DEFAULT_CACHE_NAME);
+
+            final TransactionProxyImpl<?, ?> tx = (TransactionProxyImpl<?, ?>)primary.transactions().txStart();
+
+            cache.put(key, key);
+
+            tx.tx().prepare(true);
+
+            for (Ignite grid : G.allGrids())
+                assertTrue(((IgniteEx)grid).context().cache().context().tm().activeTransactions().size() == 1);
+
+            final Collection<IgniteInternalTx> backupTransactions = new LinkedList<>();
+
+            for (Ignite backup : backups)
+                backupTransactions.addAll(((IgniteEx)backup).context().cache().context().tm().activeTransactions());
+
+            assertTrue(backupTransactions.size() == 2);
+
+            final CountDownLatch ensureBothPoolsAreBlockedLatch = new CountDownLatch(2);
+            final CountDownLatch unblockBothPoolsLatch = new CountDownLatch(1);
+
+            final Runnable poolBlockerTask = () -> {
+                try {
+                    ensureBothPoolsAreBlockedLatch.countDown();
+                    unblockBothPoolsLatch.await();
+                }
+                catch (InterruptedException ignored) {
+                    // No-op.
+                }
+            };
+
+            final IgniteEx blockedBackup = (IgniteEx)backups.get(0);
+
+            blockedBackup.context().pools().getSystemExecutorService().execute(poolBlockerTask);
+
+            blockedBackup.context().pools().getStripedExecutorService().execute(0, poolBlockerTask);
+
+            ensureBothPoolsAreBlockedLatch.await();
+
+            runAsync(primary::close);
+
+            waitForTxRecoveryRequestEnqueuedOn(blockedBackup);
+
+            waitForTxRecoveryTaskEnqueuedOn(blockedBackup);

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r943079931


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentOnPrimaryFailTest.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.Collection;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentOnPrimaryFailTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME).setCacheMode(PARTITIONED)
+            .setBackups(2).setAtomicityMode(TRANSACTIONAL).setAffinity(new RendezvousAffinityFunction(false, 1)));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        final IgniteEx grid0 = startGrid(0);
+
+        final IgniteEx grid1 = startGrid(1);
+
+        final CyclicBarrier grid1BlockerBarrier = new CyclicBarrier(3);
+
+        final Runnable grid1BlockerTask = () -> {
+            try {
+                grid1BlockerBarrier.await();
+            }
+            catch (InterruptedException | BrokenBarrierException e) {
+                // Just supress.
+            }
+        };
+
+        for (int iter = 0; iter < 100; iter++) {
+            final IgniteEx grid2 = startGrid(2);
+
+            awaitPartitionMapExchange();
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
+
+            // Key for which the grid2 node is primary.
+            final Integer grid2PrimaryKey = primaryKeys(cache, 1, 0).get(0);
+
+            cache.put(grid2PrimaryKey, Boolean.TRUE);
+
+            final TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;
+
+            p.tx().prepare(true);
+
+            final Collection<IgniteInternalTx> txs0 = grid0.context().cache().context().tm().activeTransactions();
+            final Collection<IgniteInternalTx> txs1 = grid1.context().cache().context().tm().activeTransactions();
+            final Collection<IgniteInternalTx> txs2 = grid2.context().cache().context().tm().activeTransactions();
+
+            assertTrue(txs0.size() == 1);
+            assertTrue(txs1.size() == 1);
+            assertTrue(txs2.size() == 1);
+
+            final IgniteInternalTx tx0 = txs0.iterator().next();
+            final IgniteInternalTx tx1 = txs1.iterator().next();

Review Comment:
   Fixed



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentOnPrimaryFailTest.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.Collection;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentOnPrimaryFailTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME).setCacheMode(PARTITIONED)
+            .setBackups(2).setAtomicityMode(TRANSACTIONAL).setAffinity(new RendezvousAffinityFunction(false, 1)));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        final IgniteEx grid0 = startGrid(0);
+
+        final IgniteEx grid1 = startGrid(1);
+
+        final CyclicBarrier grid1BlockerBarrier = new CyclicBarrier(3);
+
+        final Runnable grid1BlockerTask = () -> {
+            try {
+                grid1BlockerBarrier.await();
+            }
+            catch (InterruptedException | BrokenBarrierException e) {
+                // Just supress.
+            }
+        };
+
+        for (int iter = 0; iter < 100; iter++) {
+            final IgniteEx grid2 = startGrid(2);
+
+            awaitPartitionMapExchange();
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
+
+            // Key for which the grid2 node is primary.
+            final Integer grid2PrimaryKey = primaryKeys(cache, 1, 0).get(0);
+
+            cache.put(grid2PrimaryKey, Boolean.TRUE);
+
+            final TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;
+
+            p.tx().prepare(true);
+
+            final Collection<IgniteInternalTx> txs0 = grid0.context().cache().context().tm().activeTransactions();
+            final Collection<IgniteInternalTx> txs1 = grid1.context().cache().context().tm().activeTransactions();
+            final Collection<IgniteInternalTx> txs2 = grid2.context().cache().context().tm().activeTransactions();
+
+            assertTrue(txs0.size() == 1);
+            assertTrue(txs1.size() == 1);
+            assertTrue(txs2.size() == 1);
+
+            final IgniteInternalTx tx0 = txs0.iterator().next();
+            final IgniteInternalTx tx1 = txs1.iterator().next();
+
+            // Block recovery procedure processing on grid1.
+            grid1.context().pools().getSystemExecutorService().execute(grid1BlockerTask);
+
+            // Block stripe tx recovery request processing on grid1 (note that the only stripe is configured in the executor).
+            grid1.context().pools().getStripedExecutorService().execute(0, grid1BlockerTask);

Review Comment:
   Fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r941437326


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +263,120 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 100; iter++) {
+            final IgniteEx grid0 = startGrid(0);
+
+            final IgniteEx grid1 = startGrid(1, (UnaryOperator<IgniteConfiguration>)cfg -> cfg
+                .setSystemThreadPoolSize(1).setStripedPoolSize(1));
+
+            final IgniteEx grid2 = startGrid(2);
+
+            grid0.cluster().state(ACTIVE);
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
+
+            final Integer g2Key = primaryKeys(cache, 1, 0).get(0);
+
+            cache.put(g2Key, Boolean.TRUE);
+
+            final TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;
+
+            p.tx().prepare(true);
+
+            final List<IgniteInternalTx> txs0 = txs(grid0);
+            final List<IgniteInternalTx> txs1 = txs(grid1);
+            final List<IgniteInternalTx> txs2 = txs(grid2);
+
+            assertTrue(txs0.size() == 1);
+            assertTrue(txs1.size() == 1);
+            assertTrue(txs2.size() == 1);
+
+            final CountDownLatch grid1NodeLeftEventLatch = new CountDownLatch(1);
+
+            grid1.events().localListen(new PE() {
+                @Override public boolean apply(Event evt) {
+                    grid1NodeLeftEventLatch.countDown();
+
+                    return true;
+                }
+            }, EventType.EVT_NODE_LEFT);
+
+            final CountDownLatch grid1BlockLatch = new CountDownLatch(1);
+
+            // Block recovery procedure processing on grid1.
+            grid1.context().pools().getSystemExecutorService().execute(() -> U.awaitQuiet(grid1BlockLatch));
+
+            final int stripe = U.safeAbs(p.tx().xidVersion().hashCode());

Review Comment:
   Yes it will be always 0 in fact - changed to constant.



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +263,120 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 100; iter++) {
+            final IgniteEx grid0 = startGrid(0);
+
+            final IgniteEx grid1 = startGrid(1, (UnaryOperator<IgniteConfiguration>)cfg -> cfg
+                .setSystemThreadPoolSize(1).setStripedPoolSize(1));
+
+            final IgniteEx grid2 = startGrid(2);
+
+            grid0.cluster().state(ACTIVE);
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
+
+            final Integer g2Key = primaryKeys(cache, 1, 0).get(0);

Review Comment:
   Renamed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] anton-vinogradov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
anton-vinogradov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r937556299


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +259,110 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <ul>
+     *  <li>Start 3 nodes (g0, g1, g2) and cache with 2 backups.</li>
+     *  <li>Prepare a transaction with g2 as a primary node.</li>
+     *  <li>Kill g2.</li>
+     *  <li>Enforce the concurrent processing of transaction in tx recovery on g1.</li>
+     * </ul>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished both on g0 and g1
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 75; iter++) {
+            stopAllGrids();
+
+            log.info("iteration=" + iter);
+            final IgniteEx grid0 = startGrid("g0");
+            final IgniteEx grid1 = startGrid("g1",
+                    cfg -> cfg.setSystemThreadPoolSize(1).setStripedPoolSize(1));
+            final IgniteEx grid2 = startGrid("g2");
+
+            grid0.cluster().state(ACTIVE);
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Integer g2Key = primaryKeys(grid2.cache(DEFAULT_CACHE_NAME), 1, 0).get(0);
+
+            List<IgniteInternalTx> txs0 = null;
+            List<IgniteInternalTx> txs1 = null;
+
+            CountDownLatch grid1BlockLatch = new CountDownLatch(1);
+
+            try (final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                cache.put(g2Key, Boolean.TRUE);
+                TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;
+                p.tx().prepare(true);
+
+                txs0 = txs(grid0);
+                txs1 = txs(grid1);
+                List<IgniteInternalTx> txs2 = txs(grid2);
+
+                assertTrue(txs0.size() == 1);
+                assertTrue(txs1.size() == 1);
+                assertTrue(txs2.size() == 1);
+
+                // Prevent tx recovery request to be sent from grid0 to grid1.
+                spi(grid0).blockMessages(GridCacheTxRecoveryRequest.class, grid1.name());
+
+                // Block recovery procedure processing on grid1
+                grid1.context().pools().getSystemExecutorService().execute(() -> U.awaitQuiet(grid1BlockLatch));
+                // Block stripe tx recovery request processing on grid1
+                int stripe = U.safeAbs(p.tx().xidVersion().hashCode());
+                grid1.context().pools().getStripedExecutorService().execute(stripe, () -> U.awaitQuiet(grid1BlockLatch));
+
+                // Prevent finish request processing on grid0 and grid1.
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid0.name());
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid1.name());
+
+                runAsync(() -> {
+                    grid2.close();
+                    return null;
+                });
+            }
+            catch (Exception ignored) {
+                // Expected.
+            }
+
+            // Wait grid0 is ready to send the tx recovery request to grid1
+            spi(grid0).waitForBlocked();
+            // Let grid0 send the tx recovery request to grid1
+            log.info("unblock grid0");
+            spi(grid0).stopBlock();
+            // Give grid1 some time to receive the tx recovery request (processing is still blocked in grid1).
+            log.info("sleep in grid0");
+            doSleep(100);
+
+            // Unblock processing in grid1. Simultaneously in striped and system pools to start
+            // recovery procedure and the tx recovery request processing at the "same" moment
+            // (for the same transaction). This should increase chances for race condition occur in
+            // the IgniteTxAdapter::markFinalizing.
+            log.info("unblock grid1");

Review Comment:
   Plrase get rid of helpless logging.
   Use asserts "to guarantee" instead of logging "to find".



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +259,110 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <ul>
+     *  <li>Start 3 nodes (g0, g1, g2) and cache with 2 backups.</li>
+     *  <li>Prepare a transaction with g2 as a primary node.</li>
+     *  <li>Kill g2.</li>
+     *  <li>Enforce the concurrent processing of transaction in tx recovery on g1.</li>
+     * </ul>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished both on g0 and g1
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 75; iter++) {
+            stopAllGrids();
+
+            log.info("iteration=" + iter);
+            final IgniteEx grid0 = startGrid("g0");
+            final IgniteEx grid1 = startGrid("g1",
+                    cfg -> cfg.setSystemThreadPoolSize(1).setStripedPoolSize(1));
+            final IgniteEx grid2 = startGrid("g2");
+
+            grid0.cluster().state(ACTIVE);
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Integer g2Key = primaryKeys(grid2.cache(DEFAULT_CACHE_NAME), 1, 0).get(0);
+
+            List<IgniteInternalTx> txs0 = null;
+            List<IgniteInternalTx> txs1 = null;
+
+            CountDownLatch grid1BlockLatch = new CountDownLatch(1);
+
+            try (final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                cache.put(g2Key, Boolean.TRUE);
+                TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;
+                p.tx().prepare(true);
+
+                txs0 = txs(grid0);
+                txs1 = txs(grid1);
+                List<IgniteInternalTx> txs2 = txs(grid2);
+
+                assertTrue(txs0.size() == 1);
+                assertTrue(txs1.size() == 1);
+                assertTrue(txs2.size() == 1);
+
+                // Prevent tx recovery request to be sent from grid0 to grid1.
+                spi(grid0).blockMessages(GridCacheTxRecoveryRequest.class, grid1.name());
+
+                // Block recovery procedure processing on grid1
+                grid1.context().pools().getSystemExecutorService().execute(() -> U.awaitQuiet(grid1BlockLatch));
+                // Block stripe tx recovery request processing on grid1
+                int stripe = U.safeAbs(p.tx().xidVersion().hashCode());
+                grid1.context().pools().getStripedExecutorService().execute(stripe, () -> U.awaitQuiet(grid1BlockLatch));
+
+                // Prevent finish request processing on grid0 and grid1.
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid0.name());
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid1.name());
+
+                runAsync(() -> {
+                    grid2.close();
+                    return null;
+                });
+            }
+            catch (Exception ignored) {
+                // Expected.
+            }
+
+            // Wait grid0 is ready to send the tx recovery request to grid1
+            spi(grid0).waitForBlocked();
+            // Let grid0 send the tx recovery request to grid1
+            log.info("unblock grid0");
+            spi(grid0).stopBlock();
+            // Give grid1 some time to receive the tx recovery request (processing is still blocked in grid1).
+            log.info("sleep in grid0");

Review Comment:
   Please separate semantic units according to 
   https://cwiki.apache.org/confluence/display/IGNITE/Coding+Guidelines#CodingGuidelines-SemanticUnits



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +259,110 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <ul>
+     *  <li>Start 3 nodes (g0, g1, g2) and cache with 2 backups.</li>
+     *  <li>Prepare a transaction with g2 as a primary node.</li>
+     *  <li>Kill g2.</li>
+     *  <li>Enforce the concurrent processing of transaction in tx recovery on g1.</li>

Review Comment:
   Javadoc must explain ides, not to retell test content.
   Such approach is not refactoring friendly, please get rid of retelling.



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +259,110 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <ul>
+     *  <li>Start 3 nodes (g0, g1, g2) and cache with 2 backups.</li>
+     *  <li>Prepare a transaction with g2 as a primary node.</li>
+     *  <li>Kill g2.</li>
+     *  <li>Enforce the concurrent processing of transaction in tx recovery on g1.</li>
+     * </ul>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished both on g0 and g1
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 75; iter++) {

Review Comment:
   Why 75?
   Looks like a magic number.
   Use regular numbers like 10, 100, ... when you just need some iterations, or explain your choice.



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +259,110 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <ul>
+     *  <li>Start 3 nodes (g0, g1, g2) and cache with 2 backups.</li>
+     *  <li>Prepare a transaction with g2 as a primary node.</li>
+     *  <li>Kill g2.</li>
+     *  <li>Enforce the concurrent processing of transaction in tx recovery on g1.</li>
+     * </ul>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished both on g0 and g1
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 75; iter++) {
+            stopAllGrids();
+
+            log.info("iteration=" + iter);
+            final IgniteEx grid0 = startGrid("g0");
+            final IgniteEx grid1 = startGrid("g1",
+                    cfg -> cfg.setSystemThreadPoolSize(1).setStripedPoolSize(1));
+            final IgniteEx grid2 = startGrid("g2");
+
+            grid0.cluster().state(ACTIVE);
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Integer g2Key = primaryKeys(grid2.cache(DEFAULT_CACHE_NAME), 1, 0).get(0);
+
+            List<IgniteInternalTx> txs0 = null;
+            List<IgniteInternalTx> txs1 = null;
+
+            CountDownLatch grid1BlockLatch = new CountDownLatch(1);
+
+            try (final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                cache.put(g2Key, Boolean.TRUE);
+                TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;
+                p.tx().prepare(true);
+
+                txs0 = txs(grid0);
+                txs1 = txs(grid1);
+                List<IgniteInternalTx> txs2 = txs(grid2);
+
+                assertTrue(txs0.size() == 1);
+                assertTrue(txs1.size() == 1);
+                assertTrue(txs2.size() == 1);
+
+                // Prevent tx recovery request to be sent from grid0 to grid1.
+                spi(grid0).blockMessages(GridCacheTxRecoveryRequest.class, grid1.name());
+
+                // Block recovery procedure processing on grid1
+                grid1.context().pools().getSystemExecutorService().execute(() -> U.awaitQuiet(grid1BlockLatch));
+                // Block stripe tx recovery request processing on grid1
+                int stripe = U.safeAbs(p.tx().xidVersion().hashCode());
+                grid1.context().pools().getStripedExecutorService().execute(stripe, () -> U.awaitQuiet(grid1BlockLatch));
+
+                // Prevent finish request processing on grid0 and grid1.
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid0.name());
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid1.name());
+
+                runAsync(() -> {
+                    grid2.close();
+                    return null;
+                });
+            }
+            catch (Exception ignored) {
+                // Expected.
+            }
+
+            // Wait grid0 is ready to send the tx recovery request to grid1
+            spi(grid0).waitForBlocked();
+            // Let grid0 send the tx recovery request to grid1
+            log.info("unblock grid0");
+            spi(grid0).stopBlock();
+            // Give grid1 some time to receive the tx recovery request (processing is still blocked in grid1).
+            log.info("sleep in grid0");
+            doSleep(100);
+
+            // Unblock processing in grid1. Simultaneously in striped and system pools to start
+            // recovery procedure and the tx recovery request processing at the "same" moment
+            // (for the same transaction). This should increase chances for race condition occur in
+            // the IgniteTxAdapter::markFinalizing.
+            log.info("unblock grid1");
+            grid1BlockLatch.countDown();
+
+            // Wait transaction finish in grid0.
+            txs0.get(0).finishFuture().get(5_000);

Review Comment:
   Please never **get_with_timeout** without checking the result of the **get**.
   Also, try to avoid sleeps, timeouts and so on, use concurrent latches, futures, etc instead.



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +259,110 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <ul>
+     *  <li>Start 3 nodes (g0, g1, g2) and cache with 2 backups.</li>
+     *  <li>Prepare a transaction with g2 as a primary node.</li>
+     *  <li>Kill g2.</li>
+     *  <li>Enforce the concurrent processing of transaction in tx recovery on g1.</li>
+     * </ul>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished both on g0 and g1
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 75; iter++) {
+            stopAllGrids();
+
+            log.info("iteration=" + iter);
+            final IgniteEx grid0 = startGrid("g0");
+            final IgniteEx grid1 = startGrid("g1",
+                    cfg -> cfg.setSystemThreadPoolSize(1).setStripedPoolSize(1));
+            final IgniteEx grid2 = startGrid("g2");
+
+            grid0.cluster().state(ACTIVE);
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Integer g2Key = primaryKeys(grid2.cache(DEFAULT_CACHE_NAME), 1, 0).get(0);
+
+            List<IgniteInternalTx> txs0 = null;
+            List<IgniteInternalTx> txs1 = null;
+
+            CountDownLatch grid1BlockLatch = new CountDownLatch(1);
+
+            try (final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                cache.put(g2Key, Boolean.TRUE);
+                TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;
+                p.tx().prepare(true);
+
+                txs0 = txs(grid0);
+                txs1 = txs(grid1);
+                List<IgniteInternalTx> txs2 = txs(grid2);
+
+                assertTrue(txs0.size() == 1);
+                assertTrue(txs1.size() == 1);
+                assertTrue(txs2.size() == 1);
+
+                // Prevent tx recovery request to be sent from grid0 to grid1.
+                spi(grid0).blockMessages(GridCacheTxRecoveryRequest.class, grid1.name());
+
+                // Block recovery procedure processing on grid1
+                grid1.context().pools().getSystemExecutorService().execute(() -> U.awaitQuiet(grid1BlockLatch));
+                // Block stripe tx recovery request processing on grid1
+                int stripe = U.safeAbs(p.tx().xidVersion().hashCode());
+                grid1.context().pools().getStripedExecutorService().execute(stripe, () -> U.awaitQuiet(grid1BlockLatch));
+
+                // Prevent finish request processing on grid0 and grid1.
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid0.name());
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid1.name());
+
+                runAsync(() -> {
+                    grid2.close();
+                    return null;
+                });
+            }
+            catch (Exception ignored) {
+                // Expected.
+            }
+
+            // Wait grid0 is ready to send the tx recovery request to grid1
+            spi(grid0).waitForBlocked();
+            // Let grid0 send the tx recovery request to grid1
+            log.info("unblock grid0");
+            spi(grid0).stopBlock();
+            // Give grid1 some time to receive the tx recovery request (processing is still blocked in grid1).
+            log.info("sleep in grid0");
+            doSleep(100);
+
+            // Unblock processing in grid1. Simultaneously in striped and system pools to start
+            // recovery procedure and the tx recovery request processing at the "same" moment
+            // (for the same transaction). This should increase chances for race condition occur in
+            // the IgniteTxAdapter::markFinalizing.
+            log.info("unblock grid1");
+            grid1BlockLatch.countDown();
+
+            // Wait transaction finish in grid0.
+            txs0.get(0).finishFuture().get(5_000);
+
+            try {
+                // Check if transaction is finished in grid1. It wouldn't if race condition occur.
+                txs1.get(0).finishFuture().get(20_000);

Review Comment:
   Looks like after the fix you need no timeout at all, please get rid if so.



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +259,110 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <ul>
+     *  <li>Start 3 nodes (g0, g1, g2) and cache with 2 backups.</li>
+     *  <li>Prepare a transaction with g2 as a primary node.</li>
+     *  <li>Kill g2.</li>
+     *  <li>Enforce the concurrent processing of transaction in tx recovery on g1.</li>
+     * </ul>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished both on g0 and g1
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 75; iter++) {
+            stopAllGrids();
+
+            log.info("iteration=" + iter);
+            final IgniteEx grid0 = startGrid("g0");
+            final IgniteEx grid1 = startGrid("g1",
+                    cfg -> cfg.setSystemThreadPoolSize(1).setStripedPoolSize(1));
+            final IgniteEx grid2 = startGrid("g2");

Review Comment:
   Please call something only when you're going to summon it by name.
   Please get rig of useless naming.



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +259,110 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <ul>
+     *  <li>Start 3 nodes (g0, g1, g2) and cache with 2 backups.</li>
+     *  <li>Prepare a transaction with g2 as a primary node.</li>
+     *  <li>Kill g2.</li>
+     *  <li>Enforce the concurrent processing of transaction in tx recovery on g1.</li>
+     * </ul>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished both on g0 and g1
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 75; iter++) {
+            stopAllGrids();
+
+            log.info("iteration=" + iter);
+            final IgniteEx grid0 = startGrid("g0");
+            final IgniteEx grid1 = startGrid("g1",
+                    cfg -> cfg.setSystemThreadPoolSize(1).setStripedPoolSize(1));
+            final IgniteEx grid2 = startGrid("g2");
+
+            grid0.cluster().state(ACTIVE);
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Integer g2Key = primaryKeys(grid2.cache(DEFAULT_CACHE_NAME), 1, 0).get(0);
+
+            List<IgniteInternalTx> txs0 = null;
+            List<IgniteInternalTx> txs1 = null;
+
+            CountDownLatch grid1BlockLatch = new CountDownLatch(1);
+
+            try (final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                cache.put(g2Key, Boolean.TRUE);
+                TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;
+                p.tx().prepare(true);
+
+                txs0 = txs(grid0);
+                txs1 = txs(grid1);
+                List<IgniteInternalTx> txs2 = txs(grid2);
+
+                assertTrue(txs0.size() == 1);
+                assertTrue(txs1.size() == 1);
+                assertTrue(txs2.size() == 1);
+
+                // Prevent tx recovery request to be sent from grid0 to grid1.
+                spi(grid0).blockMessages(GridCacheTxRecoveryRequest.class, grid1.name());
+
+                // Block recovery procedure processing on grid1
+                grid1.context().pools().getSystemExecutorService().execute(() -> U.awaitQuiet(grid1BlockLatch));
+                // Block stripe tx recovery request processing on grid1
+                int stripe = U.safeAbs(p.tx().xidVersion().hashCode());
+                grid1.context().pools().getStripedExecutorService().execute(stripe, () -> U.awaitQuiet(grid1BlockLatch));
+
+                // Prevent finish request processing on grid0 and grid1.
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid0.name());
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid1.name());
+
+                runAsync(() -> {
+                    grid2.close();
+                    return null;
+                });
+            }
+            catch (Exception ignored) {
+                // Expected.
+            }
+
+            // Wait grid0 is ready to send the tx recovery request to grid1
+            spi(grid0).waitForBlocked();
+            // Let grid0 send the tx recovery request to grid1
+            log.info("unblock grid0");
+            spi(grid0).stopBlock();
+            // Give grid1 some time to receive the tx recovery request (processing is still blocked in grid1).
+            log.info("sleep in grid0");
+            doSleep(100);

Review Comment:
   Please never use sleeps at the tests.
   Each sleep means you can't guarantee something, but trying to hide this.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r939661885


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +264,129 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there will be 2 backup nodes
+     * to execute the tx recovery in parallel if primary one would fail. These backup nodes will send the
+     * tx recovery requests to each other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 100; iter++) {
+            stopAllGrids();
+
+            log.info("iteration=" + iter);
+
+            final IgniteEx grid0 = startGrid(0);
+
+            final IgniteEx grid1 = startGrid(1, (UnaryOperator<IgniteConfiguration>)cfg ->
+                    cfg.setSystemThreadPoolSize(1).setStripedPoolSize(1));
+
+            final IgniteEx grid2 = startGrid(2);
+
+            grid0.cluster().state(ACTIVE);
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Integer g2Key = primaryKeys(cache, 1, 0).get(0);
+
+            List<IgniteInternalTx> txs0 = null;
+            List<IgniteInternalTx> txs1 = null;
+            List<IgniteInternalTx> txs2;
+
+            final CountDownLatch grid1BlockLatch = new CountDownLatch(1);
+
+            final CountDownLatch grid1NodeLeftEventLatch = new CountDownLatch(1);
+
+            int[] stripeHolder = new int[1];
+
+            try (final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                cache.put(g2Key, Boolean.TRUE);
+
+                TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;
+
+                p.tx().prepare(true);
+
+                txs0 = txs(grid0);
+                txs1 = txs(grid1);
+                txs2 = txs(grid2);
+
+                assertTrue(txs0.size() == 1);
+                assertTrue(txs1.size() == 1);
+                assertTrue(txs2.size() == 1);
+
+                grid1.events().localListen(new PE() {
+                    @Override public boolean apply(Event evt) {
+                        grid1NodeLeftEventLatch.countDown();
+
+                        return true;
+                    }
+                }, EventType.EVT_NODE_LEFT);
+
+                // Block recovery procedure processing on grid1
+                grid1.context().pools().getSystemExecutorService().execute(() -> U.awaitQuiet(grid1BlockLatch));
+
+                int stripe = U.safeAbs(p.tx().xidVersion().hashCode());
+
+                stripeHolder[0] = stripe;
+
+                // Block stripe tx recovery request processing on grid1
+                grid1.context().pools().getStripedExecutorService().execute(stripe, () -> U.awaitQuiet(grid1BlockLatch));
+
+                // Prevent finish request processing on grid0
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid0.name());
+
+                // Prevent finish request processing on grid1
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid1.name());
+
+                runAsync(() -> {
+                    grid2.close();
+                    return null;
+                });
+            }
+            catch (Exception ignored) {
+                // Expected.
+            }
+
+            // Wait until grid1 node detects primary node left
+            grid1NodeLeftEventLatch.await();
+
+            // Wait until grid1 receives the tx recovery request and the corresponding processing task is added into the queue
+            assertTrue("tx recovery request received on grid1", GridTestUtils.waitForCondition(() ->
+                    grid1.context().pools().getStripedExecutorService().queueStripeSize(stripeHolder[0]) == 1, 5_000));
+
+            // Unblock processing in grid1. Simultaneously in striped and system pools to start
+            // recovery procedure and the tx recovery request processing at the "same" moment
+            // (for the same transaction). This should increase chances for race condition occur in
+            // the IgniteTxAdapter::markFinalizing.
+            grid1BlockLatch.countDown();
+
+            waitForTopology(2);
+
+            try {
+                awaitPartitionMapExchange();
+            }
+            catch (IgniteException ex) {
+                log.error("exception from the awaitPartitionMapExchange", ex);
+                fail("fail to wait for the partition map exchange, iteration=" + iter);
+            }

Review Comment:
   Just for iteration count logging - removed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r940252931


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +263,120 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 100; iter++) {

Review Comment:
   It's a probabilistic one indeed. 
   
   We have a very specific interleaving situation here in the concurrent calls of IgniteTxAdapter#markFinalizing. As described in the Jira ticket it is as follows:
   
   > The following steps are executed on the node1 in two threads ("procedure" which is a system pool thread
   > executing the tx recovery procedure and "handler" which is a striped pool thread processing the tx
   > recovery request sent from node0):
   
   > - tx.finalization == NONE
   > - "procedure": calls markFinalizing(RECOVERY_FINISH)
   > - "handler": calls markFinalizing(RECOVERY_FINISH)
   > - "procedure": gets old tx.finlalization - it's NONE
   > - "handler": gets old tx.finalization - it's NONE
   > - "handler": updates tx.finalization - now it's RECOVERY_FINISH
   > - "procedure": tries to update tx.finalization via compareAndSet and fails since compare fails.
   > - "procedure": stops transaction processing and does not try to commit it.
   > - Transaction remains not finished on node1.
   
   So to have the problem reproduced we need to be lucky to:
   1. Call markFinalizing at the same time from two threads.  Which is not precise, since both threads do some other work before calling the markFinalizing. And I fialed to find any other  sync points which may be exploited.
   2. Be lucky to have a particular order of operations in the markFinalizing.
   
   ***
   Anyway I did some experiments trying to reduce number of repetitions needed  on weekend.  I added several dummy threads to increase thread scheduling interleavings following the hint suggested at https://github.com/code-review-checklists/java-concurrency#test-workers-interleavings
   
   Unfortunately it doesn't help.
   
   In the experiment I perform 100 test runs in each configuration (with and without dummy threads). And collect number of repetions needed to reproduce the problem.  It took 33 repetitions on average for both cases. And there are attempts which fail to reproduce the problem even by 100 repetions (5 and 6 times respectevely). The detailed results are:
   
   
     | No dummy threads | 8 dummy threads
   -- | -- | --
   Mean | 33.32 | 32.36
   Median | 26.5 | 27
   First Quartile | 11 | 13
   Third Quartile | 47.25 | 45.25
   Standard Deviation | 27.92 | 26.01
   Range | 99 | 99
   Minimum | 1 | 1
   Maximum | 100 | 100
   Sum | 3332 | 3236
   Count | 100 | 100
   
   
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r939661819


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +264,129 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there will be 2 backup nodes
+     * to execute the tx recovery in parallel if primary one would fail. These backup nodes will send the
+     * tx recovery requests to each other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 100; iter++) {
+            stopAllGrids();
+
+            log.info("iteration=" + iter);
+
+            final IgniteEx grid0 = startGrid(0);
+
+            final IgniteEx grid1 = startGrid(1, (UnaryOperator<IgniteConfiguration>)cfg ->
+                    cfg.setSystemThreadPoolSize(1).setStripedPoolSize(1));
+
+            final IgniteEx grid2 = startGrid(2);
+
+            grid0.cluster().state(ACTIVE);
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Integer g2Key = primaryKeys(cache, 1, 0).get(0);
+
+            List<IgniteInternalTx> txs0 = null;
+            List<IgniteInternalTx> txs1 = null;
+            List<IgniteInternalTx> txs2;
+
+            final CountDownLatch grid1BlockLatch = new CountDownLatch(1);
+
+            final CountDownLatch grid1NodeLeftEventLatch = new CountDownLatch(1);
+
+            int[] stripeHolder = new int[1];
+
+            try (final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                cache.put(g2Key, Boolean.TRUE);
+
+                TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;
+
+                p.tx().prepare(true);
+
+                txs0 = txs(grid0);
+                txs1 = txs(grid1);
+                txs2 = txs(grid2);
+
+                assertTrue(txs0.size() == 1);
+                assertTrue(txs1.size() == 1);
+                assertTrue(txs2.size() == 1);
+
+                grid1.events().localListen(new PE() {
+                    @Override public boolean apply(Event evt) {
+                        grid1NodeLeftEventLatch.countDown();
+
+                        return true;
+                    }
+                }, EventType.EVT_NODE_LEFT);
+
+                // Block recovery procedure processing on grid1
+                grid1.context().pools().getSystemExecutorService().execute(() -> U.awaitQuiet(grid1BlockLatch));
+
+                int stripe = U.safeAbs(p.tx().xidVersion().hashCode());
+
+                stripeHolder[0] = stripe;

Review Comment:
   Fixed



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +264,129 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there will be 2 backup nodes
+     * to execute the tx recovery in parallel if primary one would fail. These backup nodes will send the
+     * tx recovery requests to each other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 100; iter++) {
+            stopAllGrids();
+
+            log.info("iteration=" + iter);

Review Comment:
   Fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r941442336


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +263,120 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 100; iter++) {

Review Comment:
   Looks like number of repetitions can not be reduced. 
   
   So I have optimized the single repetion. It required to put test into the separate class to have more optimized cache & node configuration.   
   
   The main idea of optimization is to restart the only node on each cycle iteration.
   
   On my laptop test execution time reduced from about 90 to 55 seconds for 100 iterations.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r941450295


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +263,120 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 100; iter++) {

Review Comment:
   In the TC environment it is now 43s vs 96s



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r943079988


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentOnPrimaryFailTest.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.Collection;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentOnPrimaryFailTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME).setCacheMode(PARTITIONED)
+            .setBackups(2).setAtomicityMode(TRANSACTIONAL).setAffinity(new RendezvousAffinityFunction(false, 1)));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        final IgniteEx grid0 = startGrid(0);
+
+        final IgniteEx grid1 = startGrid(1);
+
+        final CyclicBarrier grid1BlockerBarrier = new CyclicBarrier(3);
+
+        final Runnable grid1BlockerTask = () -> {
+            try {
+                grid1BlockerBarrier.await();
+            }
+            catch (InterruptedException | BrokenBarrierException e) {
+                // Just supress.
+            }
+        };
+
+        for (int iter = 0; iter < 100; iter++) {
+            final IgniteEx grid2 = startGrid(2);
+
+            awaitPartitionMapExchange();
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
+
+            // Key for which the grid2 node is primary.
+            final Integer grid2PrimaryKey = primaryKeys(cache, 1, 0).get(0);
+
+            cache.put(grid2PrimaryKey, Boolean.TRUE);
+
+            final TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;
+
+            p.tx().prepare(true);
+
+            final Collection<IgniteInternalTx> txs0 = grid0.context().cache().context().tm().activeTransactions();
+            final Collection<IgniteInternalTx> txs1 = grid1.context().cache().context().tm().activeTransactions();
+            final Collection<IgniteInternalTx> txs2 = grid2.context().cache().context().tm().activeTransactions();
+
+            assertTrue(txs0.size() == 1);
+            assertTrue(txs1.size() == 1);
+            assertTrue(txs2.size() == 1);
+
+            final IgniteInternalTx tx0 = txs0.iterator().next();
+            final IgniteInternalTx tx1 = txs1.iterator().next();
+
+            // Block recovery procedure processing on grid1.
+            grid1.context().pools().getSystemExecutorService().execute(grid1BlockerTask);
+
+            // Block stripe tx recovery request processing on grid1 (note that the only stripe is configured in the executor).
+            grid1.context().pools().getStripedExecutorService().execute(0, grid1BlockerTask);
+
+            // Prevent finish request processing on grid0.
+            spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid0.name());
+
+            // Prevent finish request processing on grid1.
+            spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid1.name());

Review Comment:
   Fixed



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentOnPrimaryFailTest.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.Collection;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentOnPrimaryFailTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME).setCacheMode(PARTITIONED)
+            .setBackups(2).setAtomicityMode(TRANSACTIONAL).setAffinity(new RendezvousAffinityFunction(false, 1)));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        final IgniteEx grid0 = startGrid(0);
+
+        final IgniteEx grid1 = startGrid(1);
+
+        final CyclicBarrier grid1BlockerBarrier = new CyclicBarrier(3);
+
+        final Runnable grid1BlockerTask = () -> {
+            try {
+                grid1BlockerBarrier.await();
+            }
+            catch (InterruptedException | BrokenBarrierException e) {
+                // Just supress.
+            }
+        };
+
+        for (int iter = 0; iter < 100; iter++) {
+            final IgniteEx grid2 = startGrid(2);
+
+            awaitPartitionMapExchange();
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
+
+            // Key for which the grid2 node is primary.
+            final Integer grid2PrimaryKey = primaryKeys(cache, 1, 0).get(0);
+
+            cache.put(grid2PrimaryKey, Boolean.TRUE);
+
+            final TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;
+
+            p.tx().prepare(true);
+
+            final Collection<IgniteInternalTx> txs0 = grid0.context().cache().context().tm().activeTransactions();
+            final Collection<IgniteInternalTx> txs1 = grid1.context().cache().context().tm().activeTransactions();
+            final Collection<IgniteInternalTx> txs2 = grid2.context().cache().context().tm().activeTransactions();
+
+            assertTrue(txs0.size() == 1);
+            assertTrue(txs1.size() == 1);
+            assertTrue(txs2.size() == 1);
+
+            final IgniteInternalTx tx0 = txs0.iterator().next();
+            final IgniteInternalTx tx1 = txs1.iterator().next();
+
+            // Block recovery procedure processing on grid1.
+            grid1.context().pools().getSystemExecutorService().execute(grid1BlockerTask);
+
+            // Block stripe tx recovery request processing on grid1 (note that the only stripe is configured in the executor).
+            grid1.context().pools().getStripedExecutorService().execute(0, grid1BlockerTask);
+
+            // Prevent finish request processing on grid0.
+            spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid0.name());
+
+            // Prevent finish request processing on grid1.
+            spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid1.name());
+
+            runAsync(() -> {
+                grid2.close();
+
+                return null;
+            });

Review Comment:
   Fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r941977922


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentOnPrimaryFailTest.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.Collection;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentOnPrimaryFailTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME).setCacheMode(PARTITIONED)
+            .setBackups(2).setAtomicityMode(TRANSACTIONAL).setAffinity(new RendezvousAffinityFunction(false, 1)));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        final IgniteEx grid0 = startGrid(0);
+
+        final IgniteEx grid1 = startGrid(1);
+
+        final CyclicBarrier grid1BlockerBarrier = new CyclicBarrier(3);
+
+        final Runnable grid1BlockerTask = () -> {
+            try {
+                grid1BlockerBarrier.await();
+            }
+            catch (InterruptedException | BrokenBarrierException e) {
+                // Just supress.
+            }
+        };
+
+        for (int iter = 0; iter < 100; iter++) {
+            final IgniteEx grid2 = startGrid(2);
+
+            awaitPartitionMapExchange();
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ);

Review Comment:
   Yes



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r946478672


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentTest.java:
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.Collection;
+import java.util.LinkedList;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ThreadPoolExecutor;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME).setCacheMode(PARTITIONED)
+            .setBackups(2).setAtomicityMode(TRANSACTIONAL).setAffinity(new RendezvousAffinityFunction(false, 10)));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to near and primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnNearAndPrimaryFail() throws Exception {
+        final IgniteEx backup = startGrids(2);

Review Comment:
   Reworked to calculate primary and backups from key value (0)



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentTest.java:
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.Collection;
+import java.util.LinkedList;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ThreadPoolExecutor;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME).setCacheMode(PARTITIONED)
+            .setBackups(2).setAtomicityMode(TRANSACTIONAL).setAffinity(new RendezvousAffinityFunction(false, 10)));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to near and primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnNearAndPrimaryFail() throws Exception {
+        final IgniteEx backup = startGrids(2);
+
+        final CyclicBarrier backupBlockerBarrier = new CyclicBarrier(3);
+
+        final Runnable backupBlockerTask = () -> {
+            try {
+                backupBlockerBarrier.await();
+                backupBlockerBarrier.await();
+            }
+            catch (InterruptedException | BrokenBarrierException ignored) {
+                // Just supress.
+            }
+        };
+
+        for (int iter = 0; iter < 100; iter++) {
+            final IgniteEx primary = startGrid(2);
+
+            awaitPartitionMapExchange();
+
+            final IgniteCache<Object, Object> cache = primary.cache(DEFAULT_CACHE_NAME);
+
+            final TransactionProxyImpl<?, ?> tx = (TransactionProxyImpl<?, ?>)primary.transactions().txStart();
+
+            final Integer key = primaryKeys(cache, 1, 0).get(0);
+
+            cache.put(key, key);
+
+            tx.tx().prepare(true);
+
+            for (Ignite grid : G.allGrids())
+                assertTrue(((IgniteEx)grid).context().cache().context().tm().activeTransactions().size() == 1);
+
+            Collection<IgniteInternalTx> backupTransactions = new LinkedList<>();
+
+            for (Ignite grid : backupNodes(key, DEFAULT_CACHE_NAME))
+                backupTransactions.addAll(((IgniteEx)grid).context().cache().context().tm().activeTransactions());
+
+            assertTrue(backupTransactions.size() == 2);
+
+            // Block recovery procedure processing on one of the backup nodes.
+            backup.context().pools().getSystemExecutorService().execute(backupBlockerTask);
+
+            // Block stripe tx recovery request processing on one of the backup nodes (note that the only stripe is
+            // configured in the executor).
+            backup.context().pools().getStripedExecutorService().execute(0, backupBlockerTask);
+
+            // Prevent tx finish request processing on both backup nodes.
+            for (Ignite grid : backupNodes(key, DEFAULT_CACHE_NAME))

Review Comment:
   Reworked



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] anton-vinogradov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
anton-vinogradov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r938583884


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +264,129 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there will be 2 backup nodes
+     * to execute the tx recovery in parallel if primary one would fail. These backup nodes will send the
+     * tx recovery requests to each other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 100; iter++) {
+            stopAllGrids();
+
+            log.info("iteration=" + iter);
+
+            final IgniteEx grid0 = startGrid(0);
+
+            final IgniteEx grid1 = startGrid(1, (UnaryOperator<IgniteConfiguration>)cfg ->
+                    cfg.setSystemThreadPoolSize(1).setStripedPoolSize(1));
+
+            final IgniteEx grid2 = startGrid(2);
+
+            grid0.cluster().state(ACTIVE);
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Integer g2Key = primaryKeys(cache, 1, 0).get(0);
+
+            List<IgniteInternalTx> txs0 = null;
+            List<IgniteInternalTx> txs1 = null;
+            List<IgniteInternalTx> txs2;
+
+            final CountDownLatch grid1BlockLatch = new CountDownLatch(1);
+
+            final CountDownLatch grid1NodeLeftEventLatch = new CountDownLatch(1);
+
+            int[] stripeHolder = new int[1];
+
+            try (final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                cache.put(g2Key, Boolean.TRUE);
+
+                TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;
+
+                p.tx().prepare(true);
+
+                txs0 = txs(grid0);
+                txs1 = txs(grid1);
+                txs2 = txs(grid2);
+
+                assertTrue(txs0.size() == 1);
+                assertTrue(txs1.size() == 1);
+                assertTrue(txs2.size() == 1);
+
+                grid1.events().localListen(new PE() {
+                    @Override public boolean apply(Event evt) {
+                        grid1NodeLeftEventLatch.countDown();
+
+                        return true;
+                    }
+                }, EventType.EVT_NODE_LEFT);
+
+                // Block recovery procedure processing on grid1
+                grid1.context().pools().getSystemExecutorService().execute(() -> U.awaitQuiet(grid1BlockLatch));
+
+                int stripe = U.safeAbs(p.tx().xidVersion().hashCode());
+
+                stripeHolder[0] = stripe;

Review Comment:
   Looks like a sync anti pattern.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r939662025


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +264,129 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there will be 2 backup nodes
+     * to execute the tx recovery in parallel if primary one would fail. These backup nodes will send the
+     * tx recovery requests to each other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 100; iter++) {
+            stopAllGrids();
+
+            log.info("iteration=" + iter);
+
+            final IgniteEx grid0 = startGrid(0);
+
+            final IgniteEx grid1 = startGrid(1, (UnaryOperator<IgniteConfiguration>)cfg ->
+                    cfg.setSystemThreadPoolSize(1).setStripedPoolSize(1));
+
+            final IgniteEx grid2 = startGrid(2);
+
+            grid0.cluster().state(ACTIVE);
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Integer g2Key = primaryKeys(cache, 1, 0).get(0);
+
+            List<IgniteInternalTx> txs0 = null;
+            List<IgniteInternalTx> txs1 = null;
+            List<IgniteInternalTx> txs2;
+
+            final CountDownLatch grid1BlockLatch = new CountDownLatch(1);
+
+            final CountDownLatch grid1NodeLeftEventLatch = new CountDownLatch(1);
+
+            int[] stripeHolder = new int[1];
+
+            try (final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                cache.put(g2Key, Boolean.TRUE);
+
+                TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;
+
+                p.tx().prepare(true);
+
+                txs0 = txs(grid0);
+                txs1 = txs(grid1);
+                txs2 = txs(grid2);
+
+                assertTrue(txs0.size() == 1);
+                assertTrue(txs1.size() == 1);
+                assertTrue(txs2.size() == 1);
+
+                grid1.events().localListen(new PE() {
+                    @Override public boolean apply(Event evt) {
+                        grid1NodeLeftEventLatch.countDown();
+
+                        return true;
+                    }
+                }, EventType.EVT_NODE_LEFT);
+
+                // Block recovery procedure processing on grid1
+                grid1.context().pools().getSystemExecutorService().execute(() -> U.awaitQuiet(grid1BlockLatch));
+
+                int stripe = U.safeAbs(p.tx().xidVersion().hashCode());
+
+                stripeHolder[0] = stripe;
+
+                // Block stripe tx recovery request processing on grid1
+                grid1.context().pools().getStripedExecutorService().execute(stripe, () -> U.awaitQuiet(grid1BlockLatch));
+
+                // Prevent finish request processing on grid0
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid0.name());
+
+                // Prevent finish request processing on grid1
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid1.name());
+
+                runAsync(() -> {
+                    grid2.close();
+                    return null;
+                });
+            }
+            catch (Exception ignored) {
+                // Expected.
+            }
+
+            // Wait until grid1 node detects primary node left
+            grid1NodeLeftEventLatch.await();
+
+            // Wait until grid1 receives the tx recovery request and the corresponding processing task is added into the queue
+            assertTrue("tx recovery request received on grid1", GridTestUtils.waitForCondition(() ->
+                    grid1.context().pools().getStripedExecutorService().queueStripeSize(stripeHolder[0]) == 1, 5_000));

Review Comment:
   Not sure which newline is meant here in particular.  Tryed to fix newline usages in the patch as a whole.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r939662093


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +264,129 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there will be 2 backup nodes
+     * to execute the tx recovery in parallel if primary one would fail. These backup nodes will send the
+     * tx recovery requests to each other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 100; iter++) {
+            stopAllGrids();
+
+            log.info("iteration=" + iter);
+
+            final IgniteEx grid0 = startGrid(0);
+
+            final IgniteEx grid1 = startGrid(1, (UnaryOperator<IgniteConfiguration>)cfg ->
+                    cfg.setSystemThreadPoolSize(1).setStripedPoolSize(1));
+
+            final IgniteEx grid2 = startGrid(2);
+
+            grid0.cluster().state(ACTIVE);
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Integer g2Key = primaryKeys(cache, 1, 0).get(0);
+
+            List<IgniteInternalTx> txs0 = null;
+            List<IgniteInternalTx> txs1 = null;
+            List<IgniteInternalTx> txs2;
+
+            final CountDownLatch grid1BlockLatch = new CountDownLatch(1);
+
+            final CountDownLatch grid1NodeLeftEventLatch = new CountDownLatch(1);
+
+            int[] stripeHolder = new int[1];
+
+            try (final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                cache.put(g2Key, Boolean.TRUE);
+
+                TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;
+
+                p.tx().prepare(true);
+
+                txs0 = txs(grid0);
+                txs1 = txs(grid1);
+                txs2 = txs(grid2);
+
+                assertTrue(txs0.size() == 1);
+                assertTrue(txs1.size() == 1);
+                assertTrue(txs2.size() == 1);
+
+                grid1.events().localListen(new PE() {
+                    @Override public boolean apply(Event evt) {
+                        grid1NodeLeftEventLatch.countDown();
+
+                        return true;
+                    }
+                }, EventType.EVT_NODE_LEFT);
+
+                // Block recovery procedure processing on grid1
+                grid1.context().pools().getSystemExecutorService().execute(() -> U.awaitQuiet(grid1BlockLatch));
+
+                int stripe = U.safeAbs(p.tx().xidVersion().hashCode());
+
+                stripeHolder[0] = stripe;
+
+                // Block stripe tx recovery request processing on grid1
+                grid1.context().pools().getStripedExecutorService().execute(stripe, () -> U.awaitQuiet(grid1BlockLatch));

Review Comment:
   Looks like fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] anton-vinogradov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
anton-vinogradov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r940123476


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +263,120 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 100; iter++) {
+            final IgniteEx grid0 = startGrid(0);
+
+            final IgniteEx grid1 = startGrid(1, (UnaryOperator<IgniteConfiguration>)cfg -> cfg
+                .setSystemThreadPoolSize(1).setStripedPoolSize(1));
+
+            final IgniteEx grid2 = startGrid(2);
+
+            grid0.cluster().state(ACTIVE);
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
+
+            final Integer g2Key = primaryKeys(cache, 1, 0).get(0);
+
+            cache.put(g2Key, Boolean.TRUE);
+
+            final TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;
+
+            p.tx().prepare(true);
+
+            final List<IgniteInternalTx> txs0 = txs(grid0);
+            final List<IgniteInternalTx> txs1 = txs(grid1);
+            final List<IgniteInternalTx> txs2 = txs(grid2);
+
+            assertTrue(txs0.size() == 1);
+            assertTrue(txs1.size() == 1);
+            assertTrue(txs2.size() == 1);
+
+            final CountDownLatch grid1NodeLeftEventLatch = new CountDownLatch(1);
+
+            grid1.events().localListen(new PE() {
+                @Override public boolean apply(Event evt) {
+                    grid1NodeLeftEventLatch.countDown();
+
+                    return true;
+                }
+            }, EventType.EVT_NODE_LEFT);
+
+            final CountDownLatch grid1BlockLatch = new CountDownLatch(1);
+
+            // Block recovery procedure processing on grid1.
+            grid1.context().pools().getSystemExecutorService().execute(() -> U.awaitQuiet(grid1BlockLatch));
+
+            final int stripe = U.safeAbs(p.tx().xidVersion().hashCode());
+
+            // Block stripe tx recovery request processing on grid1.
+            grid1.context().pools().getStripedExecutorService().execute(stripe, () -> U.awaitQuiet(grid1BlockLatch));
+
+            // Prevent finish request processing on grid0.
+            spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid0.name());
+
+            // Prevent finish request processing on grid1.
+            spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid1.name());
+
+            runAsync(() -> {
+                grid2.close();
+
+                return null;
+            });
+
+            try {
+                tx.close();
+            }
+            catch (Exception ignored) {
+                // Don't bother if the transaction close throws in case grid2 appear to be stopping or stopped already
+                // for this thread.
+            }
+
+            // Wait until grid1 node detects primary node left.
+            grid1NodeLeftEventLatch.await();
+
+            // Wait until grid1 receives the tx recovery request and the corresponding processing task is added into
+            // the queue.
+            assertTrue("tx recovery request received on grid1", GridTestUtils.waitForCondition(() -> grid1.context()

Review Comment:
   assert message is about situation when assert failed



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +263,120 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 100; iter++) {
+            final IgniteEx grid0 = startGrid(0);
+
+            final IgniteEx grid1 = startGrid(1, (UnaryOperator<IgniteConfiguration>)cfg -> cfg
+                .setSystemThreadPoolSize(1).setStripedPoolSize(1));
+
+            final IgniteEx grid2 = startGrid(2);
+
+            grid0.cluster().state(ACTIVE);
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
+
+            final Integer g2Key = primaryKeys(cache, 1, 0).get(0);

Review Comment:
   what is g2Key, could this be renamed according to its role?



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +263,120 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 100; iter++) {
+            final IgniteEx grid0 = startGrid(0);
+
+            final IgniteEx grid1 = startGrid(1, (UnaryOperator<IgniteConfiguration>)cfg -> cfg
+                .setSystemThreadPoolSize(1).setStripedPoolSize(1));
+
+            final IgniteEx grid2 = startGrid(2);
+
+            grid0.cluster().state(ACTIVE);
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
+
+            final Integer g2Key = primaryKeys(cache, 1, 0).get(0);
+
+            cache.put(g2Key, Boolean.TRUE);
+
+            final TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;
+
+            p.tx().prepare(true);
+
+            final List<IgniteInternalTx> txs0 = txs(grid0);
+            final List<IgniteInternalTx> txs1 = txs(grid1);
+            final List<IgniteInternalTx> txs2 = txs(grid2);
+
+            assertTrue(txs0.size() == 1);
+            assertTrue(txs1.size() == 1);
+            assertTrue(txs2.size() == 1);
+
+            final CountDownLatch grid1NodeLeftEventLatch = new CountDownLatch(1);
+
+            grid1.events().localListen(new PE() {
+                @Override public boolean apply(Event evt) {
+                    grid1NodeLeftEventLatch.countDown();
+
+                    return true;
+                }
+            }, EventType.EVT_NODE_LEFT);
+
+            final CountDownLatch grid1BlockLatch = new CountDownLatch(1);
+
+            // Block recovery procedure processing on grid1.
+            grid1.context().pools().getSystemExecutorService().execute(() -> U.awaitQuiet(grid1BlockLatch));
+
+            final int stripe = U.safeAbs(p.tx().xidVersion().hashCode());
+
+            // Block stripe tx recovery request processing on grid1.
+            grid1.context().pools().getStripedExecutorService().execute(stripe, () -> U.awaitQuiet(grid1BlockLatch));
+
+            // Prevent finish request processing on grid0.
+            spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid0.name());
+
+            // Prevent finish request processing on grid1.
+            spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid1.name());
+
+            runAsync(() -> {
+                grid2.close();
+
+                return null;
+            });
+
+            try {
+                tx.close();
+            }
+            catch (Exception ignored) {
+                // Don't bother if the transaction close throws in case grid2 appear to be stopping or stopped already
+                // for this thread.
+            }

Review Comment:
   Cound we avoid doing thins we dont bother about?
   Do we really need to close the tx here?
   Could we refactor this somehow to avoid catched ignoring?



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +263,120 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 100; iter++) {
+            final IgniteEx grid0 = startGrid(0);
+
+            final IgniteEx grid1 = startGrid(1, (UnaryOperator<IgniteConfiguration>)cfg -> cfg
+                .setSystemThreadPoolSize(1).setStripedPoolSize(1));
+
+            final IgniteEx grid2 = startGrid(2);
+
+            grid0.cluster().state(ACTIVE);
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
+
+            final Integer g2Key = primaryKeys(cache, 1, 0).get(0);
+
+            cache.put(g2Key, Boolean.TRUE);
+
+            final TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;
+
+            p.tx().prepare(true);
+
+            final List<IgniteInternalTx> txs0 = txs(grid0);
+            final List<IgniteInternalTx> txs1 = txs(grid1);
+            final List<IgniteInternalTx> txs2 = txs(grid2);
+
+            assertTrue(txs0.size() == 1);
+            assertTrue(txs1.size() == 1);
+            assertTrue(txs2.size() == 1);
+
+            final CountDownLatch grid1NodeLeftEventLatch = new CountDownLatch(1);
+
+            grid1.events().localListen(new PE() {
+                @Override public boolean apply(Event evt) {
+                    grid1NodeLeftEventLatch.countDown();
+
+                    return true;
+                }
+            }, EventType.EVT_NODE_LEFT);
+
+            final CountDownLatch grid1BlockLatch = new CountDownLatch(1);
+
+            // Block recovery procedure processing on grid1.
+            grid1.context().pools().getSystemExecutorService().execute(() -> U.awaitQuiet(grid1BlockLatch));
+
+            final int stripe = U.safeAbs(p.tx().xidVersion().hashCode());

Review Comment:
   Any chances to get this from pool's code, to make this more refactoring friendly?
   Will this always be 0 because of `setStripedPoolSize(1)`?



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +263,120 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 100; iter++) {

Review Comment:
   Do we really need to do this 100 times?
   Looks lile you describing precise situation, not probabilistic



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r941436395


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +263,120 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 100; iter++) {
+            final IgniteEx grid0 = startGrid(0);
+
+            final IgniteEx grid1 = startGrid(1, (UnaryOperator<IgniteConfiguration>)cfg -> cfg
+                .setSystemThreadPoolSize(1).setStripedPoolSize(1));
+
+            final IgniteEx grid2 = startGrid(2);
+
+            grid0.cluster().state(ACTIVE);
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
+
+            final Integer g2Key = primaryKeys(cache, 1, 0).get(0);
+
+            cache.put(g2Key, Boolean.TRUE);
+
+            final TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;
+
+            p.tx().prepare(true);
+
+            final List<IgniteInternalTx> txs0 = txs(grid0);
+            final List<IgniteInternalTx> txs1 = txs(grid1);
+            final List<IgniteInternalTx> txs2 = txs(grid2);
+
+            assertTrue(txs0.size() == 1);
+            assertTrue(txs1.size() == 1);
+            assertTrue(txs2.size() == 1);
+
+            final CountDownLatch grid1NodeLeftEventLatch = new CountDownLatch(1);
+
+            grid1.events().localListen(new PE() {
+                @Override public boolean apply(Event evt) {
+                    grid1NodeLeftEventLatch.countDown();
+
+                    return true;
+                }
+            }, EventType.EVT_NODE_LEFT);
+
+            final CountDownLatch grid1BlockLatch = new CountDownLatch(1);
+
+            // Block recovery procedure processing on grid1.
+            grid1.context().pools().getSystemExecutorService().execute(() -> U.awaitQuiet(grid1BlockLatch));
+
+            final int stripe = U.safeAbs(p.tx().xidVersion().hashCode());
+
+            // Block stripe tx recovery request processing on grid1.
+            grid1.context().pools().getStripedExecutorService().execute(stripe, () -> U.awaitQuiet(grid1BlockLatch));
+
+            // Prevent finish request processing on grid0.
+            spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid0.name());
+
+            // Prevent finish request processing on grid1.
+            spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid1.name());
+
+            runAsync(() -> {
+                grid2.close();
+
+                return null;
+            });
+
+            try {
+                tx.close();
+            }
+            catch (Exception ignored) {
+                // Don't bother if the transaction close throws in case grid2 appear to be stopping or stopped already
+                // for this thread.
+            }

Review Comment:
   I looked at what the tx close does.  And looks like we don't actually need it here.  It just will be discarded on the next repetition.



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +263,120 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 100; iter++) {
+            final IgniteEx grid0 = startGrid(0);
+
+            final IgniteEx grid1 = startGrid(1, (UnaryOperator<IgniteConfiguration>)cfg -> cfg
+                .setSystemThreadPoolSize(1).setStripedPoolSize(1));
+
+            final IgniteEx grid2 = startGrid(2);
+
+            grid0.cluster().state(ACTIVE);
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
+
+            final Integer g2Key = primaryKeys(cache, 1, 0).get(0);
+
+            cache.put(g2Key, Boolean.TRUE);
+
+            final TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;
+
+            p.tx().prepare(true);
+
+            final List<IgniteInternalTx> txs0 = txs(grid0);
+            final List<IgniteInternalTx> txs1 = txs(grid1);
+            final List<IgniteInternalTx> txs2 = txs(grid2);
+
+            assertTrue(txs0.size() == 1);
+            assertTrue(txs1.size() == 1);
+            assertTrue(txs2.size() == 1);
+
+            final CountDownLatch grid1NodeLeftEventLatch = new CountDownLatch(1);
+
+            grid1.events().localListen(new PE() {
+                @Override public boolean apply(Event evt) {
+                    grid1NodeLeftEventLatch.countDown();
+
+                    return true;
+                }
+            }, EventType.EVT_NODE_LEFT);
+
+            final CountDownLatch grid1BlockLatch = new CountDownLatch(1);
+
+            // Block recovery procedure processing on grid1.
+            grid1.context().pools().getSystemExecutorService().execute(() -> U.awaitQuiet(grid1BlockLatch));
+
+            final int stripe = U.safeAbs(p.tx().xidVersion().hashCode());
+
+            // Block stripe tx recovery request processing on grid1.
+            grid1.context().pools().getStripedExecutorService().execute(stripe, () -> U.awaitQuiet(grid1BlockLatch));
+
+            // Prevent finish request processing on grid0.
+            spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid0.name());
+
+            // Prevent finish request processing on grid1.
+            spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid1.name());
+
+            runAsync(() -> {
+                grid2.close();
+
+                return null;
+            });
+
+            try {
+                tx.close();
+            }
+            catch (Exception ignored) {
+                // Don't bother if the transaction close throws in case grid2 appear to be stopping or stopped already
+                // for this thread.
+            }
+
+            // Wait until grid1 node detects primary node left.
+            grid1NodeLeftEventLatch.await();
+
+            // Wait until grid1 receives the tx recovery request and the corresponding processing task is added into
+            // the queue.
+            assertTrue("tx recovery request received on grid1", GridTestUtils.waitForCondition(() -> grid1.context()

Review Comment:
   Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] anton-vinogradov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
anton-vinogradov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r937600089


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +259,110 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <ul>
+     *  <li>Start 3 nodes (g0, g1, g2) and cache with 2 backups.</li>
+     *  <li>Prepare a transaction with g2 as a primary node.</li>
+     *  <li>Kill g2.</li>
+     *  <li>Enforce the concurrent processing of transaction in tx recovery on g1.</li>
+     * </ul>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished both on g0 and g1
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 75; iter++) {

Review Comment:
   Another TC will have another speed. 
   Better fix is to fix test to be faster, I think.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r938099125


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +259,110 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <ul>
+     *  <li>Start 3 nodes (g0, g1, g2) and cache with 2 backups.</li>
+     *  <li>Prepare a transaction with g2 as a primary node.</li>
+     *  <li>Kill g2.</li>
+     *  <li>Enforce the concurrent processing of transaction in tx recovery on g1.</li>
+     * </ul>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished both on g0 and g1
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 75; iter++) {
+            stopAllGrids();
+
+            log.info("iteration=" + iter);
+            final IgniteEx grid0 = startGrid("g0");
+            final IgniteEx grid1 = startGrid("g1",
+                    cfg -> cfg.setSystemThreadPoolSize(1).setStripedPoolSize(1));
+            final IgniteEx grid2 = startGrid("g2");
+
+            grid0.cluster().state(ACTIVE);
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Integer g2Key = primaryKeys(grid2.cache(DEFAULT_CACHE_NAME), 1, 0).get(0);
+
+            List<IgniteInternalTx> txs0 = null;
+            List<IgniteInternalTx> txs1 = null;
+
+            CountDownLatch grid1BlockLatch = new CountDownLatch(1);
+
+            try (final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                cache.put(g2Key, Boolean.TRUE);
+                TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;
+                p.tx().prepare(true);
+
+                txs0 = txs(grid0);
+                txs1 = txs(grid1);
+                List<IgniteInternalTx> txs2 = txs(grid2);
+
+                assertTrue(txs0.size() == 1);
+                assertTrue(txs1.size() == 1);
+                assertTrue(txs2.size() == 1);
+
+                // Prevent tx recovery request to be sent from grid0 to grid1.
+                spi(grid0).blockMessages(GridCacheTxRecoveryRequest.class, grid1.name());
+
+                // Block recovery procedure processing on grid1
+                grid1.context().pools().getSystemExecutorService().execute(() -> U.awaitQuiet(grid1BlockLatch));
+                // Block stripe tx recovery request processing on grid1
+                int stripe = U.safeAbs(p.tx().xidVersion().hashCode());
+                grid1.context().pools().getStripedExecutorService().execute(stripe, () -> U.awaitQuiet(grid1BlockLatch));
+
+                // Prevent finish request processing on grid0 and grid1.
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid0.name());
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid1.name());
+
+                runAsync(() -> {
+                    grid2.close();
+                    return null;
+                });
+            }
+            catch (Exception ignored) {
+                // Expected.
+            }
+
+            // Wait grid0 is ready to send the tx recovery request to grid1
+            spi(grid0).waitForBlocked();
+            // Let grid0 send the tx recovery request to grid1
+            log.info("unblock grid0");
+            spi(grid0).stopBlock();
+            // Give grid1 some time to receive the tx recovery request (processing is still blocked in grid1).
+            log.info("sleep in grid0");
+            doSleep(100);
+
+            // Unblock processing in grid1. Simultaneously in striped and system pools to start
+            // recovery procedure and the tx recovery request processing at the "same" moment
+            // (for the same transaction). This should increase chances for race condition occur in
+            // the IgniteTxAdapter::markFinalizing.
+            log.info("unblock grid1");
+            grid1BlockLatch.countDown();
+
+            // Wait transaction finish in grid0.
+            txs0.get(0).finishFuture().get(5_000);

Review Comment:
   Reworked to use "wait" tools from the base test class without explicit timeouts in the test as such.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r938099982


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +259,110 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <ul>
+     *  <li>Start 3 nodes (g0, g1, g2) and cache with 2 backups.</li>
+     *  <li>Prepare a transaction with g2 as a primary node.</li>
+     *  <li>Kill g2.</li>
+     *  <li>Enforce the concurrent processing of transaction in tx recovery on g1.</li>
+     * </ul>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished both on g0 and g1
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 75; iter++) {
+            stopAllGrids();
+
+            log.info("iteration=" + iter);
+            final IgniteEx grid0 = startGrid("g0");
+            final IgniteEx grid1 = startGrid("g1",
+                    cfg -> cfg.setSystemThreadPoolSize(1).setStripedPoolSize(1));
+            final IgniteEx grid2 = startGrid("g2");

Review Comment:
   Fixed - use the default names.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r943079857


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentOnPrimaryFailTest.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.Collection;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentOnPrimaryFailTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME).setCacheMode(PARTITIONED)
+            .setBackups(2).setAtomicityMode(TRANSACTIONAL).setAffinity(new RendezvousAffinityFunction(false, 1)));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        final IgniteEx grid0 = startGrid(0);
+
+        final IgniteEx grid1 = startGrid(1);
+
+        final CyclicBarrier grid1BlockerBarrier = new CyclicBarrier(3);
+
+        final Runnable grid1BlockerTask = () -> {
+            try {
+                grid1BlockerBarrier.await();
+            }
+            catch (InterruptedException | BrokenBarrierException e) {
+                // Just supress.
+            }
+        };
+
+        for (int iter = 0; iter < 100; iter++) {
+            final IgniteEx grid2 = startGrid(2);
+
+            awaitPartitionMapExchange();
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
+
+            // Key for which the grid2 node is primary.
+            final Integer grid2PrimaryKey = primaryKeys(cache, 1, 0).get(0);

Review Comment:
   Fixed



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentOnPrimaryFailTest.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.Collection;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentOnPrimaryFailTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME).setCacheMode(PARTITIONED)
+            .setBackups(2).setAtomicityMode(TRANSACTIONAL).setAffinity(new RendezvousAffinityFunction(false, 1)));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        final IgniteEx grid0 = startGrid(0);
+
+        final IgniteEx grid1 = startGrid(1);
+
+        final CyclicBarrier grid1BlockerBarrier = new CyclicBarrier(3);
+
+        final Runnable grid1BlockerTask = () -> {
+            try {
+                grid1BlockerBarrier.await();
+            }
+            catch (InterruptedException | BrokenBarrierException e) {
+                // Just supress.
+            }
+        };
+
+        for (int iter = 0; iter < 100; iter++) {
+            final IgniteEx grid2 = startGrid(2);
+
+            awaitPartitionMapExchange();
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
+
+            // Key for which the grid2 node is primary.
+            final Integer grid2PrimaryKey = primaryKeys(cache, 1, 0).get(0);
+
+            cache.put(grid2PrimaryKey, Boolean.TRUE);
+
+            final TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;
+
+            p.tx().prepare(true);
+
+            final Collection<IgniteInternalTx> txs0 = grid0.context().cache().context().tm().activeTransactions();
+            final Collection<IgniteInternalTx> txs1 = grid1.context().cache().context().tm().activeTransactions();
+            final Collection<IgniteInternalTx> txs2 = grid2.context().cache().context().tm().activeTransactions();
+
+            assertTrue(txs0.size() == 1);
+            assertTrue(txs1.size() == 1);
+            assertTrue(txs2.size() == 1);

Review Comment:
   Fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on PR #10178:
URL: https://github.com/apache/ignite/pull/10178#issuecomment-1213053537

   Also added guarantee that both pools are blocked in backup node before killing the primary one.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r939661610


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +264,129 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there will be 2 backup nodes
+     * to execute the tx recovery in parallel if primary one would fail. These backup nodes will send the
+     * tx recovery requests to each other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 100; iter++) {
+            stopAllGrids();
+
+            log.info("iteration=" + iter);
+
+            final IgniteEx grid0 = startGrid(0);
+
+            final IgniteEx grid1 = startGrid(1, (UnaryOperator<IgniteConfiguration>)cfg ->
+                    cfg.setSystemThreadPoolSize(1).setStripedPoolSize(1));
+
+            final IgniteEx grid2 = startGrid(2);
+
+            grid0.cluster().state(ACTIVE);
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Integer g2Key = primaryKeys(cache, 1, 0).get(0);
+
+            List<IgniteInternalTx> txs0 = null;
+            List<IgniteInternalTx> txs1 = null;
+            List<IgniteInternalTx> txs2;
+
+            final CountDownLatch grid1BlockLatch = new CountDownLatch(1);
+
+            final CountDownLatch grid1NodeLeftEventLatch = new CountDownLatch(1);
+
+            int[] stripeHolder = new int[1];
+
+            try (final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                cache.put(g2Key, Boolean.TRUE);
+
+                TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;
+
+                p.tx().prepare(true);
+
+                txs0 = txs(grid0);
+                txs1 = txs(grid1);
+                txs2 = txs(grid2);
+
+                assertTrue(txs0.size() == 1);
+                assertTrue(txs1.size() == 1);
+                assertTrue(txs2.size() == 1);
+
+                grid1.events().localListen(new PE() {
+                    @Override public boolean apply(Event evt) {
+                        grid1NodeLeftEventLatch.countDown();
+
+                        return true;
+                    }
+                }, EventType.EVT_NODE_LEFT);
+
+                // Block recovery procedure processing on grid1
+                grid1.context().pools().getSystemExecutorService().execute(() -> U.awaitQuiet(grid1BlockLatch));
+
+                int stripe = U.safeAbs(p.tx().xidVersion().hashCode());
+
+                stripeHolder[0] = stripe;
+
+                // Block stripe tx recovery request processing on grid1
+                grid1.context().pools().getStripedExecutorService().execute(stripe, () -> U.awaitQuiet(grid1BlockLatch));
+
+                // Prevent finish request processing on grid0
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid0.name());
+
+                // Prevent finish request processing on grid1
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid1.name());
+
+                runAsync(() -> {
+                    grid2.close();

Review Comment:
   FIxed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] anton-vinogradov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
anton-vinogradov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r938581624


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +264,129 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there will be 2 backup nodes
+     * to execute the tx recovery in parallel if primary one would fail. These backup nodes will send the
+     * tx recovery requests to each other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 100; iter++) {
+            stopAllGrids();
+
+            log.info("iteration=" + iter);
+
+            final IgniteEx grid0 = startGrid(0);
+
+            final IgniteEx grid1 = startGrid(1, (UnaryOperator<IgniteConfiguration>)cfg ->
+                    cfg.setSystemThreadPoolSize(1).setStripedPoolSize(1));
+
+            final IgniteEx grid2 = startGrid(2);
+
+            grid0.cluster().state(ACTIVE);
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Integer g2Key = primaryKeys(cache, 1, 0).get(0);
+
+            List<IgniteInternalTx> txs0 = null;
+            List<IgniteInternalTx> txs1 = null;
+            List<IgniteInternalTx> txs2;
+
+            final CountDownLatch grid1BlockLatch = new CountDownLatch(1);
+
+            final CountDownLatch grid1NodeLeftEventLatch = new CountDownLatch(1);
+
+            int[] stripeHolder = new int[1];
+
+            try (final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                cache.put(g2Key, Boolean.TRUE);
+
+                TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;
+
+                p.tx().prepare(true);
+
+                txs0 = txs(grid0);
+                txs1 = txs(grid1);
+                txs2 = txs(grid2);
+
+                assertTrue(txs0.size() == 1);
+                assertTrue(txs1.size() == 1);
+                assertTrue(txs2.size() == 1);
+
+                grid1.events().localListen(new PE() {
+                    @Override public boolean apply(Event evt) {
+                        grid1NodeLeftEventLatch.countDown();
+
+                        return true;
+                    }
+                }, EventType.EVT_NODE_LEFT);
+
+                // Block recovery procedure processing on grid1
+                grid1.context().pools().getSystemExecutorService().execute(() -> U.awaitQuiet(grid1BlockLatch));
+
+                int stripe = U.safeAbs(p.tx().xidVersion().hashCode());
+
+                stripeHolder[0] = stripe;
+
+                // Block stripe tx recovery request processing on grid1
+                grid1.context().pools().getStripedExecutorService().execute(stripe, () -> U.awaitQuiet(grid1BlockLatch));
+
+                // Prevent finish request processing on grid0
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid0.name());
+
+                // Prevent finish request processing on grid1
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid1.name());
+
+                runAsync(() -> {
+                    grid2.close();
+                    return null;
+                });
+            }
+            catch (Exception ignored) {
+                // Expected.

Review Comment:
   It's not clear where it's expected. 
   A huge amount of code is wrapped by try-catch section.



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +264,129 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there will be 2 backup nodes
+     * to execute the tx recovery in parallel if primary one would fail. These backup nodes will send the
+     * tx recovery requests to each other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 100; iter++) {
+            stopAllGrids();
+
+            log.info("iteration=" + iter);
+
+            final IgniteEx grid0 = startGrid(0);
+
+            final IgniteEx grid1 = startGrid(1, (UnaryOperator<IgniteConfiguration>)cfg ->
+                    cfg.setSystemThreadPoolSize(1).setStripedPoolSize(1));
+
+            final IgniteEx grid2 = startGrid(2);
+
+            grid0.cluster().state(ACTIVE);
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Integer g2Key = primaryKeys(cache, 1, 0).get(0);
+
+            List<IgniteInternalTx> txs0 = null;
+            List<IgniteInternalTx> txs1 = null;
+            List<IgniteInternalTx> txs2;
+
+            final CountDownLatch grid1BlockLatch = new CountDownLatch(1);
+
+            final CountDownLatch grid1NodeLeftEventLatch = new CountDownLatch(1);
+
+            int[] stripeHolder = new int[1];
+
+            try (final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                cache.put(g2Key, Boolean.TRUE);
+
+                TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;
+
+                p.tx().prepare(true);
+
+                txs0 = txs(grid0);
+                txs1 = txs(grid1);
+                txs2 = txs(grid2);
+
+                assertTrue(txs0.size() == 1);
+                assertTrue(txs1.size() == 1);
+                assertTrue(txs2.size() == 1);
+
+                grid1.events().localListen(new PE() {
+                    @Override public boolean apply(Event evt) {
+                        grid1NodeLeftEventLatch.countDown();
+
+                        return true;
+                    }
+                }, EventType.EVT_NODE_LEFT);
+
+                // Block recovery procedure processing on grid1
+                grid1.context().pools().getSystemExecutorService().execute(() -> U.awaitQuiet(grid1BlockLatch));
+
+                int stripe = U.safeAbs(p.tx().xidVersion().hashCode());
+
+                stripeHolder[0] = stripe;
+
+                // Block stripe tx recovery request processing on grid1
+                grid1.context().pools().getStripedExecutorService().execute(stripe, () -> U.awaitQuiet(grid1BlockLatch));
+
+                // Prevent finish request processing on grid0
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid0.name());
+
+                // Prevent finish request processing on grid1
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid1.name());
+
+                runAsync(() -> {
+                    grid2.close();
+                    return null;
+                });
+            }
+            catch (Exception ignored) {
+                // Expected.
+            }
+
+            // Wait until grid1 node detects primary node left
+            grid1NodeLeftEventLatch.await();
+
+            // Wait until grid1 receives the tx recovery request and the corresponding processing task is added into the queue
+            assertTrue("tx recovery request received on grid1", GridTestUtils.waitForCondition(() ->
+                    grid1.context().pools().getStripedExecutorService().queueStripeSize(stripeHolder[0]) == 1, 5_000));
+
+            // Unblock processing in grid1. Simultaneously in striped and system pools to start
+            // recovery procedure and the tx recovery request processing at the "same" moment
+            // (for the same transaction). This should increase chances for race condition occur in
+            // the IgniteTxAdapter::markFinalizing.
+            grid1BlockLatch.countDown();
+
+            waitForTopology(2);
+
+            try {
+                awaitPartitionMapExchange();
+            }
+            catch (IgniteException ex) {
+                log.error("exception from the awaitPartitionMapExchange", ex);
+                fail("fail to wait for the partition map exchange, iteration=" + iter);
+            }

Review Comment:
   Any reason to rethrow an exception?
   Looks like `awaitPartitionMapExchange();` already do whan necessary.



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +264,129 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there will be 2 backup nodes
+     * to execute the tx recovery in parallel if primary one would fail. These backup nodes will send the
+     * tx recovery requests to each other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 100; iter++) {
+            stopAllGrids();
+
+            log.info("iteration=" + iter);
+
+            final IgniteEx grid0 = startGrid(0);
+
+            final IgniteEx grid1 = startGrid(1, (UnaryOperator<IgniteConfiguration>)cfg ->
+                    cfg.setSystemThreadPoolSize(1).setStripedPoolSize(1));
+
+            final IgniteEx grid2 = startGrid(2);
+
+            grid0.cluster().state(ACTIVE);
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Integer g2Key = primaryKeys(cache, 1, 0).get(0);
+
+            List<IgniteInternalTx> txs0 = null;
+            List<IgniteInternalTx> txs1 = null;
+            List<IgniteInternalTx> txs2;
+
+            final CountDownLatch grid1BlockLatch = new CountDownLatch(1);
+
+            final CountDownLatch grid1NodeLeftEventLatch = new CountDownLatch(1);
+
+            int[] stripeHolder = new int[1];
+
+            try (final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                cache.put(g2Key, Boolean.TRUE);
+
+                TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;
+
+                p.tx().prepare(true);
+
+                txs0 = txs(grid0);
+                txs1 = txs(grid1);
+                txs2 = txs(grid2);
+
+                assertTrue(txs0.size() == 1);
+                assertTrue(txs1.size() == 1);
+                assertTrue(txs2.size() == 1);
+
+                grid1.events().localListen(new PE() {
+                    @Override public boolean apply(Event evt) {
+                        grid1NodeLeftEventLatch.countDown();
+
+                        return true;
+                    }
+                }, EventType.EVT_NODE_LEFT);
+
+                // Block recovery procedure processing on grid1
+                grid1.context().pools().getSystemExecutorService().execute(() -> U.awaitQuiet(grid1BlockLatch));
+
+                int stripe = U.safeAbs(p.tx().xidVersion().hashCode());
+
+                stripeHolder[0] = stripe;
+
+                // Block stripe tx recovery request processing on grid1
+                grid1.context().pools().getStripedExecutorService().execute(stripe, () -> U.awaitQuiet(grid1BlockLatch));
+
+                // Prevent finish request processing on grid0
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid0.name());
+
+                // Prevent finish request processing on grid1
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid1.name());
+
+                runAsync(() -> {
+                    grid2.close();
+                    return null;
+                });
+            }
+            catch (Exception ignored) {
+                // Expected.
+            }
+
+            // Wait until grid1 node detects primary node left
+            grid1NodeLeftEventLatch.await();
+
+            // Wait until grid1 receives the tx recovery request and the corresponding processing task is added into the queue
+            assertTrue("tx recovery request received on grid1", GridTestUtils.waitForCondition(() ->
+                    grid1.context().pools().getStripedExecutorService().queueStripeSize(stripeHolder[0]) == 1, 5_000));

Review Comment:
   Looks like a useless newline



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +264,129 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there will be 2 backup nodes
+     * to execute the tx recovery in parallel if primary one would fail. These backup nodes will send the
+     * tx recovery requests to each other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 100; iter++) {
+            stopAllGrids();
+
+            log.info("iteration=" + iter);
+
+            final IgniteEx grid0 = startGrid(0);
+
+            final IgniteEx grid1 = startGrid(1, (UnaryOperator<IgniteConfiguration>)cfg ->
+                    cfg.setSystemThreadPoolSize(1).setStripedPoolSize(1));
+
+            final IgniteEx grid2 = startGrid(2);
+
+            grid0.cluster().state(ACTIVE);
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Integer g2Key = primaryKeys(cache, 1, 0).get(0);
+
+            List<IgniteInternalTx> txs0 = null;
+            List<IgniteInternalTx> txs1 = null;
+            List<IgniteInternalTx> txs2;
+
+            final CountDownLatch grid1BlockLatch = new CountDownLatch(1);
+
+            final CountDownLatch grid1NodeLeftEventLatch = new CountDownLatch(1);
+
+            int[] stripeHolder = new int[1];
+
+            try (final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                cache.put(g2Key, Boolean.TRUE);
+
+                TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;
+
+                p.tx().prepare(true);
+
+                txs0 = txs(grid0);
+                txs1 = txs(grid1);
+                txs2 = txs(grid2);
+
+                assertTrue(txs0.size() == 1);
+                assertTrue(txs1.size() == 1);
+                assertTrue(txs2.size() == 1);
+
+                grid1.events().localListen(new PE() {
+                    @Override public boolean apply(Event evt) {
+                        grid1NodeLeftEventLatch.countDown();
+
+                        return true;
+                    }
+                }, EventType.EVT_NODE_LEFT);
+
+                // Block recovery procedure processing on grid1
+                grid1.context().pools().getSystemExecutorService().execute(() -> U.awaitQuiet(grid1BlockLatch));
+
+                int stripe = U.safeAbs(p.tx().xidVersion().hashCode());
+
+                stripeHolder[0] = stripe;
+
+                // Block stripe tx recovery request processing on grid1
+                grid1.context().pools().getStripedExecutorService().execute(stripe, () -> U.awaitQuiet(grid1BlockLatch));
+
+                // Prevent finish request processing on grid0
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid0.name());
+
+                // Prevent finish request processing on grid1
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid1.name());
+
+                runAsync(() -> {
+                    grid2.close();

Review Comment:
   newline missed betwen the semantic units



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +264,129 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there will be 2 backup nodes
+     * to execute the tx recovery in parallel if primary one would fail. These backup nodes will send the
+     * tx recovery requests to each other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 100; iter++) {
+            stopAllGrids();
+
+            log.info("iteration=" + iter);
+
+            final IgniteEx grid0 = startGrid(0);
+
+            final IgniteEx grid1 = startGrid(1, (UnaryOperator<IgniteConfiguration>)cfg ->
+                    cfg.setSystemThreadPoolSize(1).setStripedPoolSize(1));
+
+            final IgniteEx grid2 = startGrid(2);
+
+            grid0.cluster().state(ACTIVE);
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Integer g2Key = primaryKeys(cache, 1, 0).get(0);
+
+            List<IgniteInternalTx> txs0 = null;
+            List<IgniteInternalTx> txs1 = null;
+            List<IgniteInternalTx> txs2;
+
+            final CountDownLatch grid1BlockLatch = new CountDownLatch(1);
+
+            final CountDownLatch grid1NodeLeftEventLatch = new CountDownLatch(1);
+
+            int[] stripeHolder = new int[1];
+
+            try (final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                cache.put(g2Key, Boolean.TRUE);
+
+                TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;
+
+                p.tx().prepare(true);
+
+                txs0 = txs(grid0);
+                txs1 = txs(grid1);
+                txs2 = txs(grid2);
+
+                assertTrue(txs0.size() == 1);
+                assertTrue(txs1.size() == 1);
+                assertTrue(txs2.size() == 1);
+
+                grid1.events().localListen(new PE() {
+                    @Override public boolean apply(Event evt) {
+                        grid1NodeLeftEventLatch.countDown();
+
+                        return true;
+                    }
+                }, EventType.EVT_NODE_LEFT);
+
+                // Block recovery procedure processing on grid1
+                grid1.context().pools().getSystemExecutorService().execute(() -> U.awaitQuiet(grid1BlockLatch));
+
+                int stripe = U.safeAbs(p.tx().xidVersion().hashCode());
+
+                stripeHolder[0] = stripe;

Review Comment:
   Looks like a sync antipatern.



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +264,129 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there will be 2 backup nodes
+     * to execute the tx recovery in parallel if primary one would fail. These backup nodes will send the
+     * tx recovery requests to each other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 100; iter++) {
+            stopAllGrids();
+
+            log.info("iteration=" + iter);

Review Comment:
   Helpless logging



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +264,129 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there will be 2 backup nodes
+     * to execute the tx recovery in parallel if primary one would fail. These backup nodes will send the
+     * tx recovery requests to each other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 100; iter++) {
+            stopAllGrids();
+
+            log.info("iteration=" + iter);
+
+            final IgniteEx grid0 = startGrid(0);
+
+            final IgniteEx grid1 = startGrid(1, (UnaryOperator<IgniteConfiguration>)cfg ->
+                    cfg.setSystemThreadPoolSize(1).setStripedPoolSize(1));
+
+            final IgniteEx grid2 = startGrid(2);
+
+            grid0.cluster().state(ACTIVE);
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Integer g2Key = primaryKeys(cache, 1, 0).get(0);
+
+            List<IgniteInternalTx> txs0 = null;
+            List<IgniteInternalTx> txs1 = null;
+            List<IgniteInternalTx> txs2;
+
+            final CountDownLatch grid1BlockLatch = new CountDownLatch(1);
+
+            final CountDownLatch grid1NodeLeftEventLatch = new CountDownLatch(1);
+
+            int[] stripeHolder = new int[1];
+
+            try (final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                cache.put(g2Key, Boolean.TRUE);
+
+                TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;
+
+                p.tx().prepare(true);
+
+                txs0 = txs(grid0);
+                txs1 = txs(grid1);
+                txs2 = txs(grid2);
+
+                assertTrue(txs0.size() == 1);
+                assertTrue(txs1.size() == 1);
+                assertTrue(txs2.size() == 1);
+
+                grid1.events().localListen(new PE() {
+                    @Override public boolean apply(Event evt) {
+                        grid1NodeLeftEventLatch.countDown();
+
+                        return true;
+                    }
+                }, EventType.EVT_NODE_LEFT);
+
+                // Block recovery procedure processing on grid1
+                grid1.context().pools().getSystemExecutorService().execute(() -> U.awaitQuiet(grid1BlockLatch));
+
+                int stripe = U.safeAbs(p.tx().xidVersion().hashCode());
+
+                stripeHolder[0] = stripe;
+
+                // Block stripe tx recovery request processing on grid1
+                grid1.context().pools().getStripedExecutorService().execute(stripe, () -> U.awaitQuiet(grid1BlockLatch));

Review Comment:
   useless newline



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +264,129 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there will be 2 backup nodes
+     * to execute the tx recovery in parallel if primary one would fail. These backup nodes will send the
+     * tx recovery requests to each other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;

Review Comment:
   Never used?



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +264,129 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there will be 2 backup nodes
+     * to execute the tx recovery in parallel if primary one would fail. These backup nodes will send the
+     * tx recovery requests to each other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;

Review Comment:
   Never used?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r938099321


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +259,110 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <ul>
+     *  <li>Start 3 nodes (g0, g1, g2) and cache with 2 backups.</li>
+     *  <li>Prepare a transaction with g2 as a primary node.</li>
+     *  <li>Kill g2.</li>
+     *  <li>Enforce the concurrent processing of transaction in tx recovery on g1.</li>
+     * </ul>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished both on g0 and g1
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 75; iter++) {
+            stopAllGrids();
+
+            log.info("iteration=" + iter);
+            final IgniteEx grid0 = startGrid("g0");
+            final IgniteEx grid1 = startGrid("g1",
+                    cfg -> cfg.setSystemThreadPoolSize(1).setStripedPoolSize(1));
+            final IgniteEx grid2 = startGrid("g2");
+
+            grid0.cluster().state(ACTIVE);
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Integer g2Key = primaryKeys(grid2.cache(DEFAULT_CACHE_NAME), 1, 0).get(0);
+
+            List<IgniteInternalTx> txs0 = null;
+            List<IgniteInternalTx> txs1 = null;
+
+            CountDownLatch grid1BlockLatch = new CountDownLatch(1);
+
+            try (final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                cache.put(g2Key, Boolean.TRUE);
+                TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;
+                p.tx().prepare(true);
+
+                txs0 = txs(grid0);
+                txs1 = txs(grid1);
+                List<IgniteInternalTx> txs2 = txs(grid2);
+
+                assertTrue(txs0.size() == 1);
+                assertTrue(txs1.size() == 1);
+                assertTrue(txs2.size() == 1);
+
+                // Prevent tx recovery request to be sent from grid0 to grid1.
+                spi(grid0).blockMessages(GridCacheTxRecoveryRequest.class, grid1.name());
+
+                // Block recovery procedure processing on grid1
+                grid1.context().pools().getSystemExecutorService().execute(() -> U.awaitQuiet(grid1BlockLatch));
+                // Block stripe tx recovery request processing on grid1
+                int stripe = U.safeAbs(p.tx().xidVersion().hashCode());
+                grid1.context().pools().getStripedExecutorService().execute(stripe, () -> U.awaitQuiet(grid1BlockLatch));
+
+                // Prevent finish request processing on grid0 and grid1.
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid0.name());
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid1.name());
+
+                runAsync(() -> {
+                    grid2.close();
+                    return null;
+                });
+            }
+            catch (Exception ignored) {
+                // Expected.
+            }
+
+            // Wait grid0 is ready to send the tx recovery request to grid1
+            spi(grid0).waitForBlocked();
+            // Let grid0 send the tx recovery request to grid1
+            log.info("unblock grid0");
+            spi(grid0).stopBlock();
+            // Give grid1 some time to receive the tx recovery request (processing is still blocked in grid1).
+            log.info("sleep in grid0");
+            doSleep(100);
+
+            // Unblock processing in grid1. Simultaneously in striped and system pools to start
+            // recovery procedure and the tx recovery request processing at the "same" moment
+            // (for the same transaction). This should increase chances for race condition occur in
+            // the IgniteTxAdapter::markFinalizing.
+            log.info("unblock grid1");
+            grid1BlockLatch.countDown();
+
+            // Wait transaction finish in grid0.
+            txs0.get(0).finishFuture().get(5_000);
+
+            try {
+                // Check if transaction is finished in grid1. It wouldn't if race condition occur.
+                txs1.get(0).finishFuture().get(20_000);

Review Comment:
   Fixed



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +259,110 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <ul>
+     *  <li>Start 3 nodes (g0, g1, g2) and cache with 2 backups.</li>
+     *  <li>Prepare a transaction with g2 as a primary node.</li>
+     *  <li>Kill g2.</li>
+     *  <li>Enforce the concurrent processing of transaction in tx recovery on g1.</li>
+     * </ul>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished both on g0 and g1
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 75; iter++) {
+            stopAllGrids();
+
+            log.info("iteration=" + iter);
+            final IgniteEx grid0 = startGrid("g0");
+            final IgniteEx grid1 = startGrid("g1",
+                    cfg -> cfg.setSystemThreadPoolSize(1).setStripedPoolSize(1));
+            final IgniteEx grid2 = startGrid("g2");
+
+            grid0.cluster().state(ACTIVE);
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Integer g2Key = primaryKeys(grid2.cache(DEFAULT_CACHE_NAME), 1, 0).get(0);
+
+            List<IgniteInternalTx> txs0 = null;
+            List<IgniteInternalTx> txs1 = null;
+
+            CountDownLatch grid1BlockLatch = new CountDownLatch(1);
+
+            try (final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                cache.put(g2Key, Boolean.TRUE);
+                TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;
+                p.tx().prepare(true);
+
+                txs0 = txs(grid0);
+                txs1 = txs(grid1);
+                List<IgniteInternalTx> txs2 = txs(grid2);
+
+                assertTrue(txs0.size() == 1);
+                assertTrue(txs1.size() == 1);
+                assertTrue(txs2.size() == 1);
+
+                // Prevent tx recovery request to be sent from grid0 to grid1.
+                spi(grid0).blockMessages(GridCacheTxRecoveryRequest.class, grid1.name());
+
+                // Block recovery procedure processing on grid1
+                grid1.context().pools().getSystemExecutorService().execute(() -> U.awaitQuiet(grid1BlockLatch));
+                // Block stripe tx recovery request processing on grid1
+                int stripe = U.safeAbs(p.tx().xidVersion().hashCode());
+                grid1.context().pools().getStripedExecutorService().execute(stripe, () -> U.awaitQuiet(grid1BlockLatch));
+
+                // Prevent finish request processing on grid0 and grid1.
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid0.name());
+                spi(grid2).blockMessages(GridDhtTxFinishRequest.class, grid1.name());
+
+                runAsync(() -> {
+                    grid2.close();
+                    return null;
+                });
+            }
+            catch (Exception ignored) {
+                // Expected.
+            }
+
+            // Wait grid0 is ready to send the tx recovery request to grid1
+            spi(grid0).waitForBlocked();
+            // Let grid0 send the tx recovery request to grid1
+            log.info("unblock grid0");
+            spi(grid0).stopBlock();
+            // Give grid1 some time to receive the tx recovery request (processing is still blocked in grid1).
+            log.info("sleep in grid0");

Review Comment:
   Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r938100428


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +259,110 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <ul>
+     *  <li>Start 3 nodes (g0, g1, g2) and cache with 2 backups.</li>
+     *  <li>Prepare a transaction with g2 as a primary node.</li>
+     *  <li>Kill g2.</li>
+     *  <li>Enforce the concurrent processing of transaction in tx recovery on g1.</li>

Review Comment:
   Tried to explain the idea.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r938101544


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +259,110 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <ul>
+     *  <li>Start 3 nodes (g0, g1, g2) and cache with 2 backups.</li>
+     *  <li>Prepare a transaction with g2 as a primary node.</li>
+     *  <li>Kill g2.</li>
+     *  <li>Enforce the concurrent processing of transaction in tx recovery on g1.</li>
+     * </ul>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished both on g0 and g1
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 75; iter++) {

Review Comment:
   Return 100 for now since test looks faster a bit after rework.
   Not sure how to make it more faster. Have any idea?
   
   The test reproduces the very rare situation. Sometimes it tooks upto 93 iterations to fail on my laptop.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r941987134


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentOnPrimaryFailTest.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.Collection;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentOnPrimaryFailTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME).setCacheMode(PARTITIONED)
+            .setBackups(2).setAtomicityMode(TRANSACTIONAL).setAffinity(new RendezvousAffinityFunction(false, 1)));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        final IgniteEx grid0 = startGrid(0);
+
+        final IgniteEx grid1 = startGrid(1);
+
+        final CyclicBarrier grid1BlockerBarrier = new CyclicBarrier(3);
+
+        final Runnable grid1BlockerTask = () -> {
+            try {
+                grid1BlockerBarrier.await();
+            }
+            catch (InterruptedException | BrokenBarrierException e) {
+                // Just supress.
+            }
+        };
+
+        for (int iter = 0; iter < 100; iter++) {
+            final IgniteEx grid2 = startGrid(2);
+
+            awaitPartitionMapExchange();
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
+
+            // Key for which the grid2 node is primary.
+            final Integer grid2PrimaryKey = primaryKeys(cache, 1, 0).get(0);
+
+            cache.put(grid2PrimaryKey, Boolean.TRUE);
+
+            final TransactionProxyImpl<?, ?> p = (TransactionProxyImpl<?, ?>)tx;

Review Comment:
   ok



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r947452493


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentTest.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.Collection;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ThreadPoolExecutor;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME)
+            .setCacheMode(PARTITIONED)
+            .setBackups(2)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setAffinity(new RendezvousAffinityFunction(false, 1)));
+
+        return cfg;
+    }
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to near and primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Blocking is used to force concurrent processing on one of the backup nodes. Another
+     * backup works unconstrained to provide the right environment for the blocked one.
+     * In particular, it should send a tx recovery request to the blocked backup.
+     * <p>
+     * Use several attempts to reproduce the race condition in the blocked backup node.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnNearAndPrimaryFail() throws Exception {
+        startGrids(2);
+
+        final int key = 0;
+
+        for (int iter = 0; iter < 100; iter++) {
+            startGrid(iter + 2);
+
+            awaitPartitionMapExchange();
+
+            final Ignite primary = primaryNode(key, DEFAULT_CACHE_NAME);
+            final List<Ignite> backups = backupNodes(key, DEFAULT_CACHE_NAME);
+
+            final IgniteCache<Object, Object> cache = primary.cache(DEFAULT_CACHE_NAME);
+
+            final TransactionProxyImpl<?, ?> tx = (TransactionProxyImpl<?, ?>)primary.transactions().txStart();
+
+            cache.put(key, key);
+
+            tx.tx().prepare(true);
+
+            for (Ignite grid : G.allGrids())
+                assertTrue(((IgniteEx)grid).context().cache().context().tm().activeTransactions().size() == 1);
+
+            final Collection<IgniteInternalTx> backupTransactions = new LinkedList<>();
+
+            for (Ignite backup : backups)
+                backupTransactions.addAll(((IgniteEx)backup).context().cache().context().tm().activeTransactions());
+
+            assertTrue(backupTransactions.size() == 2);
+
+            final CountDownLatch ensureBothPoolsAreBlockedLatch = new CountDownLatch(2);
+            final CountDownLatch unblockBothPoolsLatch = new CountDownLatch(1);
+
+            final Runnable poolBlockerTask = () -> {
+                try {
+                    ensureBothPoolsAreBlockedLatch.countDown();
+                    unblockBothPoolsLatch.await();
+                }
+                catch (InterruptedException ignored) {
+                    // No-op.
+                }
+            };
+
+            final IgniteEx blockedBackup = (IgniteEx)backups.get(0);
+
+            blockedBackup.context().pools().getSystemExecutorService().execute(poolBlockerTask);
+
+            blockedBackup.context().pools().getStripedExecutorService().execute(0, poolBlockerTask);
+
+            ensureBothPoolsAreBlockedLatch.await();
+
+            runAsync(primary::close);
+
+            waitForTxRecoveryRequestEnqueuedOn(blockedBackup);
+
+            waitForTxRecoveryTaskEnqueuedOn(blockedBackup);
+
+            // Unblock processing in blocked backup node. Simultaneously in striped and system pools to start recovery
+            // procedure and the tx recovery request processing at the "same" moment (for the same transaction). This
+            // should increase chances for race condition occur in the IgniteTxAdapter#markFinalizing.
+            unblockBothPoolsLatch.countDown();
+
+            waitForTopology(2);
+
+            awaitPartitionMapExchange();

Review Comment:
   done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r946807655


##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java:
##########
@@ -603,7 +603,13 @@ protected void uncommit() {
             case RECOVERY_FINISH:
                 FinalizationStatus old = finalizing;
 
-                res = old != FinalizationStatus.USER_FINISH && FINALIZING_UPD.compareAndSet(this, old, status);
+                if (old == FinalizationStatus.USER_FINISH)
+                    res = false;
+                else if (old == FinalizationStatus.RECOVERY_FINISH)
+                    res = true;
+                else
+                    res = FINALIZING_UPD.compareAndSet(this, old, status) ||
+                        finalizing == FinalizationStatus.RECOVERY_FINISH;

Review Comment:
   Applied as suggested here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r938631754


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +264,129 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there will be 2 backup nodes
+     * to execute the tx recovery in parallel if primary one would fail. These backup nodes will send the
+     * tx recovery requests to each other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;

Review Comment:
   Used in TxRecoveryWithConcurrentRollbackTest::getConfiguration to create node config



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +264,129 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there will be 2 backup nodes
+     * to execute the tx recovery in parallel if primary one would fail. These backup nodes will send the
+     * tx recovery requests to each other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;

Review Comment:
   Used in TxRecoveryWithConcurrentRollbackTest::getConfiguration to create node config



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r941987042


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentOnPrimaryFailTest.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.Collection;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentOnPrimaryFailTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME).setCacheMode(PARTITIONED)
+            .setBackups(2).setAtomicityMode(TRANSACTIONAL).setAffinity(new RendezvousAffinityFunction(false, 1)));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        final IgniteEx grid0 = startGrid(0);
+
+        final IgniteEx grid1 = startGrid(1);
+
+        final CyclicBarrier grid1BlockerBarrier = new CyclicBarrier(3);
+
+        final Runnable grid1BlockerTask = () -> {
+            try {
+                grid1BlockerBarrier.await();
+            }
+            catch (InterruptedException | BrokenBarrierException e) {
+                // Just supress.
+            }
+        };
+
+        for (int iter = 0; iter < 100; iter++) {
+            final IgniteEx grid2 = startGrid(2);
+
+            awaitPartitionMapExchange();
+
+            final IgniteCache<Object, Object> cache = grid2.cache(DEFAULT_CACHE_NAME);
+
+            final Transaction tx = grid2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
+
+            // Key for which the grid2 node is primary.
+            final Integer grid2PrimaryKey = primaryKeys(cache, 1, 0).get(0);
+
+            cache.put(grid2PrimaryKey, Boolean.TRUE);

Review Comment:
   ok



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r941442336


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +263,120 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 100; iter++) {

Review Comment:
   Looks like number of repetitions can not be reduced. 
   
   So I have optimized the single repetion. It required to put test into the separate class to have more optimized cache & node configuration.   
   
   The main idea of optimization is to restart the only node on each cycle iteration.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r938101544


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryWithConcurrentRollbackTest.java:
##########
@@ -258,6 +259,110 @@ else if (g1Keys.contains(key))
         assertEquals(s1, s2);
     }
 
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction both in the
+     * tx recovery procedure started due to primary node left and in the tx recovery request handler
+     * invoked by message from another backup node.
+     * <ul>
+     *  <li>Start 3 nodes (g0, g1, g2) and cache with 2 backups.</li>
+     *  <li>Prepare a transaction with g2 as a primary node.</li>
+     *  <li>Kill g2.</li>
+     *  <li>Enforce the concurrent processing of transaction in tx recovery on g1.</li>
+     * </ul>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished both on g0 and g1
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnPrimaryFail() throws Exception {
+        backups = 2;
+        persistence = false;
+
+        for (int iter = 0; iter < 75; iter++) {

Review Comment:
   Return 100 for now since test looks faster a bit after rework.
   Not sure how to make it more faster. Have any idea?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r946479799


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentTest.java:
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.Collection;
+import java.util.LinkedList;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ThreadPoolExecutor;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME).setCacheMode(PARTITIONED)
+            .setBackups(2).setAtomicityMode(TRANSACTIONAL).setAffinity(new RendezvousAffinityFunction(false, 10)));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to near and primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Use several attempts to reproduce the race condition.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnNearAndPrimaryFail() throws Exception {
+        final IgniteEx backup = startGrids(2);
+
+        final CyclicBarrier backupBlockerBarrier = new CyclicBarrier(3);
+
+        final Runnable backupBlockerTask = () -> {
+            try {
+                backupBlockerBarrier.await();
+                backupBlockerBarrier.await();

Review Comment:
   Renamed to use more talking names



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] skorotkov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
skorotkov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r946808268


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentTest.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.Collection;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ThreadPoolExecutor;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME)
+            .setCacheMode(PARTITIONED)
+            .setBackups(2)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setAffinity(new RendezvousAffinityFunction(false, 1)));
+
+        return cfg;
+    }
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to near and primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Blocking is used to force concurrent processing on one of the backup nodes. Another
+     * backup works unconstrained to provide the right environment for the blocked one.
+     * In particular, it should send a tx recovery request to the blocked backup.
+     * <p>
+     * Use several attempts to reproduce the race condition in the blocked backup node.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnNearAndPrimaryFail() throws Exception {
+        startGrids(2);
+
+        final CyclicBarrier ensurePoolsAreBlockedBarrier = new CyclicBarrier(3);
+
+        final CyclicBarrier unblockPoolsBarrier = new CyclicBarrier(3);
+
+        final Runnable backupBlockerTask = () -> {
+            try {
+                ensurePoolsAreBlockedBarrier.await();
+

Review Comment:
   fixed



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentTest.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.Collection;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ThreadPoolExecutor;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME)
+            .setCacheMode(PARTITIONED)
+            .setBackups(2)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setAffinity(new RendezvousAffinityFunction(false, 1)));
+
+        return cfg;
+    }
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to near and primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Blocking is used to force concurrent processing on one of the backup nodes. Another
+     * backup works unconstrained to provide the right environment for the blocked one.
+     * In particular, it should send a tx recovery request to the blocked backup.
+     * <p>
+     * Use several attempts to reproduce the race condition in the blocked backup node.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnNearAndPrimaryFail() throws Exception {
+        startGrids(2);
+
+        final CyclicBarrier ensurePoolsAreBlockedBarrier = new CyclicBarrier(3);
+
+        final CyclicBarrier unblockPoolsBarrier = new CyclicBarrier(3);
+
+        final Runnable backupBlockerTask = () -> {
+            try {
+                ensurePoolsAreBlockedBarrier.await();
+
+                unblockPoolsBarrier.await();
+            }
+            catch (InterruptedException | BrokenBarrierException ignored) {
+                // Just supress.
+            }
+        };
+
+        for (int iter = 0; iter < 100; iter++) {
+            startGrid(iter + 2);
+
+            awaitPartitionMapExchange();
+
+            final Ignite primary = primaryNode(0, DEFAULT_CACHE_NAME);
+

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] anton-vinogradov commented on a diff in pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
anton-vinogradov commented on code in PR #10178:
URL: https://github.com/apache/ignite/pull/10178#discussion_r946586354


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentTest.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.Collection;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ThreadPoolExecutor;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME)
+            .setCacheMode(PARTITIONED)
+            .setBackups(2)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setAffinity(new RendezvousAffinityFunction(false, 1)));
+
+        return cfg;
+    }
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to near and primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Blocking is used to force concurrent processing on one of the backup nodes. Another
+     * backup works unconstrained to provide the right environment for the blocked one.
+     * In particular, it should send a tx recovery request to the blocked backup.
+     * <p>
+     * Use several attempts to reproduce the race condition in the blocked backup node.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnNearAndPrimaryFail() throws Exception {
+        startGrids(2);
+
+        final CyclicBarrier ensurePoolsAreBlockedBarrier = new CyclicBarrier(3);
+
+        final CyclicBarrier unblockPoolsBarrier = new CyclicBarrier(3);
+
+        final Runnable backupBlockerTask = () -> {
+            try {
+                ensurePoolsAreBlockedBarrier.await();
+
+                unblockPoolsBarrier.await();
+            }
+            catch (InterruptedException | BrokenBarrierException ignored) {
+                // Just supress.

Review Comment:
   We have special comment for this %) 
   // No-op.



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentTest.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.Collection;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ThreadPoolExecutor;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME)
+            .setCacheMode(PARTITIONED)
+            .setBackups(2)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setAffinity(new RendezvousAffinityFunction(false, 1)));
+
+        return cfg;
+    }
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to near and primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Blocking is used to force concurrent processing on one of the backup nodes. Another
+     * backup works unconstrained to provide the right environment for the blocked one.
+     * In particular, it should send a tx recovery request to the blocked backup.
+     * <p>
+     * Use several attempts to reproduce the race condition in the blocked backup node.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnNearAndPrimaryFail() throws Exception {
+        startGrids(2);
+
+        final CyclicBarrier ensurePoolsAreBlockedBarrier = new CyclicBarrier(3);
+
+        final CyclicBarrier unblockPoolsBarrier = new CyclicBarrier(3);
+
+        final Runnable backupBlockerTask = () -> {
+            try {
+                ensurePoolsAreBlockedBarrier.await();
+
+                unblockPoolsBarrier.await();
+            }
+            catch (InterruptedException | BrokenBarrierException ignored) {
+                // Just supress.
+            }
+        };
+
+        for (int iter = 0; iter < 100; iter++) {
+            startGrid(iter + 2);
+
+            awaitPartitionMapExchange();
+
+            final Ignite primary = primaryNode(0, DEFAULT_CACHE_NAME);
+

Review Comment:
   no newline needed



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentTest.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.Collection;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ThreadPoolExecutor;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME)
+            .setCacheMode(PARTITIONED)
+            .setBackups(2)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setAffinity(new RendezvousAffinityFunction(false, 1)));
+
+        return cfg;
+    }
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to near and primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Blocking is used to force concurrent processing on one of the backup nodes. Another
+     * backup works unconstrained to provide the right environment for the blocked one.
+     * In particular, it should send a tx recovery request to the blocked backup.
+     * <p>
+     * Use several attempts to reproduce the race condition in the blocked backup node.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnNearAndPrimaryFail() throws Exception {
+        startGrids(2);
+
+        final CyclicBarrier ensurePoolsAreBlockedBarrier = new CyclicBarrier(3);
+
+        final CyclicBarrier unblockPoolsBarrier = new CyclicBarrier(3);
+
+        final Runnable backupBlockerTask = () -> {
+            try {
+                ensurePoolsAreBlockedBarrier.await();
+

Review Comment:
   no need for newline here



##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRecoveryConcurrentTest.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.Collection;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ThreadPoolExecutor;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ * Tests concurrent execution of the tx recovery.
+ */
+public class TxRecoveryConcurrentTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
+        cfg.setSystemThreadPoolSize(1);
+        cfg.setStripedPoolSize(1);
+
+        cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME)
+            .setCacheMode(PARTITIONED)
+            .setBackups(2)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setAffinity(new RendezvousAffinityFunction(false, 1)));
+
+        return cfg;
+    }
+
+    /**
+     * The test enforces the concurrent processing of the same prepared transaction
+     * both in the tx recovery procedure started due to near and primary node left and in the
+     * tx recovery request handler invoked by message from another backup node.
+     * <p>
+     * The idea is to have a 3-nodes cluster and a cache with 2 backups. So there
+     * will be 2 backup nodes to execute the tx recovery in parallel if primary one
+     * would fail. These backup nodes will send the tx recovery requests to each
+     * other, so the tx recovery request handler will be invoked as well.
+     * <p>
+     * Blocking is used to force concurrent processing on one of the backup nodes. Another
+     * backup works unconstrained to provide the right environment for the blocked one.
+     * In particular, it should send a tx recovery request to the blocked backup.
+     * <p>
+     * Use several attempts to reproduce the race condition in the blocked backup node.
+     * <p>
+     * Expected result: transaction is finished on both backup nodes and the partition
+     * map exchange is completed as well.
+     */
+    @Test
+    public void testRecoveryNotDeadLockOnNearAndPrimaryFail() throws Exception {
+        startGrids(2);
+
+        final CyclicBarrier ensurePoolsAreBlockedBarrier = new CyclicBarrier(3);
+
+        final CyclicBarrier unblockPoolsBarrier = new CyclicBarrier(3);

Review Comment:
   May be defined without newline between to show they are almost about the same



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] anton-vinogradov merged pull request #10178: IGNITE-17457 Fix cluster lock after tx recovery

Posted by GitBox <gi...@apache.org>.
anton-vinogradov merged PR #10178:
URL: https://github.com/apache/ignite/pull/10178


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org