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

Lerh Chuan Low updated KAFKA-13375:
-----------------------------------
    Description: 
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 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 terriroty 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. 

  was:
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 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 terriroty 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. 


> 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 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 terriroty 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. 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to