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

Greg Harris commented on KAFKA-14901:
-------------------------------------

cc [~jolshan] [~dajac] [~hachikuji] [~mimaison] 

I have not seen this failure mode before, and I'm worried that this might be a 
recent regression. It also doesn't look like an error that is intended to be 
surfaced by the API in normal operations (I might expect a disconnect or 
ProducerFencedException, maybe).
Just to be safe I've marked this as a blocker for 3.5.0, but we can downgrade 
the severity with some more investigation.

Since the incidence rate for this is relatively low, I've attached a couple of 
logs of the test run leading up to the error. The trace-0 log fails on 
PrepareEpochFence instead of Empty, but it appears both runs fail when the 
pending state is Ongoing.
To reproduce yourself: Run the version of test on trunk 10-50 times until it 
fails. This test is different from the other tests in the same suite as it 
makes use of active producer fencing to eliminate other tasks/producers when 
reconfiguring tasks.

> Flaky test ExactlyOnceSourceIntegrationTest.testConnectorReconfiguration
> ------------------------------------------------------------------------
>
>                 Key: KAFKA-14901
>                 URL: https://issues.apache.org/jira/browse/KAFKA-14901
>             Project: Kafka
>          Issue Type: Test
>          Components: KafkaConnect
>            Reporter: Greg Harris
>            Priority: Blocker
>              Labels: flaky-test
>             Fix For: 3.5.0
>
>         Attachments: transaction-flake-trace-0.out, transaction-flake.out
>
>
> The EOS Source test appears to be very rarely failing (<5% chance) with the 
> following error:
> {noformat}
> org.apache.kafka.common.KafkaException: Unexpected error in 
> InitProducerIdResponse; The server experienced an unexpected error when 
> processing the request.
>       at 
> app//org.apache.kafka.clients.producer.internals.TransactionManager$InitProducerIdHandler.handleResponse(TransactionManager.java:1303)
>       at 
> app//org.apache.kafka.clients.producer.internals.TransactionManager$TxnRequestHandler.onComplete(TransactionManager.java:1207)
>       at 
> app//org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:154)
>       at 
> app//org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:594)
>       at 
> app//org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:586)
>       at 
> app//org.apache.kafka.clients.producer.internals.Sender.maybeSendAndPollTransactionalRequest(Sender.java:426)
>       at 
> app//org.apache.kafka.clients.producer.internals.Sender.runOnce(Sender.java:316)
>       at 
> app//org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:243)
>       at java.base@11.0.16.1/java.lang.Thread.run(Thread.java:829){noformat}
> which appears to be triggered by the following failure inside the broker:
> {noformat}
> [2023-04-12 14:01:38,931] ERROR [KafkaApi-0] Unexpected error handling 
> request RequestHeader(apiKey=INIT_PRODUCER_ID, apiVersion=4, 
> clientId=simulated-task-producer-exactlyOnceQuestionMark-1, correlationId=5, 
> headerVersion=2) -- 
> InitProducerIdRequestData(transactionalId='exactly-once-source-integration-test-exactlyOnceQuestionMark-1',
>  transactionTimeoutMs=60000, producerId=-1, producerEpoch=-1) with context 
> RequestContext(header=RequestHeader(apiKey=INIT_PRODUCER_ID, apiVersion=4, 
> clientId=simulated-task-producer-exactlyOnceQuestionMark-1, correlationId=5, 
> headerVersion=2), connectionId='127.0.0.1:54213-127.0.0.1:54367-46', 
> clientAddress=/127.0.0.1, principal=User:ANONYMOUS, 
> listenerName=ListenerName(PLAINTEXT), securityProtocol=PLAINTEXT, 
> clientInformation=ClientInformation(softwareName=apache-kafka-java, 
> softwareVersion=3.5.0-SNAPSHOT), fromPrivilegedListener=true, 
> principalSerde=Optional[org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder@615924cd])
>  (kafka.server.KafkaApis:76)
> java.lang.IllegalStateException: Preparing transaction state transition to 
> Empty while it already a pending state Ongoing
>     at 
> kafka.coordinator.transaction.TransactionMetadata.prepareTransitionTo(TransactionMetadata.scala:380)
>     at 
> kafka.coordinator.transaction.TransactionMetadata.prepareIncrementProducerEpoch(TransactionMetadata.scala:311)
>     at 
> kafka.coordinator.transaction.TransactionCoordinator.prepareInitProducerIdTransit(TransactionCoordinator.scala:240)
>     at 
> kafka.coordinator.transaction.TransactionCoordinator.$anonfun$handleInitProducerId$3(TransactionCoordinator.scala:151)
>     at 
> kafka.coordinator.transaction.TransactionMetadata.inLock(TransactionMetadata.scala:242)
>     at 
> kafka.coordinator.transaction.TransactionCoordinator.$anonfun$handleInitProducerId$2(TransactionCoordinator.scala:150)
>     at scala.util.Either.flatMap(Either.scala:352)
>     at 
> kafka.coordinator.transaction.TransactionCoordinator.handleInitProducerId(TransactionCoordinator.scala:145)
>     at 
> kafka.server.KafkaApis.handleInitProducerIdRequest(KafkaApis.scala:2236)
>     at kafka.server.KafkaApis.handle(KafkaApis.scala:202)
>     at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:76)
>     at java.base/java.lang.Thread.run(Thread.java:829{noformat}



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

Reply via email to