@@ -211,7 +211,74 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging {
211211 Utils .timeStringAsMs(get(key, defaultValue))
212212 }
213213
214+ /**
215+ * Get a size parameter as bytes; throws a NoSuchElementException if it's not set. If no
216+ * suffix is provided then bytes are assumed.
217+ * @throws NoSuchElementException
218+ */
219+ def getSizeAsBytes (key : String ): Long = {
220+ Utils .byteStringAsBytes(get(key))
221+ }
222+
223+ /**
224+ * Get a size parameter as bytes, falling back to a default if not set. If no
225+ * suffix is provided then bytes are assumed.
226+ */
227+ def getSizeAsBytes (key : String , defaultValue : String ): Long = {
228+ Utils .byteStringAsBytes(get(key, defaultValue))
229+ }
230+
231+ /**
232+ * Get a size parameter as Kibibytes; throws a NoSuchElementException if it's not set. If no
233+ * suffix is provided then Kibibytes are assumed.
234+ * @throws NoSuchElementException
235+ */
236+ def getSizeAsKb (key : String ): Long = {
237+ Utils .byteStringAsKb(get(key))
238+ }
239+
240+ /**
241+ * Get a size parameter as Kibibytes, falling back to a default if not set. If no
242+ * suffix is provided then Kibibytes are assumed.
243+ */
244+ def getSizeAsKb (key : String , defaultValue : String ): Long = {
245+ Utils .byteStringAsKb(get(key, defaultValue))
246+ }
247+
248+ /**
249+ * Get a size parameter as Mebibytes; throws a NoSuchElementException if it's not set. If no
250+ * suffix is provided then Mebibytes are assumed.
251+ * @throws NoSuchElementException
252+ */
253+ def getSizeAsMb (key : String ): Long = {
254+ Utils .byteStringAsMb(get(key))
255+ }
256+
257+ /**
258+ * Get a size parameter as Mebibytes, falling back to a default if not set. If no
259+ * suffix is provided then Mebibytes are assumed.
260+ */
261+ def getSizeAsMb (key : String , defaultValue : String ): Long = {
262+ Utils .byteStringAsMb(get(key, defaultValue))
263+ }
264+
265+ /**
266+ * Get a size parameter as Gibibytes; throws a NoSuchElementException if it's not set. If no
267+ * suffix is provided then Gibibytes are assumed.
268+ * @throws NoSuchElementException
269+ */
270+ def getSizeAsGb (key : String ): Long = {
271+ Utils .byteStringAsGb(get(key))
272+ }
214273
274+ /**
275+ * Get a size parameter as Gibibytes, falling back to a default if not set. If no
276+ * suffix is provided then Gibibytes are assumed.
277+ */
278+ def getSizeAsGb (key : String , defaultValue : String ): Long = {
279+ Utils .byteStringAsGb(get(key, defaultValue))
280+ }
281+
215282 /** Get a parameter as an Option */
216283 def getOption (key : String ): Option [String ] = {
217284 Option (settings.get(key)).orElse(getDeprecatedConfig(key, this ))
@@ -407,7 +474,13 @@ private[spark] object SparkConf extends Logging {
407474 " The spark.cache.class property is no longer being used! Specify storage levels using " +
408475 " the RDD.persist() method instead." ),
409476 DeprecatedConfig (" spark.yarn.user.classpath.first" , " 1.3" ,
410- " Please use spark.{driver,executor}.userClassPathFirst instead." ))
477+ " Please use spark.{driver,executor}.userClassPathFirst instead." ),
478+ DeprecatedConfig (" spark.kryoserializer.buffer.mb" , " 1.4" ,
479+ " Please use spark.kryoserializer.buffer instead. The default value for " +
480+ " spark.kryoserializer.buffer.mb was previously specified as '0.064'. Fractional values " +
481+ " are no longer accepted. To specify the equivalent now, one may use '64k'." )
482+ )
483+
411484 Map (configs.map { cfg => (cfg.key -> cfg) }:_* )
412485 }
413486
@@ -432,6 +505,21 @@ private[spark] object SparkConf extends Logging {
432505 AlternateConfig (" spark.yarn.applicationMaster.waitTries" , " 1.3" ,
433506 // Translate old value to a duration, with 10s wait time per try.
434507 translation = s => s " ${s.toLong * 10 }s " )),
508+ " spark.reducer.maxSizeInFlight" -> Seq (
509+ AlternateConfig (" spark.reducer.maxMbInFlight" , " 1.4" )),
510+ " spark.kryoserializer.buffer" ->
511+ Seq (AlternateConfig (" spark.kryoserializer.buffer.mb" , " 1.4" ,
512+ translation = s => s " ${s.toDouble * 1000 }k " )),
513+ " spark.kryoserializer.buffer.max" -> Seq (
514+ AlternateConfig (" spark.kryoserializer.buffer.max.mb" , " 1.4" )),
515+ " spark.shuffle.file.buffer" -> Seq (
516+ AlternateConfig (" spark.shuffle.file.buffer.kb" , " 1.4" )),
517+ " spark.executor.logs.rolling.maxSize" -> Seq (
518+ AlternateConfig (" spark.executor.logs.rolling.size.maxBytes" , " 1.4" )),
519+ " spark.io.compression.snappy.blockSize" -> Seq (
520+ AlternateConfig (" spark.io.compression.snappy.block.size" , " 1.4" )),
521+ " spark.io.compression.lz4.blockSize" -> Seq (
522+ AlternateConfig (" spark.io.compression.lz4.block.size" , " 1.4" )),
435523 " spark.rpc.numRetries" -> Seq (
436524 AlternateConfig (" spark.akka.num.retries" , " 1.4" )),
437525 " spark.rpc.retry.wait" -> Seq (
0 commit comments