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

sk0x50 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 89c8d45a2e3 IGNITE-27702 Failure handler should not be triggered by 
failed attempt to switch tx state (#7517)
89c8d45a2e3 is described below

commit 89c8d45a2e3a1a5829d2b1c1efba5b6cb210860d
Author: Anton Laletin <[email protected]>
AuthorDate: Thu Feb 5 18:43:48 2026 +0400

    IGNITE-27702 Failure handler should not be triggered by failed attempt to 
switch tx state (#7517)
---
 .../handlers/TxFinishReplicaRequestHandler.java    |  22 ++-
 .../raft/UnexpectedTransactionStateException.java  |  42 ------
 .../raft/handlers/FinishTxCommandHandler.java      |  46 +++---
 .../TxFinishReplicaRequestHandlerTest.java         | 163 +++++++++++++++++++++
 4 files changed, 197 insertions(+), 76 deletions(-)

diff --git 
a/modules/partition-replicator/src/main/java/org/apache/ignite/internal/partition/replicator/handlers/TxFinishReplicaRequestHandler.java
 
b/modules/partition-replicator/src/main/java/org/apache/ignite/internal/partition/replicator/handlers/TxFinishReplicaRequestHandler.java
index fd1f888a534..57ac98eb864 100644
--- 
a/modules/partition-replicator/src/main/java/org/apache/ignite/internal/partition/replicator/handlers/TxFinishReplicaRequestHandler.java
+++ 
b/modules/partition-replicator/src/main/java/org/apache/ignite/internal/partition/replicator/handlers/TxFinishReplicaRequestHandler.java
@@ -46,7 +46,6 @@ import 
org.apache.ignite.internal.partition.replicator.ReplicaTxFinishMarker;
 import 
org.apache.ignite.internal.partition.replicator.ReplicationRaftCommandApplicator;
 import 
org.apache.ignite.internal.partition.replicator.network.PartitionReplicationMessagesFactory;
 import 
org.apache.ignite.internal.partition.replicator.network.command.FinishTxCommandV2Builder;
-import 
org.apache.ignite.internal.partition.replicator.raft.UnexpectedTransactionStateException;
 import 
org.apache.ignite.internal.partition.replicator.schema.ValidationSchemasSource;
 import 
org.apache.ignite.internal.partition.replicator.schemacompat.CompatValidationResult;
 import 
org.apache.ignite.internal.partition.replicator.schemacompat.SchemaCompatibilityValidator;
@@ -283,17 +282,7 @@ public class TxFinishReplicaRequestHandler {
                 ))
                 .handle((txOutcome, ex) -> {
                     if (ex != null) {
-                        // RAFT 'finish' command failed because the state has 
already been written by someone else.
-                        // In that case we throw a corresponding exception.
-                        if (ex instanceof UnexpectedTransactionStateException) 
{
-                            UnexpectedTransactionStateException utse = 
(UnexpectedTransactionStateException) ex;
-                            TransactionResult result = 
utse.transactionResult();
-
-                            replicaTxFinishMarker.markFinished(txId, 
result.transactionState(), result.commitTimestamp());
-
-                            throw new 
MismatchingTransactionOutcomeInternalException(utse.getMessage(), 
utse.transactionResult());
-                        }
-                        // Otherwise we convert from the internal exception to 
the client one.
+                        // Convert from the internal exception to the client 
one.
                         throw new TransactionException(commit ? TX_COMMIT_ERR 
: TX_ROLLBACK_ERR, ex);
                     }
 
@@ -301,6 +290,15 @@ public class TxFinishReplicaRequestHandler {
 
                     replicaTxFinishMarker.markFinished(txId, 
result.transactionState(), result.commitTimestamp());
 
+                    if (commit != (result.transactionState() == COMMITTED)) {
+                        throw new 
MismatchingTransactionOutcomeInternalException(
+                                format("Failed to change the outcome of a 
finished transaction [{}, txState={}].",
+                                        formatTxInfo(txId, txManager, false),
+                                        result.transactionState()),
+                                result
+                        );
+                    }
+
                     return result;
                 });
     }
diff --git 
a/modules/partition-replicator/src/main/java/org/apache/ignite/internal/partition/replicator/raft/UnexpectedTransactionStateException.java
 
b/modules/partition-replicator/src/main/java/org/apache/ignite/internal/partition/replicator/raft/UnexpectedTransactionStateException.java
deleted file mode 100644
index a060b72f474..00000000000
--- 
a/modules/partition-replicator/src/main/java/org/apache/ignite/internal/partition/replicator/raft/UnexpectedTransactionStateException.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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.partition.replicator.raft;
-
-import static 
org.apache.ignite.lang.ErrorGroups.Transactions.TX_UNEXPECTED_STATE_ERR;
-
-import org.apache.ignite.internal.lang.IgniteInternalException;
-import org.apache.ignite.internal.tx.TransactionResult;
-
-/**
- * The exception is thrown when a transaction state has already been set.
- */
-public class UnexpectedTransactionStateException extends 
IgniteInternalException {
-    private static final long serialVersionUID = 3791709101302280451L;
-
-    /** Stored transaction result. */
-    private final TransactionResult transactionResult;
-
-    public UnexpectedTransactionStateException(String errorMessage, 
TransactionResult transactionResult) {
-        super(TX_UNEXPECTED_STATE_ERR, errorMessage);
-        this.transactionResult = transactionResult;
-    }
-
-    public TransactionResult transactionResult() {
-        return transactionResult;
-    }
-}
diff --git 
a/modules/partition-replicator/src/main/java/org/apache/ignite/internal/partition/replicator/raft/handlers/FinishTxCommandHandler.java
 
b/modules/partition-replicator/src/main/java/org/apache/ignite/internal/partition/replicator/raft/handlers/FinishTxCommandHandler.java
index 423742c88eb..be46f3c948f 100644
--- 
a/modules/partition-replicator/src/main/java/org/apache/ignite/internal/partition/replicator/raft/handlers/FinishTxCommandHandler.java
+++ 
b/modules/partition-replicator/src/main/java/org/apache/ignite/internal/partition/replicator/raft/handlers/FinishTxCommandHandler.java
@@ -20,6 +20,7 @@ package 
org.apache.ignite.internal.partition.replicator.raft.handlers;
 import static org.apache.ignite.internal.lang.IgniteStringFormatter.format;
 import static org.apache.ignite.internal.tx.TxState.ABORTED;
 import static org.apache.ignite.internal.tx.TxState.COMMITTED;
+import static org.apache.ignite.lang.ErrorGroups.Common.INTERNAL_ERR;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -32,7 +33,6 @@ import 
org.apache.ignite.internal.partition.replicator.network.command.FinishTxC
 import 
org.apache.ignite.internal.partition.replicator.network.command.FinishTxCommandV2;
 import org.apache.ignite.internal.partition.replicator.raft.CommandResult;
 import org.apache.ignite.internal.partition.replicator.raft.RaftTxFinishMarker;
-import 
org.apache.ignite.internal.partition.replicator.raft.UnexpectedTransactionStateException;
 import org.apache.ignite.internal.replicator.ZonePartitionId;
 import org.apache.ignite.internal.tx.TransactionResult;
 import org.apache.ignite.internal.tx.TxManager;
@@ -98,18 +98,30 @@ public class FinishTxCommandHandler extends 
AbstractCommandHandler<FinishTxComma
                 commandTerm
         );
 
