[ https://issues.apache.org/jira/browse/SPARK-17380?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15680270#comment-15680270 ]
Udit Mehrotra commented on SPARK-17380: --------------------------------------- We came across this Memory Leak in the executor logs, by using the JVM option '-Dio.netty.leakDetectionLevel=advanced', which seems like a good evidence of memory leak, and tells the location where the buffer is created. 16/11/09 06:03:28 ERROR ResourceLeakDetector: LEAK: ByteBuf.release() was not called before it's garbage-collected. See http://netty.io/wiki/reference-counted-objects.html for more information. Recent access records: 0 Created at: io.netty.buffer.CompositeByteBuf.<init>(CompositeByteBuf.java:103) io.netty.buffer.Unpooled.wrappedBuffer(Unpooled.java:335) io.netty.buffer.Unpooled.wrappedBuffer(Unpooled.java:247) org.apache.spark.util.io.ChunkedByteBuffer.toNetty(ChunkedByteBuffer.scala:69) org.apache.spark.storage.BlockManager.org$apache$spark$storage$BlockManager$$replicate(BlockManager.scala:1161) org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:976) org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:910) org.apache.spark.storage.BlockManager.doPut(BlockManager.scala:866) org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:910) org.apache.spark.storage.BlockManager.putIterator(BlockManager.scala:700) org.apache.spark.streaming.receiver.BlockManagerBasedBlockHandler.storeBlock(ReceivedBlockHandler.scala:80) org.apache.spark.streaming.receiver.ReceiverSupervisorImpl.pushAndReportBlock(ReceiverSupervisorImpl.scala:158) org.apache.spark.streaming.receiver.ReceiverSupervisorImpl.pushArrayBuffer(ReceiverSupervisorImpl.scala:129) org.apache.spark.streaming.receiver.Receiver.store(Receiver.scala:133) org.apache.spark.streaming.kinesis.KinesisReceiver.org$apache$spark$streaming$kinesis$KinesisReceiver$$storeBlockWithRanges(KinesisReceiver.scala:282) org.apache.spark.streaming.kinesis.KinesisReceiver$GeneratedBlockHandler.onPushBlock(KinesisReceiver.scala:352) org.apache.spark.streaming.receiver.BlockGenerator.pushBlock(BlockGenerator.scala:297) org.apache.spark.streaming.receiver.BlockGenerator.org$apache$spark$streaming$receiver$BlockGenerator$$keepPushingBlocks(BlockGenerator.scala:269) org.apache.spark.streaming.receiver.BlockGenerator$$anon$1.run(BlockGenerator.scala:110) Can we please have some action on this JIRA ? > Spark streaming with a multi shard Kinesis freezes after several days > (memory/resource leak?) > --------------------------------------------------------------------------------------------- > > Key: SPARK-17380 > URL: https://issues.apache.org/jira/browse/SPARK-17380 > Project: Spark > Issue Type: Bug > Components: DStreams > Affects Versions: 2.0.0 > Reporter: Xeto > Attachments: exec_Leak_Hunter.zip, memory-after-freeze.png, memory.png > > > Running Spark Streaming 2.0.0 on AWS EMR 5.0.0 consuming from Kinesis (125 > shards). > Used memory keeps growing all the time according to Ganglia. > The application works properly for about 3.5 days till all free memory has > been used. > Then, micro batches start queuing up but none is served. > Spark freezes. You can see in Ganglia that some memory is being freed but it > doesn't help the job to recover. > Is it a memory/resource leak? > The job uses back pressure and Kryo. > The code has a mapToPair(), groupByKey(), flatMap(), > persist(StorageLevel.MEMORY_AND_DISK_SER_2()) and repartition(19); Then > storing to s3 using foreachRDD() > Cluster size: 20 machines > Spark cofiguration: > spark.executor.extraJavaOptions -verbose:gc -XX:+PrintGCDetails > -XX:+PrintGCDateStamps -XX:+UseConcMarkSweepGC > -XX:CMSInitiatingOccupancyFraction=70 -XX:MaxHeapFreeRatio=70 > -XX:PermSize=256M -XX:MaxPermSize=256M -XX:OnOutOfMemoryError='kill -9 %p' > spark.driver.extraJavaOptions -Dspark.driver.log.level=INFO > -XX:+UseConcMarkSweepGC -XX:PermSize=256M -XX:MaxPermSize=256M > -XX:OnOutOfMemoryError='kill -9 %p' > spark.master yarn-cluster > spark.executor.instances 19 > spark.executor.cores 7 > spark.executor.memory 7500M > spark.driver.memory 7500M > spark.default.parallelism 133 > spark.yarn.executor.memoryOverhead 2950 > spark.yarn.driver.memoryOverhead 2950 > spark.eventLog.enabled false > spark.eventLog.dir hdfs:///spark-logs/ -- This message was sent by Atlassian JIRA (v6.3.4#6332) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org