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

vpyatkov pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git


The following commit(s) were added to refs/heads/main by this push:
     new bba00639c6 IGNITE-21173 Repeated call to commit/abort should not emit 
exceptions (#3064)
bba00639c6 is described below

commit bba00639c68970007b4ec837e03cf0aa10cb0b87
Author: Cyrill <cyrill.si...@gmail.com>
AuthorDate: Fri Jan 19 19:09:52 2024 +0300

    IGNITE-21173 Repeated call to commit/abort should not emit exceptions 
(#3064)
---
 .../internal/table/ItTransactionRecoveryTest.java  |   8 +-
 .../replicator/PartitionReplicaListener.java       |  10 +-
 .../replication/PartitionReplicaListenerTest.java  |   8 +-
 .../ignite/internal/table/TxAbstractTest.java      |  23 ++--
 ...=> MismatchingTransactionOutcomeException.java} |  15 ++-
 .../internal/tx/impl/ReadWriteTransactionImpl.java |   7 +-
 .../ignite/internal/tx/impl/TxManagerImpl.java     |  12 +-
 .../apache/ignite/internal/tx/TxManagerTest.java   | 121 ++++++++++++++++++++-
 8 files changed, 164 insertions(+), 40 deletions(-)

diff --git 
a/modules/table/src/integrationTest/java/org/apache/ignite/internal/table/ItTransactionRecoveryTest.java
 
b/modules/table/src/integrationTest/java/org/apache/ignite/internal/table/ItTransactionRecoveryTest.java
index d72df1b78e..2f5af30348 100644
--- 
a/modules/table/src/integrationTest/java/org/apache/ignite/internal/table/ItTransactionRecoveryTest.java
+++ 
b/modules/table/src/integrationTest/java/org/apache/ignite/internal/table/ItTransactionRecoveryTest.java
@@ -53,7 +53,7 @@ import 
org.apache.ignite.internal.table.distributed.replication.request.ReadWrit
 import org.apache.ignite.internal.testframework.IgniteTestUtils;
 import org.apache.ignite.internal.tx.HybridTimestampTracker;
 import org.apache.ignite.internal.tx.InternalTransaction;
-import org.apache.ignite.internal.tx.TransactionAlreadyFinishedException;
+import org.apache.ignite.internal.tx.MismatchingTransactionOutcomeException;
 import org.apache.ignite.internal.tx.TxMeta;
 import org.apache.ignite.internal.tx.TxState;
 import org.apache.ignite.internal.tx.TxStateMeta;
@@ -502,7 +502,7 @@ public class ItTransactionRecoveryTest extends 
ClusterPerTestIntegrationTest {
 
         ErrorTimestampAwareReplicaResponse errorResponse = 
(ErrorTimestampAwareReplicaResponse) response;
 
-        assertInstanceOf(TransactionAlreadyFinishedException.class, 
ExceptionUtils.unwrapCause(errorResponse.throwable()));
+        assertInstanceOf(MismatchingTransactionOutcomeException.class, 
ExceptionUtils.unwrapCause(errorResponse.throwable()));
 
         assertEquals(TxState.ABORTED, txStoredState(commitPartNode, 
orphanTx.id()));
     }
@@ -618,7 +618,7 @@ public class ItTransactionRecoveryTest extends 
ClusterPerTestIntegrationTest {
                 rwTx1Id
         );
 
-        assertThat(finish2, 
willThrow(TransactionAlreadyFinishedException.class));
+        assertThat(finish2, 
willThrow(MismatchingTransactionOutcomeException.class));
     }
 
     @Test
