denis-chudov commented on code in PR #2856:
URL: https://github.com/apache/ignite-3/pull/2856#discussion_r1430385186


##########
modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/TxManagerImpl.java:
##########
@@ -439,20 +472,26 @@ private CompletableFuture<Void> durableFinish(
                     if (ex != null) {
                         Throwable cause = ExceptionUtils.unwrapCause(ex);
 
-                        if (cause instanceof TransactionException) {
-                            TransactionException transactionException = 
(TransactionException) cause;
+                        if (cause instanceof 
TransactionAlreadyFinishedException) {
+                            TransactionAlreadyFinishedException 
transactionException = (TransactionAlreadyFinishedException) cause;
+
+                            TransactionResult result = 
transactionException.transactionResult();
 
-                            if (transactionException.code() == 
TX_WAS_ABORTED_ERR) {
-                                updateTxMeta(txId, old -> {
-                                    TxStateMeta txStateMeta = new 
TxStateMeta(ABORTED, old.txCoordinatorId(), commitPartition, null);
+                            updateTxMeta(txId, old -> {
+                                TxStateMeta txStateMeta =
+                                        new TxStateMeta(
+                                                result.transactionState(),
+                                                old.txCoordinatorId(),
+                                                commitPartition,
+                                                result.commitTimestamp()
+                                        );
 
-                                    txFinishFuture.complete(txStateMeta);
+                                txFinishFuture.complete(txStateMeta);

Review Comment:
   By the way, let's better move this future completion out of the meta updater 
(and hence from the map.compute closure)



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/PartitionListener.java:
##########
@@ -576,7 +583,11 @@ private static void onTxStateStorageCasFail(UUID txId, 
TxMeta txMetaBeforeCas, T
                 txMetaToSet
         );
 
-        IgniteInternalException stateChangeException = new 
IgniteInternalException(TX_UNEXPECTED_STATE_ERR, errorMsg);
+        IgniteInternalException stateChangeException =
+                new UnexpectedTransactionStateException(
+                        errorMsg,
+                        new TransactionResult(txMetaBeforeCas.txState(), 
txMetaBeforeCas.commitTimestamp())

Review Comment:
   txMetaBeforeCas can be null, still CAS can fail.



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java:
##########
@@ -1575,33 +1608,38 @@ private CompletableFuture<Void> finishAndCleanup(
             // The transaction is finished, but the locks are not released.
             // If we got here, it means we are retrying the finish request.
             // Let's make sure the desired state is valid.
-            // Tx logic does not allow to send a rollback over a finished 
transaction:
             // - The Coordinator calls use same tx state over retries, both 
abort and commit are possible.
-            // - Server side recovery (which is not implemented yet) may only 
change tx state to aborted.
+            // - Server side recovery may only change tx state to aborted.
             // - The Coordinator itself should prevent user calls with 
different proposed state to the one,
-            //   that was already triggered (e.g. the client side -> 
txCoordinator.commitAsync(); txCoordinator.rollbackAsync())
-            //
-            // To sum it up, the possible states that a 'commit' is allowed to 
see:
+            //   that was already triggered (e.g. the client side -> 
txCoordinator.commitAsync(); txCoordinator.rollbackAsync()).
+            // - A coordinator might send a commit, then die, but the commit 
message might still arrive at the commit partition primary.
+            //   If it arrived with a delay, another node might come across a 
write intent/lock from that tx
+            //   and realize that the coordinator is no longer available and 
start tx recovery.
+            //   The original commit message might arrive later than the 
recovery one,
+            //   hence a 'commit over rollback' case.
+            // The possible states that a 'commit' is allowed to see:
             // - null (if it's the first change state attempt)
             // - committed (if it was already updated in the previous attempt)
             // - aborted (if it was aborted by the initiate recovery logic,
             //   though this is a very unlikely case because initiate recovery 
will only roll back the tx if coordinator is dead).
             //
             // Within 'roll back' it's allowed to see:
-            // - null
-            // - aborted
-            // Other combinations of states are not possible.
-
-            // First, throw an exception if we are trying to abort an already 
committed tx.
-            assert !(txMeta.txState() == COMMITTED && !commit) : "Not allowed 
to abort an already committed transaction.";
+            // - null (if it's the first change state attempt)
+            // - aborted  (if it was already updated in the previous attempt 
or the result of a concurrent recovery)
+            // - commit (if initiate recovery has started, but a delayed 
message from the coordinator finally arrived and executed earlier).
 
-            // If a 'commit' sees a tx in the ABORTED state (valid as per the 
explanation above), let the client know with an exception.
-            if (commit && txMeta.txState() == ABORTED) {
-                LOG.error("Failed to commit a transaction that is already 
aborted [txId={}].", txId);
+            // Let the client know a transaction has finished with a different 
outcome.
+            if (commit != (txMeta.txState() == COMMITTED)) {
+                LOG.error("Failed to finish a transaction that is already 
finished [txId={}, expectedState={}, actualState={}].",
+                        txId,
+                        commit ? COMMITTED : ABORTED,
+                        txMeta.txState()
+                );
 
-                throw new TransactionException(TX_WAS_ABORTED_ERR,
-                        "Failed to change the outcome of a finished 
transaction"
-                                + " [txId=" + txId + ", txState=" + 
txMeta.txState() + "].");
+                throw new TransactionAlreadyFinishedException(
+                        "Failed to change the outcome of a finished 
transaction [txId=" + txId + "].",
+                        new TransactionResult(txMeta.txState(), 
txMeta.commitTimestamp())
+                );

Review Comment:
   Don't you think that the exception message should include the transaction 
result?



##########
modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/TxManagerImpl.java:
##########
@@ -375,6 +397,17 @@ public CompletableFuture<Void> finish(
                 ));
     }
 
+    private static CompletableFuture<Void> checkTxOutcome(boolean commit, UUID 
txId, TransactionMeta stateMeta) {
+        if ((stateMeta.txState() == COMMITTED) == commit) {
+            return nullCompletedFuture();
+        }
+
+        return CompletableFuture.failedFuture(new 
TransactionAlreadyFinishedException(
+                "Failed to change the outcome of a finished transaction 
[txId=" + txId + "].",

Review Comment:
   Don't you think that the exception message should include the transaction 
result?



##########
modules/transactions/src/main/java/org/apache/ignite/internal/tx/TransactionResult.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.tx;
+
+import static 
org.apache.ignite.internal.hlc.HybridTimestamp.hybridTimestampToLong;
+import static 
org.apache.ignite.internal.hlc.HybridTimestamp.nullableHybridTimestamp;
+
+import java.io.Serializable;
+import org.apache.ignite.internal.hlc.HybridTimestamp;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Represents transaction result - the state and the commit timestamp.

Review Comment:
   ```suggestion
    * Represents transaction final result - the state and the commit timestamp.
   ```



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to