aboutsummaryrefslogtreecommitdiff
path: root/docs/configuration.md
diff options
context:
space:
mode:
authorIlya Ganelin <ilya.ganelin@capitalone.com>2015-04-28 12:18:55 -0700
committerAndrew Or <andrew@databricks.com>2015-04-28 12:18:55 -0700
commit2d222fb39dd978e5a33cde6ceb59307cbdf7b171 (patch)
tree8c235538b86a9bf5f6decd2f14e3073b75d348b5 /docs/configuration.md
parent8aab94d8984e9d12194dbda47b2e7d9dbc036889 (diff)
downloadspark-2d222fb39dd978e5a33cde6ceb59307cbdf7b171.tar.gz
spark-2d222fb39dd978e5a33cde6ceb59307cbdf7b171.tar.bz2
spark-2d222fb39dd978e5a33cde6ceb59307cbdf7b171.zip
[SPARK-5932] [CORE] Use consistent naming for size properties
I've added an interface to JavaUtils to do byte conversion and added hooks within Utils.scala to handle conversion within Spark code (like for time strings). I've added matching tests for size conversion, and then updated all deprecated configs and documentation as per SPARK-5933. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Closes #5574 from ilganeli/SPARK-5932 and squashes the following commits: 11f6999 [Ilya Ganelin] Nit fixes 49a8720 [Ilya Ganelin] Whitespace fix 2ab886b [Ilya Ganelin] Scala style fc85733 [Ilya Ganelin] Got rid of floating point math 852a407 [Ilya Ganelin] [SPARK-5932] Added much improved overflow handling. Can now handle sizes up to Long.MAX_VALUE Petabytes instead of being capped at Long.MAX_VALUE Bytes 9ee779c [Ilya Ganelin] Simplified fraction matches 22413b1 [Ilya Ganelin] Made MAX private 3dfae96 [Ilya Ganelin] Fixed some nits. Added automatic conversion of old paramter for kryoserializer.mb to new values. e428049 [Ilya Ganelin] resolving merge conflict 8b43748 [Ilya Ganelin] Fixed error in pattern matching for doubles 84a2581 [Ilya Ganelin] Added smoother handling of fractional values for size parameters. This now throws an exception and added a warning for old spark.kryoserializer.buffer d3d09b6 [Ilya Ganelin] [SPARK-5932] Fixing error in KryoSerializer fe286b4 [Ilya Ganelin] Resolved merge conflict c7803cd [Ilya Ganelin] Empty lines 54b78b4 [Ilya Ganelin] Simplified byteUnit class 69e2f20 [Ilya Ganelin] Updates to code f32bc01 [Ilya Ganelin] [SPARK-5932] Fixed error in API in SparkConf.scala where Kb conversion wasn't being done properly (was Mb). Added test cases for both timeUnit and ByteUnit conversion f15f209 [Ilya Ganelin] Fixed conversion of kryo buffer size 0f4443e [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-5932 35a7fa7 [Ilya Ganelin] Minor formatting 928469e [Ilya Ganelin] [SPARK-5932] Converted some longs to ints 5d29f90 [Ilya Ganelin] [SPARK-5932] Finished documentation updates 7a6c847 [Ilya Ganelin] [SPARK-5932] Updated spark.shuffle.file.buffer afc9a38 [Ilya Ganelin] [SPARK-5932] Updated spark.broadcast.blockSize and spark.storage.memoryMapThreshold ae7e9f6 [Ilya Ganelin] [SPARK-5932] Updated spark.io.compression.snappy.block.size 2d15681 [Ilya Ganelin] [SPARK-5932] Updated spark.executor.logs.rolling.size.maxBytes 1fbd435 [Ilya Ganelin] [SPARK-5932] Updated spark.broadcast.blockSize eba4de6 [Ilya Ganelin] [SPARK-5932] Updated spark.shuffle.file.buffer.kb b809a78 [Ilya Ganelin] [SPARK-5932] Updated spark.kryoserializer.buffer.max 0cdff35 [Ilya Ganelin] [SPARK-5932] Updated to use bibibytes in method names. Updated spark.kryoserializer.buffer.mb and spark.reducer.maxMbInFlight 475370a [Ilya Ganelin] [SPARK-5932] Simplified ByteUnit code, switched to using longs. Updated docs to clarify that we use kibi, mebi etc instead of kilo, mega 851d691 [Ilya Ganelin] [SPARK-5932] Updated memoryStringToMb to use new interfaces a9f4fcf [Ilya Ganelin] [SPARK-5932] Added unit tests for unit conversion 747393a [Ilya Ganelin] [SPARK-5932] Added unit tests for ByteString conversion 09ea450 [Ilya Ganelin] [SPARK-5932] Added byte string conversion to Jav utils 5390fd9 [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-5932 db9a963 [Ilya Ganelin] Closing second spark context 1dc0444 [Ilya Ganelin] Added ref equality check 8c884fa [Ilya Ganelin] Made getOrCreate synchronized cb0c6b7 [Ilya Ganelin] Doc updates and code cleanup 270cfe3 [Ilya Ganelin] [SPARK-6703] Documentation fixes 15e8dea [Ilya Ganelin] Updated comments and added MiMa Exclude 0e1567c [Ilya Ganelin] Got rid of unecessary option for AtomicReference dfec4da [Ilya Ganelin] Changed activeContext to AtomicReference 733ec9f [Ilya Ganelin] Fixed some bugs in test code 8be2f83 [Ilya Ganelin] Replaced match with if e92caf7 [Ilya Ganelin] [SPARK-6703] Added test to ensure that getOrCreate both allows creation, retrieval, and a second context if desired a99032f [Ilya Ganelin] Spacing fix d7a06b8 [Ilya Ganelin] Updated SparkConf class to add getOrCreate method. Started test suite implementation
Diffstat (limited to 'docs/configuration.md')
-rw-r--r--docs/configuration.md60
1 files changed, 35 insertions, 25 deletions
diff --git a/docs/configuration.md b/docs/configuration.md
index d587b91124..72105feba4 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -48,6 +48,17 @@ The following format is accepted:
5d (days)
1y (years)
+
+Properties that specify a byte size should be configured with a unit of size.
+The following format is accepted:
+
+ 1b (bytes)
+ 1k or 1kb (kibibytes = 1024 bytes)
+ 1m or 1mb (mebibytes = 1024 kibibytes)
+ 1g or 1gb (gibibytes = 1024 mebibytes)
+ 1t or 1tb (tebibytes = 1024 gibibytes)
+ 1p or 1pb (pebibytes = 1024 tebibytes)
+
## Dynamically Loading Spark Properties
In some cases, you may want to avoid hard-coding certain configurations in a `SparkConf`. For
instance, if you'd like to run the same application with different masters or different
@@ -272,12 +283,11 @@ Apart from these, the following properties are also available, and may be useful
</td>
</tr>
<tr>
- <td><code>spark.executor.logs.rolling.size.maxBytes</code></td>
+ <td><code>spark.executor.logs.rolling.maxSize</code></td>
<td>(none)</td>
<td>
Set the max size of the file by which the executor logs will be rolled over.
- Rolling is disabled by default. Value is set in terms of bytes.
- See <code>spark.executor.logs.rolling.maxRetainedFiles</code>
+ Rolling is disabled by default. See <code>spark.executor.logs.rolling.maxRetainedFiles</code>
for automatic cleaning of old logs.
</td>
</tr>
@@ -366,10 +376,10 @@ Apart from these, the following properties are also available, and may be useful
<table class="table">
<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
<tr>
- <td><code>spark.reducer.maxMbInFlight</code></td>
- <td>48</td>
+ <td><code>spark.reducer.maxSizeInFlight</code></td>
+ <td>48m</td>
<td>
- Maximum size (in megabytes) of map outputs to fetch simultaneously from each reduce task. Since
+ Maximum size of map outputs to fetch simultaneously from each reduce task. Since
each output requires us to create a buffer to receive it, this represents a fixed memory
overhead per reduce task, so keep it small unless you have a large amount of memory.
</td>
@@ -403,10 +413,10 @@ Apart from these, the following properties are also available, and may be useful
</td>
</tr>
<tr>
- <td><code>spark.shuffle.file.buffer.kb</code></td>
- <td>32</td>
+ <td><code>spark.shuffle.file.buffer</code></td>
+ <td>32k</td>
<td>
- Size of the in-memory buffer for each shuffle file output stream, in kilobytes. These buffers
+ Size of the in-memory buffer for each shuffle file output stream. These buffers
reduce the number of disk seeks and system calls made in creating intermediate shuffle files.
</td>
</tr>
@@ -582,18 +592,18 @@ Apart from these, the following properties are also available, and may be useful
</td>
</tr>
<tr>
- <td><code>spark.io.compression.lz4.block.size</code></td>
- <td>32768</td>
+ <td><code>spark.io.compression.lz4.blockSize</code></td>
+ <td>32k</td>
<td>
- Block size (in bytes) used in LZ4 compression, in the case when LZ4 compression codec
+ Block size used in LZ4 compression, in the case when LZ4 compression codec
is used. Lowering this block size will also lower shuffle memory usage when LZ4 is used.
</td>
</tr>
<tr>
- <td><code>spark.io.compression.snappy.block.size</code></td>
- <td>32768</td>
+ <td><code>spark.io.compression.snappy.blockSize</code></td>
+ <td>32k</td>
<td>
- Block size (in bytes) used in Snappy compression, in the case when Snappy compression codec
+ Block size used in Snappy compression, in the case when Snappy compression codec
is used. Lowering this block size will also lower shuffle memory usage when Snappy is used.
</td>
</tr>
@@ -641,19 +651,19 @@ Apart from these, the following properties are also available, and may be useful
</td>
</tr>
<tr>
- <td><code>spark.kryoserializer.buffer.max.mb</code></td>
- <td>64</td>
+ <td><code>spark.kryoserializer.buffer.max</code></td>
+ <td>64m</td>
<td>
- Maximum allowable size of Kryo serialization buffer, in megabytes. This must be larger than any
+ Maximum allowable size of Kryo serialization buffer. This must be larger than any
object you attempt to serialize. Increase this if you get a "buffer limit exceeded" exception
inside Kryo.
</td>
</tr>
<tr>
- <td><code>spark.kryoserializer.buffer.mb</code></td>
- <td>0.064</td>
+ <td><code>spark.kryoserializer.buffer</code></td>
+ <td>64k</td>
<td>
- Initial size of Kryo's serialization buffer, in megabytes. Note that there will be one buffer
+ Initial size of Kryo's serialization buffer. Note that there will be one buffer
<i>per core</i> on each worker. This buffer will grow up to
<code>spark.kryoserializer.buffer.max.mb</code> if needed.
</td>
@@ -698,9 +708,9 @@ Apart from these, the following properties are also available, and may be useful
<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
<tr>
<td><code>spark.broadcast.blockSize</code></td>
- <td>4096</td>
+ <td>4m</td>
<td>
- Size of each piece of a block in kilobytes for <code>TorrentBroadcastFactory</code>.
+ Size of each piece of a block for <code>TorrentBroadcastFactory</code>.
Too large a value decreases parallelism during broadcast (makes it slower); however, if it is
too small, <code>BlockManager</code> might take a performance hit.
</td>
@@ -816,9 +826,9 @@ Apart from these, the following properties are also available, and may be useful
</tr>
<tr>
<td><code>spark.storage.memoryMapThreshold</code></td>
- <td>2097152</td>
+ <td>2m</td>
<td>
- Size of a block, in bytes, above which Spark memory maps when reading a block from disk.
+ Size of a block above which Spark memory maps when reading a block from disk.
This prevents Spark from memory mapping very small blocks. In general, memory
mapping has high overhead for blocks close to or below the page size of the operating system.
</td>