You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Sean Owen (JIRA)" <ji...@apache.org> on 2016/08/23 17:12:22 UTC

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

    [ 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(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: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)
> {noformat}
> This can happen anytime the shuffle read side requires more memory than what is available for the task.  Since the shuffle-read side doubles its memory request each time, it can easily end up acquiring all of the available memory, even if it does not use it.  Eg., say that after the final spill, the shuffle-read side requires 10 MB more memory, and there is 15 MB of memory available.  But if it starts at 2 MB, it will double to 4, 8, and then request 16 MB of memory, and in fact get all available 15 MB.  Since the 15 MB of memory is sufficient, it will not spill, and will continue holding on to all available memory.  But this leaves *no* memory available for the shuffle-write side.  Since the shuffle-write side cannot request the shuffle-read side to free up memory, this leads to an OOM.
> The simple solution is to make {{Spillable}} implement {{MemoryConsumer}} as well, so RDDs can benefit from the cooperative memory management introduced by SPARK-10342.
> Note that an additional improvement would be for the shuffle-read side to simple release unused memory, without spilling, in case that would leave enough memory, and only spill if that was inadequate.  However that can come as a later improvement.
> *Workaround*:  You can set {{spark.shuffle.spill.numElementsForceSpillThreshold=N}} to force spilling to occur every {{N}} elements, thus preventing the shuffle-read side from ever grabbing all of the available memory.  However, this requires careful tuning of {{N}} to specific workloads: too big, and you will still get an OOM; too small, and there will be so much spilling that performance will suffer drastically.  Furthermore, this workaround uses an *undocumented* configuration with *no compatibility guarantees* for future versions of spark.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org