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

Justine Olshan commented on KAFKA-14830:
----------------------------------------

So the race is likely between the sender handling produce calls and the 
transaction manager that is handling the other transactional apis?

 

My understanding is that an ideal client should be flushing/completing all 
produce requests before the abort/commit request is sent out. So while I can 
imagine maybe a different transactional api (ie, offset commit/add partitions) 
could race with a produce request, it would be surprising if a produce response 
came in after the abort/commit is sent. I know previously we had concerns about 
a late response coming in, but my understanding is that previous change 
prevented us from failing the batch twice. We only fail if completed 
exceptionally. See changes here: 
[https://github.com/apache/kafka/blame/f61719f96262d1dd36c99d49861c75ed39c964cb/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java#L831]



I will also take a look at the PR.

> Illegal state error in transactional producer
> ---------------------------------------------
>
>                 Key: KAFKA-14830
>                 URL: https://issues.apache.org/jira/browse/KAFKA-14830
>             Project: Kafka
>          Issue Type: Bug
>          Components: clients, producer 
>    Affects Versions: 3.1.2
>            Reporter: Jason Gustafson
>            Assignee: Kirk True
>            Priority: Critical
>              Labels: transactions
>             Fix For: 4.0.0
>
>
> We have seen the following illegal state error in the producer:
> {code:java}
> [Producer clientId=client-id2, transactionalId=transactional-id] Transiting 
> to abortable error state due to 
> org.apache.kafka.common.errors.TimeoutException: Expiring 1 record(s) for 
> topic-0:120027 ms has passed since batch creation
> [Producer clientId=client-id2, transactionalId=transactional-id] Transiting 
> to abortable error state due to 
> org.apache.kafka.common.errors.TimeoutException: Expiring 1 record(s) for 
> topic-1:120026 ms has passed since batch creation
> [Producer clientId=client-id2, transactionalId=transactional-id] Aborting 
> incomplete transaction
> [Producer clientId=client-id2, transactionalId=transactional-id] Invoking 
> InitProducerId with current producer ID and epoch 
> ProducerIdAndEpoch(producerId=191799, epoch=0) in order to bump the epoch
> [Producer clientId=client-id2, transactionalId=transactional-id] ProducerId 
> set to 191799 with epoch 1
> [Producer clientId=client-id2, transactionalId=transactional-id] Transiting 
> to abortable error state due to 
> org.apache.kafka.common.errors.NetworkException: Disconnected from node 4
> [Producer clientId=client-id2, transactionalId=transactional-id] Transiting 
> to abortable error state due to 
> org.apache.kafka.common.errors.TimeoutException: The request timed out.
> [Producer clientId=client-id2, transactionalId=transactional-id] Uncaught 
> error in request completion:
> java.lang.IllegalStateException: TransactionalId transactional-id: Invalid 
> transition attempted from state READY to state ABORTABLE_ERROR
>         at 
> org.apache.kafka.clients.producer.internals.TransactionManager.transitionTo(TransactionManager.java:1089)
>         at 
> org.apache.kafka.clients.producer.internals.TransactionManager.transitionToAbortableError(TransactionManager.java:508)
>         at 
> org.apache.kafka.clients.producer.internals.TransactionManager.maybeTransitionToErrorState(TransactionManager.java:734)
>         at 
> org.apache.kafka.clients.producer.internals.TransactionManager.handleFailedBatch(TransactionManager.java:739)
>         at 
> org.apache.kafka.clients.producer.internals.Sender.failBatch(Sender.java:753)
>         at 
> org.apache.kafka.clients.producer.internals.Sender.failBatch(Sender.java:743)
>         at 
> org.apache.kafka.clients.producer.internals.Sender.failBatch(Sender.java:695)
>         at 
> org.apache.kafka.clients.producer.internals.Sender.completeBatch(Sender.java:634)
>         at 
> org.apache.kafka.clients.producer.internals.Sender.lambda$null$1(Sender.java:575)
>         at java.base/java.util.ArrayList.forEach(ArrayList.java:1541)
>         at 
> org.apache.kafka.clients.producer.internals.Sender.lambda$handleProduceResponse$2(Sender.java:562)
>         at java.base/java.lang.Iterable.forEach(Iterable.java:75)
>         at 
> org.apache.kafka.clients.producer.internals.Sender.handleProduceResponse(Sender.java:562)
>         at 
> org.apache.kafka.clients.producer.internals.Sender.lambda$sendProduceRequest$5(Sender.java:836)
>         at 
> org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:109)
>         at 
> org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:583)
>         at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:575)
>         at 
> org.apache.kafka.clients.producer.internals.Sender.runOnce(Sender.java:328)
>         at 
> org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:243)
>         at java.base/java.lang.Thread.run(Thread.java:829)
>  {code}
> The producer hits timeouts which cause it to abort an active transaction. 
> After aborting, the producer bumps its epoch, which transitions it back to 
> the `READY` state. Following this, there are two errors for inflight 
> requests, which cause an illegal state transition to `ABORTABLE_ERROR`. But 
> how could the transaction ABORT complete if there were still inflight 
> requests? 



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

Reply via email to