[ https://issues.apache.org/jira/browse/KAFKA-4778?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15886778#comment-15886778 ]
Guozhang Wang commented on KAFKA-4778: -------------------------------------- [~peoplemerge] What I was looking for is actually the StreamsConfig you used in your app, not the broker side configs. Also I'd like to know if you are using any state stores (seems so as you mentioned rocksDB on local SSD), and if yes how they are being used, for example for joins / aggregations / others? >From the jhat diagram that class java.io.EOFException 501606 20064240 class java.util.ArrayDeque 501941 8031056 And the number of consumer records are high I'd suspect that the apps did encounter some exceptions and hence were restarting while not clearly cleaning up the previous instances. To validate if this is the case, I'd ask a few questions: 1. Did you set an UncaughtExceptionHandler at your KafkaStreams object, and if yes, did you close the instance and restart it? 2. Did you observe multiple DEBUG log entries with {{Starting Kafka Stream process.}} at app's log files? > OOM on kafka-streams instances with high numbers of unreaped Record classes > --------------------------------------------------------------------------- > > Key: KAFKA-4778 > URL: https://issues.apache.org/jira/browse/KAFKA-4778 > Project: Kafka > Issue Type: Bug > Components: consumer, streams > Affects Versions: 0.10.1.1 > Environment: AWS m3.large Ubuntu 16.04.1 LTS. rocksDB on local SSD. > Kafka has 3 zk, 5 brokers. > stream processors are run with: > -XX:+UseG1GC -XX:+HeapDumpOnOutOfMemoryError -XX:+PrintGCDetails > Java(TM) SE Runtime Environment (build 1.8.0_101-b13) > Java HotSpot(TM) 64-Bit Server VM (build 25.101-b13, mixed mode) > Stream processors written in scala 2.11.8 > Reporter: Dave Thomas > Attachments: oom-killer.txt > > > We have a stream processing app with ~8 source/sink stages operating roughly > at the rate of 500k messages ingested/day (~4M across the 8 stages). > We get OOM eruptions once every ~18 hours. Note it is Linux triggering the > OOM-killer, not the JVM terminating itself. > It may be worth noting that stream processing uses ~50 mbytes while > processing normally for hours on end, until the problem surfaces; then in > ~20-30 sec memory grows suddenly from under 100 mbytes to >1 gig and does not > shrink until the process is killed. > We are using supervisor to restart the instances. Sometimes, the process > dies immediately once stream processing resumes for the same reason, a > process which could continue for minutes or hours. This extended window has > enabled us to capture a heap dump using jmap. > jhat's histogram feature reveals the following top objects in memory: > Class Instance Count Total Size > class [B 4070487 867857833 > class [Ljava.lang.Object; 2066986 268036184 > class [C 539519 92010932 > class [S 1003290 80263028 > class [I 508208 77821516 > class java.nio.HeapByteBuffer 1506943 58770777 > class org.apache.kafka.common.record.Record 1506783 36162792 > class org.apache.kafka.clients.consumer.ConsumerRecord 528652 35948336 > class org.apache.kafka.common.record.MemoryRecords$RecordsIterator 501742 > 32613230 > class org.apache.kafka.common.record.LogEntry 2009373 32149968 > class org.xerial.snappy.SnappyInputStream 501600 20565600 > class java.io.DataInputStream 501742 20069680 > class java.io.EOFException 501606 20064240 > class java.util.ArrayDeque 501941 8031056 > class java.lang.Long 516463 4131704 > Note high on the list include org.apache.kafka.common.record.Record, > org.apache.kafka.clients.consumer.ConsumerRecord, > org.apache.kafka.common.record.MemoryRecords$RecordsIterator, > org.apache.kafka.common.record.LogEntry > All of these contain 500k-1.5M instances. > There is nothing in stream processing logs that is distinctive (log levels > are still at default). > Could it be references (weak, phantom, etc) causing these instances to not be > garbage-collected? > Edit: to request a full heap dump (created using `jmap > -dump:format=b,file=`), contact me directly at opensou...@peoplemerge.com. > It is 2G. -- This message was sent by Atlassian JIRA (v6.3.15#6346)