@@ -762,7 +762,7 @@ public class ItTransactionRecoveryTest extends 
ClusterPerTestIntegrationTest {
 
         cancelLease(commitPartNode, tblReplicationGrp);
 
-        assertThat(commitFut, 
willThrow(TransactionAlreadyFinishedException.class, 30, SECONDS));
+        assertThat(commitFut, 
willThrow(MismatchingTransactionOutcomeException.class, 30, SECONDS));
 
         RecordView<Tuple> view = 
txCrdNode.tables().table(TABLE_NAME).recordView();
 
diff --git 
a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java
 
b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java
index 2b2135675d..b2454b2f80 100644
--- 
a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java
+++ 
b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java
@@ -152,7 +152,7 @@ import org.apache.ignite.internal.tx.Lock;
 import org.apache.ignite.internal.tx.LockKey;
 import org.apache.ignite.internal.tx.LockManager;
 import org.apache.ignite.internal.tx.LockMode;
-import org.apache.ignite.internal.tx.TransactionAlreadyFinishedException;
+import org.apache.ignite.internal.tx.MismatchingTransactionOutcomeException;
 import org.apache.ignite.internal.tx.TransactionIds;
 import org.apache.ignite.internal.tx.TransactionMeta;
 import org.apache.ignite.internal.tx.TransactionResult;
@@ -1529,13 +1529,13 @@ public class PartitionReplicaListener implements 
ReplicaListener {
         if (!validationResult.isSuccessful()) {
             if (validationResult.isTableDropped()) {
                 // TODO: IGNITE-20966 - improve error message.
-                throw new TransactionAlreadyFinishedException(
+                throw new MismatchingTransactionOutcomeException(
                         format("Commit failed because a table was already 
dropped [tableId={}]", validationResult.failedTableId()),
                         txResult
                 );
             } else {
                 // TODO: IGNITE-20966 - improve error message.
-                throw new TransactionAlreadyFinishedException(
+                throw new MismatchingTransactionOutcomeException(
                         "Commit failed because schema "
                                 + validationResult.fromSchemaVersion() + " is 
not forward-compatible with "
                                 + validationResult.toSchemaVersion() + " for 
table " + validationResult.failedTableId(),
@@ -1589,7 +1589,7 @@ public class PartitionReplicaListener implements 
ReplicaListener {
                         txMeta.txState()
                 );
 
-                throw new TransactionAlreadyFinishedException(
+                throw new MismatchingTransactionOutcomeException(
                         "Failed to change the outcome of a finished 
transaction [txId=" + txId + ", txState=" + txMeta.txState() + "].",
                         new TransactionResult(txMeta.txState(), 
txMeta.commitTimestamp())
                 );
@@ -1648,7 +1648,7 @@ public class PartitionReplicaListener implements 
ReplicaListener {
 
                             markFinished(txId, result.transactionState(), 
result.commitTimestamp());
 
-                            throw new 
TransactionAlreadyFinishedException(utse.getMessage(), 
utse.transactionResult());
+                            throw new 
MismatchingTransactionOutcomeException(utse.getMessage(), 
utse.transactionResult());
                         }
                         // Otherwise we convert from the internal exception to 
the client one.
                         throw new TransactionException(commit ? TX_COMMIT_ERR 
: TX_ROLLBACK_ERR, ex);
diff --git 
a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerTest.java
 
b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerTest.java
index 5806bc9d80..932d9dc86f 100644
--- 
a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerTest.java
+++ 
b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerTest.java
@@ -158,7 +158,7 @@ import 
org.apache.ignite.internal.testframework.IgniteAbstractTest;
 import org.apache.ignite.internal.tostring.IgniteToStringInclude;
 import org.apache.ignite.internal.tostring.S;
 import org.apache.ignite.internal.tx.LockManager;
-import org.apache.ignite.internal.tx.TransactionAlreadyFinishedException;
+import org.apache.ignite.internal.tx.MismatchingTransactionOutcomeException;
 import org.apache.ignite.internal.tx.TransactionIds;
 import org.apache.ignite.internal.tx.TransactionMeta;
 import org.apache.ignite.internal.tx.TransactionResult;
@@ -1624,8 +1624,8 @@ public class PartitionReplicaListenerTest extends 
IgniteAbstractTest {
 
         CompletableFuture<?> future = beginAndCommitTx();
 
-        TransactionAlreadyFinishedException ex = assertWillThrowFast(future,
-                TransactionAlreadyFinishedException.class);
+        MismatchingTransactionOutcomeException ex = assertWillThrowFast(future,
+                MismatchingTransactionOutcomeException.class);
 
         assertThat(ex.getMessage(), containsString("Commit failed because 
schema 1 is not forward-compatible with 2"));
 
@@ -2323,7 +2323,7 @@ public class PartitionReplicaListenerTest extends 
IgniteAbstractTest {
                 localNode.id()
         );
 
-        TransactionAlreadyFinishedException ex = assertWillThrowFast(future, 
TransactionAlreadyFinishedException.class);
+        MismatchingTransactionOutcomeException ex = 
assertWillThrowFast(future, MismatchingTransactionOutcomeException.class);
 
         assertThat(ex.getMessage(), is("Commit failed because a table was 
already dropped [tableId=" + tableToBeDroppedId + "]"));
 
diff --git 
a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/TxAbstractTest.java
 
b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/TxAbstractTest.java
index b3505d2d21..6566df1ccd 100644
--- 
a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/TxAbstractTest.java
+++ 
b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/TxAbstractTest.java
@@ -162,16 +162,14 @@ public abstract class TxAbstractTest extends 
IgniteAbstractTest {
         InternalTransaction tx = (InternalTransaction) 
igniteTransactions.begin();
 
         CompletableFuture<Void> fut = accounts.recordView().upsertAsync(tx, 
makeValue(1, 100.));
-        assertThrows(Exception.class, () -> fut.join());
 
-        CompletableFuture<Void> fut0 = tx.commitAsync();
-        assertThrows(Exception.class, () -> fut0.join());
+        assertThrows(Exception.class, fut::join);
 
-        CompletableFuture<Void> fut1 = tx.rollbackAsync();
-        assertThrows(Exception.class, () -> fut1.join());
+        tx.commitAsync().join();
 
-        CompletableFuture<Void> fut2 = tx.commitAsync();
-        assertThrows(Exception.class, () -> fut2.join());
+        tx.rollbackAsync().join();
+
+        tx.commitAsync().join();
     }
 
     @Test
@@ -183,16 +181,13 @@ public abstract class TxAbstractTest extends 
IgniteAbstractTest {
         injectFailureOnNextOperation(accounts);
 
         CompletableFuture<Void> fut = tx.rollbackAsync();
-        assertThrows(Exception.class, () -> fut.join());
+        assertThrows(Exception.class, fut::join);
 
-        CompletableFuture<Void> fut0 = tx.commitAsync();
-        assertThrows(Exception.class, () -> fut0.join());
+        tx.commitAsync().join();
 
-        CompletableFuture<Void> fut1 = tx.rollbackAsync();
-        assertThrows(Exception.class, () -> fut1.join());
+        tx.rollbackAsync().join();
 
-        CompletableFuture<Void> fut2 = tx.commitAsync();
-        assertThrows(Exception.class, () -> fut2.join());
+        tx.commitAsync().join();
     }
 
     @Test
diff --git 
a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/TransactionAlreadyFinishedException.java
 
b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/MismatchingTransactionOutcomeException.java
similarity index 67%
rename from 
modules/transactions/src/main/java/org/apache/ignite/internal/tx/TransactionAlreadyFinishedException.java
rename to 
modules/transactions/src/main/java/org/apache/ignite/internal/tx/MismatchingTransactionOutcomeException.java
index 6d14d467fa..5ae4ea70d3 100644
--- 
a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/TransactionAlreadyFinishedException.java
+++ 
b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/MismatchingTransactionOutcomeException.java
@@ -22,21 +22,28 @@ import static 
org.apache.ignite.lang.ErrorGroups.Transactions.TX_UNEXPECTED_STAT
 import org.apache.ignite.tx.TransactionException;
 
 /**
- * The exception is thrown when a transaction has already been finished.
+ * The exception is thrown when the transaction result differs from the 
intended one.
+ *
+ * <p>For example, {@code tx.commit()} is called for a transaction, but the 
verification logic decided to abort it instead. The transaction
+ * will be finished with {@link TxState#ABORTED} and the call to {@code 
tx.commit()} will throw this exception.
  */
-public class TransactionAlreadyFinishedException extends TransactionException {
+public class MismatchingTransactionOutcomeException extends 
TransactionException {
 
     private static final long serialVersionUID = -7953057695915339651L;
 
     /** Stored transaction result. */
     private final TransactionResult transactionResult;
 
-    public TransactionAlreadyFinishedException(int errorCode, String message, 
TransactionResult transactionResult, Throwable cause) {
+    /**
+     * Constructor.
+     */
+    public MismatchingTransactionOutcomeException(int errorCode, String 
message, TransactionResult transactionResult, Throwable cause) {
         super(errorCode, message, cause);
+
         this.transactionResult = transactionResult;
     }
 
-    public TransactionAlreadyFinishedException(String message, 
TransactionResult transactionResult) {
+    public MismatchingTransactionOutcomeException(String message, 
TransactionResult transactionResult) {
         this(TX_UNEXPECTED_STATE_ERR, message, transactionResult, null);
     }
 
diff --git 
a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/ReadWriteTransactionImpl.java
 
b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/ReadWriteTransactionImpl.java
index 3b021fd192..b585cf2f5e 100644
--- 
a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/ReadWriteTransactionImpl.java
+++ 
b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/ReadWriteTransactionImpl.java
@@ -145,7 +145,12 @@ public class ReadWriteTransactionImpl extends 
IgniteAbstractTransactionImpl {
             if (!hasTxFinalizationBegun()) {
                 assert finishFuture == null : "Transaction is already finished 
[id=" + id() + ", state=" + state() + "].";
 
-                finishFuture = finishInternal(commit);
+                CompletableFuture<Void> finishFutureInternal = 
finishInternal(commit);
+
+                finishFuture = finishFutureInternal.handle((unused, throwable) 
-> null);
+
+                // Return the real future first time.
+                return finishFutureInternal;
             }
 
             return finishFuture;
diff --git 
a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/TxManagerImpl.java
 
b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/TxManagerImpl.java
index e40523e56d..8a3db333d9 100644
--- 
a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/TxManagerImpl.java
+++ 
b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/TxManagerImpl.java
@@ -80,7 +80,7 @@ import org.apache.ignite.internal.thread.NamedThreadFactory;
 import org.apache.ignite.internal.tx.HybridTimestampTracker;
 import org.apache.ignite.internal.tx.InternalTransaction;
 import org.apache.ignite.internal.tx.LockManager;
-import org.apache.ignite.internal.tx.TransactionAlreadyFinishedException;
+import org.apache.ignite.internal.tx.MismatchingTransactionOutcomeException;
 import org.apache.ignite.internal.tx.TransactionMeta;
 import org.apache.ignite.internal.tx.TransactionResult;
 import org.apache.ignite.internal.tx.TxManager;
@@ -444,7 +444,7 @@ public class TxManagerImpl implements TxManager, 
NetworkMessageHandler {
             return nullCompletedFuture();
         }
 
-        return failedFuture(new TransactionAlreadyFinishedException(
+        return failedFuture(new MismatchingTransactionOutcomeException(
                 "Failed to change the outcome of a finished transaction 
[txId=" + txId + ", txState=" + stateMeta.txState() + "].",
                 new TransactionResult(stateMeta.txState(), 
stateMeta.commitTimestamp()))
         );
@@ -514,8 +514,8 @@ public class TxManagerImpl implements TxManager, 
NetworkMessageHandler {
                     if (ex != null) {
                         Throwable cause = ExceptionUtils.unwrapCause(ex);
 
-                        if (cause instanceof 
TransactionAlreadyFinishedException) {
-                            TransactionAlreadyFinishedException 
transactionException = (TransactionAlreadyFinishedException) cause;
+                        if (cause instanceof 
MismatchingTransactionOutcomeException) {
+                            MismatchingTransactionOutcomeException 
transactionException = (MismatchingTransactionOutcomeException) cause;
 
                             TransactionResult result = 
transactionException.transactionResult();
 
@@ -602,7 +602,7 @@ public class TxManagerImpl implements TxManager, 
NetworkMessageHandler {
                     txResult.transactionState()
             );
 
-            throw new TransactionAlreadyFinishedException(
+            throw new MismatchingTransactionOutcomeException(
                     "Failed to change the outcome of a finished transaction 
[txId=" + txId + ", txState=" + txResult.transactionState()
                             + "].",
                     txResult
@@ -855,7 +855,7 @@ public class TxManagerImpl implements TxManager, 
NetworkMessageHandler {
                 }
             } else {
                 if (commit && readyToFinishException instanceof 
PrimaryReplicaExpiredException) {
-                    finishInProgressFuture.completeExceptionally(new 
TransactionAlreadyFinishedException(
+                    finishInProgressFuture.completeExceptionally(new 
MismatchingTransactionOutcomeException(
                             TX_PRIMARY_REPLICA_EXPIRED_ERR,
                             "Failed to commit the transaction.",
                             new TransactionResult(ABORTED, null),
diff --git 
a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/TxManagerTest.java
 
b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/TxManagerTest.java
index 287dde26cf..b1f7b629b8 100644
--- 
a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/TxManagerTest.java
+++ 
b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/TxManagerTest.java
@@ -32,6 +32,7 @@ import static 
org.apache.ignite.lang.ErrorGroups.Transactions.TX_PRIMARY_REPLICA
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertInstanceOf;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertSame;
 import static org.junit.jupiter.api.Assertions.assertThrows;
@@ -48,6 +49,7 @@ import static org.mockito.Mockito.when;
 
 import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
 import java.util.function.LongSupplier;
 import 
org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
 import 
org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
@@ -57,6 +59,7 @@ import org.apache.ignite.internal.hlc.HybridTimestamp;
 import org.apache.ignite.internal.lang.IgniteBiTuple;
 import org.apache.ignite.internal.lang.IgniteInternalException;
 import org.apache.ignite.internal.placementdriver.PlacementDriver;
+import org.apache.ignite.internal.placementdriver.ReplicaMeta;
 import org.apache.ignite.internal.placementdriver.TestReplicaMetaImpl;
 import org.apache.ignite.internal.replicator.ReplicaService;
 import org.apache.ignite.internal.replicator.TablePartitionId;
@@ -69,6 +72,7 @@ import 
org.apache.ignite.internal.tx.impl.TransactionIdGenerator;
 import org.apache.ignite.internal.tx.impl.TxManagerImpl;
 import org.apache.ignite.internal.tx.message.TxFinishReplicaRequest;
 import org.apache.ignite.internal.tx.test.TestTransactionIds;
+import org.apache.ignite.internal.util.ExceptionUtils;
 import org.apache.ignite.lang.ErrorGroups.Transactions;
 import org.apache.ignite.network.ClusterNode;
 import org.apache.ignite.network.ClusterNodeImpl;
@@ -235,6 +239,119 @@ public class TxManagerTest extends IgniteAbstractTest {
         assertThat(txManager.updateLowWatermark(clock.now()), 
willSucceedFast());
     }
 
+    @Test
+    public void testRepeatedCommitRollbackAfterCommit() throws Exception {
+        ReplicaMeta meta = mock(ReplicaMeta.class);
+        when(meta.getStartTime()).thenReturn(hybridTimestamp(1));
+
+        when(placementDriver.currentLease(any())).thenReturn(meta);
+        when(placementDriver.getPrimaryReplica(any(), 
any())).thenReturn(completedFuture(
+                new TestReplicaMetaImpl(LOCAL_NODE, hybridTimestamp(1), 
HybridTimestamp.MAX_VALUE)));
+        when(placementDriver.awaitPrimaryReplica(any(), any(), anyLong(), 
any())).thenReturn(completedFuture(
+                new TestReplicaMetaImpl(LOCAL_NODE, hybridTimestamp(1), 
HybridTimestamp.MAX_VALUE)));
+
+        HybridTimestamp commitTimestamp = clock.now();
+        when(replicaService.invoke(anyString(), 
any(TxFinishReplicaRequest.class)))
+                .thenReturn(completedFuture(new 
TransactionResult(TxState.COMMITTED, commitTimestamp)));
+
+        InternalTransaction tx = txManager.begin(hybridTimestampTracker);
+
+        ClusterNode node = mock(ClusterNode.class);
+
+        TablePartitionId tablePartitionId1 = new TablePartitionId(1, 0);
+
+        tx.enlist(tablePartitionId1, new IgniteBiTuple<>(node, 1L));
+        tx.assignCommitPartition(tablePartitionId1);
+
+        tx.commit();
+
+        tx.commitAsync().get(3, TimeUnit.SECONDS);
+        tx.rollbackAsync().get(3, TimeUnit.SECONDS);
+    }
+
+    @Test
+    public void testRepeatedCommitRollbackAfterRollback() throws Exception {
+        when(placementDriver.awaitPrimaryReplica(any(), any(), anyLong(), 
any())).thenReturn(completedFuture(
+                new TestReplicaMetaImpl(LOCAL_NODE, hybridTimestamp(1), 
HybridTimestamp.MAX_VALUE)));
+
+        when(replicaService.invoke(anyString(), 
any(TxFinishReplicaRequest.class)))
+                .thenReturn(completedFuture(new 
TransactionResult(TxState.ABORTED, null)));
+
+        InternalTransaction tx = txManager.begin(hybridTimestampTracker);
+
+        ClusterNode node = mock(ClusterNode.class);
+
+        TablePartitionId tablePartitionId1 = new TablePartitionId(1, 0);
+
+        tx.enlist(tablePartitionId1, new IgniteBiTuple<>(node, 1L));
+        tx.assignCommitPartition(tablePartitionId1);
+
+        tx.rollback();
+
+        tx.commitAsync().get(3, TimeUnit.SECONDS);
+        tx.rollbackAsync().get(3, TimeUnit.SECONDS);
+    }
+
+    @Test
+    void testRepeatedCommitRollbackAfterCommitWithException() throws Exception 
{
+        when(placementDriver.awaitPrimaryReplica(any(), any(), anyLong(), 
any())).thenReturn(completedFuture(
+                new TestReplicaMetaImpl(LOCAL_NODE, hybridTimestamp(1), 
HybridTimestamp.MAX_VALUE)));
+
+        when(replicaService.invoke(anyString(), 
any(TxFinishReplicaRequest.class)))
+                .thenReturn(failedFuture(
+                        new MismatchingTransactionOutcomeException(
+                                "Test exception",
+                                new TransactionResult(TxState.ABORTED, null
+                                )
+                        )));
+
+        InternalTransaction tx = txManager.begin(hybridTimestampTracker);
+
+        ClusterNode node = mock(ClusterNode.class);
+
+        TablePartitionId tablePartitionId1 = new TablePartitionId(1, 0);
+
+        tx.enlist(tablePartitionId1, new IgniteBiTuple<>(node, 1L));
+        tx.assignCommitPartition(tablePartitionId1);
+
+        TransactionException transactionException = 
assertThrows(TransactionException.class, tx::commit);
+
+        assertInstanceOf(MismatchingTransactionOutcomeException.class, 
ExceptionUtils.unwrapCause(transactionException.getCause()));
+
+        tx.commitAsync().get(3, TimeUnit.SECONDS);
+        tx.rollbackAsync().get(3, TimeUnit.SECONDS);
+    }
+
+    @Test
+    public void testRepeatedCommitRollbackAfterRollbackWithException() throws 
Exception {
+        when(placementDriver.awaitPrimaryReplica(any(), any(), anyLong(), 
any())).thenReturn(completedFuture(
+                new TestReplicaMetaImpl(LOCAL_NODE, hybridTimestamp(1), 
HybridTimestamp.MAX_VALUE)));
+
+        when(replicaService.invoke(anyString(), 
any(TxFinishReplicaRequest.class)))
+                .thenReturn(failedFuture(
+                        new MismatchingTransactionOutcomeException(
+                                "Test exception",
+                                new TransactionResult(TxState.ABORTED, null
+                                )
+                        )));
+
+        InternalTransaction tx = txManager.begin(hybridTimestampTracker);
+
+        ClusterNode node = mock(ClusterNode.class);
+
+        TablePartitionId tablePartitionId1 = new TablePartitionId(1, 0);
+
+        tx.enlist(tablePartitionId1, new IgniteBiTuple<>(node, 1L));
+        tx.assignCommitPartition(tablePartitionId1);
+
+        TransactionException transactionException = 
assertThrows(TransactionException.class, tx::rollback);
+
+        assertInstanceOf(MismatchingTransactionOutcomeException.class, 
ExceptionUtils.unwrapCause(transactionException.getCause()));
+
+        tx.commitAsync().get(3, TimeUnit.SECONDS);
+        tx.rollbackAsync().get(3, TimeUnit.SECONDS);
+    }
+
     @ParameterizedTest
     @ValueSource(booleans = {true, false})
     public void testTestOnlyPendingCommit(boolean startReadOnlyTransaction) {
@@ -442,14 +559,14 @@ public class TxManagerTest extends IgniteAbstractTest {
         when(placementDriver.awaitPrimaryReplica(any(), any(), anyLong(), 
any())).thenReturn(completedFuture(
                 new TestReplicaMetaImpl(LOCAL_NODE, hybridTimestamp(1), 
hybridTimestamp(10))));
         when(replicaService.invoke(anyString(), 
any(TxFinishReplicaRequest.class)))
-                .thenReturn(failedFuture(new 
TransactionAlreadyFinishedException(
+                .thenReturn(failedFuture(new 
MismatchingTransactionOutcomeException(
                         "TX already finished.",
                         new TransactionResult(TxState.ABORTED, null)
                 )));
 
         InternalTransaction committedTransaction = prepareTransaction();
 
-        assertThrowsWithCause(committedTransaction::commit, 
TransactionAlreadyFinishedException.class);
+        assertThrowsWithCause(committedTransaction::commit, 
MismatchingTransactionOutcomeException.class);
 
         assertEquals(TxState.ABORTED, 
txManager.stateMeta(committedTransaction.id()).txState());
 

Reply via email to