[ https://issues.apache.org/jira/browse/SPARK-11293?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15208203#comment-15208203 ]
William Dee commented on SPARK-11293: ------------------------------------- I don't have such verbose logging as above but can confirm that the test case given produces memory leaks in Spark 1.6.1 {code} scala> sc.parallelize(0 to 10000000, 2).map(x => x % 10000 -> x).groupByKey.mapPartitions { it => it.take(1) }.collect 16/03/23 10:26:11 INFO SparkContext: Starting job: collect at <console>:28 16/03/23 10:26:11 INFO DAGScheduler: Registering RDD 1 (map at <console>:28) 16/03/23 10:26:11 INFO DAGScheduler: Got job 0 (collect at <console>:28) with 2 output partitions 16/03/23 10:26:11 INFO DAGScheduler: Final stage: ResultStage 1 (collect at <console>:28) 16/03/23 10:26:11 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 0) 16/03/23 10:26:11 INFO DAGScheduler: Missing parents: List(ShuffleMapStage 0) 16/03/23 10:26:11 INFO DAGScheduler: Submitting ShuffleMapStage 0 (MapPartitionsRDD[1] at map at <console>:28), which has no missing parents 16/03/23 10:26:11 INFO MemoryStore: Block broadcast_0 stored as values in memory (estimated size 3.5 KB, free 3.5 KB) 16/03/23 10:26:11 INFO MemoryStore: Block broadcast_0_piece0 stored as bytes in memory (estimated size 1955.0 B, free 5.4 KB) 16/03/23 10:26:11 INFO BlockManagerInfo: Added broadcast_0_piece0 in memory on localhost:51671 (size: 1955.0 B, free: 511.5 MB) 16/03/23 10:26:11 INFO SparkContext: Created broadcast 0 from broadcast at DAGScheduler.scala:1006 16/03/23 10:26:11 INFO DAGScheduler: Submitting 2 missing tasks from ShuffleMapStage 0 (MapPartitionsRDD[1] at map at <console>:28) 16/03/23 10:26:11 INFO TaskSchedulerImpl: Adding task set 0.0 with 2 tasks 16/03/23 10:26:11 INFO TaskSetManager: Starting task 0.0 in stage 0.0 (TID 0, localhost, partition 0,PROCESS_LOCAL, 2067 bytes) 16/03/23 10:26:11 INFO TaskSetManager: Starting task 1.0 in stage 0.0 (TID 1, localhost, partition 1,PROCESS_LOCAL, 2124 bytes) 16/03/23 10:26:11 INFO Executor: Running task 0.0 in stage 0.0 (TID 0) 16/03/23 10:26:11 INFO Executor: Running task 1.0 in stage 0.0 (TID 1) 16/03/23 10:26:21 INFO Executor: Finished task 1.0 in stage 0.0 (TID 1). 1159 bytes result sent to driver 16/03/23 10:26:21 INFO TaskSetManager: Finished task 1.0 in stage 0.0 (TID 1) in 9152 ms on localhost (1/2) 16/03/23 10:26:21 INFO Executor: Finished task 0.0 in stage 0.0 (TID 0). 1159 bytes result sent to driver 16/03/23 10:26:21 INFO TaskSetManager: Finished task 0.0 in stage 0.0 (TID 0) in 9292 ms on localhost (2/2) 16/03/23 10:26:21 INFO DAGScheduler: ShuffleMapStage 0 (map at <console>:28) finished in 9.303 s 16/03/23 10:26:21 INFO TaskSchedulerImpl: Removed TaskSet 0.0, whose tasks have all completed, from pool 16/03/23 10:26:21 INFO DAGScheduler: looking for newly runnable stages 16/03/23 10:26:21 INFO DAGScheduler: running: Set() 16/03/23 10:26:21 INFO DAGScheduler: waiting: Set(ResultStage 1) 16/03/23 10:26:21 INFO DAGScheduler: failed: Set() 16/03/23 10:26:21 INFO DAGScheduler: Submitting ResultStage 1 (MapPartitionsRDD[3] at mapPartitions at <console>:28), which has no missing parents 16/03/23 10:26:21 INFO MemoryStore: Block broadcast_1 stored as values in memory (estimated size 4.5 KB, free 9.8 KB) 16/03/23 10:26:21 INFO MemoryStore: Block broadcast_1_piece0 stored as bytes in memory (estimated size 2.3 KB, free 12.1 KB) 16/03/23 10:26:21 INFO BlockManagerInfo: Added broadcast_1_piece0 in memory on localhost:51671 (size: 2.3 KB, free: 511.5 MB) 16/03/23 10:26:21 INFO SparkContext: Created broadcast 1 from broadcast at DAGScheduler.scala:1006 16/03/23 10:26:21 INFO DAGScheduler: Submitting 2 missing tasks from ResultStage 1 (MapPartitionsRDD[3] at mapPartitions at <console>:28) 16/03/23 10:26:21 INFO TaskSchedulerImpl: Adding task set 1.0 with 2 tasks 16/03/23 10:26:21 INFO TaskSetManager: Starting task 0.0 in stage 1.0 (TID 2, localhost, partition 0,NODE_LOCAL, 1894 bytes) 16/03/23 10:26:21 INFO TaskSetManager: Starting task 1.0 in stage 1.0 (TID 3, localhost, partition 1,NODE_LOCAL, 1894 bytes) 16/03/23 10:26:21 INFO Executor: Running task 1.0 in stage 1.0 (TID 3) 16/03/23 10:26:21 INFO Executor: Running task 0.0 in stage 1.0 (TID 2) 16/03/23 10:26:21 INFO ShuffleBlockFetcherIterator: Getting 2 non-empty blocks out of 2 blocks 16/03/23 10:26:21 INFO ShuffleBlockFetcherIterator: Getting 2 non-empty blocks out of 2 blocks 16/03/23 10:26:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 4 ms 16/03/23 10:26:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms 16/03/23 10:26:30 ERROR Executor: Managed memory leak detected; size = 36707226 bytes, TID = 3 16/03/23 10:26:30 ERROR Executor: Managed memory leak detected; size = 20935176 bytes, TID = 2 ...{code} Can we add 1.6.1 to the affected versions? > 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