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

Guozhang Wang commented on KAFKA-13375:
---------------------------------------

There are actually multiple known issues on the broker side and one of them is 
https://issues.apache.org/jira/browse/KAFKA-7408. But more frequently we found 
that a lot of the root causes is on clients (not java clients, but e.g. 
third-party clients, non-java buggy clients etc) that do not strictly follow 
the EOS protocol.

Here's our current thinking on further improving EOS design: The transaction 
protocol relies on well-behaved clients. Before writing transactional data to 
the partition, it is up to the client to ensure that the partition has been 
added to the transaction using the `AddPartitionsToTxn` API. The broker today 
has no way to ensure that the client does this properly which means that a 
buggy client could cause hanging transactions. And the fact that the client has 
to do this in the first place means the client implementation is more complex 
and likely to contain bugs.

We can consider moving the responsibility of adding the partition to the 
transaction to the partition leader. When the leader encounters the first 
append of a transaction from a given producer, it can itself send 
`AddPartitionsToTxn` to the transaction coordinator. Notably, the produce 
request already contains the transactionalId, so it is possible to find the 
coordinator efficiently. And the produce data contains the producer epoch, so 
we can still fence at the transaction coordinator. 

> Kafka streams apps w/EOS unable to start at InitProducerId
> ----------------------------------------------------------
>
>                 Key: KAFKA-13375
>                 URL: https://issues.apache.org/jira/browse/KAFKA-13375
>             Project: Kafka
>          Issue Type: Bug
>          Components: core
>    Affects Versions: 2.8.0
>            Reporter: Lerh Chuan Low
>            Priority: Major
>
> Hello, I'm wondering if this is a Kafka bug. Our environment setup is as 
> follows:
> Kafka streams 2.8 - with *EXACTLY_ONCE* turned on (Not *EOS_BETA*, but I 
> don't think the changes introduced in EOS beta should affect this). 
> Transaction timeout = 60s.
>  Kafka broker 2.8. 
> We have this situation where we were doing a rolling restart of the broker to 
> apply some security changes. After we finished, 4 out of some 15 Stream Apps 
> are unable to start. They can never succeed, no matter what we do. 
> They fail with the error:
> {code:java}
>  2021-10-14 07:20:13,548 WARN 
> [srn-rec-feeder-802c18a1-9512-4a2a-8c2e-00e37550199d-StreamThread-3] 
> o.a.k.s.p.i.StreamsProducer stream-thread 
> [srn-rec-feeder-802c18a1-9512-4a2a-8c2e-00e37550199d-StreamThread-3] task 
> [0_6] Timeout exception caught trying to initialize transactions. The broker 
> is either slow or in bad state (like not having enough replicas) in 
> responding to the request, or the connection to broker was interrupted 
> sending the request or receiving the response. Will retry initializing the 
> task in the next loop. Consider overwriting max.block.ms to a larger value to 
> avoid timeout errors{code}
> We found a previous Jira describing the issue here: 
> https://issues.apache.org/jira/browse/KAFKA-8803. It seems like back then 
> what people did was to rolling restart the brokers. We tried that - we 
> targeted the group coordinators for our failing apps, then transaction 
> coordinators, then all of them. It hasn't resolved our issue so far. 
> A few interesting things we've found so far:
>  - What I can see is that all the failing apps only fail on certain 
> partitions. E.g for the app above, only partition 6 never succeeds. Partition 
> 6 shares the same coordinator as some of the other partitions and those work, 
> so it seems like the issue isn't related to broker memory state. 
>  - All the failing apps have a message similar to this 
> {code:java}
> [2021-10-14 00:54:51,569] INFO [Transaction Marker Request Completion Handler 
> 103]: Sending srn-rec-feeder-0_6's transaction marker for partition 
> srn-bot-003-14 has permanently failed with error 
> org.apache.kafka.common.errors.InvalidProducerEpochException with the current 
> coordinator epoch 143; cancel sending any more transaction markers 
> TxnMarkerEntry{producerId=7001, producerEpoch=610, coordinatorEpoch=143, 
> result=ABORT, partitions=[srn-bot-003-14]} to the brokers 
> (kafka.coordinator.transaction.TransactionMarkerRequestCompletionHandler) 
> {code}
> While we were restarting the brokers. They all failed shortly after. No other 
> consumer groups for the other working partitions/working stream apps logged 
> this message. 
> On digging around in git blame and reading through the source, it looks like 
> this is meant to be benign. 
>  - We tried DEBUG logging for the TransactionCoordinator and 
> TransactionStateManager. We can see (assigner is a functioning app)
> {code:java}
> [2021-10-14 06:48:23,813] DEBUG [TransactionCoordinator id=105] Returning 
> CONCURRENT_TRANSACTIONS error code to client for srn-assigner-0_14's 
> AddPartitions request (kafka.coordinator.transaction.TransactionCoordinator) 
> {code}
> I've seen those before during steady state. I do believe they are benign. We 
> never see it for the problematic partitions/consumer groups for some reason.
>  - We tried turning on TRACE for KafkaApis. We can see
> {code:java}
> [2021-10-14 06:56:58,408] TRACE [KafkaApi-105] Completed srn-rec-feeder-0_6's 
> InitProducerIdRequest with result 
> InitProducerIdResult(-1,-1,CONCURRENT_TRANSACTIONS) from client 
> srn-rec-feeder-802c18a1-9512-4a2a-8c2e-00e37550199d-StreamThread-4-0_6-producer.
>  (kafka.server.KafkaApis) {code}
> It starts to make me wonder if there's a situation where Kafka is unable to 
> abort the transactions if there is never any success in initializing a 
> producer ID. But this is diving deep into insider knowledge territory that I 
> simply don't have. I'm wondering if anyone with more knowledge of how 
> transactions work can shed some light here if we are in the wrong path, or if 
> there's any way to restore operations at all short of a streams reset? From a 
> cursory look at
> {noformat}
> TransactionCoordinator#handleInitProducerId {noformat}
> it looks like any old transactions should just be aborted and life goes on, 
> but it's not happening. 
> I wonder if there's some corrupted state of the transaction Kafka log files. 
> I'm not sure if it's possible to edit them or how to resolve the issue though 
> if that's true. 
> Update:
> We temporarily managed to get our producers/consumers working again by 
> removing EOS and making them ALO (At least once). 
> After we had left it running for a bit (> 15 minutes), we decided to turn on 
> EOS again to see if the problem had gone away. It still failed with the exact 
> same error. 
> So we've turned it back to ALO. Not sure if that helps or gives a clue into 
> what may have gone awry.
> Thanks!



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

Reply via email to