You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "Tom Lee (Jira)" <ji...@apache.org> on 2019/10/14 01:03:00 UTC

[jira] [Commented] (KAFKA-8950) KafkaConsumer stops fetching

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

Tom Lee commented on KAFKA-8950:
--------------------------------

Believe we're also running into this. High throughput topics/partitions seem to be more frequently impacted than others. Like [~wtjames] we saw _nodesWithPendingFetchRequests_ indicate that a request was in-flight and we saw poll() regularly return an empty record set at our configured timeout interval. We were able to dig a little deeper via a heap dump and see there were no requests actually in-flight according to ConsumerNetworkClient/NetworkClient/Selector, nor were there any indications of pending/completed/failed/aborted/unsent requests at the time of the heap dump. inFlightRequestsCount was zero. The state of the fetcher's nodesWithPendingFetchRequests map just seems to disagree with the underlying ConsumerNetworkClient/NetworkClient/Selector. The Fetcher believes a fetch response (or timeout) will eventually come and it never does.

Perhaps worst of all the heartbeat/coordinator thread continued sending heartbeats etc. as normal so no rebalances occurred and the "stuck" partition was not released by the consumer. The state of the leader's broker connection was still READY. No broker  connect/reconnect activity as best I can tell from metrics. No errors in the logs or exceptions thrown as far as I could see. 

Given how simple the synchronization around the nodesWithPendingFetchRequests map is, it's as though we're somehow failing to invoke [this request listener|https://github.com/apache/kafka/blob/2.3.0/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java#L248-L249] or something like that, but the underlying networking code is tricky enough that it's difficult for me to speculate where. I've been eyeballing the _UnsentRequests_ data structure since it seems to be one of the few situations that we're manipulating state outside of coarse synchronized blocks in both the coordinator & consumer threads and it seems there's been bugs reported against it in the past, but so far I've come up empty.

I can imagine that an exception thrown at the wrong time/place could cause the RequestFuture listener not to be invoked, but I have no evidence of that actually happening.

> KafkaConsumer stops fetching
> ----------------------------
>
>                 Key: KAFKA-8950
>                 URL: https://issues.apache.org/jira/browse/KAFKA-8950
>             Project: Kafka
>          Issue Type: Bug
>          Components: clients
>    Affects Versions: 2.3.0
>         Environment: linux
>            Reporter: Will James
>            Priority: Major
>
> We have a KafkaConsumer consuming from a single partition with enable.auto.commit set to true.
> Very occasionally, the consumer goes into a broken state. It returns no records from the broker with every poll, and from most of the Kafka metrics in the consumer it looks like it is fully caught up to the end of the log. 
> We see that we are long polling for the max poll timeout, and that there is zero lag. In addition, we see that the heartbeat rate stays unchanged from before the issue begins (so the consumer stays a part of the consumer group).
> In addition, from looking at the __consumer_offsets topic, it is possible to see that the consumer is committing the same offset on the auto commit interval, however, the offset does not move, and the lag from the broker's perspective continues to increase.
> The issue is only resolved by restarting our application (which restarts the KafkaConsumer instance).
> From a heap dump of an application in this state, I can see that the Fetcher is in a state where it believes there are nodesWithPendingFetchRequests.
> However, I can see the state of the fetch latency sensor, specifically, the fetch rate, and see that the samples were not updated for a long period of time (actually, precisely the amount of time that the problem in our application was occurring, around 50 hours - we have alerting on other metrics but not the fetch rate, so we didn't notice the problem until a customer complained).
> In this example, the consumer was processing around 40 messages per second, with an average size of about 10kb, although most of the other examples of this have happened with higher volume (250 messages / second, around 23kb per message on average).
> I have spent some time investigating the issue on our end, and will continue to do so as time allows, however I wanted to raise this as an issue because it may be affecting other people.
> Please let me know if you have any questions or need additional information. I doubt I can provide heap dumps unfortunately, but I can provide further information as needed.



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