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

Edoardo Comar commented on KAFKA-4669:
--------------------------------------

We have found a strong correlation between the clients getting 

{code}
Uncaught error in kafka producer I/O thread: 
java.lang.IllegalStateException: Correlation id for response (703766) does not 
match request (703764)
{code}

and an NPE in one of our 10.0.1 brokers
{code}
[2017-03-06 17:46:29,827] ERROR Processor got uncaught exception. 
(kafka.network.Processor)
java.lang.NullPointerException
    at 
kafka.network.Processor$$anonfun$processCompletedReceives$1.apply(SocketServer.scala:486)
    at 
kafka.network.Processor$$anonfun$processCompletedReceives$1.apply(SocketServer.scala:483)
    at scala.collection.Iterator$class.foreach(Iterator.scala:742)
    at scala.collection.AbstractIterator.foreach(Iterator.scala:1194)
    at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
    at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
    at kafka.network.Processor.processCompletedReceives(SocketServer.scala:483)
    at kafka.network.Processor.run(SocketServer.scala:413)
    at java.lang.Thread.run(Thread.java:809)
{code}

that suggest that somehow 
{code}
  private def processCompletedReceives() {
    selector.completedReceives.asScala.foreach { receive =>
      try {
        val channel = selector.channel(receive.source)
        val session = RequestChannel.Session(new 
KafkaPrincipal(KafkaPrincipal.USER_TYPE, channel.principal.getName),
        //NPE if channel is null
...
{code}

[~ijuma] the only clients that are getting the occasional IllegalStateException 
are the ones producing to a partition that has as leader a broker where that 
NPE is appearing in our logs.

> 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
>            Priority: Critical
>              Labels: reliability
>             Fix For: 0.11.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.3.15#6346)

Reply via email to