-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-5932][CORE] Use consistent naming for size properties #5574
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
d7a06b8
a99032f
e92caf7
8be2f83
733ec9f
dfec4da
0e1567c
15e8dea
270cfe3
cb0c6b7
8c884fa
1dc0444
db9a963
5390fd9
09ea450
747393a
a9f4fcf
851d691
475370a
0cdff35
b809a78
eba4de6
1fbd435
2d15681
ae7e9f6
afc9a38
7a6c847
5d29f90
928469e
35a7fa7
0f4443e
f15f209
f32bc01
69e2f20
54b78b4
c7803cd
fe286b4
d3d09b6
84a2581
8b43748
e428049
3dfae96
22413b1
9ee779c
852a407
fc85733
2ab886b
49a8720
11f6999
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -211,7 +211,74 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { | |
| Utils.timeStringAsMs(get(key, defaultValue)) | ||
| } | ||
|
|
||
| /** | ||
| * Get a size parameter as bytes; throws a NoSuchElementException if it's not set. If no | ||
| * suffix is provided then bytes are assumed. | ||
| * @throws NoSuchElementException | ||
| */ | ||
| def getSizeAsBytes(key: String): Long = { | ||
| Utils.byteStringAsBytes(get(key)) | ||
| } | ||
|
|
||
| /** | ||
| * Get a size parameter as bytes, falling back to a default if not set. If no | ||
| * suffix is provided then bytes are assumed. | ||
| */ | ||
| def getSizeAsBytes(key: String, defaultValue: String): Long = { | ||
| Utils.byteStringAsBytes(get(key, defaultValue)) | ||
| } | ||
|
|
||
| /** | ||
| * Get a size parameter as Kibibytes; throws a NoSuchElementException if it's not set. If no | ||
| * suffix is provided then Kibibytes are assumed. | ||
| * @throws NoSuchElementException | ||
| */ | ||
| def getSizeAsKb(key: String): Long = { | ||
| Utils.byteStringAsKb(get(key)) | ||
| } | ||
|
|
||
| /** | ||
| * Get a size parameter as Kibibytes, falling back to a default if not set. If no | ||
| * suffix is provided then Kibibytes are assumed. | ||
| */ | ||
| def getSizeAsKb(key: String, defaultValue: String): Long = { | ||
| Utils.byteStringAsKb(get(key, defaultValue)) | ||
| } | ||
|
|
||
| /** | ||
| * Get a size parameter as Mebibytes; throws a NoSuchElementException if it's not set. If no | ||
| * suffix is provided then Mebibytes are assumed. | ||
| * @throws NoSuchElementException | ||
| */ | ||
| def getSizeAsMb(key: String): Long = { | ||
| Utils.byteStringAsMb(get(key)) | ||
| } | ||
|
|
||
| /** | ||
| * Get a size parameter as Mebibytes, falling back to a default if not set. If no | ||
| * suffix is provided then Mebibytes are assumed. | ||
| */ | ||
| def getSizeAsMb(key: String, defaultValue: String): Long = { | ||
| Utils.byteStringAsMb(get(key, defaultValue)) | ||
| } | ||
|
|
||
| /** | ||
| * Get a size parameter as Gibibytes; throws a NoSuchElementException if it's not set. If no | ||
| * suffix is provided then Gibibytes are assumed. | ||
| * @throws NoSuchElementException | ||
| */ | ||
| def getSizeAsGb(key: String): Long = { | ||
| Utils.byteStringAsGb(get(key)) | ||
| } | ||
|
|
||
| /** | ||
| * Get a size parameter as Gibibytes, falling back to a default if not set. If no | ||
| * suffix is provided then Gibibytes are assumed. | ||
| */ | ||
| def getSizeAsGb(key: String, defaultValue: String): Long = { | ||
| Utils.byteStringAsGb(get(key, defaultValue)) | ||
| } | ||
|
|
||
| /** Get a parameter as an Option */ | ||
| def getOption(key: String): Option[String] = { | ||
| Option(settings.get(key)).orElse(getDeprecatedConfig(key, this)) | ||
|
|
@@ -407,7 +474,13 @@ private[spark] object SparkConf extends Logging { | |
| "The spark.cache.class property is no longer being used! Specify storage levels using " + | ||
| "the RDD.persist() method instead."), | ||
| DeprecatedConfig("spark.yarn.user.classpath.first", "1.3", | ||
| "Please use spark.{driver,executor}.userClassPathFirst instead.")) | ||
| "Please use spark.{driver,executor}.userClassPathFirst instead."), | ||
| DeprecatedConfig("spark.kryoserializer.buffer.mb", "1.4", | ||
| "Please use spark.kryoserializer.buffer instead. The default value for " + | ||
| "spark.kryoserializer.buffer.mb was previously specified as '0.064'. Fractional values " + | ||
| "are no longer accepted. To specify the equivalent now, one may use '64k'.") | ||
| ) | ||
|
|
||
| Map(configs.map { cfg => (cfg.key -> cfg) }:_*) | ||
| } | ||
|
|
||
|
|
@@ -432,6 +505,21 @@ private[spark] object SparkConf extends Logging { | |
| AlternateConfig("spark.yarn.applicationMaster.waitTries", "1.3", | ||
| // Translate old value to a duration, with 10s wait time per try. | ||
| translation = s => s"${s.toLong * 10}s")), | ||
| "spark.reducer.maxSizeInFlight" -> Seq( | ||
| AlternateConfig("spark.reducer.maxMbInFlight", "1.4")), | ||
| "spark.kryoserializer.buffer" -> | ||
| Seq(AlternateConfig("spark.kryoserializer.buffer.mb", "1.4", | ||
| translation = s => s"${s.toDouble * 1000}k")), | ||
|
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This automatic translation may throw a NumberFormatException if someone tries to use the .mb parameter as "64k" (e.g. the correct new format). Is that a case we should be concerned with? There will be enough warnings and errors thrown for them to readily track down the problem and fix the erroneous config so this should be ok but want to confirm that.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think since you're adding the new config, it's fine to not allow the old config take the new style. If you really want to support that, you could try to parse the config using the new API in an exception handler.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. very small nit, but I would put the |
||
| "spark.kryoserializer.buffer.max" -> Seq( | ||
| AlternateConfig("spark.kryoserializer.buffer.max.mb", "1.4")), | ||
| "spark.shuffle.file.buffer" -> Seq( | ||
| AlternateConfig("spark.shuffle.file.buffer.kb", "1.4")), | ||
| "spark.executor.logs.rolling.maxSize" -> Seq( | ||
| AlternateConfig("spark.executor.logs.rolling.size.maxBytes", "1.4")), | ||
| "spark.io.compression.snappy.blockSize" -> Seq( | ||
| AlternateConfig("spark.io.compression.snappy.block.size", "1.4")), | ||
| "spark.io.compression.lz4.blockSize" -> Seq( | ||
| AlternateConfig("spark.io.compression.lz4.block.size", "1.4")), | ||
| "spark.rpc.numRetries" -> Seq( | ||
| AlternateConfig("spark.akka.num.retries", "1.4")), | ||
| "spark.rpc.retry.wait" -> Seq( | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -49,16 +49,17 @@ class KryoSerializer(conf: SparkConf) | |
| with Logging | ||
| with Serializable { | ||
|
|
||
| private val bufferSizeMb = conf.getDouble("spark.kryoserializer.buffer.mb", 0.064) | ||
| if (bufferSizeMb >= 2048) { | ||
| throw new IllegalArgumentException("spark.kryoserializer.buffer.mb must be less than " + | ||
| s"2048 mb, got: + $bufferSizeMb mb.") | ||
| private val bufferSizeKb = conf.getSizeAsKb("spark.kryoserializer.buffer", "64k") | ||
|
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. All - I don't know what the right solution here is. The old value simply can't work using the new framework. Fractional values are no longer supported and near as I can tell this is is the only instance of such usage. The only way to truly maintain backwards compatibility (short of throwing an exception) is to leave this as conf.getDouble but then this is an exception to the rule for how we handle size variables. Proper handling of fractional values throughout the code will be more intrusive and will require a lot of code modification all for the sake of this one parameter. This needs to be |
||
|
|
||
| if (bufferSizeKb >= 2048) { | ||
| throw new IllegalArgumentException("spark.kryoserializer.buffer must be less than " + | ||
| s"2048 mb, got: + $bufferSizeKb mb.") | ||
| } | ||
| private val bufferSize = (bufferSizeMb * 1024 * 1024).toInt | ||
| private val bufferSize = (bufferSizeKb * 1024).toInt | ||
|
|
||
| val maxBufferSizeMb = conf.getInt("spark.kryoserializer.buffer.max.mb", 64) | ||
| val maxBufferSizeMb = conf.getSizeAsMb("spark.kryoserializer.buffer.max", "64m").toInt | ||
| if (maxBufferSizeMb >= 2048) { | ||
| throw new IllegalArgumentException("spark.kryoserializer.buffer.max.mb must be less than " + | ||
| throw new IllegalArgumentException("spark.kryoserializer.buffer.max must be less than " + | ||
| s"2048 mb, got: + $maxBufferSizeMb mb.") | ||
| } | ||
| private val maxBufferSize = maxBufferSizeMb * 1024 * 1024 | ||
|
|
@@ -173,7 +174,7 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends Serializ | |
| } catch { | ||
| case e: KryoException if e.getMessage.startsWith("Buffer overflow") => | ||
| throw new SparkException(s"Kryo serialization failed: ${e.getMessage}. To avoid this, " + | ||
| "increase spark.kryoserializer.buffer.max.mb value.") | ||
| "increase spark.kryoserializer.buffer.max value.") | ||
| } | ||
| ByteBuffer.wrap(output.toBytes) | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -1041,21 +1041,48 @@ private[spark] object Utils extends Logging { | |
| } | ||
|
|
||
| /** | ||
| * Convert a Java memory parameter passed to -Xmx (such as 300m or 1g) to a number of megabytes. | ||
| * Convert a passed byte string (e.g. 50b, 100k, or 250m) to bytes for internal use. | ||
| * | ||
| * If no suffix is provided, the passed number is assumed to be in bytes. | ||
| */ | ||
| def byteStringAsBytes(str: String): Long = { | ||
| JavaUtils.byteStringAsBytes(str) | ||
| } | ||
|
|
||
| /** | ||
| * Convert a passed byte string (e.g. 50b, 100k, or 250m) to kibibytes for internal use. | ||
| * | ||
| * If no suffix is provided, the passed number is assumed to be in kibibytes. | ||
| */ | ||
| def byteStringAsKb(str: String): Long = { | ||
| JavaUtils.byteStringAsKb(str) | ||
| } | ||
|
|
||
| /** | ||
| * Convert a passed byte string (e.g. 50b, 100k, or 250m) to mebibytes for internal use. | ||
| * | ||
| * If no suffix is provided, the passed number is assumed to be in mebibytes. | ||
| */ | ||
| def byteStringAsMb(str: String): Long = { | ||
| JavaUtils.byteStringAsMb(str) | ||
| } | ||
|
|
||
| /** | ||
| * Convert a passed byte string (e.g. 50b, 100k, or 250m, 500g) to gibibytes for internal use. | ||
| * | ||
| * If no suffix is provided, the passed number is assumed to be in gibibytes. | ||
| */ | ||
| def byteStringAsGb(str: String): Long = { | ||
| JavaUtils.byteStringAsGb(str) | ||
| } | ||
|
|
||
| /** | ||
| * Convert a Java memory parameter passed to -Xmx (such as 300m or 1g) to a number of mebibytes. | ||
| */ | ||
| def memoryStringToMb(str: String): Int = { | ||
| val lower = str.toLowerCase | ||
| if (lower.endsWith("k")) { | ||
| (lower.substring(0, lower.length-1).toLong / 1024).toInt | ||
| } else if (lower.endsWith("m")) { | ||
| lower.substring(0, lower.length-1).toInt | ||
| } else if (lower.endsWith("g")) { | ||
| lower.substring(0, lower.length-1).toInt * 1024 | ||
| } else if (lower.endsWith("t")) { | ||
| lower.substring(0, lower.length-1).toInt * 1024 * 1024 | ||
| } else {// no suffix, so it's just a number in bytes | ||
| (lower.toLong / 1024 / 1024).toInt | ||
| } | ||
| // Convert to bytes, rather than directly to MB, because when no units are specified the unit | ||
| // is assumed to be bytes | ||
| (JavaUtils.byteStringAsBytes(str) / 1024 / 1024).toInt | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. super nit: you could drop the |
||
| } | ||
|
|
||
| /** | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
You could have a translation function for this, no? Like
spark.yarn.applicationMaster.waitTriesbelow.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Automatic translation would fail with a NumberFormatException though if they try to pass in a correctly formatted new value for the old config (e.g. 64k) and there's nowhere to catch that exception in this code.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Since you added the alternate config, you can remove this entry.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I wanted to still provide this message to clarify that fractional values are no longer allowed.