[
https://issues.apache.org/jira/browse/KAFKA-8803?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17022192#comment-17022192
]
Raman Gupta commented on KAFKA-8803:
------------------------------------
[~oleksii.boiko]'s last message inspired me to check for
`IllegalStateException` in our logs. I don't see the same error as he, but I do
see 12 of these errors on our `kafka-2` broker a few hours before the last
timeout error we experienced on Jan 15th – these errors always seem to occur on
stream restart. The `kafka-2` broker is the one which is the same broker
restarted just before the stream recovered.
{code:java}
[2020-01-15 09:27:08,689] ERROR Uncaught exception in scheduled task
'load-txns-for-partition-__transaction_state-22' (kafka.utils.KafkaScheduler)
java.lang.IllegalStateException: The metadata cache for txn partition 22 has
already exist with epoch 567 and 9 entries while trying to add to it; this
should not happen
at
kafka.coordinator.transaction.TransactionStateManager.addLoadedTransactionsToCache(TransactionStateManager.scala:369)
at
kafka.coordinator.transaction.TransactionStateManager.$anonfun$loadTransactionsForTxnTopicPartition$3(TransactionStateManager.scala:394)
at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:253)
at kafka.utils.CoreUtils$.inWriteLock(CoreUtils.scala:261)
at
kafka.coordinator.transaction.TransactionStateManager.loadTransactions$1(TransactionStateManager.scala:393)
at
kafka.coordinator.transaction.TransactionStateManager.$anonfun$loadTransactionsForTxnTopicPartition$7(TransactionStateManager.scala:426)
at kafka.utils.KafkaScheduler.$anonfun$schedule$2(KafkaScheduler.scala:114)
at kafka.utils.CoreUtils$$anon$1.run(CoreUtils.scala:65)
at
java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
at
java.base/java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:304)
at
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
at
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
at java.base/java.lang.Thread.run(Thread.java:834)
{code}
> 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)