[
https://issues.apache.org/jira/browse/KAFKA-4669?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16271727#comment-16271727
]
aarti gupta commented on KAFKA-4669:
------------------------------------
We saw this in production today
ERROR c.v.v.h.m.k.KafkaEventConsumerDelegate- Error fetching next record
java.lang.Exception: Error fetching next new record from kafka queue
at
com.vmware.vchs.hybridity.messaging.kafka.KafkaEventConsumerDelegate.getNextEventWithTimeout(KafkaEventConsumerDelegate.java:121)
at
com.vmware.vchs.hybridity.messaging.kafka.KafkaEventConsumerDelegate.getNextEvent(KafkaEventConsumerDelegate.java:64)
at
com.vmware.vchs.hybridity.messaging.adapter.EventListenerAdapter.getNextEvent(EventListenerAdapter.java:76)
at
com.vmware.vchs.hybridity.messaging.adapter.EventListenerAdapter.getNextEventAndAck(EventListenerAdapter.java:94)
at
com.vmware.vchs.hybridity.messaging.adapter.EventListenerAdapter$1.run(EventListenerAdapter.java:125)
at java.lang.Thread.run(Thread.java:748)
Caused by: java.lang.IllegalStateException: Correlation id for response
(386681) does not match request (386680), request header:
{api_key=9,api_version=3,correlation_id=386680,client_id=consumer-36}
at
org.apache.kafka.clients.NetworkClient.correlate(NetworkClient.java:752)
at
org.apache.kafka.clients.NetworkClient.parseStructMaybeUpdateThrottleTimeMetrics(NetworkClient.java:561)
at
org.apache.kafka.clients.NetworkClient.handleCompletedReceives(NetworkClient.java:657)
at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:442)
at
org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:232)
at
org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:208)
at
org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:168)
at
org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.fetchCommittedOffsets(ConsumerCoordinator.java:477)
at
org.apache.kafka.clients.consumer.KafkaConsumer.committed(KafkaConsumer.java:1346)
at
com.vmware.vchs.hybridity.messaging.kafka.KafkaEventConsumerDelegate.getNextEventWithTimeout(KafkaEventConsumerDelegate.java:86)
... 5 common frames omitted
> KafkaProducer.flush hangs when NetworkClient.handleCompletedReceives throws
> exception
> -------------------------------------------------------------------------------------
>
> Key: KAFKA-4669
> URL: https://issues.apache.org/jira/browse/KAFKA-4669
> Project: Kafka
> Issue Type: Bug
> Components: clients
> Affects Versions: 0.9.0.1
> Reporter: Cheng Ju
> Assignee: Rajini Sivaram
> Priority: Critical
> Labels: reliability
> Fix For: 0.11.0.1, 1.0.0
>
>
> There is no try catch in NetworkClient.handleCompletedReceives. If an
> exception is thrown after inFlightRequests.completeNext(source), then the
> corresponding RecordBatch's done will never get called, and
> KafkaProducer.flush will hang on this RecordBatch.
> I've checked 0.10 code and think this bug does exist in 0.10 versions.
> A real case. First a correlateId not match exception happens:
> 13 Jan 2017 17:08:24,059 ERROR [kafka-producer-network-thread | producer-21]
> (org.apache.kafka.clients.producer.internals.Sender.run:130) - Uncaught
> error in kafka producer I/O thread:
> java.lang.IllegalStateException: Correlation id for response (703766) does
> not match request (703764)
> at
> org.apache.kafka.clients.NetworkClient.correlate(NetworkClient.java:477)
> at
> org.apache.kafka.clients.NetworkClient.handleCompletedReceives(NetworkClient.java:440)
> at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:265)
> at
> org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:216)
> at
> org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:128)
> at java.lang.Thread.run(Thread.java:745)
> Then jstack shows the thread is hanging on:
> at java.util.concurrent.CountDownLatch.await(CountDownLatch.java:231)
> at
> org.apache.kafka.clients.producer.internals.ProduceRequestResult.await(ProduceRequestResult.java:57)
> at
> org.apache.kafka.clients.producer.internals.RecordAccumulator.awaitFlushCompletion(RecordAccumulator.java:425)
> at
> org.apache.kafka.clients.producer.KafkaProducer.flush(KafkaProducer.java:544)
> at org.apache.flume.sink.kafka.KafkaSink.process(KafkaSink.java:224)
> at
> org.apache.flume.sink.DefaultSinkProcessor.process(DefaultSinkProcessor.java:67)
> at org.apache.flume.SinkRunner$PollingRunner.run(SinkRunner.java:145)
> at java.lang.Thread.run(Thread.java:745)
> client code
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)