[ https://issues.apache.org/jira/browse/SPARK-11293?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15276723#comment-15276723 ]
Miles Crawford commented on SPARK-11293: ---------------------------------------- Also biting us in 1.6.1 - we have to repartition our dataset into many thousands of partitions to avoid the following stack: {code} 2016-05-08 16:05:11,941 ERROR org.apache.spark.executor.Executor: Managed memory leak detected; size = 5748783225 bytes, TID = 1283 2016-05-08 16:05:11,948 ERROR org.apache.spark.executor.Executor: Exception in task 116.4 in stage 1.0 (TID 1283) java.lang.OutOfMemoryError: Unable to acquire 2383 bytes of memory, got 0 at org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:120) ~[d56f3336b4a0fcc71fe8beb90052dbafd0e88a749bdb4bbb15d37894cf443364-spark-core_2.11-1.6.1.jar:1.6.1] at org.apache.spark.shuffle.sort.ShuffleExternalSorter.acquireNewPageIfNecessary(ShuffleExternalSorter.java:346) ~[d56f3336b4a0fcc71fe8beb90052dbafd0e88a749bdb4bbb15d37894cf443364-spark-core_2.11-1.6.1.jar:1.6.1] at org.apache.spark.shuffle.sort.ShuffleExternalSorter.insertRecord(ShuffleExternalSorter.java:367) ~[d56f3336b4a0fcc71fe8beb90052dbafd0e88a749bdb4bbb15d37894cf443364-spark-core_2.11-1.6.1.jar:1.6.1] at org.apache.spark.shuffle.sort.UnsafeShuffleWriter.insertRecordIntoSorter(UnsafeShuffleWriter.java:237) ~[d56f3336b4a0fcc71fe8beb90052dbafd0e88a749bdb4bbb15d37894cf443364-spark-core_2.11-1.6.1.jar:1.6.1] at org.apache.spark.shuffle.sort.UnsafeShuffleWriter.write(UnsafeShuffleWriter.java:164) ~[d56f3336b4a0fcc71fe8beb90052dbafd0e88a749bdb4bbb15d37894cf443364-spark-core_2.11-1.6.1.jar:1.6.1] at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:73) ~[d56f3336b4a0fcc71fe8beb90052dbafd0e88a749bdb4bbb15d37894cf443364-spark-core_2.11-1.6.1.jar:1.6.1] at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) ~[d56f3336b4a0fcc71fe8beb90052dbafd0e88a749bdb4bbb15d37894cf443364-spark-core_2.11-1.6.1.jar:1.6.1] at org.apache.spark.scheduler.Task.run(Task.scala:89) ~[d56f3336b4a0fcc71fe8beb90052dbafd0e88a749bdb4bbb15d37894cf443364-spark-core_2.11-1.6.1.jar:1.6.1] at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214) ~[d56f3336b4a0fcc71fe8beb90052dbafd0e88a749bdb4bbb15d37894cf443364-spark-core_2.11-1.6.1.jar:1.6.1] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_91] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_91] at java.lang.Thread.run(Thread.java:745) [na:1.8.0_91]{code} > 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, 1.6.1 > 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