[
https://issues.apache.org/jira/browse/IGNITE-27255?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Denis Chudov updated IGNITE-27255:
----------------------------------
Summary: TxIdMismatchException due to write attempt by RW transaction after
write intent resolution (was: TxIdMismatchException due to possible concurrent
write intent insertion after resolution before locking)
> TxIdMismatchException due to write attempt by RW transaction after write
> intent resolution
> ------------------------------------------------------------------------------------------
>
> Key: IGNITE-27255
> URL: https://issues.apache.org/jira/browse/IGNITE-27255
> Project: Ignite
> Issue Type: Bug
> Reporter: Denis Chudov
> Assignee: Denis Chudov
> Priority: Major
> Labels: ignite-3
> Time Spent: 10m
> Remaining Estimate: 0h
>
> This happens on the processing of single row request:
> {code:java}
> 2025-12-03 19:43:07:418 +0000
> [WARNING][%cac-dpd-cde-gg-aks-dev-2%partition-operations-3][ReplicaManager]
> Failed to process replica request
> [request=ReadWriteSingleRowReplicaRequestImpl
> [commitPartitionId=ZonePartitionIdMessageImpl [partitionId=19, zoneId=24],
> coordinatorId=7fa86716-8652-42df-a7b1-8e8b55f19cf9, delayedAckProcessor=null,
> enlistmentConsistencyToken=115657324018139159, full=false,
> groupId=ZonePartitionIdMessageImpl [partitionId=5, zoneId=27],
> requestType=RW_UPSERT, schemaVersion=1, skipDelayedAck=false, tableId=1049,
> timestamp=HybridTimestamp [physical=2025-12-03 19:42:45:567 +0000,
> logical=49, composite=115657340719398961],
> transactionId=019ae5bd-153f-0031-399b-326e00000001]].java.util.concurrent.CompletionException:
> org.apache.ignite.internal.storage.TxIdMismatchException: IGN-CMN-65535
> Mismatched transaction id [expectedTxId=019ae57c-1599-0079-399b-326e00000001,
> actualTxId=019ae5bd-153f-0031-399b-326e00000001] TraceId:437a68a4 at
> java.base/java.util.concurrent.CompletableFuture.encodeThrowable(Unknown
> Source) at
> java.base/java.util.concurrent.CompletableFuture.completeThrowable(Unknown
> Source) at
> java.base/java.util.concurrent.CompletableFuture$UniCompose.tryFire(Unknown
> Source) at
> java.base/java.util.concurrent.CompletableFuture.postComplete(Unknown Source)
> at java.base/java.util.concurrent.CompletableFuture.postFire(Unknown
> Source) at
> java.base/java.util.concurrent.CompletableFuture$UniApply.tryFire(Unknown
> Source) at
> java.base/java.util.concurrent.CompletableFuture$Completion.run(Unknown
> Source) at
> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source)
> at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown
> Source) at java.base/java.lang.Thread.run(Unknown Source)Caused by:
> org.apache.ignite.internal.storage.TxIdMismatchException: IGN-CMN-65535
> Mismatched transaction id [expectedTxId=019ae57c-1599-0079-399b-326e00000001,
> actualTxId=019ae5bd-153f-0031-399b-326e00000001] TraceId:437a68a4 at
> org.apache.ignite.internal.table.distributed.StorageUpdateHandler.performAddWriteWithCleanup(StorageUpdateHandler.java:484)
> at
> org.apache.ignite.internal.table.distributed.StorageUpdateHandler.tryProcessRow(StorageUpdateHandler.java:173)
> at
> org.apache.ignite.internal.table.distributed.StorageUpdateHandler.lambda$handleUpdate$0(StorageUpdateHandler.java:123)
> at
> org.apache.ignite.internal.storage.pagememory.mv.PersistentPageMemoryMvPartitionStorage.lambda$runConsistently$0(PersistentPageMemoryMvPartitionStorage.java:187)
> at
> org.apache.ignite.internal.storage.pagememory.mv.AbstractPageMemoryMvPartitionStorage.busy(AbstractPageMemoryMvPartitionStorage.java:1062)
> at
> org.apache.ignite.internal.storage.pagememory.mv.PersistentPageMemoryMvPartitionStorage.runConsistently(PersistentPageMemoryMvPartitionStorage.java:177)
> at
> org.apache.ignite.internal.table.distributed.raft.snapshot.SnapshotAwarePartitionDataStorage.runConsistently(SnapshotAwarePartitionDataStorage.java:84)
> at
> org.apache.ignite.internal.table.distributed.StorageUpdateHandler.handleUpdate(StorageUpdateHandler.java:120)
> at
> org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.applyUpdateCommand(PartitionReplicaListener.java:3275)
> at
> org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.applyUpdateCommand(PartitionReplicaListener.java:3347)
> at
> org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.lambda$processSingleEntryAction$160(PartitionReplicaListener.java:3664)
> ... 8 more {code}
> The code:
> {code:java}
> case RW_UPSERT: {
> return resolveRowByPk(extractPk(searchRow), txId, (rowId, row,
> lastCommitTime) -> {
> boolean insert = rowId == null;
> RowId rowId0 = insert ? new RowId(partId(), RowIdGenerator.next()) :
> rowId;
> CompletableFuture<IgniteBiTuple<RowId, Collection<Lock>>> lockFut =
> insert
> ? takeLocksForInsert(searchRow, rowId0, txId)
> : takeLocksForUpdate(searchRow, rowId0, txId);
> {code}
> Exception is thrown in thenCompose of lockFut.
> Scenario:
> * some row x is stored in a table, in a partition with primary replica on
> node-0
> * tx0 started by node-1, makes upsert on row x but doesn’t finish
> * primary replica for partition of row x is moved from node-0 to node-1
> * after write intent by tx0 is replicated to node-1, node-0 is stopped
> * tx1 tries to make upsert on row x and fails (coordinator of tx0 is alive
> and considers tx0 as pending, so write intent on node-1 is not resolved, and
> there is no lock, so tx1 tries to create new write intent and stumbles into
> the old one)
>
>
>
--
This message was sent by Atlassian Jira
(v8.20.10#820010)