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

Oleksii Boiko commented on KAFKA-8803:
--------------------------------------

Original exception full stacktrace(Broker 2.0.1)


{noformat}
[2020-01-02 03:03:06,262] ERROR [KafkaApi-0] Error when handling request 
{transactional_id=<id>,producer_id=61432,producer_epoch=0,group_id=<group-id>} 
(kafka.server.KafkaApis)java.lang.IllegalStateException: TransactionalId <id> 
failed transition to state TxnTransitMetadata(producerId=61432, 
producerEpoch=0, txnTimeoutMs=60000, txnState=Ongoing, 
topicPartitions=Set(__consumer_offsets-10), txnStartTimestamp=1577934186261, 
txnLastUpdateTimestamp=1577934186261) due to unexpected metadata at 
kafka.coordinator.transaction.TransactionMetadata.throwStateTransitionFailure(TransactionMetadata.scala:390)
 at 
kafka.coordinator.transaction.TransactionMetadata.completeTransitionTo(TransactionMetadata.scala:326)
 at 
kafka.coordinator.transaction.TransactionStateManager$$anonfun$kafka$coordinator$transaction$TransactionStateManager$$updateCacheCallback$1$1.apply$mcV$sp(TransactionStateManager.scala:542)
 at 
kafka.coordinator.transaction.TransactionStateManager$$anonfun$kafka$coordinator$transaction$TransactionStateManager$$updateCacheCallback$1$1.apply(TransactionStateManager.scala:534)
 at 
kafka.coordinator.transaction.TransactionStateManager$$anonfun$kafka$coordinator$transaction$TransactionStateManager$$updateCacheCallback$1$1.apply(TransactionStateManager.scala:534)
 at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:251) at 
kafka.coordinator.transaction.TransactionMetadata.inLock(TransactionMetadata.scala:172)
 at 
kafka.coordinator.transaction.TransactionStateManager.kafka$coordinator$transaction$TransactionStateManager$$updateCacheCallback$1(TransactionStateManager.scala:533)
 at 
kafka.coordinator.transaction.TransactionStateManager$$anonfun$appendTransactionToLog$1$$anonfun$apply$mcV$sp$11.apply(TransactionStateManager.scala:628)
 at 
kafka.coordinator.transaction.TransactionStateManager$$anonfun$appendTransactionToLog$1$$anonfun$apply$mcV$sp$11.apply(TransactionStateManager.scala:628)
 at kafka.server.DelayedProduce.onComplete(DelayedProduce.scala:129) at 
kafka.server.DelayedOperation.forceComplete(DelayedOperation.scala:70) at 
kafka.server.DelayedProduce.tryComplete(DelayedProduce.scala:110) at 
kafka.server.DelayedOperationPurgatory.tryCompleteElseWatch(DelayedOperation.scala:232)
 at kafka.server.ReplicaManager.appendRecords(ReplicaManager.scala:495) at 
kafka.coordinator.transaction.TransactionStateManager$$anonfun$appendTransactionToLog$1.apply$mcV$sp(TransactionStateManager.scala:622)
 at 
kafka.coordinator.transaction.TransactionStateManager$$anonfun$appendTransactionToLog$1.apply(TransactionStateManager.scala:599)
 at 
kafka.coordinator.transaction.TransactionStateManager$$anonfun$appendTransactionToLog$1.apply(TransactionStateManager.scala:599)
 at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:251) at 
kafka.utils.CoreUtils$.inReadLock(CoreUtils.scala:257) at 
kafka.coordinator.transaction.TransactionStateManager.appendTransactionToLog(TransactionStateManager.scala:593)
 at 
kafka.coordinator.transaction.TransactionCoordinator.handleAddPartitionsToTransaction(TransactionCoordinator.scala:272)
 at kafka.server.KafkaApis.handleAddOffsetsToTxnRequest(KafkaApis.scala:1852) 
at kafka.server.KafkaApis.handle(KafkaApis.scala:138) at 
kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:69) at 
java.lang.Thread.run(Thread.java:748)
{noformat}

> 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