[ https://issues.apache.org/jira/browse/SPARK-11293?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15200543#comment-15200543 ]
Nezih Yigitbasi commented on SPARK-11293: ----------------------------------------- [~joshrosen] any plans to fix this? I believe we are hitting this issue with {{1.6.0}} {code} 16/03/17 21:38:38 INFO memory.TaskMemoryManager: Acquired by org.apache.spark.shuffle.sort.ShuffleExternalSorter@6ecba8f1: 32.0 KB 16/03/17 21:38:38 INFO memory.TaskMemoryManager: 1528015093 bytes of memory were used by task 103134 but are not associated with specific consumers 16/03/17 21:38:38 INFO memory.TaskMemoryManager: 1528047861 bytes of memory are used for execution and 80608434 bytes of memory are used for storage 16/03/17 21:38:38 ERROR executor.Executor: Managed memory leak detected; size = 1528015093 bytes, TID = 103134 16/03/17 21:38:38 ERROR executor.Executor: Exception in task 448.0 in stage 273.0 (TID 103134) java.lang.OutOfMemoryError: Unable to acquire 128 bytes of memory, got 0 at org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:120) at org.apache.spark.shuffle.sort.ShuffleExternalSorter.acquireNewPageIfNecessary(ShuffleExternalSorter.java:354) at org.apache.spark.shuffle.sort.ShuffleExternalSorter.insertRecord(ShuffleExternalSorter.java:375) at org.apache.spark.shuffle.sort.UnsafeShuffleWriter.insertRecordIntoSorter(UnsafeShuffleWriter.java:237) at org.apache.spark.shuffle.sort.UnsafeShuffleWriter.write(UnsafeShuffleWriter.java:164) 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:213) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) 16/03/17 21:38:38 ERROR util.SparkUncaughtExceptionHandler: Uncaught exception in thread Thread[Executor task launch worker-0,5,main] java.lang.OutOfMemoryError: Unable to acquire 128 bytes of memory, got 0 at org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:120) at org.apache.spark.shuffle.sort.ShuffleExternalSorter.acquireNewPageIfNecessary(ShuffleExternalSorter.java:354) at org.apache.spark.shuffle.sort.ShuffleExternalSorter.insertRecord(ShuffleExternalSorter.java:375) at org.apache.spark.shuffle.sort.UnsafeShuffleWriter.insertRecordIntoSorter(UnsafeShuffleWriter.java:237) at org.apache.spark.shuffle.sort.UnsafeShuffleWriter.write(UnsafeShuffleWriter.java:164) 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:213) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) 16/03/17 21:38:38 INFO storage.DiskBlockManager: Shutdown hook called 16/03/17 21:38:38 INFO util.ShutdownHookManager: Shutdown hook called {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 > 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