[jira] [Comment Edited] (SPARK-14560) Cooperative Memory Management for Spillables

2016-11-23 Thread Roy Wang (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-14560?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15689843#comment-15689843
 ] 

Roy Wang edited comment on SPARK-14560 at 11/23/16 11:35 AM:
-

I found my tasks run into this problem after them was running nearly 2 hours 
since one week ago. I am using spark 2.0.2 on hadoop 2.6.
Here is the stack:

org.apache.spark.SparkException:Job aborted due to stage failure: Task 7 in 
stage 219.0 failed 4 times, most recent failure: Lost task 7.3 in stage 219.0 
(TID 73989, datanode16.bi): org.apache.spark.SparkException: Task failed while 
writing rows.
at 
org.apache.spark.sql.hive.SparkHiveDynamicPartitionWriterContainer.writeToFile(hiveWriterContainers.scala:330)
at 
org.apache.spark.sql.hive.execution.InsertIntoHiveTable$$anonfun$saveAsHiveFile$3.apply(InsertIntoHiveTable.scala:131)
at 
org.apache.spark.sql.hive.execution.InsertIntoHiveTable$$anonfun$saveAsHiveFile$3.apply(InsertIntoHiveTable.scala:131)
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)
at org.apache.spark.scheduler.Task.run(Task.scala:86)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)
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)
Caused by: java.lang.OutOfMemoryError: Unable to acquire 276 bytes of memory, 
got 0
at 
org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:129)
at 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.acquireNewPageIfNecessary(UnsafeExternalSorter.java:376)
at 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.insertRecord(UnsafeExternalSorter.java:408)
at 
org.apache.spark.sql.execution.UnsafeExternalRowSorter.insertRow(UnsafeExternalRowSorter.java:94)
at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.sort_addToSorter$(Unknown
 Source)
at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown
 Source)
at 
org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
at 
org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370)
at 
org.apache.spark.sql.execution.RowIteratorFromScala.advanceNext(RowIterator.scala:83)
at 
org.apache.spark.sql.execution.joins.SortMergeJoinScanner.advancedBufferedToRowWithNullFreeJoinKey(SortMergeJoinExec.scala:730)
at 
org.apache.spark.sql.execution.joins.SortMergeJoinScanner.(SortMergeJoinExec.scala:605)
at 
org.apache.spark.sql.execution.joins.SortMergeJoinExec$$anonfun$doExecute$1.apply(SortMergeJoinExec.scala:162)
at 
org.apache.spark.sql.execution.joins.SortMergeJoinExec$$anonfun$doExecute$1.apply(SortMergeJoinExec.scala:100)
at 
org.apache.spark.rdd.ZippedPartitionsRDD2.compute(ZippedPartitionsRDD.scala:89)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:283)
at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:283)
at 
org.apache.spark.rdd.CoalescedRDD$$anonfun$compute$1.apply(CoalescedRDD.scala:96)
at 
org.apache.spark.rdd.CoalescedRDD$$anonfun$compute$1.apply(CoalescedRDD.scala:95)
at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434)
at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:440)
at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408)
at 
org.apache.spark.sql.hive.SparkHiveDynamicPartitionWriterContainer.writeToFile(hiveWriterContainers.scala:288)
... 8 more

Driver stacktrace:
at 
org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1454)
at 
org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1442)
at 
org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1441)
at 
scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1441)
at 
org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:811)
at 
org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:811)
at scala.Option.foreach(Option.scala:257)
at 
org.apache.spark.scheduler.

[jira] [Comment Edited] (SPARK-14560) Cooperative Memory Management for Spillables

2016-11-23 Thread Roy Wang (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-14560?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15689843#comment-15689843
 ] 

Roy Wang edited comment on SPARK-14560 at 11/23/16 11:36 AM:
-

I found my tasks run into this problem after them was running nearly 2 hours 
since one week ago. I am using spark 2.0.0 on hadoop 2.6. Then I used spark 
2.0.2 instead of spark 2.0.0, but it still happend.Here is the stack:

