[ https://issues.apache.org/jira/browse/KAFKA-3552?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15240051#comment-15240051 ]
Kanak Biscuitwala commented on KAFKA-3552: ------------------------------------------ Looks like there is a full GC immediately before the OOM (and that's confusing because shouldn't the completion of a full GC be enough to free up memory?): {code} 2016-04-13T18:19:47.181+0000: 5318.045: [GC [1 CMS-initial-mark: 702209K(900444K)] 809543K(1844188K), 0.0330340 secs] [Times: user=0.03 sys=0.00, real=0.03 secs] 2016-04-13T18:19:47.214+0000: 5318.078: [CMS-concurrent-mark-start] 2016-04-13T18:19:47.563+0000: 5318.427: [CMS-concurrent-mark: 0.349/0.349 secs] [Times: user=0.58 sys=0.16, real=0.35 secs] 2016-04-13T18:19:47.564+0000: 5318.427: [CMS-concurrent-preclean-start] 2016-04-13T18:19:47.573+0000: 5318.436: [CMS-concurrent-preclean: 0.008/0.009 secs] [Times: user=0.01 sys=0.00, real=0.01 secs] 2016-04-13T18:19:47.573+0000: 5318.437: [CMS-concurrent-abortable-preclean-start] 2016-04-13T18:19:49.670+0000: 5320.534: [Full GC2016-04-13T18:19:49.671+0000: 5320.534: [CMS2016-04-13T18:19:49.677+0000: 5320.541: [CMS-concurrent-abortable-preclean: 2.079/2.104 secs] [Times: user=3.46 sys=0.22, real=2.11 secs] (concurrent mode interrupted): 702209K->385639K(900444K), 0.8443580 secs] 1507659K->385639K(1844188K), [CMS Perm : 56002K->56002K(83968K)], 0.8448730 secs] [Times: user=0.86 sys=0.00, real=0.84 secs] 2016-04-13T18:19:52.619+0000: 5323.483: [Full GC2016-04-13T18:19:52.620+0000: 5323.483: [CMS: 385639K->384334K(900444K), 0.6693420 secs] 714628K->384334K(1844188K), [CMS Perm : 56002K->56002K(83968K)], 0.6698370 secs] [Times: user=0.68 sys=0.00, real=0.67 secs] 2016-04-13T18:19:55.395+0000: 5326.259: [Full GC2016-04-13T18:19:55.395+0000: 5326.259: [CMS: 384334K->383389K(900444K), 0.6660360 secs] 695662K->383389K(1844188K), [CMS Perm : 56002K->56002K(83968K)], 0.6665300 secs] [Times: user=0.68 sys=0.00, real=0.67 secs] 2016-04-13T18:19:58.166+0000: 5329.030: [Full GC2016-04-13T18:19:58.166+0000: 5329.030: [CMS: 383389K->382675K(900444K), 0.6607420 secs] 624249K->382675K(1844188K), [CMS Perm : 56002K->56002K(83968K)], 0.6612310 secs] [Times: user=0.67 sys=0.00, real=0.66 secs] 2016-04-13T18:20:01.171+0000: 5332.035: [GC2016-04-13T18:20:01.171+0000: 5332.035: [ParNew: 838912K->90048K(943744K), 0.0167690 secs] 1221587K->472723K(1844188K), 0 .0172720 secs] [Times: user=0.06 sys=0.00, real=0.01 secs] 2016-04-13T18:20:07.407+0000: 5338.270: [GC2016-04-13T18:20:07.407+0000: 5338.271: [ParNew: 928960K->25607K(943744K), 0.0232340 secs] 1311635K->408283K(1844188K), 0 .0237360 secs] [Times: user=0.07 sys=0.00, real=0.03 secs] {code} And the OOM occurs at 2016-04-13/18:19:58.928 UTC I do see that my host is running somewhat low on physical memory (but has plenty of swap) -- the heap dump is too large to attach, but I will attach a couple screenshots of byte[] allocations taking much more space than I would expect. Is it possible that there is a memory leak here? > New Consumer: java.lang.OutOfMemoryError: Direct buffer memory > -------------------------------------------------------------- > > Key: KAFKA-3552 > URL: https://issues.apache.org/jira/browse/KAFKA-3552 > Project: Kafka > Issue Type: Bug > Components: consumer > Affects Versions: 0.9.0.1 > Reporter: Kanak Biscuitwala > Assignee: Liquan Pei > Attachments: Screen Shot 2016-04-13 at 11.56.05 AM.png, Screen Shot > 2016-04-13 at 2.17.48 PM.png > > > I'm running Kafka's new consumer with message handlers that can sometimes > take a lot of time to return, and combining that with manual offset > management (to get at-least-once semantics). Since poll() is the only way to > heartbeat with the consumer, I have a thread that runs every 500 milliseconds > that does the following: > 1) Pause all partitions > 2) Call poll(0) > 3) Resume all partitions > For the record, all accesses to KafkaConsumer are protected by synchronized > blocks. This generally works, but I'm occasionally seeing messages like this: > {code} > java.lang.OutOfMemoryError: Direct buffer memory > at java.nio.Bits.reserveMemory(Bits.java:658) > at java.nio.DirectByteBuffer.<init>(DirectByteBuffer.java:123) > at java.nio.ByteBuffer.allocateDirect(ByteBuffer.java:306) > at sun.nio.ch.Util.getTemporaryDirectBuffer(Util.java:174) > at sun.nio.ch.IOUtil.read(IOUtil.java:195) > at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:379) > at > org.apache.kafka.common.network.PlaintextTransportLayer.read(PlaintextTransportLayer.java:108) > at > org.apache.kafka.common.network.NetworkReceive.readFromReadableChannel(NetworkReceive.java:97) > 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 > org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.clientPoll(ConsumerNetworkClient.java:320) > at > org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:213) > at > org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:193) > at > org.apache.kafka.clients.consumer.KafkaConsumer.pollOnce(KafkaConsumer.java:908) > at > org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:853) > {code} > In addition, when I'm reporting offsets, I'm seeing: > {code} > java.lang.OutOfMemoryError: Direct buffer memory > at java.nio.Bits.reserveMemory(Bits.java:658) > at java.nio.DirectByteBuffer.<init>(DirectByteBuffer.java:123) > at java.nio.ByteBuffer.allocateDirect(ByteBuffer.java:306) > at sun.nio.ch.Util.getTemporaryDirectBuffer(Util.java:174) > at sun.nio.ch.IOUtil.read(IOUtil.java:195) > at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:379) > at > org.apache.kafka.common.network.PlaintextTransportLayer.read(PlaintextTransportLayer.java:108) > at > org.apache.kafka.common.network.NetworkReceive.readFromReadableChannel(NetworkReceive.java:97) > 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 > org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.clientPoll(ConsumerNetworkClient.java:320) > at > org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:213) > at > org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:193) > at > org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:163) > at > org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.commitOffsetsSync(ConsumerCoordinator.java:358) > at > org.apache.kafka.clients.consumer.KafkaConsumer.commitSync(KafkaConsumer.java:968) > {code} > Given that I'm just calling the library, this behavior is unexpected. -- This message was sent by Atlassian JIRA (v6.3.4#6332)