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

Nick Travers commented on KAFKA-4669:
-------------------------------------

[~rsivaram] [~hachikuji] - I did some digging on the broker side. While we 
usually see a steady stream of exceptions in the broker logs due to oversized 
messages, here's another exception from around the time we say the exception in 
the producer client.

There were over 100 of these exception on the same broker network thread within 
the same second. A few milliseconds after this, the exception was seen in the 
producer client. Hard to say definitively on the ordering, given clock skew 
between the different broker and client hosts.

{code}
2017-11-28 09:18:20,599 
kafka-network-thread-28-ListenerName(PLAINTEXT)-PLAINTEXT-4 Processor got 
uncaught exception.
java.nio.BufferUnderflowException
        at java.nio.Buffer.nextGetIndex(Buffer.java:506)
        at java.nio.HeapByteBuffer.getShort(HeapByteBuffer.java:310)
        at kafka.network.RequestChannel$Request.<init>(RequestChannel.scala:70)
        at 
kafka.network.Processor$$anonfun$processCompletedReceives$1.apply(SocketServer.scala:518)
        at 
kafka.network.Processor$$anonfun$processCompletedReceives$1.apply(SocketServer.scala:511)
        at scala.collection.Iterator$class.foreach(Iterator.scala:893)
        at scala.collection.AbstractIterator.foreach(Iterator.scala:1336)
        at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
        at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
        at 
kafka.network.Processor.processCompletedReceives(SocketServer.scala:511)
        at kafka.network.Processor.run(SocketServer.scala:436)
        at java.lang.Thread.run(Thread.java:748)
{code}

Here's an example of the oversized message exceptions in the broker:

{code}
2017-11-28 09:18:12,379 
kafka-network-thread-28-ListenerName(PLAINTEXT)-PLAINTEXT-4 Unexpected error 
from /10.4.12.70; closing connection
org.apache.kafka.common.network.InvalidReceiveException: Invalid receive (size 
= 335544320 larger than 104857600)
        at 
org.apache.kafka.common.network.NetworkReceive.readFromReadableChannel(NetworkReceive.java:95)
        at 
org.apache.kafka.common.network.NetworkReceive.readFrom(NetworkReceive.java:75)
        at 
org.apache.kafka.common.network.KafkaChannel.receive(KafkaChannel.java:203)
        at 
org.apache.kafka.common.network.KafkaChannel.read(KafkaChannel.java:167)
        at 
org.apache.kafka.common.network.Selector.pollSelectionKeys(Selector.java:381)
        at org.apache.kafka.common.network.Selector.poll(Selector.java:326)
        at kafka.network.Processor.poll(SocketServer.scala:500)
        at kafka.network.Processor.run(SocketServer.scala:435)
        at java.lang.Thread.run(Thread.java:748)
{code}

> 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)

Reply via email to