You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ignite.apache.org by " Kirill Sizov (Jira)" <ji...@apache.org> on 2024/03/29 10:46:00 UTC

[jira] [Commented] (IGNITE-21861) Unexpected "Transaction is already finished" exception

    [ https://issues.apache.org/jira/browse/IGNITE-21861?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17832169#comment-17832169 ] 

 Kirill Sizov commented on IGNITE-21861:
----------------------------------------

Seems like a command was trying to execute on a finished transaction, hard to say COMMITTED or ABORTED.

> Unexpected "Transaction is already finished" exception 
> -------------------------------------------------------
>
>                 Key: IGNITE-21861
>                 URL: https://issues.apache.org/jira/browse/IGNITE-21861
>             Project: Ignite
>          Issue Type: Bug
>            Reporter: Denis Chudov
>            Priority: Major
>              Labels: ignite-3
>         Attachments: _Integration_Tests_Module_SQL_Engine_4133_.log
>
>
> Exception in log:
> {code:java}
> [2024-03-27T01:24:46,636][WARN ][%idt_n_1%partition-operations-4][ReplicaManager] Failed to process replica request [request=ReadWriteScanRetrieveBatchReplicaRequestImpl [batchSize=512, columnsToInclude=null, commitPartitionId=TablePartitionIdMessageImpl [partitionId=17, tableId=90], coordinatorId=125b397c-0404-4dcf-a28b-625fe010ecef, enlistmentConsistencyToken=112165039282455690, exactKey=null, flags=0, full=false, groupId=92_part_7, indexToUse=null, lowerBoundPrefix=null, scanId=20361, timestampLong=112165039967305730, transactionId=018e7d82-647b-0030-63a2-6a1900000001, upperBoundPrefix=null]].
> java.util.concurrent.CompletionException: org.apache.ignite.tx.TransactionException: IGN-TX-14 TraceId:6612dad8-4a32-4453-8af0-0139e336aad9 Transaction is already finished.
>     at java.base/java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:331) ~[?:?]
>     at java.base/java.util.concurrent.CompletableFuture.uniComposeStage(CompletableFuture.java:1099) ~[?:?]
>     at java.base/java.util.concurrent.CompletableFuture.thenCompose(CompletableFuture.java:2235) ~[?:?]
>     at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.processOperationRequest(PartitionReplicaListener.java:660) ~[ignite-table-3.0.0-SNAPSHOT.jar:?]
>     at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.processOperationRequestWithTxRwCounter(PartitionReplicaListener.java:3860) ~[ignite-table-3.0.0-SNAPSHOT.jar:?]
>     at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.lambda$processRequest$5(PartitionReplicaListener.java:436) ~[ignite-table-3.0.0-SNAPSHOT.jar:?]
>     at java.base/java.util.concurrent.CompletableFuture$UniCompose.tryFire(CompletableFuture.java:1072) ~[?:?]
>     at java.base/java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:506) [?:?]
>     at java.base/java.util.concurrent.CompletableFuture.postFire(CompletableFuture.java:610) [?:?]
>     at java.base/java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:649) [?:?]
>     at java.base/java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:478) [?:?]
>     at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) [?:?]
>     at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) [?:?]
>     at java.base/java.lang.Thread.run(Thread.java:834) [?:?]
> Caused by: org.apache.ignite.tx.TransactionException: Transaction is already finished.
>     at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.appendTxCommand(PartitionReplicaListener.java:1937) ~[ignite-table-3.0.0-SNAPSHOT.jar:?]
>     at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.processOperationRequest(PartitionReplicaListener.java:659) ~[ignite-table-3.0.0-SNAPSHOT.jar:?]
>     ... 10 more{code}
>  
> It happens in PartitionReplicaListener because the local volatile tx state is null or final when trying to compute a value for txCleanupReadyFutures map:
> {code:java}
> txCleanupReadyFutures.compute(txId, (id, txOps) -> {
>     // First check whether the transaction has already been finished.
>     // And complete cleanupReadyFut with exception if it is the case.
>     TxStateMeta txStateMeta = txManager.stateMeta(txId);
>     if (txStateMeta == null || isFinalState(txStateMeta.txState())) {
>         cleanupReadyFut.completeExceptionally(new Exception());
>         return txOps;
>     }
>     // Otherwise collect cleanupReadyFut in the transaction's futures.
>     if (txOps == null) {
>         txOps = new TxCleanupReadyFutureList();
>     }
>     txOps.futures.computeIfAbsent(cmdType, type -> new HashMap<>()).put(opId, cleanupReadyFut);
>     return txOps;
> });
> if (cleanupReadyFut.isCompletedExceptionally()) {
>     return failedFuture(new TransactionException(TX_ALREADY_FINISHED_ERR, "Transaction is already finished."));
> }{code}
> First problem is that we don't actually know the real state from this exception.
> The second one is the exception itself, because it shouldn't happen. We shouldn't meet a null state, because it's updated to pending just before, and it can be vacuumized only after it becomes final. 
> Committed state is also not possible because we wait for all in-flights before the state transition. It can be Aborted state here, but there should be no exception in logs in this case.
> In our case, the transaction is most likely aborted because of replication timeout exception happened before (it would be nice to see a tx id in this exception as well).
> Full log is attached.
> *Defitinion of done:*
>  * no TransactionException in log in case of aborted transaction (if the hypothesis is correct)
>  * in case of exception the real tx state is displayed
>  * in case of replication exception the tx id is present, if applicable.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)