GitHub user andrewor14 opened a pull request:
https://github.com/apache/spark/pull/3353
[SPARK-4480] Avoid many small spills in external data structures
**Summary.** Currently, we may spill many small files in
`ExternalAppendOnlyMap` and `ExternalSorter`. The underlying root cause of this
is summarized in
[SPARK-4452](https://issues.apache.org/jira/browse/SPARK-4452). This PR does
not address this root cause, but simply provides the guarantee that we never
spill the in-memory data structure if its size is less than a hard-coded
threshold of `5MB`.
**Symptom.** Each spill is orders of magnitude smaller than 1MB, and there
are many spills. In environments where the ulimit is set, this frequently
causes "too many open file" exceptions observed in
[SPARK-3633](https://issues.apache.org/jira/browse/SPARK-3633).
```
14/11/13 19:20:43 INFO collection.ExternalSorter: Thread 60 spilling
in-memory batch of 4792 B to disk (292769 spills so far)
14/11/13 19:20:43 INFO collection.ExternalSorter: Thread 60 spilling
in-memory batch of 4760 B to disk (292770 spills so far)
14/11/13 19:20:43 INFO collection.ExternalSorter: Thread 60 spilling
in-memory batch of 4520 B to disk (292771 spills so far)
14/11/13 19:20:43 INFO collection.ExternalSorter: Thread 60 spilling
in-memory batch of 4560 B to disk (292772 spills so far)
14/11/13 19:20:43 INFO collection.ExternalSorter: Thread 60 spilling
in-memory batch of 4792 B to disk (292773 spills so far)
14/11/13 19:20:43 INFO collection.ExternalSorter: Thread 60 spilling
in-memory batch of 4784 B to disk (292774 spills so far)
```
**Reproduction.** I ran the following on a small 5-node cluster with
`512MB` executors. Note that the back-to-back shuffle here is necessary for
reasons described in
[SPARK-4522](https://issues.apache.org/jira/browse/SPARK-4452). The second
shuffle is a `reduceByKey` because it performs a map-side combine.
```
sc.parallelize(1 to 100000000, 100)
.map { i => (i, i) }
.groupByKey()
.reduceByKey(_ ++ _)
.count()
```
Before the change, I notice that each thread may spill up to 1000 times,
and the size of each spill is on the order of `10KB`. After the change, each
thread spills only up to 20 times in the worst case, and the size of each spill
is on the order of `1MB`.
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/andrewor14/spark avoid-small-spills
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/spark/pull/3353.patch
To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:
This closes #3353
----
commit a9197768432ef8446d4c1072e247c313356ae229
Author: Andrew Or <[email protected]>
Date: 2014-11-19T01:46:09Z
Avoid many small spills
----
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]