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

Guozhang Wang commented on KAFKA-8803:
--------------------------------------

[~rocketraman] I've looked into the logs you shared, and here are some findings:

1. There are a total of 9 TimeoutExceptions from the client side, the first two 
are on task 3 and 7, while the rest of them are on task 36. Also the task 3 / 7 
failures seem not been handled with auto creation / resumption of the instance, 
while task 36's timeout are triggering auto creation / resumption and are 
migrating between thread 1, 2, and 3.

{code}
cis-55d8b78cf-zqr4q     2019-08-13 21:04:10,501 ERROR --- 
[540a9-StreamThread-1] org.apa.kaf.str.pro.int.StreamTask                : task 
[0_3] Timeout exception caught when initializing transactions for task 0_3.
cis-55d8b78cf-2ws25     2019-08-13 21:04:10,867 ERROR --- 
[111fd-StreamThread-1] org.apa.kaf.str.pro.int.StreamTask                : task 
[0_7] Timeout exception caught when initializing transactions for task 0_7.
cis-55d8b78cf-zqr4q     2019-08-13 21:06:01,930 ERROR --- 
[d950d-StreamThread-1] org.apa.kaf.str.pro.int.StreamTask                : task 
[0_36] Timeout exception caught when initializing transactions for task 0_36.
cis-55d8b78cf-zqr4q     2019-08-13 21:09:21,640 ERROR --- 
[daa53-StreamThread-1] org.apa.kaf.str.pro.int.StreamTask                : task 
[0_36] Timeout exception caught when initializing transactions for task 0_36.
cis-55d8b78cf-zqr4q     2019-08-13 21:11:19,263 ERROR --- 
[f4cfc-StreamThread-1] org.apa.kaf.str.pro.int.StreamTask                : task 
[0_36] Timeout exception caught when initializing transactions for task 0_36.
cis-55d8b78cf-2ws25     2019-08-13 21:13:27,471 ERROR --- 
[e2227-StreamThread-2] org.apa.kaf.str.pro.int.StreamTask                : task 
[0_36] Timeout exception caught when initializing transactions for task 0_36.
cis-55d8b78cf-zqr4q     2019-08-13 21:14:28,419 ERROR --- 
[249f0-StreamThread-2] org.apa.kaf.str.pro.int.StreamTask                : task 
[0_36] Timeout exception caught when initializing transactions for task 0_36.
cis-55d8b78cf-2ws25     2019-08-13 21:16:37,501 ERROR --- 
[a6e65-StreamThread-2] org.apa.kaf.str.pro.int.StreamTask                : task 
[0_36] Timeout exception caught when initializing transactions for task 0_36.
cis-55d8b78cf-zqr4q     2019-08-13 21:18:35,811 ERROR --- 
[ad405-StreamThread-3] org.apa.kaf.str.pro.int.StreamTask                : task 
[0_36] Timeout exception caught when initializing transactions for task 0_36.
{code}

2. The first two errors auto recovered later, as the txn coordinator 
successfully completed future initProducerId requests:

{code}
kafka-2 [2019-08-13 21:04:28,820] INFO [TransactionCoordinator id=2] 
Initialized transactionalId dev-cisFileIndexer-stream-0_7 with producerId 
121077 and producer epoch 21 on partition __transaction_state-18 
(kafka.coordinator.transaction.TransactionCoordinator)
kafka-2 [2019-08-13 21:04:29,448] INFO [TransactionCoordinator id=2] 
Initialized transactionalId dev-cisFileIndexer-stream-0_3 with producerId 
121071 and producer epoch 21 on partition __transaction_state-14 
(kafka.coordinator.transaction.TransactionCoordinator)
{code}

And although it is not directly deductionable from the logs, I believe it is 
because there are a bunch of topic-partitions (especially 
__transaction_state-XX partitions) and the broker is swamped handling those 
migrations and did not respond to the initProducerId requests in time, and 
hence the timeout exception, these failures successfully recovers as the log 
indicates.

3. For task 0_36 though, they never recovers, but the corresponding txn 
topic-partition, which is 21, is actually healthy on broker-1 and it is 
responsible for both task 81 and task 36, and we can see successful entries for 
task 81 before and after the period of task 36's timeout:

