aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorReynold Xin <rxin@apache.org>2014-08-05 16:24:50 -0700
committerReynold Xin <rxin@apache.org>2014-08-05 16:24:50 -0700
commitacff9a7f13b98f10a08aea1d11cfa685c3419367 (patch)
tree8c0226b6eb12d4e480a240686f33e9b0d88d3361
parentcc491f69cd239ae7572f1f5f55a2452f7f417dc1 (diff)
downloadspark-acff9a7f13b98f10a08aea1d11cfa685c3419367.tar.gz
spark-acff9a7f13b98f10a08aea1d11cfa685c3419367.tar.bz2
spark-acff9a7f13b98f10a08aea1d11cfa685c3419367.zip
[SPARK-2503] Lower shuffle output buffer (spark.shuffle.file.buffer.kb) to 32KB.
This can substantially reduce memory usage during shuffle. Author: Reynold Xin <rxin@apache.org> Closes #1781 from rxin/SPARK-2503-spark.shuffle.file.buffer.kb and squashes the following commits: 104b8d8 [Reynold Xin] [SPARK-2503] Lower shuffle output buffer (spark.shuffle.file.buffer.kb) to 32KB.
-rw-r--r--core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala2
-rw-r--r--docs/configuration.md2
5 files changed, 5 insertions, 5 deletions
diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala
index 9a356d0dba..24db2f287a 100644
--- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala
+++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala
@@ -40,7 +40,7 @@ private[spark] class SortShuffleWriter[K, V, C](
private val ser = Serializer.getSerializer(dep.serializer.orNull)
private val conf = SparkEnv.get.conf
- private val fileBufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 100) * 1024
+ private val fileBufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 32) * 1024
private var sorter: ExternalSorter[K, V, _] = null
private var outputFile: File = null
diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala
index 28aa35bc7e..f9fdffae8b 100644
--- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala
@@ -73,7 +73,7 @@ class ShuffleBlockManager(blockManager: BlockManager) extends Logging {
val sortBasedShuffle =
conf.get("spark.shuffle.manager", "") == classOf[SortShuffleManager].getName
- private val bufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 100) * 1024
+ private val bufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 32) * 1024
/**
* Contains all the state related to a particular shuffle. This includes a pool of unused
diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala
index cc0423856c..260a5c3888 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala
@@ -101,7 +101,7 @@ class ExternalAppendOnlyMap[K, V, C](
private var _memoryBytesSpilled = 0L
private var _diskBytesSpilled = 0L
- private val fileBufferSize = sparkConf.getInt("spark.shuffle.file.buffer.kb", 100) * 1024
+ private val fileBufferSize = sparkConf.getInt("spark.shuffle.file.buffer.kb", 32) * 1024
private val keyComparator = new HashComparator[K]
private val ser = serializer.newInstance()
diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala
index 101c83b264..3f93afd57b 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala
@@ -84,7 +84,7 @@ private[spark] class ExternalSorter[K, V, C](
private val conf = SparkEnv.get.conf
private val spillingEnabled = conf.getBoolean("spark.shuffle.spill", true)
- private val fileBufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 100) * 1024
+ private val fileBufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 32) * 1024
// Size of object batches when reading/writing from serializers.
//
diff --git a/docs/configuration.md b/docs/configuration.md
index 5e7556c08e..7cd7f4124d 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -266,7 +266,7 @@ Apart from these, the following properties are also available, and may be useful
</tr>
<tr>
<td><code>spark.shuffle.file.buffer.kb</code></td>
- <td>100</td>
+ <td>32</td>
<td>
Size of the in-memory buffer for each shuffle file output stream, in kilobytes. These buffers
reduce the number of disk seeks and system calls made in creating intermediate shuffle files.