You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "huxi (JIRA)" <ji...@apache.org> on 2017/01/23 08:24:26 UTC

[jira] [Commented] (KAFKA-4669) KafkaProducer.flush hangs when NetworkClient.handleCompletedReceives throws exception

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

huxi commented on KAFKA-4669:
-----------------------------

Maybe we could invoke an alternative method `ProduceRequestResult#await(long, TimeUnit)` and specify a reasonable timeout value in RecordAccumulator#awaitFlushCompletion. If await times out, we record a warn log to indicate users that some request did not get marked as complete, and flush will not be stuck anymore, as show below:

{code:borderStyle=solid}
public void awaitFlushCompletion() throws InterruptedException {
	try {
		for (RecordBatch batch : this.incomplete.all()) {
			if (!batch.produceFuture.await(30, TimeUnit.SECONDS)) {
				log.warn("Did not complete the produce request for {} in 30 seconds.", batch.produceFuture.topicPartition());
			}
		}
	} finally {
		this.flushesInProgress.decrementAndGet();
	}
}
{code}

[~becket_qin] Does it make sense?

> 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.10.2.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.4#6332)