{code}
kafka-1 [2019-08-13 21:05:01,486] INFO [TransactionCoordinator id=1] 
Initialized transactionalId dev-cisSegmenter-stream-0_81 with producerId 120078 
and producer epoch 9 on partition __transaction_state-21 
(kafka.coordinator.transaction.TransactionCoordinator)
kafka-1 [2019-08-13 21:10:18,817] INFO [TransactionCoordinator id=1] 
Initialized transactionalId dev-cisSegmenter-stream-0_81 with producerId 120078 
and producer epoch 12 on partition __transaction_state-21 
(kafka.coordinator.transaction.TransactionCoordinator)
kafka-1 [2019-08-13 21:17:35,514] INFO [TransactionCoordinator id=1] 
Initialized transactionalId dev-cisSegmenter-stream-0_81 with producerId 120078 
and producer epoch 16 on partition __transaction_state-21 
(kafka.coordinator.transaction.TransactionCoordinator)
kafka-1 [2019-08-13 21:19:45,073] INFO [TransactionCoordinator id=1] 
Initialized transactionalId dev-cisSegmenter-stream-0_81 with producerId 120078 
and producer epoch 17 on partition __transaction_state-21 
(kafka.coordinator.transaction.TransactionCoordinator)
{code}

Since you mentioned after bouncing the brokers handles this, while at the same 
time the partition is healthy and can handle other tasks, I suspect it is 
blocked on finishing the current dangling txn for the same txn-id from "task 
0_36" as in {{handleEndTransaction}} which needs to call 
{{appendTransactionToLog}} to append the abort txn entry, but without more logs 
I cannot investigate further at this point.

I'm wondering which version of Kafka are you using on the broker side?

> Stream will not start due to TimeoutException: Timeout expired after 
> 60000milliseconds while awaiting InitProducerId
> --------------------------------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-8803
>                 URL: https://issues.apache.org/jira/browse/KAFKA-8803
>             Project: Kafka
>          Issue Type: Bug
>          Components: streams
>            Reporter: Raman Gupta
>            Assignee: Boyang Chen
>            Priority: Major
>         Attachments: logs.txt.gz, screenshot-1.png
>
>
> One streams app is consistently failing at startup with the following 
> exception:
> {code}
> 2019-08-14 17:02:29,568 ERROR --- [2ce1b-StreamThread-2] 
> org.apa.kaf.str.pro.int.StreamTask                : task [0_36] Timeout 
> exception caught when initializing transactions for task 0_36. This might 
> happen if the broker is slow to respond, if the network connection to the 
> broker was interrupted, or if similar circumstances arise. You can increase 
> producer parameter `max.block.ms` to increase this timeout.
> org.apache.kafka.common.errors.TimeoutException: Timeout expired after 
> 60000milliseconds while awaiting InitProducerId
> {code}
> These same brokers are used by many other streams without any issue, 
> including some in the very same processes for the stream which consistently 
> throws this exception.
> *UPDATE 08/16:*
> The very first instance of this error is August 13th 2019, 17:03:36.754 and 
> it happened for 4 different streams. For 3 of these streams, the error only 
> happened once, and then the stream recovered. For the 4th stream, the error 
> has continued to happen, and continues to happen now.
> I looked up the broker logs for this time, and see that at August 13th 2019, 
> 16:47:43, two of four brokers started reporting messages like this, for 
> multiple partitions:
> [2019-08-13 20:47:43,658] INFO [ReplicaFetcher replicaId=3, leaderId=1, 
> fetcherId=0] Retrying leaderEpoch request for partition xxx-1 as the leader 
> reported an error: UNKNOWN_LEADER_EPOCH (kafka.server.ReplicaFetcherThread)
> The UNKNOWN_LEADER_EPOCH messages continued for some time, and then stopped, 
> here is a view of the count of these messages over time:
>  !screenshot-1.png! 
> However, as noted, the stream task timeout error continues to happen.
> I use the static consumer group protocol with Kafka 2.3.0 clients and 2.3.0 
> broker. The broker has a patch for KAFKA-8773.



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

Reply via email to