Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -107,7 +107,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()

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -83,7 +83,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.
//
Expand Down
2 changes: 1 addition & 1 deletion docs/configuration.md
Original file line number Diff line number Diff line change
Expand Up @@ -258,7 +258,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.
Expand Down