org.apache.spark.SparkException:Job aborted due to stage failure: Task 7 in 
stage 219.0 failed 4 times, most recent failure: Lost task 7.3 in stage 219.0 
(TID 73989, datanode16.bi): org.apache.spark.SparkException: Task failed while 
writing rows.
at 
org.apache.spark.sql.hive.SparkHiveDynamicPartitionWriterContainer.writeToFile(hiveWriterContainers.scala:330)
at 
org.apache.spark.sql.hive.execution.InsertIntoHiveTable$$anonfun$saveAsHiveFile$3.apply(InsertIntoHiveTable.scala:131)
at 
org.apache.spark.sql.hive.execution.InsertIntoHiveTable$$anonfun$saveAsHiveFile$3.apply(InsertIntoHiveTable.scala:131)
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)
at org.apache.spark.scheduler.Task.run(Task.scala:86)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)
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)
Caused by: java.lang.OutOfMemoryError: Unable to acquire 276 bytes of memory, 
got 0
at 
org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:129)
at 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.acquireNewPageIfNecessary(UnsafeExternalSorter.java:376)
at 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.insertRecord(UnsafeExternalSorter.java:408)
at 
org.apache.spark.sql.execution.UnsafeExternalRowSorter.insertRow(UnsafeExternalRowSorter.java:94)
at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.sort_addToSorter$(Unknown
 Source)
at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown
 Source)
at 
org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
at 
org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370)
at 
org.apache.spark.sql.execution.RowIteratorFromScala.advanceNext(RowIterator.scala:83)
at 
org.apache.spark.sql.execution.joins.SortMergeJoinScanner.advancedBufferedToRowWithNullFreeJoinKey(SortMergeJoinExec.scala:730)
at 
org.apache.spark.sql.execution.joins.SortMergeJoinScanner.(SortMergeJoinExec.scala:605)
at 
org.apache.spark.sql.execution.joins.SortMergeJoinExec$$anonfun$doExecute$1.apply(SortMergeJoinExec.scala:162)
at 
org.apache.spark.sql.execution.joins.SortMergeJoinExec$$anonfun$doExecute$1.apply(SortMergeJoinExec.scala:100)
at 
org.apache.spark.rdd.ZippedPartitionsRDD2.compute(ZippedPartitionsRDD.scala:89)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:283)
at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:283)
at 
org.apache.spark.rdd.CoalescedRDD$$anonfun$compute$1.apply(CoalescedRDD.scala:96)
at 
org.apache.spark.rdd.CoalescedRDD$$anonfun$compute$1.apply(CoalescedRDD.scala:95)
at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434)
at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:440)
at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408)
at 
org.apache.spark.sql.hive.SparkHiveDynamicPartitionWriterContainer.writeToFile(hiveWriterContainers.scala:288)
... 8 more

Driver stacktrace:
at 
org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1454)
at 
org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1442)
at 
org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1441)
at 
scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1441)
at 
org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:811)
at 
org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:811)
at 

[jira] [Comment Edited] (SPARK-14560) Cooperative Memory Management for Spillables

2016-11-23 Thread Roy Wang (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-14560?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15689843#comment-15689843
 ] 

Roy Wang edited comment on SPARK-14560 at 11/23/16 11:37 AM:
-

I found my tasks run into this problem after them was running nearly 2 hours 
since one week ago. I am using spark 2.0.0 on hadoop 2.6. Then I used spark 
2.0.2 instead of spark 2.0.0, but it still happend. Here is the stack:

org.apache.spark.SparkException:Job aborted due to stage failure: Task 7 in 
stage 219.0 failed 4 times, most recent failure: Lost task 7.3 in stage 219.0 
(TID 73989, datanode16.bi): org.apache.spark.SparkException: Task failed while 
writing rows.
at 
org.apache.spark.sql.hive.SparkHiveDynamicPartitionWriterContainer.writeToFile(hiveWriterContainers.scala:330)
at 
org.apache.spark.sql.hive.execution.InsertIntoHiveTable$$anonfun$saveAsHiveFile$3.apply(InsertIntoHiveTable.scala:131)
at 
org.apache.spark.sql.hive.execution.InsertIntoHiveTable$$anonfun$saveAsHiveFile$3.apply(InsertIntoHiveTable.scala:131)
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)
at org.apache.spark.scheduler.Task.run(Task.scala:86)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)
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)
Caused by: java.lang.OutOfMemoryError: Unable to acquire 276 bytes of memory, 
got 0
at 
org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:129)
at 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.acquireNewPageIfNecessary(UnsafeExternalSorter.java:376)
at 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.insertRecord(UnsafeExternalSorter.java:408)
at 
org.apache.spark.sql.execution.UnsafeExternalRowSorter.insertRow(UnsafeExternalRowSorter.java:94)
at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.sort_addToSorter$(Unknown
 Source)
at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown
 Source)
