[ https://issues.apache.org/jira/browse/SPARK-11293?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15206635#comment-15206635 ]
Yi Zhou commented on SPARK-11293: --------------------------------- seem to hit the issue with Spark 1.6.1 not sure if this is relative to this..if yes, it can be fixed in Spark 1.6.2 ? 16/03/22 23:10:26 INFO memory.TaskMemoryManager: Allocate page number 16 (67108864 bytes) 16/03/22 23:10:26 INFO sort.UnsafeExternalSorter: Thread 221 spilling sort data of 1472.0 MB to disk (0 time so far) 16/03/22 23:11:26 INFO memory.TaskMemoryManager: Allocate page number 1 (1060044737 bytes) 16/03/22 23:11:26 INFO memory.TaskMemoryManager: Memory used in task 9302 16/03/22 23:11:26 INFO memory.TaskMemoryManager: Acquired by org.apache.spark.shuffle.sort.ShuffleExternalSorter@8bac554: 32.0 KB 16/03/22 23:11:26 INFO memory.TaskMemoryManager: Acquired by org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@7a117b4f: 512.0 MB 16/03/22 23:11:26 INFO memory.TaskMemoryManager: 0 bytes of memory were used by task 9302 but are not associated with specific consumers 16/03/22 23:11:26 INFO memory.TaskMemoryManager: 14909439433 bytes of memory are used for execution and 1376877 bytes of memory are used for storage 16/03/22 23:11:26 WARN memory.TaskMemoryManager: leak 32.0 KB memory from org.apache.spark.shuffle.sort.ShuffleExternalSorter@8bac554 16/03/22 23:11:26 ERROR executor.Executor: Managed memory leak detected; size = 32768 bytes, TID = 9302 16/03/22 23:11:26 ERROR executor.Executor: Exception in task 192.0 in stage 153.0 (TID 9302) java.lang.OutOfMemoryError: Unable to acquire 1073741824 bytes of memory, got 1060044737 at org.apache.spark.memory.MemoryConsumer.allocateArray(MemoryConsumer.java:91) at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.growPointerArrayIfNecessary(UnsafeExternalSorter.java:295) at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.insertRecord(UnsafeExternalSorter.java:330) at org.apache.spark.sql.execution.UnsafeExternalRowSorter.insertRow(UnsafeExternalRowSorter.java:91) at org.apache.spark.sql.execution.UnsafeExternalRowSorter.sort(UnsafeExternalRowSorter.java:168) at org.apache.spark.sql.execution.Sort$$anonfun$1.apply(Sort.scala:90) at org.apache.spark.sql.execution.Sort$$anonfun$1.apply(Sort.scala:64) at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$21.apply(RDD.scala:728) at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$21.apply(RDD.scala:728) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306) at org.apache.spark.rdd.RDD.iterator(RDD.scala:270) at org.apache.spark.rdd.ZippedPartitionsRDD2.compute(ZippedPartitionsRDD.scala:88) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306) at org.apache.spark.rdd.RDD.iterator(RDD.scala:270) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306) at org.apache.spark.rdd.RDD.iterator(RDD.scala:270) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306) at org.apache.spark.rdd.RDD.iterator(RDD.scala:270) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306) at org.apache.spark.rdd.RDD.iterator(RDD.scala:270) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:73) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) at org.apache.spark.scheduler.Task.run(Task.scala:89) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:745) > Spillable collections leak shuffle memory > ----------------------------------------- > > Key: SPARK-11293 > URL: https://issues.apache.org/jira/browse/SPARK-11293 > Project: Spark > Issue Type: Bug > Components: Spark Core > Affects Versions: 1.3.1, 1.4.1, 1.5.1, 1.6.0 > Reporter: Josh Rosen > Assignee: Josh Rosen > Priority: Critical > > I discovered multiple leaks of shuffle memory while working on my memory > manager consolidation patch, which added the ability to do strict memory leak > detection for the bookkeeping that used to be performed by the > ShuffleMemoryManager. This uncovered a handful of places where tasks can > acquire execution/shuffle memory but never release it, starving themselves of > memory. > Problems that I found: > * {{ExternalSorter.stop()}} should release the sorter's shuffle/execution > memory. > * BlockStoreShuffleReader should call {{ExternalSorter.stop()}} using a > {{CompletionIterator}}. > * {{ExternalAppendOnlyMap}} exposes no equivalent of {{stop()}} for freeing > its resources. -- 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