[ 
https://issues.apache.org/jira/browse/IGNITE-21861?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Denis Chudov updated IGNITE-21861:
----------------------------------
    Description: 
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 state is displayed
 * in case of replication exception the tx id is present, if applicable.

  was:
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 state is displayed
 * in case of replication exception the tx id is present.


> 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 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)

Reply via email to