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

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

This specific bug was new in 2.3.0, yep. IIRC the map that would get into an 
inconsistent state and cause all the trouble was introduced in 2.3.0. Looking 
at 0.10.2.2 specifically, I think the code as implemented would have been fine.

If it's helpful, we've also had various issues with earlier versions of the 
client libs too but more often than not the issues were mitigated by config, 
upgrading client libs or system-level tuning: sysctls like tcp_retries2 & 
tcp_syn_retries are often set too high, misconfigured NICs can be an issue 
because of packet loss, stuff like that.

Request timeouts for consumers in ~0.10 were extremely high because of some 
sort of coupling with group rebalances and iirc this didn't get fixed until 
2.0.0. See 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-266%3A+Fix+consumer+indefinite+blocking+behavior.
 Producers had similar but different issues with certain configuration options. 
Some of this was more difficult to work around directly without the 2.3.x 
upgrade.

Not to say there are no more issues, but a custom build of 2.3.0 with Will's 
patch has been solid for us so far. By comparison, "vanilla" 2.3.0 would cause 
us trouble maybe once or twice a day.

> 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
>             Fix For: 2.4.0, 2.3.1
>
>
> 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)

Reply via email to