[jira] [Comment Edited] (KAFKA-3990) Kafka New Producer may raise an OutOfMemoryError

2016-07-29 Thread Brice Dutheil (JIRA)

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

Brice Dutheil edited comment on KAFKA-3990 at 7/29/16 4:04 PM:
---

Hi after further investigation we found out the issue came because we switched 
from bamboo to marathon-lb, and marathon-lb opens the 9091 HTTP port 
(https://github.com/mesosphere/marathon-lb#operational-best-practices), we 
missed that during the upgrade.

Doing a curl on this port supposed to be configured for a kafka broker we got 
an HTTP response : 

{code}
> curl -v dockerhost:9091
* About to connect() to dockerhost port 9091 (#0)
*   Trying 172.17.42.1...
* Connected to dockerhost (172.17.42.1) port 9091 (#0)
> GET / HTTP/1.1
> User-Agent: curl/7.29.0
> Host: dockerhost:9091
> Accept: */*
>
* Empty reply from server
* Connection #0 to host dockerhost left intact
curl: (52) Empty reply from server
{code}

*However I'm surprised Kafka / clients don't check the validity of the payload, 
at least upon establishment of the connection.*


was (Author: bric3):
Hi after further investigation we found out the issue came because we switched 
from bamboo to marathon-lb, and marathon-lb opens the 9091 HTTP port 
(https://github.com/mesosphere/marathon-lb#operational-best-practices), we 
missed that during the upgrade.

Doing a curl on this port supposed to be configured for a kafka broker we got 
an HTTP response : 

{code}
> curl -v dockerhost:9091
* About to connect() to dockerhost port 9091 (#0)
*   Trying 172.17.42.1...
* Connected to dockerhost (172.17.42.1) port 9091 (#0)
> GET / HTTP/1.1
> User-Agent: curl/7.29.0
> Host: dockerhost:9091
> Accept: */*
>
* Empty reply from server
* Connection #0 to host dockerhost left intact
curl: (52) Empty reply from server
{code}

However I'm surprised Kafka / clients don't check the validity of the payload.

> Kafka New Producer may raise an OutOfMemoryError
> 
>
> Key: KAFKA-3990
> URL: https://issues.apache.org/jira/browse/KAFKA-3990
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Affects Versions: 0.9.0.1
> Environment: Docker, Base image : CentOS
> Java 8u77
>Reporter: Brice Dutheil
> Attachments: app-producer-config.log, kafka-broker-logs.zip
>
>
> We are regularly seeing OOME errors on a kafka producer, we first saw :
> {code}
> java.lang.OutOfMemoryError: Java heap space
> at java.nio.HeapByteBuffer.(HeapByteBuffer.java:57) ~[na:1.8.0_77]
> at java.nio.ByteBuffer.allocate(ByteBuffer.java:335) ~[na:1.8.0_77]
> at 
> org.apache.kafka.common.network.NetworkReceive.readFromReadableChannel(NetworkReceive.java:93)
>  ~[kafka-clients-0.9.0.1.jar:na]
> at 
> org.apache.kafka.common.network.NetworkReceive.readFrom(NetworkReceive.java:71)
>  ~[kafka-clients-0.9.0.1.jar:na]
> at 
> org.apache.kafka.common.network.KafkaChannel.receive(KafkaChannel.java:153) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at 
> org.apache.kafka.common.network.KafkaChannel.read(KafkaChannel.java:134) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at org.apache.kafka.common.network.Selector.poll(Selector.java:286) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:256) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at 
> org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:216) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at 
> org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:128) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_77]
> {code}
> This line refer to a buffer allocation {{ByteBuffer.allocate(receiveSize)}} 
> (see 
> https://github.com/apache/kafka/blob/0.9.0.1/clients/src/main/java/org/apache/kafka/common/network/NetworkReceive.java#L93)
> Usually the app runs fine within 200/400 MB heap and a 64 MB Metaspace. And 
> we are producing small messages 500B at most.
> Also the error don't appear on the devlopment environment, in order to 
> identify the issue we tweaked the code to give us actual data of the 
> allocation size, we got this stack :
> {code}
> 09:55:49.484 [auth] [kafka-producer-network-thread | producer-1] WARN  
> o.a.k.c.n.NetworkReceive HEAP-ISSUE: constructor : Integer='-1', String='-1'
> 09:55:49.485 [auth] [kafka-producer-network-thread | producer-1] WARN  
> o.a.k.c.n.NetworkReceive HEAP-ISSUE: method : 
> NetworkReceive.readFromReadableChannel.receiveSize=1213486160
> java.lang.OutOfMemoryError: Java heap space
> Dumping heap to /tmp/tomcat.hprof ...
> Heap dump file created [69583827 bytes in 0.365 secs]
> 09:55:50.324 [auth] [kafka-producer-network-thread | producer-1] ERROR 
> o.a.k.c.utils.KafkaThread Uncaught exception in kafka-producer-network-thread 
> | producer-1: 
> 

[jira] [Updated] (KAFKA-3990) Kafka New Producer may raise an OutOfMemoryError

2016-07-29 Thread Brice Dutheil (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-3990?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Brice Dutheil updated KAFKA-3990:
-
Environment: 
Docker, Base image : CentOS
Java 8u77
Marathon

  was:
Docker, Base image : CentOS
Java 8u77


> Kafka New Producer may raise an OutOfMemoryError
> 
>
> Key: KAFKA-3990
> URL: https://issues.apache.org/jira/browse/KAFKA-3990
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Affects Versions: 0.9.0.1
> Environment: Docker, Base image : CentOS
> Java 8u77
> Marathon
>Reporter: Brice Dutheil
> Attachments: app-producer-config.log, kafka-broker-logs.zip
>
>
> We are regularly seeing OOME errors on a kafka producer, we first saw :
> {code}
> java.lang.OutOfMemoryError: Java heap space
> at java.nio.HeapByteBuffer.(HeapByteBuffer.java:57) ~[na:1.8.0_77]
> at java.nio.ByteBuffer.allocate(ByteBuffer.java:335) ~[na:1.8.0_77]
> at 
> org.apache.kafka.common.network.NetworkReceive.readFromReadableChannel(NetworkReceive.java:93)
>  ~[kafka-clients-0.9.0.1.jar:na]
> at 
> org.apache.kafka.common.network.NetworkReceive.readFrom(NetworkReceive.java:71)
>  ~[kafka-clients-0.9.0.1.jar:na]
> at 
> org.apache.kafka.common.network.KafkaChannel.receive(KafkaChannel.java:153) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at 
> org.apache.kafka.common.network.KafkaChannel.read(KafkaChannel.java:134) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at org.apache.kafka.common.network.Selector.poll(Selector.java:286) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:256) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at 
> org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:216) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at 
> org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:128) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_77]
> {code}
> This line refer to a buffer allocation {{ByteBuffer.allocate(receiveSize)}} 
> (see 
> https://github.com/apache/kafka/blob/0.9.0.1/clients/src/main/java/org/apache/kafka/common/network/NetworkReceive.java#L93)
> Usually the app runs fine within 200/400 MB heap and a 64 MB Metaspace. And 
> we are producing small messages 500B at most.
> Also the error don't appear on the devlopment environment, in order to 
> identify the issue we tweaked the code to give us actual data of the 
> allocation size, we got this stack :
> {code}
> 09:55:49.484 [auth] [kafka-producer-network-thread | producer-1] WARN  
> o.a.k.c.n.NetworkReceive HEAP-ISSUE: constructor : Integer='-1', String='-1'
> 09:55:49.485 [auth] [kafka-producer-network-thread | producer-1] WARN  
> o.a.k.c.n.NetworkReceive HEAP-ISSUE: method : 
> NetworkReceive.readFromReadableChannel.receiveSize=1213486160
> java.lang.OutOfMemoryError: Java heap space
> Dumping heap to /tmp/tomcat.hprof ...
> Heap dump file created [69583827 bytes in 0.365 secs]
> 09:55:50.324 [auth] [kafka-producer-network-thread | producer-1] ERROR 
> o.a.k.c.utils.KafkaThread Uncaught exception in kafka-producer-network-thread 
> | producer-1: 
> java.lang.OutOfMemoryError: Java heap space
>   at java.nio.HeapByteBuffer.(HeapByteBuffer.java:57) ~[na:1.8.0_77]
>   at java.nio.ByteBuffer.allocate(ByteBuffer.java:335) ~[na:1.8.0_77]
>   at 
> org.apache.kafka.common.network.NetworkReceive.readFromReadableChannel(NetworkReceive.java:93)
>  ~[kafka-clients-0.9.0.1.jar:na]
>   at 
> org.apache.kafka.common.network.NetworkReceive.readFrom(NetworkReceive.java:71)
>  ~[kafka-clients-0.9.0.1.jar:na]
>   at 
> org.apache.kafka.common.network.KafkaChannel.receive(KafkaChannel.java:153) 
> ~[kafka-clients-0.9.0.1.jar:na]
>   at org.apache.kafka.common.network.KafkaChannel.read(KafkaChannel.java:134) 
> ~[kafka-clients-0.9.0.1.jar:na]
>   at org.apache.kafka.common.network.Selector.poll(Selector.java:286) 
> ~[kafka-clients-0.9.0.1.jar:na]
>   at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:256) 
> ~[kafka-clients-0.9.0.1.jar:na]
>   at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:216) 
> ~[kafka-clients-0.9.0.1.jar:na]
>   at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:128) 
> ~[kafka-clients-0.9.0.1.jar:na]
>   at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_77]
> {code}
> Notice the size to allocate {{1213486160}} ~1.2 GB. I'm not yet sure how this 
> size is initialised.
> Notice as well that every time this OOME appear the {{NetworkReceive}} 
> constructor at 
> https://github.com/apache/kafka/blob/0.9.0.1/clients/src/main/java/org/apache/kafka/common/network/NetworkReceive.java#L49
>  receive the parameters : {{maxSize=-1}}, {{source="-1"}}
> We may have missed configuration in our setup but kafka clients shouldn't 
> raise 

