Is there a known workaround for this? I believe I'm also encountering this issue. We are being affected by a separate issue that causes the network to drop periodically. When this happens, my producer hangs for up to 10 minutes minutes. The retries work immediately. Also, this seems to only affect the connection to one of the brokers (and not the same one each time). During this period, messages are successfully produced to other brokers. The logs from this are below. I've only included the messages from one partition for brevity, but other partitions are also affected. Many thanks if someone could confirm if this is the same issue and/or provide a workaround.
Rafi 2015-08-16 13:46:29,397 TRACE [kafka-producer-network-thread | producer-1] o.a.k.c.p.internals.Sender : Created 1 produce requests: [ClientRequest(expectResponse=true, payload={slimhaag-118=RecordBatch(topicPartition=slimhaag-118, recordCount=1)}, request=RequestSend(header={api_key=0,api_version=0,correlation_id=35666,client_id=producer-1}, body={acks=-1,timeout=30000,topic_data=[{topic=slimhaag,data=[{partition=118,record_set=java.nio.HeapByteBuffer[pos=0 lim=308 cap=16384]}]}]}))] 2015-08-16 13:53:26,608 WARN [kafka-producer-network-thread | producer-1] o.a.k.common.network.Selector : Error in I/O with /172.25.103.251 java.io.IOException: Connection timed out at sun.nio.ch.FileDispatcherImpl.read0(Native Method) ~[na:1.8.0_51] at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:39) ~[na:1.8.0_51] at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223) ~[na:1.8.0_51] at sun.nio.ch.IOUtil.read(IOUtil.java:197) ~[na:1.8.0_51] at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:380) ~[na:1.8.0_51] at org.apache.kafka.common.network.NetworkReceive.readFrom(NetworkReceive.java:60 at org.apache.kafka.common.network.Selector.poll(Selector.java:248) at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:192) at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:191) at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:122) at java.lang.Thread.run(Thread.java:745) [na:1.8.0_51] 2015-08-16 13:53:26,608 DEBUG [kafka-producer-network-thread | producer-1] o.a.k.clients.NetworkClient : Node 1073741824 disconnected. 2015-08-16 13:53:26,608 TRACE [kafka-producer-network-thread | producer-1] o.a.k.clients.NetworkClient : Cancelled request ClientRequest(expectResponse=true, payload={slimhaag-118=RecordBatch(topicPartition=slimhaag-118, recordCount=1)}, request=RequestSend(header={api_key=0,api_version=0,correlation_id=35666,client_id=producer-1}, body={acks=-1,timeout=30000,topic_data=[{topic=slimhaag,data=[{partition=118,record_set=java.nio.HeapByteBuffer[pos=0 lim=308 cap=16384]}]}]})) due to node 1073741824 being disconnected 2015-08-16 13:53:26,608 TRACE [kafka-producer-network-thread | producer-1] o.a.k.c.p.internals.Sender : Cancelled request ClientResponse(received=1439733197983, disconnected=true, request=ClientRequest(expectResponse=true, payload={slimhaag-118=RecordBatch(topicPartition=slimhaag-118, recordCount=1)}, request=RequestSend(header={api_key=0,api_version=0,correlation_id=35666,client_id=producer-1}, body={acks=-1,timeout=30000,topic_data=[{topic=slimhaag,data=[{partition=118,record_set=java.nio.HeapByteBuffer[pos=0 lim=308 cap=16384]}]}]})), responseBody=null) due to node 1073741824 being disconnected 2015-08-16 13:53:26,608 WARN [kafka-producer-network-thread | producer-1] o.a.k.c.p.internals.Sender : Got error produce response with correlation id 35666 on topic-partition slimhaag-118, retrying (2 attempts left). Error: NETWORK_EXCEPTION Rafi On Tue, Jul 21, 2015 at 12:17 PM, Ewen Cheslack-Postava <e...@confluent.io> wrote: > This is a known issue. There are a few relevant JIRAs and a KIP: > > https://issues.apache.org/jira/browse/KAFKA-1788 > https://issues.apache.org/jira/browse/KAFKA-2120 > https://cwiki.apache.org/confluence/display/KAFKA/KIP-19+-+Add+a+request+timeout+to+NetworkClient > > -Ewen > > On Tue, Jul 21, 2015 at 7:05 AM, Stevo Slavić <ssla...@gmail.com> wrote: > >> Hello Apache Kafka community, >> >> Just noticed that : >> - message is successfully published using new 0.8.2.1 producer >> - and then Kafka is stopped >> >> next attempt to publish message using same instance of new producer hangs >> forever, and following stacktrace gets logged repeatedly: >> >> [WARN ] [o.a.kafka.common.network.Selector] [] Error in I/O with localhost/ >> 127.0.0.1 >> java.net.ConnectException: Connection refused >> at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method) ~[na:1.8.0_31] >> at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:716) >> ~[na:1.8.0_31] >> at org.apache.kafka.common.network.Selector.poll(Selector.java:238) >> ~[kafka-clients-0.8.2.1.jar:na] >> at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:192) >> [kafka-clients-0.8.2.1.jar:na] >> at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:191) >> [kafka-clients-0.8.2.1.jar:na] >> at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:122) >> [kafka-clients-0.8.2.1.jar:na] >> at java.lang.Thread.run(Thread.java:745) [na:1.8.0_31] >> >> >> I expect producer to respect timeout settings even in this connection lost >> scenario. >> >> Is this a known bug? Is there something I can do/configure as a workaround? >> >> Kind regards, >> Stevo Slavic. >> > > > > -- > Thanks, > Ewen