aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorPatrick Wendell <pwendell@gmail.com>2014-01-13 23:25:25 -0800
committerPatrick Wendell <pwendell@gmail.com>2014-01-13 23:25:25 -0800
commit0984647aaefffcecf85ebfbdb45e41ecd1e49a8c (patch)
treec415972d92d91fb6ce015df9948ae1d758b693e4
parentfdaabdc67387524ffb84354f87985f48bd31cf60 (diff)
downloadspark-0984647aaefffcecf85ebfbdb45e41ecd1e49a8c.tar.gz
spark-0984647aaefffcecf85ebfbdb45e41ecd1e49a8c.tar.bz2
spark-0984647aaefffcecf85ebfbdb45e41ecd1e49a8c.zip
Enable compression by default for spills
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockManager.scala2
-rw-r--r--docs/configuration.md2
2 files changed, 2 insertions, 2 deletions
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
index 6461deee32..ed53558566 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
@@ -81,7 +81,7 @@ private[spark] class BlockManager(
// Whether to compress RDD partitions that are stored serialized
val compressRdds = conf.getBoolean("spark.rdd.compress", false)
// Whether to compress shuffle output temporarily spilled to disk
- val compressShuffleSpill = conf.getBoolean("spark.shuffle.spill.compress", false)
+ val compressShuffleSpill = conf.getBoolean("spark.shuffle.spill.compress", true)
val heartBeatFrequency = BlockManager.getHeartBeatFrequency(conf)
diff --git a/docs/configuration.md b/docs/configuration.md
index be06bd19be..da70cabba2 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -156,7 +156,7 @@ Apart from these, the following properties are also available, and may be useful
</tr>
<tr>
<td>spark.shuffle.spill.compress</td>
- <td>false</td>
+ <td>true</td>
<td>
Whether to compress data spilled during shuffles.
</td>