at 
org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
at 
org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370)
at 
org.apache.spark.sql.execution.RowIteratorFromScala.advanceNext(RowIterator.scala:83)
at 
org.apache.spark.sql.execution.joins.SortMergeJoinScanner.advancedBufferedToRowWithNullFreeJoinKey(SortMergeJoinExec.scala:730)
at 
org.apache.spark.sql.execution.joins.SortMergeJoinScanner.(SortMergeJoinExec.scala:605)
at 
org.apache.spark.sql.execution.joins.SortMergeJoinExec$$anonfun$doExecute$1.apply(SortMergeJoinExec.scala:162)
at 
org.apache.spark.sql.execution.joins.SortMergeJoinExec$$anonfun$doExecute$1.apply(SortMergeJoinExec.scala:100)
at 
org.apache.spark.rdd.ZippedPartitionsRDD2.compute(ZippedPartitionsRDD.scala:89)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:283)
at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:283)
at 
org.apache.spark.rdd.CoalescedRDD$$anonfun$compute$1.apply(CoalescedRDD.scala:96)
at 
org.apache.spark.rdd.CoalescedRDD$$anonfun$compute$1.apply(CoalescedRDD.scala:95)
at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434)
at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:440)
at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408)
at 
org.apache.spark.sql.hive.SparkHiveDynamicPartitionWriterContainer.writeToFile(hiveWriterContainers.scala:288)
... 8 more

Driver stacktrace:
at 
org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1454)
at 
org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1442)
at 
org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1441)
at 
scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1441)
at 
org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:811)
at 
org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:811)
at

[jira] [Comment Edited] (SPARK-14560) Cooperative Memory Management for Spillables

2016-11-24 Thread Roy Wang (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-14560?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15694677#comment-15694677
 ] 

Roy Wang edited comment on SPARK-14560 at 11/25/16 2:56 AM:


UnsafeExternalSorter used a lot of memory but it looked like that 
TaskMemoryManager couldn't spill these consumers when acquireExecutionMemory 
method was called.

Here is the debug log when OOM happened:

16/11/24 06:20:26 INFO TaskMemoryManager: 0 bytes of memory were used by task 
72969 but are not associated with specific consumers
16/11/24 06:20:26 INFO TaskMemoryManager: 18911656686 bytes of memory are used 
for execution and 1326241517 bytes of memory are used for storage
16/11/24 06:20:26 INFO TaskMemoryManager: Memory used in task 72969
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@45f5aaf1: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@25bc0eb9: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@7ca441ad: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@7c4b6a45: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@152d313e: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@f936c87: 64.0 
MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@578f36d: 64.0 
MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@4f4ab652: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@6947ad58: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@38eb70df: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@6c4c0f79: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@1d2f3c19: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@5d010e6: 64.0 
MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@19a2279b: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@7536c80b: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@53b3318d: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@509328c4: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@4b175ef: 64.0 
MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@69d667b7: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@313de45f: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@468cefe9: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@5d8b57ee: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@2f594f30: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@19e5588b: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@157c760c: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@237d740c: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@73f20a80: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@4be26b90: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@30394641: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsa

[jira] [Comment Edited] (SPARK-14560) Cooperative Memory Management for Spillables

2016-11-24 Thread Roy Wang (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-14560?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15694677#comment-15694677
 ] 

Roy Wang edited comment on SPARK-14560 at 11/25/16 3:01 AM:


UnsafeExternalSorter used a lot of memory but it looked like that 
TaskMemoryManager couldn't spill these consumers when acquireExecutionMemory 
method was called.
Theoretically, TaskMemoryManager should spill a comsumer itself at first and 
then spill other comsumers on the same memory mode if a comsumer want more 
memory. But it didn't work.

Here is the debug log when OOM happened:

16/11/24 06:20:26 INFO TaskMemoryManager: 0 bytes of memory were used by task 
72969 but are not associated with specific consumers
16/11/24 06:20:26 INFO TaskMemoryManager: 18911656686 bytes of memory are used 
for execution and 1326241517 bytes of memory are used for storage
16/11/24 06:20:26 INFO TaskMemoryManager: Memory used in task 72969
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@45f5aaf1: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@25bc0eb9: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@7ca441ad: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@7c4b6a45: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@152d313e: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@f936c87: 64.0 
MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@578f36d: 64.0 
MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@4f4ab652: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@6947ad58: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@38eb70df: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@6c4c0f79: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@1d2f3c19: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@5d010e6: 64.0 
MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@19a2279b: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@7536c80b: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@53b3318d: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@509328c4: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@4b175ef: 64.0 
MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@69d667b7: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@313de45f: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@468cefe9: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@5d8b57ee: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@2f594f30: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@19e5588b: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@157c760c: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@237d740c: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@73f20a80: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired by 
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter@4be26b90: 
64.0 MB
16/11/24 06:20:26 INFO TaskMemoryManager: Acquired

