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

sanpwc 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 6d30089d84 IGNITE-18324 Calling tx.rollback() or tx.commit() is not 
failed for committed transaction. (#1503)
6d30089d84 is described below

commit 6d30089d84463ef655ae1147c45c8f8f705f5ab4
Author: Sergey Uttsel <utt...@gmail.com>
AuthorDate: Fri Jan 20 16:42:04 2023 +0300

    IGNITE-18324 Calling tx.rollback() or tx.commit() is not failed for 
committed transaction. (#1503)
---
 .../java/org/apache/ignite/tx/Transaction.java     |  12 +-
 .../internal/client/tx/ClientTransaction.java      |  45 ++--
 .../RepeatedFinishClientTransactionTest.java       | 214 +++++++++++++++
 .../app/client/ItThinClientTransactionsTest.java   |  15 +-
 .../RepeatedFinishReadWriteTransactionTest.java    | 288 +++++++++++++++++++++
 .../ignite/internal/table/TxAbstractTest.java      |  35 ++-
 .../tx/impl/IgniteAbstractTransactionImpl.java     |   3 +-
 .../internal/tx/impl/ReadOnlyTransactionImpl.java  |   4 +-
 .../internal/tx/impl/ReadWriteTransactionImpl.java |  17 +-
 9 files changed, 583 insertions(+), 50 deletions(-)

diff --git a/modules/api/src/main/java/org/apache/ignite/tx/Transaction.java 
b/modules/api/src/main/java/org/apache/ignite/tx/Transaction.java
index 3caa02de44..0070413267 100644
--- a/modules/api/src/main/java/org/apache/ignite/tx/Transaction.java
+++ b/modules/api/src/main/java/org/apache/ignite/tx/Transaction.java
@@ -25,28 +25,32 @@ import org.apache.ignite.internal.hlc.HybridTimestamp;
  */
 public interface Transaction {
     /**
-     * Synchronously commits a transaction.
+     * Synchronously commits a transaction. A commit of a completed or ending 
transaction has no effect
+     * and always succeeds when the transaction is completed.
      *
      * @throws TransactionException If a transaction can't be committed.
      */
     void commit() throws TransactionException;
 
     /**
-     * Asynchronously commits a transaction.
+     * Asynchronously commits a transaction. A commit of a completed or ending 
transaction has no effect
+     * and always succeeds when the transaction is completed.
      *
      * @return The future.
      */
     CompletableFuture<Void> commitAsync();
 
     /**
-     * Synchronously rolls back a transaction.
+     * Synchronously rolls back a transaction. A rollback of a completed or 
ending transaction has no effect
+     * and always succeeds when the transaction is completed.
      *
      * @throws TransactionException If a transaction can't be rolled back.
      */
     void rollback() throws TransactionException;
 
     /**
-     * Asynchronously rolls back a transaction.
+     * Asynchronously rolls back a transaction. A rollback of a completed or 
ending transaction has no effect
+     * and always succeeds when the transaction is completed.
      *
      * @return The future.
      */
diff --git 
a/modules/client/src/main/java/org/apache/ignite/internal/client/tx/ClientTransaction.java
 
b/modules/client/src/main/java/org/apache/ignite/internal/client/tx/ClientTransaction.java
index 74dce7fb19..c144263db1 100644
--- 
a/modules/client/src/main/java/org/apache/ignite/internal/client/tx/ClientTransaction.java
+++ 
b/modules/client/src/main/java/org/apache/ignite/internal/client/tx/ClientTransaction.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.client.tx;
 import static org.apache.ignite.internal.client.ClientUtils.sync;
 
 import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
 import org.apache.ignite.internal.client.ClientChannel;
 import org.apache.ignite.internal.client.proto.ClientOp;
 import org.apache.ignite.internal.hlc.HybridTimestamp;
@@ -31,23 +31,14 @@ import org.apache.ignite.tx.TransactionException;
  * Client transaction.
  */
 public class ClientTransaction implements Transaction {
-    /** Open state. */
-    private static final int STATE_OPEN = 0;
-
-    /** Committed state. */
-    private static final int STATE_COMMITTED = 1;
-
-    /** Rolled back state. */
-    private static final int STATE_ROLLED_BACK = 2;
-
     /** Channel that the transaction belongs to. */
     private final ClientChannel ch;
 
     /** Transaction id. */
     private final long id;
 
-    /** State. */
-    private final AtomicInteger state = new AtomicInteger(STATE_OPEN);
+    /** The future used on repeated commit/rollback. */
+    private final AtomicReference<CompletableFuture<Void>> finishFut = new 
AtomicReference<>();
 
     /**
      * Constructor.
@@ -87,9 +78,15 @@ public class ClientTransaction implements Transaction {
     /** {@inheritDoc} */
     @Override
     public CompletableFuture<Void> commitAsync() {
-        setState(STATE_COMMITTED);
+        if (!finishFut.compareAndSet(null, new CompletableFuture<>())) {
+            return finishFut.get();
+        }
+
+        CompletableFuture<Void> mainFinishFut = 
ch.serviceAsync(ClientOp.TX_COMMIT, w -> w.out().packLong(id), r -> null);
+
+        mainFinishFut.handle((res, e) -> finishFut.get().complete(null));
 
-        return ch.serviceAsync(ClientOp.TX_COMMIT, w -> w.out().packLong(id), 
r -> null);
+        return mainFinishFut;
     }
 
     /** {@inheritDoc} */
@@ -101,23 +98,15 @@ public class ClientTransaction implements Transaction {
     /** {@inheritDoc} */
     @Override
     public CompletableFuture<Void> rollbackAsync() {
-        setState(STATE_ROLLED_BACK);
-
-        return ch.serviceAsync(ClientOp.TX_ROLLBACK, w -> 
w.out().packLong(id), r -> null);
-    }
-
-    private void setState(int state) {
-        int oldState = this.state.compareAndExchange(STATE_OPEN, state);
-
-        if (oldState == STATE_OPEN) {
-            return;
+        if (!finishFut.compareAndSet(null, new CompletableFuture<>())) {
+            return finishFut.get();
         }
 
-        String message = oldState == STATE_COMMITTED
-                ? "Transaction is already committed."
-                : "Transaction is already rolled back.";
+        CompletableFuture<Void> mainFinishFut = 
ch.serviceAsync(ClientOp.TX_ROLLBACK, w -> w.out().packLong(id), r -> null);
+
+        mainFinishFut.handle((res, e) -> finishFut.get().complete(null));
 
-        throw new TransactionException(message);
+        return mainFinishFut;
     }
 
     /** {@inheritDoc} */
diff --git 
a/modules/client/src/test/java/org/apache/ignite/internal/client/RepeatedFinishClientTransactionTest.java
 
b/modules/client/src/test/java/org/apache/ignite/internal/client/RepeatedFinishClientTransactionTest.java
new file mode 100644
index 0000000000..7ffdf1010a
--- /dev/null
+++ 
b/modules/client/src/test/java/org/apache/ignite/internal/client/RepeatedFinishClientTransactionTest.java
@@ -0,0 +1,214 @@
+/*
+ * 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.client;
+
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.CompletableFuture.failedFuture;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotSame;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import org.apache.ignite.internal.client.tx.ClientTransaction;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Tests repeated commit/rollback operations.
+ */
+public class RepeatedFinishClientTransactionTest {
+    @Test
+    public void testRepeatedCommitRollbackAfterCommit() throws Exception {
+        CountDownLatch txFinishStartedLatch = new CountDownLatch(1);
+        CountDownLatch secondFinishLatch = new CountDownLatch(1);
+
+        TestClientChannel clientChannel = new 
TestClientChannel(txFinishStartedLatch, secondFinishLatch);
+
+        ClientTransaction tx = new ClientTransaction(clientChannel, 1);
+
+        CompletableFuture<Object> fut = new CompletableFuture<>();
+
+        CompletableFuture<Void> firstCommitFut = 
fut.thenComposeAsync((ignored) -> tx.commitAsync());
+
+        fut.complete(null);
+
+        txFinishStartedLatch.await();
+
+        CompletableFuture<Void> secondCommitFut = tx.commitAsync();
+
+        CompletableFuture<Void> rollbackFut = tx.rollbackAsync();
+
+        assertNotSame(firstCommitFut, secondCommitFut);
+        assertSame(secondCommitFut, rollbackFut);
+        assertSame(secondCommitFut, tx.commitAsync());
+        assertSame(rollbackFut, tx.rollbackAsync());
+
+        assertFalse(firstCommitFut.isDone());
+        assertFalse(secondCommitFut.isDone());
+        assertFalse(rollbackFut.isDone());
+
+        secondFinishLatch.countDown();
+
+        firstCommitFut.get(3, TimeUnit.SECONDS);
+        assertTrue(firstCommitFut.isDone());
+        assertTrue(secondCommitFut.isDone());
+        assertTrue(rollbackFut.isDone());
+    }
+
+    @Test
+    public void testRepeatedCommitRollbackAfterRollback() throws Exception {
+        CountDownLatch txFinishStartedLatch = new CountDownLatch(1);
+        CountDownLatch secondFinishLatch = new CountDownLatch(1);
+
+        TestClientChannel clientChannel = new 
TestClientChannel(txFinishStartedLatch, secondFinishLatch);
+
+        ClientTransaction tx = new ClientTransaction(clientChannel, 1);
+
+        CompletableFuture<Object> fut = new CompletableFuture<>();
+
+        CompletableFuture<Void> firstRollbackFut = 
fut.thenComposeAsync((ignored) -> tx.rollbackAsync());
+
+        fut.complete(null);
+
+        txFinishStartedLatch.await();
+
+        CompletableFuture<Void> commitFut = tx.commitAsync();
+
+        CompletableFuture<Void> secondRollbackFut = tx.rollbackAsync();
+
+        assertNotSame(firstRollbackFut, secondRollbackFut);
+        assertSame(secondRollbackFut, commitFut);
+        assertSame(commitFut, tx.commitAsync());
+        assertSame(secondRollbackFut, tx.rollbackAsync());
+
+        assertFalse(firstRollbackFut.isDone());
+        assertFalse(secondRollbackFut.isDone());
+        assertFalse(commitFut.isDone());
+
+        secondFinishLatch.countDown();
+
+        firstRollbackFut.get(3, TimeUnit.SECONDS);
+        assertTrue(firstRollbackFut.isDone());
+        assertTrue(secondRollbackFut.isDone());
+        assertTrue(commitFut.isDone());
+    }
+
+    @Test
+    public void testRepeatedCommitRollbackAfterCommitWithException() throws 
Exception {
+        TestClientChannel clientChannel = mock(TestClientChannel.class);
+
+        when(clientChannel.serviceAsync(anyInt(), any(), 
any())).thenReturn(failedFuture(new Exception("Expected exception.")));
+
+        ClientTransaction tx = new ClientTransaction(clientChannel, 1);
+
+        CompletableFuture<Object> fut = new CompletableFuture<>();
+
+        CompletableFuture<Void> commitFut = fut.thenComposeAsync((ignored) -> 
tx.commitAsync());
+
+        fut.complete(null);
+
+        try {
+            commitFut.get(3, TimeUnit.SECONDS);
+
+            fail();
+        } catch (Exception ignored) {
+            // No op.
+        }
+
+        tx.commitAsync().get(3, TimeUnit.SECONDS);
+        tx.rollbackAsync().get(3, TimeUnit.SECONDS);
+    }
+
+    @Test
+    public void testRepeatedCommitRollbackAfterRollbackWithException() throws 
Exception {
+        TestClientChannel clientChannel = mock(TestClientChannel.class);
+
+        when(clientChannel.serviceAsync(anyInt(), any(), 
any())).thenReturn(failedFuture(new Exception("Expected exception.")));
+
+        ClientTransaction tx = new ClientTransaction(clientChannel, 1);
+
+        CompletableFuture<Object> fut = new CompletableFuture<>();
+
+        CompletableFuture<Void> rollbackFut = fut.thenComposeAsync((ignored) 
-> tx.rollbackAsync());
+
+        fut.complete(null);
+
+        try {
+            rollbackFut.get(3, TimeUnit.SECONDS);
+
+            fail();
+        } catch (Exception ignored) {
+            // No op.
+        }
+
+        tx.commitAsync().get(3, TimeUnit.SECONDS);
+        tx.rollbackAsync().get(3, TimeUnit.SECONDS);
+    }
+
+    private static class TestClientChannel implements ClientChannel {
+        private final CountDownLatch txFinishStartedLatch;
+
+        private final CountDownLatch secondFinishLatch;
+
+        TestClientChannel(CountDownLatch txFinishStartedLatch, CountDownLatch 
secondFinishLatch) {
+            this.txFinishStartedLatch = txFinishStartedLatch;
+            this.secondFinishLatch = secondFinishLatch;
+        }
+
+        @Override
+        public <T> CompletableFuture<T> serviceAsync(int opCode, PayloadWriter 
payloadWriter, PayloadReader<T> payloadReader) {
+            txFinishStartedLatch.countDown();
+
+            try {
+                secondFinishLatch.await();
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            }
+
+            return completedFuture(null);
+        }
+
+        @Override
+        public boolean closed() {
+            return false;
+        }
+
+        @Override
+        public ProtocolContext protocolContext() {
+            return null;
+        }
+
+        @Override
+        public void 
addTopologyAssignmentChangeListener(Consumer<ClientChannel> listener) {
+
+        }
+
+        @Override
+        public void close() throws Exception {
+
+        }
+    }
+}
diff --git 
a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItThinClientTransactionsTest.java
 
b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItThinClientTransactionsTest.java
index a4eae7616e..0dbf9ef0c5 100644
--- 
a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItThinClientTransactionsTest.java
+++ 
b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItThinClientTransactionsTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.runner.app.client;
 
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.containsString;
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertThrows;
@@ -199,21 +200,23 @@ public class ItThinClientTransactionsTest extends 
ItAbstractThinClientTest {
     }
 
     @Test
-    void testCommitRollbackSameTxThrows() {
+    void testCommitRollbackSameTxDoesNotThrow() {
         Transaction tx = client().transactions().begin();
         tx.commit();
 
-        TransactionException ex = assertThrows(TransactionException.class, 
tx::rollback);
-        assertThat(ex.getMessage(), containsString("Transaction is already 
committed"));
+        assertDoesNotThrow(tx::rollback, "Unexpected exception was thrown.");
+        assertDoesNotThrow(tx::commit, "Unexpected exception was thrown.");
+        assertDoesNotThrow(tx::rollback, "Unexpected exception was thrown.");
     }
 
     @Test
-    void testRollbackCommitSameTxThrows() {
+    void testRollbackCommitSameTxDoesNotThrow() {
         Transaction tx = client().transactions().begin();
         tx.rollback();
 
-        TransactionException ex = assertThrows(TransactionException.class, 
tx::commit);
-        assertThat(ex.getMessage(), containsString("Transaction is already 
rolled back"));
+        assertDoesNotThrow(tx::commit, "Unexpected exception was thrown.");
+        assertDoesNotThrow(tx::rollback, "Unexpected exception was thrown.");
+        assertDoesNotThrow(tx::commit, "Unexpected exception was thrown.");
     }
 
     @Test
diff --git 
a/modules/table/src/test/java/org/apache/ignite/internal/table/RepeatedFinishReadWriteTransactionTest.java
 
b/modules/table/src/test/java/org/apache/ignite/internal/table/RepeatedFinishReadWriteTransactionTest.java
new file mode 100644
index 0000000000..4c0581582e
--- /dev/null
+++ 
b/modules/table/src/test/java/org/apache/ignite/internal/table/RepeatedFinishReadWriteTransactionTest.java
@@ -0,0 +1,288 @@
+/*
+ * 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.table;
+
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.CompletableFuture.failedFuture;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotSame;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import org.apache.ignite.internal.hlc.HybridTimestamp;
+import org.apache.ignite.internal.replicator.ReplicationGroupId;
+import 
org.apache.ignite.internal.table.distributed.replicator.TablePartitionId;
+import org.apache.ignite.internal.tx.InternalTransaction;
+import org.apache.ignite.internal.tx.LockManager;
+import org.apache.ignite.internal.tx.TxManager;
+import org.apache.ignite.internal.tx.TxState;
+import org.apache.ignite.internal.tx.impl.ReadWriteTransactionImpl;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.network.ClusterNode;
+import org.jetbrains.annotations.Nullable;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Tests repeated commit/rollback operations.
+ */
+public class RepeatedFinishReadWriteTransactionTest {
+    @Test
+    public void testRepeatedCommitRollbackAfterCommit() throws Exception {
+        CountDownLatch txFinishStartedLatch = new CountDownLatch(1);
+        CountDownLatch secondFinishLatch = new CountDownLatch(1);
+
+        TestTxManager txManager = new TestTxManager(txFinishStartedLatch, 
secondFinishLatch);
+
+        ReadWriteTransactionImpl tx = new ReadWriteTransactionImpl(txManager, 
UUID.randomUUID());
+
+        TablePartitionId partId = new TablePartitionId(UUID.randomUUID(), 1);
+
+        tx.enlist(partId, new IgniteBiTuple<>(null, null));
+
+        tx.assignCommitPartition(partId);
+
+        tx.enlistResultFuture(completedFuture(null));
+
+        CompletableFuture<Object> fut = new CompletableFuture<>();
+
+        CompletableFuture<Void> firstCommitFut = 
fut.thenComposeAsync((ignored) -> tx.commitAsync());
+
+        fut.complete(null);
+
+        txFinishStartedLatch.await();
+
+        CompletableFuture<Void> secondCommitFut = tx.commitAsync();
+
+        CompletableFuture<Void> rollbackFut = tx.rollbackAsync();
+
+        assertNotSame(firstCommitFut, secondCommitFut);
+        assertSame(secondCommitFut, rollbackFut);
+        assertSame(secondCommitFut, tx.commitAsync());
+        assertSame(rollbackFut, tx.rollbackAsync());
+
+        assertFalse(firstCommitFut.isDone());
+        assertFalse(secondCommitFut.isDone());
+        assertFalse(rollbackFut.isDone());
+
+        secondFinishLatch.countDown();
+
+        firstCommitFut.get(3, TimeUnit.SECONDS);
+        assertTrue(firstCommitFut.isDone());
+        assertTrue(secondCommitFut.isDone());
+        assertTrue(rollbackFut.isDone());
+    }
+
+    @Test
+    public void testRepeatedCommitRollbackAfterRollback() throws Exception {
+        CountDownLatch txFinishStartedLatch = new CountDownLatch(1);
+        CountDownLatch secondFinishLatch = new CountDownLatch(1);
+
+        TestTxManager txManager = new TestTxManager(txFinishStartedLatch, 
secondFinishLatch);
+
+        ReadWriteTransactionImpl tx = new ReadWriteTransactionImpl(txManager, 
UUID.randomUUID());
+
+        TablePartitionId partId = new TablePartitionId(UUID.randomUUID(), 1);
+
+        tx.enlist(partId, new IgniteBiTuple<>(null, null));
+
+        tx.assignCommitPartition(partId);
+
+        tx.enlistResultFuture(completedFuture(null));
+
+        CompletableFuture<Object> fut = new CompletableFuture<>();
+
+        CompletableFuture<Void> firstRollbackFut = 
fut.thenComposeAsync((ignored) -> tx.rollbackAsync());
+
+        fut.complete(null);
+
+        txFinishStartedLatch.await();
+
+        CompletableFuture<Void> commitFut = tx.commitAsync();
+
+        CompletableFuture<Void> secondRollbackFut = tx.rollbackAsync();
+
+        assertNotSame(firstRollbackFut, secondRollbackFut);
+        assertSame(secondRollbackFut, commitFut);
+        assertSame(commitFut, tx.commitAsync());
+        assertSame(secondRollbackFut, tx.rollbackAsync());
+
+        assertFalse(firstRollbackFut.isDone());
+        assertFalse(secondRollbackFut.isDone());
+        assertFalse(commitFut.isDone());
+
+        secondFinishLatch.countDown();
+
+        firstRollbackFut.get(3, TimeUnit.SECONDS);
+        assertTrue(firstRollbackFut.isDone());
+        assertTrue(secondRollbackFut.isDone());
+        assertTrue(commitFut.isDone());
+    }
+
+    @Test
+    public void testRepeatedCommitRollbackAfterCommitWithException() throws 
Exception {
+        TestTxManager txManager = mock(TestTxManager.class);
+
+        when(txManager.finish(any(), any(), any(), anyBoolean(), any(), any()))
+                .thenReturn(failedFuture(new Exception("Expected 
exception.")));
+
+        ReadWriteTransactionImpl tx = new ReadWriteTransactionImpl(txManager, 
UUID.randomUUID());
+
+        TablePartitionId partId = new TablePartitionId(UUID.randomUUID(), 1);
+
+        tx.enlist(partId, new IgniteBiTuple<>(null, null));
+
+        tx.assignCommitPartition(partId);
+
+        tx.enlistResultFuture(completedFuture(null));
+
+        CompletableFuture<Object> fut = new CompletableFuture<>();
+
+        CompletableFuture<Void> firstCommitFut = 
fut.thenComposeAsync((ignored) -> tx.commitAsync());
+
+        fut.complete(null);
+
+        try {
+            firstCommitFut.get(3, TimeUnit.SECONDS);
+
+            fail();
+        } catch (Exception ignored) {
+            // No op.
+        }
+
+        tx.commitAsync().get(3, TimeUnit.SECONDS);
+        tx.rollbackAsync().get(3, TimeUnit.SECONDS);
+    }
+
+    @Test
+    public void testRepeatedCommitRollbackAfterRollbackWithException() throws 
Exception {
+        TestTxManager txManager = mock(TestTxManager.class);
+
+        when(txManager.finish(any(), any(), any(), anyBoolean(), any(), any()))
+                .thenReturn(failedFuture(new Exception("Expected 
exception.")));
+
+        ReadWriteTransactionImpl tx = new ReadWriteTransactionImpl(txManager, 
UUID.randomUUID());
+
+        TablePartitionId partId = new TablePartitionId(UUID.randomUUID(), 1);
+
+        tx.enlist(partId, new IgniteBiTuple<>(null, null));
+
+        tx.assignCommitPartition(partId);
+
+        tx.enlistResultFuture(completedFuture(null));
+
+        CompletableFuture<Object> fut = new CompletableFuture<>();
+
+        CompletableFuture<Void> rollbackFut = fut.thenComposeAsync((ignored) 
-> tx.rollbackAsync());
+
+        fut.complete(null);
+
+        try {
+            rollbackFut.get(3, TimeUnit.SECONDS);
+
+            fail();
+        } catch (Exception ignored) {
+            // No op.
+        }
+
+        tx.commitAsync().get(3, TimeUnit.SECONDS);
+        tx.rollbackAsync().get(3, TimeUnit.SECONDS);
+    }
+
+    private static class TestTxManager implements TxManager {
+        private final CountDownLatch txFinishStartedLatch;
+
+        private final CountDownLatch secondFinishLatch;
+
+        TestTxManager(CountDownLatch txFinishStartedLatch, CountDownLatch 
secondFinishLatch) {
+            this.txFinishStartedLatch = txFinishStartedLatch;
+            this.secondFinishLatch = secondFinishLatch;
+        }
+
+        @Override
+        public InternalTransaction begin() {
+            return null;
+        }
+
+        @Override
+        public InternalTransaction begin(boolean readOnly) {
+            return null;
+        }
+
+        @Override
+        public @Nullable TxState state(UUID txId) {
+            return null;
+        }
+
+        @Override
+        public boolean changeState(UUID txId, @Nullable TxState before, 
TxState after) {
+            return false;
+        }
+
+        @Override
+        public LockManager lockManager() {
+            return null;
+        }
+
+        @Override
+        public CompletableFuture<Void> finish(ReplicationGroupId 
commitPartition, ClusterNode recipientNode, Long term, boolean commit,
+                Map<ClusterNode, List<IgniteBiTuple<ReplicationGroupId, 
Long>>> groups, UUID txId) {
+            txFinishStartedLatch.countDown();
+
+            try {
+                secondFinishLatch.await();
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            }
+
+            return completedFuture(null);
+        }
+
+        @Override
+        public CompletableFuture<Void> cleanup(ClusterNode recipientNode,
+                List<IgniteBiTuple<ReplicationGroupId, Long>> 
replicationGroupIds, UUID txId, boolean commit,
+                HybridTimestamp commitTimestamp) {
+            return null;
+        }
+
+        @Override
+        public int finished() {
+            return 0;
+        }
+
+        @Override
+        public void start() {
+
+        }
+
+        @Override
+        public void stop() throws Exception {
+
+        }
+    }
+}
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 44ad856d9d..5a356045f7 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
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.table;
 import static java.util.concurrent.CompletableFuture.allOf;
 import static java.util.concurrent.CompletableFuture.completedFuture;
 import static 
org.apache.ignite.internal.testframework.IgniteTestUtils.assertThrowsWithCause;
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
@@ -120,6 +121,32 @@ public abstract class TxAbstractTest extends 
IgniteAbstractTest {
     @BeforeEach
     public abstract void before() throws Exception;
 
+    @Test
+    public void testCommitRollbackSameTxDoesNotThrow() throws 
TransactionException {
+        InternalTransaction tx = (InternalTransaction) 
igniteTransactions.begin();
+
+        accounts.recordView().upsert(tx, makeValue(1, 100.));
+
+        tx.commit();
+
+        assertDoesNotThrow(tx::rollback, "Unexpected exception was thrown.");
+        assertDoesNotThrow(tx::commit, "Unexpected exception was thrown.");
+        assertDoesNotThrow(tx::rollback, "Unexpected exception was thrown.");
+    }
+
+    @Test
+    public void testRollbackCommitSameTxDoesNotThrow() throws 
TransactionException {
+        InternalTransaction tx = (InternalTransaction) 
igniteTransactions.begin();
+
+        accounts.recordView().upsert(tx, makeValue(1, 100.));
+
+        tx.rollback();
+
+        assertDoesNotThrow(tx::commit, "Unexpected exception was thrown.");
+        assertDoesNotThrow(tx::rollback, "Unexpected exception was thrown.");
+        assertDoesNotThrow(tx::commit, "Unexpected exception was thrown.");
+    }
+
     @Test
     public void testDeleteUpsertCommit() throws TransactionException {
         deleteUpsert().commit();
@@ -532,13 +559,7 @@ public abstract class TxAbstractTest extends 
IgniteAbstractTest {
 
         tx1.commit();
 
-        try {
-            tx2.commit();
-
-            fail();
-        } catch (TransactionException e) {
-            // Expected.
-        }
+        tx2.commit();
 
         assertEquals(101., accounts.recordView().get(null, 
makeKey(1)).doubleValue("balance"));
     }
diff --git 
a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/IgniteAbstractTransactionImpl.java
 
b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/IgniteAbstractTransactionImpl.java
index 0cf02f7fa6..42e69344c3 100644
--- 
a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/IgniteAbstractTransactionImpl.java
+++ 
b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/IgniteAbstractTransactionImpl.java
@@ -98,7 +98,8 @@ public abstract class IgniteAbstractTransactionImpl 
implements InternalTransacti
     }
 
     /**
-     * Finishes a transaction.
+     * Finishes a transaction. A finish of a completed or ending transaction 
has no effect
+     * and always succeeds when the transaction is completed.
      *
      * @param commit {@code true} to commit, false to rollback.
      * @return The future.
diff --git 
a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/ReadOnlyTransactionImpl.java
 
b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/ReadOnlyTransactionImpl.java
index 7d8308821e..fb6709ccfe 100644
--- 
a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/ReadOnlyTransactionImpl.java
+++ 
b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/ReadOnlyTransactionImpl.java
@@ -64,7 +64,7 @@ public class ReadOnlyTransactionImpl extends 
IgniteAbstractTransactionImpl {
     /** {@inheritDoc} */
     @Override
     public IgniteBiTuple<ClusterNode, Long> enlist(ReplicationGroupId 
replicationGroupId, IgniteBiTuple<ClusterNode, Long> nodeAndTerm) {
-        // TODO: IGNITE-17666 Close cursor tx finish.
+        // TODO: IGNITE-17666 Close cursor tx finish and do it on the first 
finish invocation only.
         return null;
     }
 
@@ -95,7 +95,7 @@ public class ReadOnlyTransactionImpl extends 
IgniteAbstractTransactionImpl {
     /** {@inheritDoc} */
     @Override
     protected CompletableFuture<Void> finish(boolean commit) {
-        // TODO: IGNITE-17666 Close cursor tx finish.
+        // TODO: IGNITE-17666 Close cursor tx finish and do it on the first 
finish invocation only.
         return CompletableFuture.completedFuture(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 32d10ee997..48cc3a41a8 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
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.internal.tx.impl;
 
+import static java.util.concurrent.CompletableFuture.completedFuture;
+
 import java.util.ArrayList;
 import java.util.LinkedHashMap;
 import java.util.List;
@@ -51,6 +53,9 @@ public class ReadWriteTransactionImpl extends 
IgniteAbstractTransactionImpl {
     /** Reference to the partition that stores the transaction state. */
     private final AtomicReference<ReplicationGroupId> commitPartitionRef = new 
AtomicReference<>();
 
+    /** The future used on repeated commit/rollback. */
+    private final AtomicReference<CompletableFuture<Void>> finishFut = new 
AtomicReference<>();
+
     /**
      * The constructor.
      *
@@ -88,8 +93,12 @@ public class ReadWriteTransactionImpl extends 
IgniteAbstractTransactionImpl {
     /** {@inheritDoc} */
     @Override
     protected CompletableFuture<Void> finish(boolean commit) {
+        if (!finishFut.compareAndSet(null, new CompletableFuture<>())) {
+            return finishFut.get();
+        }
+
         // TODO: https://issues.apache.org/jira/browse/IGNITE-17688 Add proper 
exception handling.
-        return CompletableFuture
+        CompletableFuture<Void> mainFinishFut = CompletableFuture
                 .allOf(enlistedResults.toArray(new CompletableFuture[0]))
                 .thenCompose(
                         ignored -> {
@@ -126,10 +135,14 @@ public class ReadWriteTransactionImpl extends 
IgniteAbstractTransactionImpl {
                                         id()
                                 );
                             } else {
-                                return CompletableFuture.completedFuture(null);
+                                return completedFuture(null);
                             }
                         }
                 );
+
+        mainFinishFut.handle((res, e) -> finishFut.get().complete(null));
+
+        return mainFinishFut;
     }
 
     /** {@inheritDoc} */

Reply via email to