-        // Assume that we handle the finish command only on the commit 
partition.
-        txFinishMarker.markFinished(txId, command.commit(), 
command.commitTimestamp(), this.replicationGroupId);
-
         LOG.debug("Finish the transaction txId = {}, state = {}, 
txStateChangeRes = {}", txId, txMetaToSet, txStateChangeRes);
 
-        if (!txStateChangeRes) {
-            assert txMetaBeforeCas != null : "txMetaBeforeCase is null, but 
CAS has failed for " + txId;
+        if (txStateChangeRes) {
+            // Assume that we handle the finish command only on the commit 
partition.
+            txFinishMarker.markFinished(txId, command.commit(), 
command.commitTimestamp(), this.replicationGroupId);
+
+            TransactionResult result = new TransactionResult(stateToSet, 
command.commitTimestamp());
 
-            onTxStateStorageCasFail(txId, txMetaBeforeCas, txMetaToSet);
+            return new CommandResult(result, true);
         }
 
-        return new CommandResult(new TransactionResult(stateToSet, 
command.commitTimestamp()), true);
+        if (txMetaBeforeCas == null) {
+            throw new IgniteInternalException(
+                    INTERNAL_ERR,
+                    "txMetaBeforeCas is null, but CAS has failed for {}",
+                    txId
+            );
+        }
+
+        TransactionResult existingResult = new 
TransactionResult(txMetaBeforeCas.txState(), txMetaBeforeCas.commitTimestamp());
+
+        logTxStateStorageCasFail(txId, txMetaBeforeCas, txMetaToSet);
+
+        return new CommandResult(existingResult, false);
     }
 
     private static List<EnlistedPartitionGroup> 
fromPartitionMessages(List<EnlistedPartitionGroupMessage> messages) {
@@ -122,24 +134,14 @@ public class FinishTxCommandHandler extends 
AbstractCommandHandler<FinishTxComma
         return list;
     }
 
-    private static void onTxStateStorageCasFail(UUID txId, TxMeta 
txMetaBeforeCas, TxMeta txMetaToSet) {
-        String errorMsg = format("Failed to update tx state in the storage, 
transaction txId = {} because of inconsistent state,"
-                        + " expected state = {}, state to set = {}",
+    private static void logTxStateStorageCasFail(UUID txId, TxMeta 
txMetaBeforeCas, TxMeta txMetaToSet) {
+        String errorMsg = format("Finish command skipped, transaction txId = 
{}, because transaction state is already set,"
+                        + " existing state = {}, state to set = {}",
                 txId,
                 txMetaBeforeCas,
                 txMetaToSet
         );
-
-        IgniteInternalException stateChangeException =
-                new UnexpectedTransactionStateException(
-                        errorMsg,
-                        new TransactionResult(txMetaBeforeCas.txState(), 
txMetaBeforeCas.commitTimestamp())
-                );
-
-        // Exception is explicitly logged because otherwise it can be lost if 
it did not occur on the leader.
-        LOG.error(errorMsg);
-
-        throw stateChangeException;
+        LOG.info(errorMsg);
     }
 
     private static List<EnlistedPartitionGroup> 
enlistedPartitions(FinishTxCommand command) {
diff --git 
a/modules/partition-replicator/src/test/java/org/apache/ignite/internal/partition/replicator/handlers/TxFinishReplicaRequestHandlerTest.java
 
b/modules/partition-replicator/src/test/java/org/apache/ignite/internal/partition/replicator/handlers/TxFinishReplicaRequestHandlerTest.java
new file mode 100644
index 00000000000..981b505cfb2
--- /dev/null
+++ 
b/modules/partition-replicator/src/test/java/org/apache/ignite/internal/partition/replicator/handlers/TxFinishReplicaRequestHandlerTest.java
@@ -0,0 +1,163 @@
+/*
+ * 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.partition.replicator.handlers;
+
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static 
org.apache.ignite.internal.replicator.message.ReplicaMessageUtils.toZonePartitionIdMessage;
+import static 
org.apache.ignite.internal.testframework.matchers.CompletableFutureExceptionMatcher.willThrow;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.ignite.internal.catalog.CatalogService;
+import org.apache.ignite.internal.hlc.ClockService;
+import org.apache.ignite.internal.hlc.HybridTimestamp;
+import 
org.apache.ignite.internal.partition.replicator.schema.ValidationSchemasSource;
+import org.apache.ignite.internal.raft.service.RaftCommandRunner;
+import org.apache.ignite.internal.replicator.ZonePartitionId;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.schema.SchemaSyncService;
+import org.apache.ignite.internal.testframework.BaseIgniteAbstractTest;
+import 
org.apache.ignite.internal.tx.MismatchingTransactionOutcomeInternalException;
+import org.apache.ignite.internal.tx.TransactionIds;
+import org.apache.ignite.internal.tx.TxManager;
+import org.apache.ignite.internal.tx.TxMeta;
+import org.apache.ignite.internal.tx.TxState;
+import org.apache.ignite.internal.tx.message.PartitionEnlistmentMessage;
+import org.apache.ignite.internal.tx.message.TxFinishReplicaRequest;
+import org.apache.ignite.internal.tx.message.TxMessagesFactory;
+import org.apache.ignite.internal.tx.storage.state.TxStatePartitionStorage;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+@ExtendWith(MockitoExtension.class)
+class TxFinishReplicaRequestHandlerTest extends BaseIgniteAbstractTest {
+    private static final long ANY_ENLISTMENT_CONSISTENCY_TOKEN = 1L;
+
+    private final TxMessagesFactory txMessagesFactory = new 
TxMessagesFactory();
+    private final ReplicaMessagesFactory replicaMessagesFactory = new 
ReplicaMessagesFactory();
+
+    private final ZonePartitionId replicationGroupId = new ZonePartitionId(1, 
1);
+
+    @Mock
+    private TxStatePartitionStorage txStatePartitionStorage;
+
+    @Mock
+    private ClockService clockService;
+
+    @Mock
+    private TxManager txManager;
+
+    @Mock
+    private ValidationSchemasSource validationSchemasSource;
+
+    @Mock
+    private SchemaSyncService schemaSyncService;
+
+    @Mock
+    private CatalogService catalogService;
+
+    @Mock
+    private RaftCommandRunner raftCommandRunner;
+
+    private TxFinishReplicaRequestHandler handler;
+
+    @BeforeEach
+    void setUp() {
+        handler = new TxFinishReplicaRequestHandler(
+                txStatePartitionStorage,
+                clockService,
+                txManager,
+                validationSchemasSource,
+                schemaSyncService,
+                catalogService,
+                raftCommandRunner,
+                replicationGroupId
+        );
+    }
+
+    @Test
+    void finishReturnsMismatchingOutcomeWhenCommitRequestedButAlreadyAborted() 
{
+        HybridTimestamp beginTimestamp = new HybridTimestamp(1, 1);
+        UUID txId = TransactionIds.transactionId(beginTimestamp, 1);
+        HybridTimestamp commitTimestamp = new HybridTimestamp(123, 456);
+
+        
when(schemaSyncService.waitForMetadataCompleteness(any())).thenReturn(completedFuture(null));
+        when(txStatePartitionStorage.get(txId)).thenReturn(new 
TxMeta(TxState.ABORTED, List.of(), null));
+
+        TxFinishReplicaRequest request = 
txMessagesFactory.txFinishReplicaRequest()
+                .commitTimestamp(commitTimestamp)
+                .groupId(toZonePartitionIdMessage(replicaMessagesFactory, 
replicationGroupId))
+                
.commitPartitionId(toZonePartitionIdMessage(replicaMessagesFactory, 
replicationGroupId))
+                .txId(txId)
+                .groups(Map.of(
+                        toZonePartitionIdMessage(replicaMessagesFactory, 
replicationGroupId),
+                        partitionEnlistmentMessage("node", Set.of())
+                ))
+                .commit(true)
+                .enlistmentConsistencyToken(ANY_ENLISTMENT_CONSISTENCY_TOKEN)
+                .build();
+
+        assertThat(handler.handle(request), 
willThrow(MismatchingTransactionOutcomeInternalException.class));
+
+        verify(txManager, never()).cleanup(any(), any(Map.class), 
anyBoolean(), any(), any());
+    }
+
+    @Test
+    void 
finishReturnsMismatchingOutcomeWhenAbortRequestedButAlreadyCommitted() {
+        UUID txId = UUID.randomUUID();
+        HybridTimestamp commitTimestamp = new HybridTimestamp(1, 1);
+
+        when(txStatePartitionStorage.get(txId)).thenReturn(new 
TxMeta(TxState.COMMITTED, List.of(), commitTimestamp));
+
+        TxFinishReplicaRequest request = 
txMessagesFactory.txFinishReplicaRequest()
+                .groupId(toZonePartitionIdMessage(replicaMessagesFactory, 
replicationGroupId))
+                
.commitPartitionId(toZonePartitionIdMessage(replicaMessagesFactory, 
replicationGroupId))
+                .txId(txId)
+                .groups(Map.of(
+                        toZonePartitionIdMessage(replicaMessagesFactory, 
replicationGroupId),
+                        partitionEnlistmentMessage("node", Set.of(1))
+                ))
+                .commit(false)
+                .enlistmentConsistencyToken(ANY_ENLISTMENT_CONSISTENCY_TOKEN)
+                .build();
+
+        assertThrows(MismatchingTransactionOutcomeInternalException.class, () 
-> handler.handle(request));
+
+        verify(txManager, never()).cleanup(any(), any(Map.class), 
anyBoolean(), any(), any());
+    }
+
+    private PartitionEnlistmentMessage partitionEnlistmentMessage(String 
primaryConsistentId, Set<Integer> tableIds) {
+        return txMessagesFactory.partitionEnlistmentMessage()
+                .primaryConsistentId(primaryConsistentId)
+                .tableIds(tableIds)
+                .build();
+    }
+}
+

Reply via email to