[jira] [Comment Edited] (SPARK-14560) Cooperative Memory Management for Spillables

2016-08-23 Thread Sean Owen (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-14560?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15433210#comment-15433210
 ] 

Sean Owen edited comment on SPARK-14560 at 8/23/16 5:11 PM:


I have a few somewhat-specific additional data points:

More memory didn't seem to help. A job that ran comfortably with tens of 
gigabytes total with Java serialization would fail even with almost a terabyte 
of memory available. The memory fraction was at the default of 0.75, or up to 
0.9. I don't think we tried less, on the theory that the shuffle memory ought 
to be tracked as part of the 'storage' memory?

But the same thing happened with the legacy memory manager.

Unhelpfully, the heap appeared full of byte[] and String.

The shuffle involved user classes that were reasonably complex: nested objects 
involving case classes, third-party library classes, etc. None of them were 
registered with Kryo. I tried registering most of them, on the theory that this 
was causing some in-memory serialized representation to become huge. It didn't 
seem to help, but I still wonder if there's a lead there. When Kryo doesn't 
know about a class it serializes its class name first, but not the class names 
of everything in the graph (right?) so it can only make so much difference. 
Java serialization does the same.

For the record, it's just this Spark app that reproduces it:
https://github.com/sryza/aas/blob/1st-edition/ch08-geotime/src/main/scala/com/cloudera/datascience/geotime/RunGeoTime.scala

I have not tried on Spark 2, only 1.6 (CDH 5.8 flavor).


was (Author: srowen):
I have a few somewhat-specific additional data points:

More memory didn't seem to help. A job that ran comfortably with tens of 
gigabytes total with Java serialization would fail even with almost a terabyte 
of memory available. The memory fraction was at the default of 0.75, or up to 
0.9. I don't think we tried less, on the theory that the shuffle memory ought 
to be tracked as part of the 'storage' memory?

But the same thing happened with the legacy memory manager.

Unhelpfully, the heap appeared full of byte[] and String.

The shuffle involved user classes that were reasonably complex: nested objects 
involving case classes, third-party library classes, etc. None of them were 
registered with Kryo. I tried registering most of them, on the theory that this 
was causing some in-memory serialized representation to become huge. It didn't 
seem to help, but I still wonder if there's a lead there. When Kryo doesn't 
know about a class it serializes its class name first, but not the class names 
of everything in the graph (right?) so it can only make so much difference. 
Java serialization does the same.

For the record, it's just this Spark app that reproduces it:
https://github.com/sryza/aas/blob/master/ch08-geotime/src/main/scala/com/cloudera/datascience/geotime/RunGeoTime.scala

I have not tried on Spark 2, only 1.6 (CDH 5.8 flavor).

> Cooperative Memory Management for Spillables
> 
>
> Key: SPARK-14560
> URL: https://issues.apache.org/jira/browse/SPARK-14560
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core
>Affects Versions: 1.6.1
>Reporter: Imran Rashid
>Assignee: Lianhui Wang
> Fix For: 2.0.0
>
>
> SPARK-10432 introduced cooperative memory management for SQL operators that 
> can spill; however, {{Spillable}} s used by the old RDD api still do not 
> cooperate.  This can lead to memory starvation, in particular on a 
> shuffle-to-shuffle stage, eventually resulting in errors like:
> {noformat}
> 16/03/28 08:59:54 INFO memory.TaskMemoryManager: Memory used in task 3081
> 16/03/28 08:59:54 INFO memory.TaskMemoryManager: Acquired by 
> org.apache.spark.shuffle.sort.ShuffleExternalSorter@69ab0291: 32.0 KB
> 16/03/28 08:59:54 INFO memory.TaskMemoryManager: 1317230346 bytes of memory 
> were used by task 3081 but are not associated with specific consumers
> 16/03/28 08:59:54 INFO memory.TaskMemoryManager: 1317263114 bytes of memory 
> are used for execution and 1710484 bytes of memory are used for storage
> 16/03/28 08:59:54 ERROR executor.Executor: Managed memory leak detected; size 
> = 1317230346 bytes, TID = 3081
> 16/03/28 08:59:54 ERROR executor.Executor: Exception in task 533.0 in stage 
> 3.0 (TID 3081)
> java.lang.OutOfMemoryError: Unable to acquire 75 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:346)
> at 
> org.apache.spark.shuffle.sort.ShuffleExternalSorter.insertRecord(ShuffleExternalSorter.java:367)
> at 
> org.apache.spark.shuffle.sort.UnsafeShuffleWriter.insertRecordIntoSorter