[jira] [Commented] (KAFKA-2352) Possible memory leak in MirrorMaker and/or new Producer

2015-07-23 Thread Kostya Golikov (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-2352?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14638656#comment-14638656
 ] 

Kostya Golikov commented on KAFKA-2352:
---

Indeed, the problem was caused by upstream kafka cluster: some partitions lost 
their leaders and since _backpressure_ (block on full pool) was disabled 
consumer overrun producer, which was looping, trying to get leaders. Thanks, 
[~becket_qin]

 Possible memory leak in MirrorMaker and/or new Producer
 ---

 Key: KAFKA-2352
 URL: https://issues.apache.org/jira/browse/KAFKA-2352
 Project: Kafka
  Issue Type: Bug
Affects Versions: 0.8.2.1
Reporter: Kostya Golikov
 Attachments: consumer.conf, output.log, producer.conf


 I've been playing around with Mirror Maker (version from trunk, dated July 
 7th) and got a few problems, most noticeable of which is that MirrorMaker 
 exhausts it's memory pool, even though it's size set to relatively huge value 
 of 132 MB, and individual messages are around 2 KB. Batch size is set to just 
 2 messages (full configs are attached). 
 {code}
 [2015-07-21 15:19:52,915] FATAL [mirrormaker-thread-1] Mirror maker thread 
 failure due to  (kafka.tools.MirrorMaker$MirrorMakerThread)
 org.apache.kafka.clients.producer.BufferExhaustedException: You have 
 exhausted the 134217728 bytes of memory you configured for the client and the 
 client is configured to error rather than block when memory is exhausted.
   at 
 org.apache.kafka.clients.producer.internals.BufferPool.allocate(BufferPool.java:124)
   at 
 org.apache.kafka.clients.producer.internals.RecordAccumulator.append(RecordAccumulator.java:172)
   at 
 org.apache.kafka.clients.producer.KafkaProducer.send(KafkaProducer.java:388)
   at 
 kafka.tools.MirrorMaker$MirrorMakerProducer.send(MirrorMaker.scala:380)
   at 
 kafka.tools.MirrorMaker$MirrorMakerThread$$anonfun$run$3.apply(MirrorMaker.scala:311)
   at 
 kafka.tools.MirrorMaker$MirrorMakerThread$$anonfun$run$3.apply(MirrorMaker.scala:311)
   at scala.collection.Iterator$class.foreach(Iterator.scala:727)
   at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
   at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
   at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
   at kafka.tools.MirrorMaker$MirrorMakerThread.run(MirrorMaker.scala:311)
 {code}
 Am I doing something wrong? Any help in further diagnosing of this problem 
 might be handy.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-2352) Possible memory leak in MirrorMaker and/or new Producer

2015-07-22 Thread Jiangjie Qin (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-2352?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14637208#comment-14637208
 ] 

Jiangjie Qin commented on KAFKA-2352:
-

Can you verify whether the messages you passed to producer has been sent or 
not? In mirror maker the default configuration was set to avoid data loss. So 
it producer cannot send a message it will retry infinitely. In that case, you 
might exhaust the producer's buffer if you keep appending messages to producer.

 Possible memory leak in MirrorMaker and/or new Producer
 ---

 Key: KAFKA-2352
 URL: https://issues.apache.org/jira/browse/KAFKA-2352
 Project: Kafka
  Issue Type: Bug
Affects Versions: 0.8.2.1
Reporter: Kostya Golikov
 Attachments: consumer.conf, output.log, producer.conf


 I've been playing around with Mirror Maker (version from trunk, dated July 
 7th) and got a few problems, most noticeable of which is that MirrorMaker 
 exhausts it's memory pool, even though it's size set to relatively huge value 
 of 132 MB, and individual messages are around 2 KB. Batch size is set to just 
 2 messages (full configs are attached). 
 {code}
 [2015-07-21 15:19:52,915] FATAL [mirrormaker-thread-1] Mirror maker thread 
 failure due to  (kafka.tools.MirrorMaker$MirrorMakerThread)
 org.apache.kafka.clients.producer.BufferExhaustedException: You have 
 exhausted the 134217728 bytes of memory you configured for the client and the 
 client is configured to error rather than block when memory is exhausted.
   at 
 org.apache.kafka.clients.producer.internals.BufferPool.allocate(BufferPool.java:124)
   at 
 org.apache.kafka.clients.producer.internals.RecordAccumulator.append(RecordAccumulator.java:172)
   at 
 org.apache.kafka.clients.producer.KafkaProducer.send(KafkaProducer.java:388)
   at 
 kafka.tools.MirrorMaker$MirrorMakerProducer.send(MirrorMaker.scala:380)
   at 
 kafka.tools.MirrorMaker$MirrorMakerThread$$anonfun$run$3.apply(MirrorMaker.scala:311)
   at 
 kafka.tools.MirrorMaker$MirrorMakerThread$$anonfun$run$3.apply(MirrorMaker.scala:311)
   at scala.collection.Iterator$class.foreach(Iterator.scala:727)
   at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
   at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
   at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
   at kafka.tools.MirrorMaker$MirrorMakerThread.run(MirrorMaker.scala:311)
 {code}
 Am I doing something wrong? Any help in further diagnosing of this problem 
 might be handy.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)