[jira] [Comment Edited] (KAFKA-3990) Kafka New Producer may raise an OutOfMemoryError

2016-07-29 Thread Brice Dutheil (JIRA)

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

Brice Dutheil edited comment on KAFKA-3990 at 7/29/16 4:03 PM:
---

Hi after further investigation we found out the issue came because we switched 
from bamboo to marathon-lb, and marathon-lb opens the 9091 HTTP port 
(https://github.com/mesosphere/marathon-lb#operational-best-practices), we 
missed that during the upgrade.

Doing a curl on this port supposed to be configured for a kafka broker we got 
an HTTP response : 

{code}
> curl -v dockerhost:9091
* About to connect() to dockerhost port 9091 (#0)
*   Trying 172.17.42.1...
* Connected to dockerhost (172.17.42.1) port 9091 (#0)
> GET / HTTP/1.1
> User-Agent: curl/7.29.0
> Host: dockerhost:9091
> Accept: */*
>
* Empty reply from server
* Connection #0 to host dockerhost left intact
curl: (52) Empty reply from server
{code}

However I'm surprised Kafka / clients don't check the validity of the payload.


was (Author: bric3):
Hi after further investigation we found out the issue came because we switched 
from bamboo to marathon-lb, and marathon-lb opens the 9091 HTTP port 
(https://github.com/mesosphere/marathon-lb#operational-best-practices), we 
missed that during the upgrade.

{code}
> curl -v dockerhost:9091
* About to connect() to dockerhost port 9091 (#0)
*   Trying 172.17.42.1...
* Connected to dockerhost (172.17.42.1) port 9091 (#0)
> GET / HTTP/1.1
> User-Agent: curl/7.29.0
> Host: dockerhost:9091
> Accept: */*
>
* Empty reply from server
* Connection #0 to host dockerhost left intact
curl: (52) Empty reply from server
{code}

However I'm surprised Kafka / clients don't check the validity of the payload.

> Kafka New Producer may raise an OutOfMemoryError
> 
>
> Key: KAFKA-3990
> URL: https://issues.apache.org/jira/browse/KAFKA-3990
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Affects Versions: 0.9.0.1
> Environment: Docker, Base image : CentOS
> Java 8u77
>Reporter: Brice Dutheil
> Attachments: app-producer-config.log, kafka-broker-logs.zip
>
>
> We are regularly seeing OOME errors on a kafka producer, we first saw :
> {code}
> java.lang.OutOfMemoryError: Java heap space
> at java.nio.HeapByteBuffer.(HeapByteBuffer.java:57) ~[na:1.8.0_77]
> at java.nio.ByteBuffer.allocate(ByteBuffer.java:335) ~[na:1.8.0_77]
> at 
> org.apache.kafka.common.network.NetworkReceive.readFromReadableChannel(NetworkReceive.java:93)
>  ~[kafka-clients-0.9.0.1.jar:na]
> at 
> org.apache.kafka.common.network.NetworkReceive.readFrom(NetworkReceive.java:71)
>  ~[kafka-clients-0.9.0.1.jar:na]
> at 
> org.apache.kafka.common.network.KafkaChannel.receive(KafkaChannel.java:153) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at 
> org.apache.kafka.common.network.KafkaChannel.read(KafkaChannel.java:134) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at org.apache.kafka.common.network.Selector.poll(Selector.java:286) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:256) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at 
> org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:216) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at 
> org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:128) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_77]
> {code}
> This line refer to a buffer allocation {{ByteBuffer.allocate(receiveSize)}} 
> (see 
> https://github.com/apache/kafka/blob/0.9.0.1/clients/src/main/java/org/apache/kafka/common/network/NetworkReceive.java#L93)
> Usually the app runs fine within 200/400 MB heap and a 64 MB Metaspace. And 
> we are producing small messages 500B at most.
> Also the error don't appear on the devlopment environment, in order to 
> identify the issue we tweaked the code to give us actual data of the 
> allocation size, we got this stack :
> {code}
> 09:55:49.484 [auth] [kafka-producer-network-thread | producer-1] WARN  
> o.a.k.c.n.NetworkReceive HEAP-ISSUE: constructor : Integer='-1', String='-1'
> 09:55:49.485 [auth] [kafka-producer-network-thread | producer-1] WARN  
> o.a.k.c.n.NetworkReceive HEAP-ISSUE: method : 
> NetworkReceive.readFromReadableChannel.receiveSize=1213486160
> java.lang.OutOfMemoryError: Java heap space
> Dumping heap to /tmp/tomcat.hprof ...
> Heap dump file created [69583827 bytes in 0.365 secs]
> 09:55:50.324 [auth] [kafka-producer-network-thread | producer-1] ERROR 
> o.a.k.c.utils.KafkaThread Uncaught exception in kafka-producer-network-thread 
> | producer-1: 
> java.lang.OutOfMemoryError: Java heap space
>   at java.nio.HeapByteBuffer.(HeapByteBuffer.java:57) ~[na:1.8.0_77]
>   at java.nio.ByteBuffer.allocate(ByteBuffer.java:335) 

[jira] [Commented] (KAFKA-3990) Kafka New Producer may raise an OutOfMemoryError

2016-07-29 Thread Brice Dutheil (JIRA)

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

Brice Dutheil commented on KAFKA-3990:
--

Hi after further investigation we found out the issue came because we switched 
from bamboo to marathon-lb, and marathon-lb opens the 9091 HTTP port 
(https://github.com/mesosphere/marathon-lb#operational-best-practices), we 
missed that during the upgrade.

{code}
> curl -v dockerhost:9091
* About to connect() to dockerhost port 9091 (#0)
*   Trying 172.17.42.1...
* Connected to dockerhost (172.17.42.1) port 9091 (#0)
> GET / HTTP/1.1
> User-Agent: curl/7.29.0
> Host: dockerhost:9091
> Accept: */*
>
* Empty reply from server
* Connection #0 to host dockerhost left intact
curl: (52) Empty reply from server
{code}

However I'm surprised Kafka / clients don't check the validity of the payload.

> Kafka New Producer may raise an OutOfMemoryError
> 
>
> Key: KAFKA-3990
> URL: https://issues.apache.org/jira/browse/KAFKA-3990
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Affects Versions: 0.9.0.1
> Environment: Docker, Base image : CentOS
> Java 8u77
>Reporter: Brice Dutheil
> Attachments: app-producer-config.log, kafka-broker-logs.zip
>
>
> We are regularly seeing OOME errors on a kafka producer, we first saw :
> {code}
> java.lang.OutOfMemoryError: Java heap space
> at java.nio.HeapByteBuffer.(HeapByteBuffer.java:57) ~[na:1.8.0_77]
> at java.nio.ByteBuffer.allocate(ByteBuffer.java:335) ~[na:1.8.0_77]
> at 
> org.apache.kafka.common.network.NetworkReceive.readFromReadableChannel(NetworkReceive.java:93)
>  ~[kafka-clients-0.9.0.1.jar:na]
> at 
> org.apache.kafka.common.network.NetworkReceive.readFrom(NetworkReceive.java:71)
>  ~[kafka-clients-0.9.0.1.jar:na]
> at 
> org.apache.kafka.common.network.KafkaChannel.receive(KafkaChannel.java:153) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at 
> org.apache.kafka.common.network.KafkaChannel.read(KafkaChannel.java:134) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at org.apache.kafka.common.network.Selector.poll(Selector.java:286) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:256) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at 
> org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:216) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at 
> org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:128) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_77]
> {code}
> This line refer to a buffer allocation {{ByteBuffer.allocate(receiveSize)}} 
> (see 
> https://github.com/apache/kafka/blob/0.9.0.1/clients/src/main/java/org/apache/kafka/common/network/NetworkReceive.java#L93)
> Usually the app runs fine within 200/400 MB heap and a 64 MB Metaspace. And 
> we are producing small messages 500B at most.
> Also the error don't appear on the devlopment environment, in order to 
> identify the issue we tweaked the code to give us actual data of the 
> allocation size, we got this stack :
> {code}
> 09:55:49.484 [auth] [kafka-producer-network-thread | producer-1] WARN  
> o.a.k.c.n.NetworkReceive HEAP-ISSUE: constructor : Integer='-1', String='-1'
> 09:55:49.485 [auth] [kafka-producer-network-thread | producer-1] WARN  
> o.a.k.c.n.NetworkReceive HEAP-ISSUE: method : 
> NetworkReceive.readFromReadableChannel.receiveSize=1213486160
> java.lang.OutOfMemoryError: Java heap space
> Dumping heap to /tmp/tomcat.hprof ...
> Heap dump file created [69583827 bytes in 0.365 secs]
> 09:55:50.324 [auth] [kafka-producer-network-thread | producer-1] ERROR 
> o.a.k.c.utils.KafkaThread Uncaught exception in kafka-producer-network-thread 
> | producer-1: 
> java.lang.OutOfMemoryError: Java heap space
>   at java.nio.HeapByteBuffer.(HeapByteBuffer.java:57) ~[na:1.8.0_77]
>   at java.nio.ByteBuffer.allocate(ByteBuffer.java:335) ~[na:1.8.0_77]
>   at 
> org.apache.kafka.common.network.NetworkReceive.readFromReadableChannel(NetworkReceive.java:93)
>  ~[kafka-clients-0.9.0.1.jar:na]
>   at 
> org.apache.kafka.common.network.NetworkReceive.readFrom(NetworkReceive.java:71)
>  ~[kafka-clients-0.9.0.1.jar:na]
>   at 
> org.apache.kafka.common.network.KafkaChannel.receive(KafkaChannel.java:153) 
> ~[kafka-clients-0.9.0.1.jar:na]
>   at org.apache.kafka.common.network.KafkaChannel.read(KafkaChannel.java:134) 
> ~[kafka-clients-0.9.0.1.jar:na]
>   at org.apache.kafka.common.network.Selector.poll(Selector.java:286) 
> ~[kafka-clients-0.9.0.1.jar:na]
>   at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:256) 
> ~[kafka-clients-0.9.0.1.jar:na]
>   at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:216) 
> ~[kafka-clients-0.9.0.1.jar:na]
>   at 

[jira] [Commented] (KAFKA-3990) Kafka New Producer may raise an OutOfMemoryError

2016-07-29 Thread Brice Dutheil (JIRA)

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

Brice Dutheil commented on KAFKA-3990:
--

Hi all, sorry for the delayed response I have busy with other stuff.

Yes the broker is 0.9.0.1 as well. It runs in a docker container too.
I attached the broker logs. We restarted the single instance cluster (~ 13:20), 
and a few minutes later (~ 13:34) we ran the application and the app face same 
problem with this big message.

This got me curious, I only looked at the server.log, however controller.log 
show OOME as well right at the broker instance start :

{code}
[2016-07-29 13:20:34,366] WARN [Controller-1-to-broker-1-send-thread], 
Controller 1 epoch 1 fails to send request 
{controller_id=1,controller_epoch=1,partition_states=[],live_brokers=[{id=1,end_points=[{port=9091,host=dockerhost,security_protocol_type=0}]}]}
 to broker Node(1, dockerhost, 9091). Reconnecting to broker. 
(kafka.controller.RequestSendThread)
java.lang.OutOfMemoryError: Java heap space
at java.nio.HeapByteBuffer.(HeapByteBuffer.java:57)
at java.nio.ByteBuffer.allocate(ByteBuffer.java:335)
at 
org.apache.kafka.common.network.NetworkReceive.readFromReadableChannel(NetworkReceive.java:93)
at 
org.apache.kafka.common.network.NetworkReceive.readFrom(NetworkReceive.java:71)
at 
org.apache.kafka.common.network.KafkaChannel.receive(KafkaChannel.java:153)
at 
org.apache.kafka.common.network.KafkaChannel.read(KafkaChannel.java:134)
at org.apache.kafka.common.network.Selector.poll(Selector.java:286)
at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:256)
at 
kafka.utils.NetworkClientBlockingOps$.recurse$1(NetworkClientBlockingOps.scala:128)
at 
kafka.utils.NetworkClientBlockingOps$.kafka$utils$NetworkClientBlockingOps$$pollUntilFound$extension(NetworkClientBlockingOps.scala:139)
at 
kafka.utils.NetworkClientBlockingOps$.blockingSendAndReceive$extension(NetworkClientBlockingOps.scala:80)
at 
kafka.controller.RequestSendThread.liftedTree1$1(ControllerChannelManager.scala:180)
at 
kafka.controller.RequestSendThread.doWork(ControllerChannelManager.scala:171)
at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:63)
{code}








> Kafka New Producer may raise an OutOfMemoryError
> 
>
> Key: KAFKA-3990
> URL: https://issues.apache.org/jira/browse/KAFKA-3990
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Affects Versions: 0.9.0.1
> Environment: Docker, Base image : CentOS
> Java 8u77
>Reporter: Brice Dutheil
> Attachments: app-producer-config.log, kafka-broker-logs.zip
>
>
> We are regularly seeing OOME errors on a kafka producer, we first saw :
> {code}
> java.lang.OutOfMemoryError: Java heap space
> at java.nio.HeapByteBuffer.(HeapByteBuffer.java:57) ~[na:1.8.0_77]
> at java.nio.ByteBuffer.allocate(ByteBuffer.java:335) ~[na:1.8.0_77]
> at 
> org.apache.kafka.common.network.NetworkReceive.readFromReadableChannel(NetworkReceive.java:93)
>  ~[kafka-clients-0.9.0.1.jar:na]
> at 
> org.apache.kafka.common.network.NetworkReceive.readFrom(NetworkReceive.java:71)
>  ~[kafka-clients-0.9.0.1.jar:na]
> at 
> org.apache.kafka.common.network.KafkaChannel.receive(KafkaChannel.java:153) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at 
> org.apache.kafka.common.network.KafkaChannel.read(KafkaChannel.java:134) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at org.apache.kafka.common.network.Selector.poll(Selector.java:286) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:256) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at 
> org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:216) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at 
> org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:128) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_77]
> {code}
> This line refer to a buffer allocation {{ByteBuffer.allocate(receiveSize)}} 
> (see 
> https://github.com/apache/kafka/blob/0.9.0.1/clients/src/main/java/org/apache/kafka/common/network/NetworkReceive.java#L93)
> Usually the app runs fine within 200/400 MB heap and a 64 MB Metaspace. And 
> we are producing small messages 500B at most.
> Also the error don't appear on the devlopment environment, in order to 
> identify the issue we tweaked the code to give us actual data of the 
> allocation size, we got this stack :
> {code}
> 09:55:49.484 [auth] [kafka-producer-network-thread | producer-1] WARN  
> o.a.k.c.n.NetworkReceive HEAP-ISSUE: constructor : Integer='-1', String='-1'
> 09:55:49.485 [auth] [kafka-producer-network-thread | producer-1] WARN  
> 

[jira] [Updated] (KAFKA-3990) Kafka New Producer may raise an OutOfMemoryError

2016-07-29 Thread Brice Dutheil (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-3990?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Brice Dutheil updated KAFKA-3990:
-
Attachment: app-producer-config.log
kafka-broker-logs.zip

> Kafka New Producer may raise an OutOfMemoryError
> 
>
> Key: KAFKA-3990
> URL: https://issues.apache.org/jira/browse/KAFKA-3990
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Affects Versions: 0.9.0.1
> Environment: Docker, Base image : CentOS
> Java 8u77
>Reporter: Brice Dutheil
> Attachments: app-producer-config.log, kafka-broker-logs.zip
>
>
> We are regularly seeing OOME errors on a kafka producer, we first saw :
> {code}
> java.lang.OutOfMemoryError: Java heap space
> at java.nio.HeapByteBuffer.(HeapByteBuffer.java:57) ~[na:1.8.0_77]
> at java.nio.ByteBuffer.allocate(ByteBuffer.java:335) ~[na:1.8.0_77]
> at 
> org.apache.kafka.common.network.NetworkReceive.readFromReadableChannel(NetworkReceive.java:93)
>  ~[kafka-clients-0.9.0.1.jar:na]
> at 
> org.apache.kafka.common.network.NetworkReceive.readFrom(NetworkReceive.java:71)
>  ~[kafka-clients-0.9.0.1.jar:na]
> at 
> org.apache.kafka.common.network.KafkaChannel.receive(KafkaChannel.java:153) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at 
> org.apache.kafka.common.network.KafkaChannel.read(KafkaChannel.java:134) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at org.apache.kafka.common.network.Selector.poll(Selector.java:286) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:256) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at 
> org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:216) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at 
> org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:128) 
> ~[kafka-clients-0.9.0.1.jar:na]
> at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_77]
> {code}
> This line refer to a buffer allocation {{ByteBuffer.allocate(receiveSize)}} 
> (see 
> https://github.com/apache/kafka/blob/0.9.0.1/clients/src/main/java/org/apache/kafka/common/network/NetworkReceive.java#L93)
> Usually the app runs fine within 200/400 MB heap and a 64 MB Metaspace. And 
> we are producing small messages 500B at most.
> Also the error don't appear on the devlopment environment, in order to 
> identify the issue we tweaked the code to give us actual data of the 
> allocation size, we got this stack :
> {code}
> 09:55:49.484 [auth] [kafka-producer-network-thread | producer-1] WARN  
> o.a.k.c.n.NetworkReceive HEAP-ISSUE: constructor : Integer='-1', String='-1'
> 09:55:49.485 [auth] [kafka-producer-network-thread | producer-1] WARN  
> o.a.k.c.n.NetworkReceive HEAP-ISSUE: method : 
> NetworkReceive.readFromReadableChannel.receiveSize=1213486160
> java.lang.OutOfMemoryError: Java heap space
> Dumping heap to /tmp/tomcat.hprof ...
> Heap dump file created [69583827 bytes in 0.365 secs]
> 09:55:50.324 [auth] [kafka-producer-network-thread | producer-1] ERROR 
> o.a.k.c.utils.KafkaThread Uncaught exception in kafka-producer-network-thread 
> | producer-1: 
> java.lang.OutOfMemoryError: Java heap space
>   at java.nio.HeapByteBuffer.(HeapByteBuffer.java:57) ~[na:1.8.0_77]
>   at java.nio.ByteBuffer.allocate(ByteBuffer.java:335) ~[na:1.8.0_77]
>   at 
> org.apache.kafka.common.network.NetworkReceive.readFromReadableChannel(NetworkReceive.java:93)
>  ~[kafka-clients-0.9.0.1.jar:na]
>   at 
> org.apache.kafka.common.network.NetworkReceive.readFrom(NetworkReceive.java:71)
>  ~[kafka-clients-0.9.0.1.jar:na]
>   at 
> org.apache.kafka.common.network.KafkaChannel.receive(KafkaChannel.java:153) 
> ~[kafka-clients-0.9.0.1.jar:na]
>   at org.apache.kafka.common.network.KafkaChannel.read(KafkaChannel.java:134) 
> ~[kafka-clients-0.9.0.1.jar:na]
>   at org.apache.kafka.common.network.Selector.poll(Selector.java:286) 
> ~[kafka-clients-0.9.0.1.jar:na]
>   at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:256) 
> ~[kafka-clients-0.9.0.1.jar:na]
>   at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:216) 
> ~[kafka-clients-0.9.0.1.jar:na]
>   at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:128) 
> ~[kafka-clients-0.9.0.1.jar:na]
>   at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_77]
> {code}
> Notice the size to allocate {{1213486160}} ~1.2 GB. I'm not yet sure how this 
> size is initialised.
> Notice as well that every time this OOME appear the {{NetworkReceive}} 
> constructor at 
> https://github.com/apache/kafka/blob/0.9.0.1/clients/src/main/java/org/apache/kafka/common/network/NetworkReceive.java#L49
>  receive the parameters : {{maxSize=-1}}, {{source="-1"}}
> We may have missed configuration in our setup but kafka clients shouldn't 
> raise an OOME. For reference the producer is 

[jira] [Updated] (KAFKA-3990) Kafka New Producer may raise an OutOfMemoryError

2016-07-26 Thread Brice Dutheil (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-3990?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Brice Dutheil updated KAFKA-3990:
-
Description: 
We are regularly seeing OOME errors on a kafka producer, we first saw :

{code}
java.lang.OutOfMemoryError: Java heap space
at java.nio.HeapByteBuffer.(HeapByteBuffer.java:57) ~[na:1.8.0_77]
at java.nio.ByteBuffer.allocate(ByteBuffer.java:335) ~[na:1.8.0_77]
at 
org.apache.kafka.common.network.NetworkReceive.readFromReadableChannel(NetworkReceive.java:93)
 ~[kafka-clients-0.9.0.1.jar:na]
at 
org.apache.kafka.common.network.NetworkReceive.readFrom(NetworkReceive.java:71) 
~[kafka-clients-0.9.0.1.jar:na]
at 
org.apache.kafka.common.network.KafkaChannel.receive(KafkaChannel.java:153) 
~[kafka-clients-0.9.0.1.jar:na]
at org.apache.kafka.common.network.KafkaChannel.read(KafkaChannel.java:134) 
~[kafka-clients-0.9.0.1.jar:na]
at org.apache.kafka.common.network.Selector.poll(Selector.java:286) 
~[kafka-clients-0.9.0.1.jar:na]
at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:256) 
~[kafka-clients-0.9.0.1.jar:na]
at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:216) 
~[kafka-clients-0.9.0.1.jar:na]
at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:128) 
~[kafka-clients-0.9.0.1.jar:na]
at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_77]
{code}


This line refer to a buffer allocation {{ByteBuffer.allocate(receiveSize)}} 
(see 
https://github.com/apache/kafka/blob/0.9.0.1/clients/src/main/java/org/apache/kafka/common/network/NetworkReceive.java#L93)

Usually the app runs fine within 200/400 MB heap and a 64 MB Metaspace. And we 
are producing small messages 500B at most.


Also the error don't appear on the devlopment environment, in order to identify 
the issue we tweaked the code to give us actual data of the allocation size, we 
got this stack :

{code}
09:55:49.484 [auth] [kafka-producer-network-thread | producer-1] WARN  
o.a.k.c.n.NetworkReceive HEAP-ISSUE: constructor : Integer='-1', String='-1'
09:55:49.485 [auth] [kafka-producer-network-thread | producer-1] WARN  
o.a.k.c.n.NetworkReceive HEAP-ISSUE: method : 
NetworkReceive.readFromReadableChannel.receiveSize=1213486160
java.lang.OutOfMemoryError: Java heap space
Dumping heap to /tmp/tomcat.hprof ...
Heap dump file created [69583827 bytes in 0.365 secs]
09:55:50.324 [auth] [kafka-producer-network-thread | producer-1] ERROR 
o.a.k.c.utils.KafkaThread Uncaught exception in kafka-producer-network-thread | 
producer-1: 
java.lang.OutOfMemoryError: Java heap space
  at java.nio.HeapByteBuffer.(HeapByteBuffer.java:57) ~[na:1.8.0_77]
  at java.nio.ByteBuffer.allocate(ByteBuffer.java:335) ~[na:1.8.0_77]
  at 
org.apache.kafka.common.network.NetworkReceive.readFromReadableChannel(NetworkReceive.java:93)
 ~[kafka-clients-0.9.0.1.jar:na]
  at 
org.apache.kafka.common.network.NetworkReceive.readFrom(NetworkReceive.java:71) 
~[kafka-clients-0.9.0.1.jar:na]
  at 
org.apache.kafka.common.network.KafkaChannel.receive(KafkaChannel.java:153) 
~[kafka-clients-0.9.0.1.jar:na]
  at org.apache.kafka.common.network.KafkaChannel.read(KafkaChannel.java:134) 
~[kafka-clients-0.9.0.1.jar:na]
  at org.apache.kafka.common.network.Selector.poll(Selector.java:286) 
~[kafka-clients-0.9.0.1.jar:na]
  at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:256) 
~[kafka-clients-0.9.0.1.jar:na]
  at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:216) 
~[kafka-clients-0.9.0.1.jar:na]
  at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:128) 
~[kafka-clients-0.9.0.1.jar:na]
  at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_77]
{code}


Notice the size to allocate {{1213486160}} ~1.2 GB. I'm not yet sure how this 
size is initialised.
Notice as well that every time this OOME appear the {{NetworkReceive}} 
constructor at 
https://github.com/apache/kafka/blob/0.9.0.1/clients/src/main/java/org/apache/kafka/common/network/NetworkReceive.java#L49
 receive the parameters : {{maxSize=-1}}, {{source="-1"}}

We may have missed configuration in our setup but kafka clients shouldn't raise 
an OOME. For reference the producer is initialised with :

{code}
Properties props = new Properties();
props.put(BOOTSTRAP_SERVERS_CONFIG, properties.bootstrapServers);
props.put(ACKS_CONFIG, "ONE");
props.put(RETRIES_CONFIG, 0);
props.put(BATCH_SIZE_CONFIG, 16384);
props.put(LINGER_MS_CONFIG, 0);
props.put(BUFFER_MEMORY_CONFIG, 33554432);
props.put(REQUEST_TIMEOUT_MS_CONFIG, 1000);
props.put(MAX_BLOCK_MS_CONFIG, 1000);
props.put(KEY_SERIALIZER_CLASS_CONFIG, 
StringSerializer.class.getName());
props.put(VALUE_SERIALIZER_CLASS_CONFIG, 
JSONSerializer.class.getName());
{code}

For reference while googling for the issue we found a similar stack trace with 
the new 

[jira] [Created] (KAFKA-3990) Kafka New Producer may raise an OutOfMemoryError

2016-07-26 Thread Brice Dutheil (JIRA)
Brice Dutheil created KAFKA-3990:


 Summary: Kafka New Producer may raise an OutOfMemoryError
 Key: KAFKA-3990
 URL: https://issues.apache.org/jira/browse/KAFKA-3990
 Project: Kafka
  Issue Type: Bug
  Components: clients
Affects Versions: 0.9.0.1
 Environment: Docker, Base image : CentOS
Java 8u77
Reporter: Brice Dutheil


We are regularly seeing OOME errors on a kafka producer, we first saw :

{code}
java.lang.OutOfMemoryError: Java heap space
at java.nio.HeapByteBuffer.(HeapByteBuffer.java:57) ~[na:1.8.0_77]
at java.nio.ByteBuffer.allocate(ByteBuffer.java:335) ~[na:1.8.0_77]
at 
org.apache.kafka.common.network.NetworkReceive.readFromReadableChannel(NetworkReceive.java:93)
 ~[kafka-clients-0.9.0.1.jar:na]
at 
org.apache.kafka.common.network.NetworkReceive.readFrom(NetworkReceive.java:71) 
~[kafka-clients-0.9.0.1.jar:na]
at 
org.apache.kafka.common.network.KafkaChannel.receive(KafkaChannel.java:153) 
~[kafka-clients-0.9.0.1.jar:na]
at org.apache.kafka.common.network.KafkaChannel.read(KafkaChannel.java:134) 
~[kafka-clients-0.9.0.1.jar:na]
at org.apache.kafka.common.network.Selector.poll(Selector.java:286) 
~[kafka-clients-0.9.0.1.jar:na]
at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:256) 
~[kafka-clients-0.9.0.1.jar:na]
at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:216) 
~[kafka-clients-0.9.0.1.jar:na]
at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:128) 
~[kafka-clients-0.9.0.1.jar:na]
at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_77]
{code}


This line refer to a buffer allocation {{ByteBuffer.allocate(receiveSize)}} 
(see 
https://github.com/apache/kafka/blob/0.9.0.1/clients/src/main/java/org/apache/kafka/common/network/NetworkReceive.java#L93)

Usually the app runs fine within 200/400 MB heap and a 64 MB Metaspace. And we 
are producing small messages 500B at most.


Also the error don't appear on the devlopment environment, in order to identify 
the issue we tweaked the code to give us actual data of the allocation size, we 
got this stack :

{code}
09:55:49.484 [auth] [kafka-producer-network-thread | producer-1] WARN  
o.a.k.c.n.NetworkReceive HEAP-ISSUE: constructor : Integer='-1', String='-1'
09:55:49.485 [auth] [kafka-producer-network-thread | producer-1] WARN  
o.a.k.c.n.NetworkReceive HEAP-ISSUE: method : 
NetworkReceive.readFromReadableChannel.receiveSize=1213486160
java.lang.OutOfMemoryError: Java heap space
Dumping heap to /tmp/tomcat.hprof ...
Heap dump file created [69583827 bytes in 0.365 secs]
09:55:50.324 [auth] [kafka-producer-network-thread | producer-1] ERROR 
o.a.k.c.utils.KafkaThread Uncaught exception in kafka-producer-network-thread | 
producer-1: 
java.lang.OutOfMemoryError: Java heap space
  at java.nio.HeapByteBuffer.(HeapByteBuffer.java:57) ~[na:1.8.0_77]
  at java.nio.ByteBuffer.allocate(ByteBuffer.java:335) ~[na:1.8.0_77]
  at 
org.apache.kafka.common.network.NetworkReceive.readFromReadableChannel(NetworkReceive.java:93)
 ~[kafka-clients-0.9.0.1.jar:na]
  at 
org.apache.kafka.common.network.NetworkReceive.readFrom(NetworkReceive.java:71) 
~[kafka-clients-0.9.0.1.jar:na]
  at 
org.apache.kafka.common.network.KafkaChannel.receive(KafkaChannel.java:153) 
~[kafka-clients-0.9.0.1.jar:na]
  at org.apache.kafka.common.network.KafkaChannel.read(KafkaChannel.java:134) 
~[kafka-clients-0.9.0.1.jar:na]
  at org.apache.kafka.common.network.Selector.poll(Selector.java:286) 
~[kafka-clients-0.9.0.1.jar:na]
  at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:256) 
~[kafka-clients-0.9.0.1.jar:na]
  at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:216) 
~[kafka-clients-0.9.0.1.jar:na]
  at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:128) 
~[kafka-clients-0.9.0.1.jar:na]
  at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_77]
{code}


Notice the size to allocate {{1213486160}} ~1.2 GB. I'm not yet sure how this 
size is initialised.
Notice as well that every time this OOME appear the NetworkReceive constructor 
receive the parameters : {{maxSize=-1}}, {{source="-1"}} 
(https://github.com/apache/kafka/blob/0.9.0.1/clients/src/main/java/org/apache/kafka/common/network/NetworkReceive.java#L49)

We may have missed configuration in our setup but kafka clients shouldn't raise 
an OOME. For reference the producer is initialised with :

{code}
Properties props = new Properties();
props.put(BOOTSTRAP_SERVERS_CONFIG, properties.bootstrapServers);
props.put(ACKS_CONFIG, "ONE");
props.put(RETRIES_CONFIG, 0);
props.put(BATCH_SIZE_CONFIG, 16384);
props.put(LINGER_MS_CONFIG, 0);
props.put(BUFFER_MEMORY_CONFIG, 33554432);
props.put(REQUEST_TIMEOUT_MS_CONFIG, 1000);
props.put(MAX_BLOCK_MS_CONFIG, 1000);