From d7a06b8b773da4923f5504ee4387906a53249fa3 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Mon, 13 Apr 2015 16:50:41 -0700 Subject: [PATCH 01/45] Updated SparkConf class to add getOrCreate method. Started test suite implementation --- .../scala/org/apache/spark/SparkContext.scala | 15 ++++++++++++++- .../org/apache/spark/SparkContextSuite.scala | 17 +++++++++++++++++ 2 files changed, 31 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 3f1a7dd99d63..f3eb988fbd0d 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1813,7 +1813,7 @@ object SparkContext extends Logging { * Access to this field is guarded by SPARK_CONTEXT_CONSTRUCTOR_LOCK */ private var contextBeingConstructed: Option[SparkContext] = None - + /** * Called to ensure that no other SparkContext is running in this JVM. * @@ -1854,6 +1854,19 @@ object SparkContext extends Logging { } } + /** + * Because we can only have one active Spark Context per JVM and there are times when multiple + * applications may wish to share a SparkContext, this function may be used to get or instantiate + * a SparkContext and register it as a singleton object. + */ + private[spark] def getOrCreate(config: SparkConf = new SparkConf()): SparkContext = { + activeContext match { + case None => setActiveContext(new SparkContext(config), + config.getBoolean("spark.driver.allowMultipleContexts", false)) + } + activeContext.get + } + /** * Called at the beginning of the SparkContext constructor to ensure that no SparkContext is * running. Throws an exception if a running context is detected and logs a warning if another diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 94be1c6d6397..3422dcf2f564 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -67,6 +67,23 @@ class SparkContextSuite extends FunSuite with LocalSparkContext { } } + test("Test getOrCreateContext") { + SparkContext.clearActiveContext() + var context: SparkContext = + SparkContext.getOrCreate(new SparkConf().setAppName("test").setMaster("local") + .set("spark.driver.allowMultipleContexts", "true")) + + assert(context.getConf.getAppId) + try { + val conf = new SparkConf().setAppName("test").setMaster("local") + .set("spark.driver.allowMultipleContexts", "true") + sc = new SparkContext(conf) + secondSparkContext = new SparkContext(conf) + } finally { + Option(secondSparkContext).foreach(_.stop()) + } + } + test("BytesWritable implicit conversion is correct") { // Regression test for SPARK-3121 val bytesWritable = new BytesWritable() From a99032f4614589bbc9ad7dd1990b0953254bf649 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Tue, 14 Apr 2015 09:26:00 -0700 Subject: [PATCH 02/45] Spacing fix --- core/src/main/scala/org/apache/spark/SparkContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index f3eb988fbd0d..61d0203c365e 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1813,7 +1813,7 @@ object SparkContext extends Logging { * Access to this field is guarded by SPARK_CONTEXT_CONSTRUCTOR_LOCK */ private var contextBeingConstructed: Option[SparkContext] = None - + /** * Called to ensure that no other SparkContext is running in this JVM. * From e92caf7b4ffb4b5d3ef0e5a90bda0d5dafe96cab Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Tue, 14 Apr 2015 09:52:43 -0700 Subject: [PATCH 03/45] [SPARK-6703] Added test to ensure that getOrCreate both allows creation, retrieval, and a second context if desired --- .../scala/org/apache/spark/SparkContext.scala | 13 +++++++++-- .../org/apache/spark/SparkContextSuite.scala | 23 +++++++++---------- 2 files changed, 22 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 61d0203c365e..bc14b9a725a0 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1857,15 +1857,24 @@ object SparkContext extends Logging { /** * Because we can only have one active Spark Context per JVM and there are times when multiple * applications may wish to share a SparkContext, this function may be used to get or instantiate - * a SparkContext and register it as a singleton object. + * a SparkContext and register it as a singleton object. + * + * Note: This function cannot be used to create multiple spark contexts even if multiple contexts + * are allowed. Multiple contexts will still need to be created with explicit calls to the + * SparkContext constructor. */ - private[spark] def getOrCreate(config: SparkConf = new SparkConf()): SparkContext = { + def getOrCreate(config: SparkConf): SparkContext = { activeContext match { case None => setActiveContext(new SparkContext(config), config.getBoolean("spark.driver.allowMultipleContexts", false)) } activeContext.get } + + /** Allow not passing a SparkConf (useful if just retrieving) */ + def getOrCreate(): SparkContext = { + getOrCreate(new SparkConf()) + } /** * Called at the beginning of the SparkContext constructor to ensure that no SparkContext is diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 3422dcf2f564..a00206a2bb0a 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -68,20 +68,19 @@ class SparkContextSuite extends FunSuite with LocalSparkContext { } test("Test getOrCreateContext") { + var sc2: SparkContext = null SparkContext.clearActiveContext() - var context: SparkContext = - SparkContext.getOrCreate(new SparkConf().setAppName("test").setMaster("local") - .set("spark.driver.allowMultipleContexts", "true")) + val conf = new SparkConf().setAppName("test").setMaster("local") + .set("spark.driver.allowMultipleContexts", "true") - assert(context.getConf.getAppId) - try { - val conf = new SparkConf().setAppName("test").setMaster("local") - .set("spark.driver.allowMultipleContexts", "true") - sc = new SparkContext(conf) - secondSparkContext = new SparkContext(conf) - } finally { - Option(secondSparkContext).foreach(_.stop()) - } + sc = SparkContext.getOrCreate(conf) + + assert(sc.getConf.get("spark.app.name").equals("test")) + sc2 = SparkContext.getOrCreate(new SparkConf().setAppName("test2").setMaster("local")) + assert(sc2.getConf.get("spark.app.name").equals("test")) + + // Try creating second context to confirm that it's still possible, if desired + sc2 = new SparkContext(new SparkConf().setAppName("test3")) } test("BytesWritable implicit conversion is correct") { From 8be2f83c16f03b82a00f7e5b4eb21c42db9b77d2 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Tue, 14 Apr 2015 09:55:06 -0700 Subject: [PATCH 04/45] Replaced match with if --- core/src/main/scala/org/apache/spark/SparkContext.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index bc14b9a725a0..72ad082e1e31 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1864,9 +1864,9 @@ object SparkContext extends Logging { * SparkContext constructor. */ def getOrCreate(config: SparkConf): SparkContext = { - activeContext match { - case None => setActiveContext(new SparkContext(config), - config.getBoolean("spark.driver.allowMultipleContexts", false)) + if (activeContext.isEmpty) { + setActiveContext(new SparkContext(config), + config.getBoolean("spark.driver.allowMultipleContexts", false)) } activeContext.get } From 733ec9ff69dbc4dde5b9c6809133308248d16657 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Tue, 14 Apr 2015 09:56:22 -0700 Subject: [PATCH 05/45] Fixed some bugs in test code --- core/src/test/scala/org/apache/spark/SparkContextSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index a00206a2bb0a..582a3146ccd5 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -80,7 +80,8 @@ class SparkContextSuite extends FunSuite with LocalSparkContext { assert(sc2.getConf.get("spark.app.name").equals("test")) // Try creating second context to confirm that it's still possible, if desired - sc2 = new SparkContext(new SparkConf().setAppName("test3")) + sc2 = new SparkContext(new SparkConf().setAppName("test3").setMaster("local") + .set("spark.driver.allowMultipleContexts", "true")) } test("BytesWritable implicit conversion is correct") { From dfec4dae763e888a2e8ae267b1a2e21180c9decc Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Tue, 14 Apr 2015 10:11:57 -0700 Subject: [PATCH 06/45] Changed activeContext to AtomicReference --- .../scala/org/apache/spark/SparkContext.scala | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 72ad082e1e31..022f70ecbaa6 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -23,7 +23,7 @@ import java.io._ import java.lang.reflect.Constructor import java.net.URI import java.util.{Arrays, Properties, UUID} -import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} +import java.util.concurrent.atomic.{AtomicReference, AtomicBoolean, AtomicInteger} import java.util.UUID.randomUUID import scala.collection.{Map, Set} @@ -1804,7 +1804,8 @@ object SparkContext extends Logging { * * Access to this field is guarded by SPARK_CONTEXT_CONSTRUCTOR_LOCK */ - private var activeContext: Option[SparkContext] = None + private val activeContext: AtomicReference[Option[SparkContext]] = + new AtomicReference[Option[SparkContext]](None) /** * Points to a partially-constructed SparkContext if some thread is in the SparkContext @@ -1839,7 +1840,7 @@ object SparkContext extends Logging { logWarning(warnMsg) } - activeContext.foreach { ctx => + activeContext.get().foreach { ctx => val errMsg = "Only one SparkContext may be running in this JVM (see SPARK-2243)." + " To ignore this error, set spark.driver.allowMultipleContexts = true. " + s"The currently running SparkContext was created at:\n${ctx.creationSite.longForm}" @@ -1864,11 +1865,11 @@ object SparkContext extends Logging { * SparkContext constructor. */ def getOrCreate(config: SparkConf): SparkContext = { - if (activeContext.isEmpty) { + if (activeContext.get().isEmpty) { setActiveContext(new SparkContext(config), config.getBoolean("spark.driver.allowMultipleContexts", false)) } - activeContext.get + activeContext.get().get } /** Allow not passing a SparkConf (useful if just retrieving) */ @@ -1902,7 +1903,7 @@ object SparkContext extends Logging { SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized { assertNoOtherContextIsRunning(sc, allowMultipleContexts) contextBeingConstructed = None - activeContext = Some(sc) + activeContext.set(Some(sc)) } } @@ -1913,7 +1914,7 @@ object SparkContext extends Logging { */ private[spark] def clearActiveContext(): Unit = { SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized { - activeContext = None + activeContext.set(None) } } From 0e1567cd8dfd7cfc39940cfe8b17fe30f63242e2 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Tue, 14 Apr 2015 10:58:50 -0700 Subject: [PATCH 07/45] Got rid of unecessary option for AtomicReference --- .../scala/org/apache/spark/SparkContext.scala | 15 ++++++++------- project/MimaExcludes.scala | 4 ++++ 2 files changed, 12 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 022f70ecbaa6..5774c309853b 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1804,8 +1804,8 @@ object SparkContext extends Logging { * * Access to this field is guarded by SPARK_CONTEXT_CONSTRUCTOR_LOCK */ - private val activeContext: AtomicReference[Option[SparkContext]] = - new AtomicReference[Option[SparkContext]](None) + private val activeContext: AtomicReference[SparkContext] = + new AtomicReference[SparkContext](null) /** * Points to a partially-constructed SparkContext if some thread is in the SparkContext @@ -1840,7 +1840,8 @@ object SparkContext extends Logging { logWarning(warnMsg) } - activeContext.get().foreach { ctx => + if(activeContext.get() != null) { + val ctx = activeContext.get() val errMsg = "Only one SparkContext may be running in this JVM (see SPARK-2243)." + " To ignore this error, set spark.driver.allowMultipleContexts = true. " + s"The currently running SparkContext was created at:\n${ctx.creationSite.longForm}" @@ -1865,11 +1866,11 @@ object SparkContext extends Logging { * SparkContext constructor. */ def getOrCreate(config: SparkConf): SparkContext = { - if (activeContext.get().isEmpty) { + if (activeContext.get() != null) { setActiveContext(new SparkContext(config), config.getBoolean("spark.driver.allowMultipleContexts", false)) } - activeContext.get().get + activeContext.get() } /** Allow not passing a SparkConf (useful if just retrieving) */ @@ -1903,7 +1904,7 @@ object SparkContext extends Logging { SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized { assertNoOtherContextIsRunning(sc, allowMultipleContexts) contextBeingConstructed = None - activeContext.set(Some(sc)) + activeContext.set(sc) } } @@ -1914,7 +1915,7 @@ object SparkContext extends Logging { */ private[spark] def clearActiveContext(): Unit = { SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized { - activeContext.set(None) + activeContext.set(null) } } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 1564babefa62..7ef363a2f07a 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -68,6 +68,10 @@ object MimaExcludes { // SPARK-6693 add tostring with max lines and width for matrix ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.mllib.linalg.Matrix.toString") + )++ Seq( + // SPARK-6703 Add getOrCreate method to SparkContext + ProblemFilters.exclude[IncompatibleResultTypeProblem] + ("org.apache.spark.SparkContext.org$apache$spark$SparkContext$$activeContext") ) case v if v.startsWith("1.3") => From 15e8dea2ba4dc8965fe2da8cf88714021e217d19 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Tue, 14 Apr 2015 11:01:21 -0700 Subject: [PATCH 08/45] Updated comments and added MiMa Exclude --- core/src/main/scala/org/apache/spark/SparkContext.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 5774c309853b..1336f5e22e38 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1800,9 +1800,9 @@ object SparkContext extends Logging { private val SPARK_CONTEXT_CONSTRUCTOR_LOCK = new Object() /** - * The active, fully-constructed SparkContext. If no SparkContext is active, then this is `None`. + * The active, fully-constructed SparkContext. If no SparkContext is active, then this is `null`. * - * Access to this field is guarded by SPARK_CONTEXT_CONSTRUCTOR_LOCK + * Access to this field is guarded by SPARK_CONTEXT_CONSTRUCTOR_LOCK. */ private val activeContext: AtomicReference[SparkContext] = new AtomicReference[SparkContext](null) @@ -1840,7 +1840,7 @@ object SparkContext extends Logging { logWarning(warnMsg) } - if(activeContext.get() != null) { + if (activeContext.get() != null) { val ctx = activeContext.get() val errMsg = "Only one SparkContext may be running in this JVM (see SPARK-2243)." + " To ignore this error, set spark.driver.allowMultipleContexts = true. " + From 270cfe30da1a216278027276c3366d80271efa0b Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Tue, 14 Apr 2015 11:29:18 -0700 Subject: [PATCH 09/45] [SPARK-6703] Documentation fixes --- .../scala/org/apache/spark/SparkContext.scala | 24 ++++++++++++------- .../org/apache/spark/SparkContextSuite.scala | 3 +-- 2 files changed, 17 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 1336f5e22e38..1227b44f5b02 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1857,23 +1857,31 @@ object SparkContext extends Logging { } /** - * Because we can only have one active Spark Context per JVM and there are times when multiple - * applications may wish to share a SparkContext, this function may be used to get or instantiate - * a SparkContext and register it as a singleton object. + * This function may be used to get or instantiate a SparkContext and register it as a + * singleton object. Because we can only have one active SparkContext per JVM, + * this is useful when applications may wish to share a SparkContext. * - * Note: This function cannot be used to create multiple spark contexts even if multiple contexts - * are allowed. Multiple contexts will still need to be created with explicit calls to the - * SparkContext constructor. + * Note: This function cannot be used to create multiple SparkContexts even if multiple contexts + * are allowed. */ def getOrCreate(config: SparkConf): SparkContext = { - if (activeContext.get() != null) { + if (activeContext.get() == null) { setActiveContext(new SparkContext(config), config.getBoolean("spark.driver.allowMultipleContexts", false)) } activeContext.get() } - /** Allow not passing a SparkConf (useful if just retrieving) */ + /** + * This function may be used to get or instantiate a SparkContext and register it as a + * singleton object. Because we can only have one active SparkContext per JVM, + * this is useful when applications may wish to share a SparkContext. + * + * This method allows not passing a SparkConf (useful if just retrieving) + * + * Note: This function cannot be used to create multiple SparkContexts even if multiple contexts + * are allowed. + */ def getOrCreate(): SparkContext = { getOrCreate(new SparkConf()) } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 582a3146ccd5..a8be2bad25ee 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -67,11 +67,10 @@ class SparkContextSuite extends FunSuite with LocalSparkContext { } } - test("Test getOrCreateContext") { + test("Test getOrCreate") { var sc2: SparkContext = null SparkContext.clearActiveContext() val conf = new SparkConf().setAppName("test").setMaster("local") - .set("spark.driver.allowMultipleContexts", "true") sc = SparkContext.getOrCreate(conf) From cb0c6b73e212312ae990fc9c61a4ef5eb8555cd9 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Tue, 14 Apr 2015 11:32:23 -0700 Subject: [PATCH 10/45] Doc updates and code cleanup --- .../scala/org/apache/spark/SparkContext.scala | 15 +++++++-------- .../org/apache/spark/SparkContextSuite.scala | 1 + 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 1227b44f5b02..d8b476a9fe0d 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1860,14 +1860,13 @@ object SparkContext extends Logging { * This function may be used to get or instantiate a SparkContext and register it as a * singleton object. Because we can only have one active SparkContext per JVM, * this is useful when applications may wish to share a SparkContext. - * - * Note: This function cannot be used to create multiple SparkContexts even if multiple contexts - * are allowed. + * + * Note: This function cannot be used to create multiple SparkContext instances + * even if multiple contexts are allowed. */ def getOrCreate(config: SparkConf): SparkContext = { if (activeContext.get() == null) { - setActiveContext(new SparkContext(config), - config.getBoolean("spark.driver.allowMultipleContexts", false)) + setActiveContext(new SparkContext(config), allowMultipleContexts = false) } activeContext.get() } @@ -1877,10 +1876,10 @@ object SparkContext extends Logging { * singleton object. Because we can only have one active SparkContext per JVM, * this is useful when applications may wish to share a SparkContext. * - * This method allows not passing a SparkConf (useful if just retrieving) + * This method allows not passing a SparkConf (useful if just retrieving). * - * Note: This function cannot be used to create multiple SparkContexts even if multiple contexts - * are allowed. + * Note: This function cannot be used to create multiple SparkContext instances + * even if multiple contexts are allowed. */ def getOrCreate(): SparkContext = { getOrCreate(new SparkConf()) diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index a8be2bad25ee..a3723f9f30ec 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -77,6 +77,7 @@ class SparkContextSuite extends FunSuite with LocalSparkContext { assert(sc.getConf.get("spark.app.name").equals("test")) sc2 = SparkContext.getOrCreate(new SparkConf().setAppName("test2").setMaster("local")) assert(sc2.getConf.get("spark.app.name").equals("test")) + assert(sc === sc2) // Try creating second context to confirm that it's still possible, if desired sc2 = new SparkContext(new SparkConf().setAppName("test3").setMaster("local") From 8c884fae2c4f018e76e2b07a94891fca34657cf8 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Tue, 14 Apr 2015 11:37:19 -0700 Subject: [PATCH 11/45] Made getOrCreate synchronized --- .../src/main/scala/org/apache/spark/SparkContext.scala | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index d8b476a9fe0d..7a562e9e2bd4 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1865,10 +1865,14 @@ object SparkContext extends Logging { * even if multiple contexts are allowed. */ def getOrCreate(config: SparkConf): SparkContext = { - if (activeContext.get() == null) { - setActiveContext(new SparkContext(config), allowMultipleContexts = false) + // Synchronize to ensure that multiple create requests don't trigger an exception + // from assertNoOtherContextIsRunning within setActiveContext + SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized { + if (activeContext.get() == null) { + setActiveContext(new SparkContext(config), allowMultipleContexts = false) + } + activeContext.get() } - activeContext.get() } /** From 1dc044452faf11062aa6f54f01c5d29094ebc113 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Tue, 14 Apr 2015 11:38:23 -0700 Subject: [PATCH 12/45] Added ref equality check --- core/src/test/scala/org/apache/spark/SparkContextSuite.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index a3723f9f30ec..d1fee15d84ad 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -78,6 +78,7 @@ class SparkContextSuite extends FunSuite with LocalSparkContext { sc2 = SparkContext.getOrCreate(new SparkConf().setAppName("test2").setMaster("local")) assert(sc2.getConf.get("spark.app.name").equals("test")) assert(sc === sc2) + assert(sc eq sc2) // Try creating second context to confirm that it's still possible, if desired sc2 = new SparkContext(new SparkConf().setAppName("test3").setMaster("local") From db9a9631415d86ee36add876770fbde510530ead Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Fri, 17 Apr 2015 14:23:22 -0700 Subject: [PATCH 13/45] Closing second spark context --- core/src/test/scala/org/apache/spark/SparkContextSuite.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index d1fee15d84ad..728558a42478 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -83,6 +83,8 @@ class SparkContextSuite extends FunSuite with LocalSparkContext { // Try creating second context to confirm that it's still possible, if desired sc2 = new SparkContext(new SparkConf().setAppName("test3").setMaster("local") .set("spark.driver.allowMultipleContexts", "true")) + + sc2.stop() } test("BytesWritable implicit conversion is correct") { From 09ea450fd5dfa8849c71dbd7420e955d9c22887f Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Sat, 18 Apr 2015 09:15:36 -0700 Subject: [PATCH 14/45] [SPARK-5932] Added byte string conversion to Jav utils --- .../apache/spark/network/util/ByteUnit.java | 172 ++++++++++++++++++ .../apache/spark/network/util/JavaUtils.java | 90 ++++++++- 2 files changed, 261 insertions(+), 1 deletion(-) create mode 100644 network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java diff --git a/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java b/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java new file mode 100644 index 000000000000..6b97b346bee5 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java @@ -0,0 +1,172 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.network.util; + +/** + * Code based on https://github.com/fabian-barney/Utils + * Copyright 2011 Fabian Barney + * + * @author Fabian Barney + */ +public enum ByteUnit { + /** + *
+   * Byte (B)
+   * 1 Byte
+   */
+  BYTE {
+    @Override
+    public double toBytes(double d) {
+      return d;
+    }
+
+    @Override
+    public double convert(double d, ByteUnit u) {
+      return u.toBytes(d);
+    }
+  },
+
+  /**
+   * 
+   * Kilobyte (kB)
+   * 10^3 Byte = 1.000 Byte
+   */
+  KB {
+    @Override
+    public double toBytes(double d) {
+      return safeMulti(d, C_KB);
+    }
+
+    @Override
+    public double convert(double d, ByteUnit u) {
+      return u.toKB(d);
+    }
+  },
+
+  /**
+   * 
+   * Megabyte (MB)
+   * 10^6 Byte = 1.000.000 Byte
+   */
+  MB {
+    @Override
+    public double toBytes(double d) {
+      return safeMulti(d, C_MB);
+    }
+
+    @Override
+    public double convert(double d, ByteUnit u) {
+      return u.toMB(d);
+    }
+  },
+
+  /**
+   * 
+   * Gigabyte (GB)
+   * 10^9 Byte = 1.000.000.000 Byte
+   */
+  GB {
+    @Override
+    public double toBytes(double d) {
+      return safeMulti(d, C_GB);
+    }
+
+    @Override
+    public double convert(double d, ByteUnit u) {
+      return u.toGB(d);
+    }
+  },
+
+  /**
+   * 
+   * Terabyte (TB)
+   * 10^12 Byte = 1.000.000.000.000 Byte
+   */
+  TB {
+    @Override
+    public double toBytes(double d) {
+      return safeMulti(d, C_TB);
+    }
+
+    @Override
+    public double convert(double d, ByteUnit u) {
+      return u.toTB(d);
+    }
+  },
+
+  /**
+   * 
+   * Petabyte (PB)
+   * 10^15 Byte = 1.000.000.000.000.000 Byte
+   */
+  PB {
+    @Override
+    public double toBytes(double d) {
+      return safeMulti(d, C_PB);
+    }
+
+    @Override
+    public double convert(double d, ByteUnit u) {
+      return u.toPB(d);
+    }
+  };
+
+  static final double C_KB = Math.pow(10d, 3d);
+  static final double C_MB = Math.pow(10d, 6d);
+  static final double C_GB = Math.pow(10d, 9d);
+  static final double C_TB = Math.pow(10d, 12d);
+  static final double C_PB = Math.pow(10d, 15d);
+
+  private static final double MAX = Double.MAX_VALUE;
+
+  static double safeMulti(double d, double multi) {
+    double limit = MAX / multi;
+
+    if (d > limit) {
+      return Double.MAX_VALUE;
+    }
+    if (d < -limit) {
+      return Double.MIN_VALUE;
+    }
+
+    return d * multi;
+  }
+
+  public abstract double toBytes(double d);
+
+  public final double toKB(double d) {
+    return toBytes(d) / C_KB;
+  }
+
+  public final double toMB(double d) {
+    return toBytes(d) / C_MB;
+  }
+
+  public final double toGB(double d) {
+    return toBytes(d) / C_GB;
+  }
+
+  public final double toTB(double d) {
+    return toBytes(d) / C_TB;
+  }
+
+  public final double toPB(double d) {
+    return toBytes(d) / C_PB;
+  }
+
+  public abstract double convert(double d, ByteUnit u);
+}
diff --git a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
index b6fbace509a0..78970cd735e3 100644
--- a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
+++ b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
@@ -22,12 +22,14 @@
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.concurrent.TimeUnit;
+
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import com.google.common.base.Charsets;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableMap;
+import com.sun.javafx.css.SizeUnits;
 import io.netty.buffer.Unpooled;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -137,6 +139,16 @@ private static boolean isSymlink(File file) throws IOException {
       .put("d", TimeUnit.DAYS)
       .build();
 
+  private static ImmutableMap byteSuffixes =
+    ImmutableMap.builder()
+      .put("b", ByteUnit.BYTE)
+      .put("kb", ByteUnit.KB)
+      .put("mb", ByteUnit.MB)
+      .put("gb", ByteUnit.GB)
+      .put("tb", ByteUnit.TB)
+      .put("pb", ByteUnit.PB)
+      .build();
+
   /**
    * Convert a passed time string (e.g. 50s, 100ms, or 250us) to a time count for
    * internal use. If no suffix is provided a direct conversion is attempted.
@@ -164,7 +176,7 @@ private static long parseTimeString(String str, TimeUnit unit) {
       return unit.convert(val, suffix != null ? timeSuffixes.get(suffix) : unit);
     } catch (NumberFormatException e) {
       String timeError = "Time must be specified as seconds (s), " +
-              "milliseconds (ms), microseconds (us), minutes (m or min) hour (h), or day (d). " +
+              "milliseconds (ms), microseconds (us), minutes (m or min), hour (h), or day (d). " +
               "E.g. 50s, 100ms, or 250us.";
       
       throw new NumberFormatException(timeError + "\n" + e.getMessage());
@@ -186,5 +198,81 @@ public static long timeStringAsMs(String str) {
   public static long timeStringAsSec(String str) {
     return parseTimeString(str, TimeUnit.SECONDS);
   }
+  
+  /**
+   * Convert a passed byte string (e.g. 50b, 100kb, or 250mb) to a ByteUnit for
+   * internal use. If no suffix is provided a direct conversion of the provided default is 
+   * attempted.
+   */
+  private static long parseByteString(String str, ByteUnit unit) {
+    String lower = str.toLowerCase().trim();
+
+    try {
+      String suffix;
+      long val;
+      Matcher m = Pattern.compile("([0-9]+)([a-z]+)?").matcher(lower);
+      if (m.matches()) {
+        val = Long.parseLong(m.group(1));
+        suffix = m.group(2);
+      } else {
+        throw new NumberFormatException("Failed to parse byte string: " + str);
+      }
+
+      // Check for invalid suffixes
+      if (suffix != null && !byteSuffixes.containsKey(suffix)) {
+        throw new NumberFormatException("Invalid suffix: \"" + suffix + "\"");
+      }
+
+      // If suffix is valid use that, otherwise none was provided and use the default passed
+      return (long) unit.convert(val, suffix != null ? byteSuffixes.get(suffix) : unit);
+    } catch (NumberFormatException e) {
+      String timeError = "Size must be specified as bytes (b), " +
+        "kilobytes (kb), megabytes (mb), gigabytes (gb), terabytes (tb), or petabytes(pb). " +
+        "E.g. 50b, 100kb, or 250mb.";
+
+      throw new NumberFormatException(timeError + "\n" + e.getMessage());
+    }
+  }
+
+  /**
+   * Convert a passed byte string (e.g. 50b, 100kb, or 250mb) to bytes for
+   * internal use.
+   * 
+   * If no suffix is provided, the passed number is assumed to be in bytes.
+   */
+  public static long byteStringAsBytes(String str) {
+    return parseByteString(str, ByteUnit.BYTE);
+  }
+
+  /**
+   * Convert a passed byte string (e.g. 50b, 100kb, or 250mb) to kilobytes for
+   * internal use.
+   *
+   * If no suffix is provided, the passed number is assumed to be in kilobytes.
+   */
+  public static long byteStringAsKB(String str) {
+    return parseByteString(str, ByteUnit.KB);
+  }
+  
+  /**
+   * Convert a passed byte string (e.g. 50b, 100kb, or 250mb) to megabytes for
+   * internal use.
+   *
+   * If no suffix is provided, the passed number is assumed to be in megabytes.
+   */
+  public static long byteStringAsMB(String str) {
+    return parseByteString(str, ByteUnit.MB);
+  }
 
+  /**
+   * Convert a passed byte string (e.g. 50b, 100kb, or 250mb) to gigabytes for
+   * internal use.
+   *
+   * If no suffix is provided, the passed number is assumed to be in gigabytes.
+   */
+  public static long byteStringAsGB(String str) {
+    return parseByteString(str, ByteUnit.GB);
+  }
+
+  
 }

From 747393a601b44e309d680f88a6f373c203faa980 Mon Sep 17 00:00:00 2001
From: Ilya Ganelin 
Date: Sat, 18 Apr 2015 09:41:01 -0700
Subject: [PATCH 15/45] [SPARK-5932] Added unit tests for ByteString conversion

---
 .../scala/org/apache/spark/util/Utils.scala   | 40 ++++++++++++
 .../org/apache/spark/util/UtilsSuite.scala    | 63 +++++++++++++++++++
 .../apache/spark/network/util/JavaUtils.java  | 14 ++---
 3 files changed, 108 insertions(+), 9 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala
index 1029b0f9fce1..ff5231e2e997 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -1036,6 +1036,46 @@ private[spark] object Utils extends Logging {
     JavaUtils.timeStringAsSec(str)
   }
 
+  /**
+   * Convert a passed byte string (e.g. 50b, 100kb, or 250mb) 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, 100kb, or 250mb) to kilobytes for
+   * internal use.
+   *
+   * If no suffix is provided, the passed number is assumed to be in kilobytes.
+   */
+  def byteStringAsKB(str: String): Long = {
+    JavaUtils.byteStringAsKB(str)
+  }
+
+  /**
+   * Convert a passed byte string (e.g. 50b, 100kb, or 250mb) to megabytes for
+   * internal use.
+   *
+   * If no suffix is provided, the passed number is assumed to be in megabytes.
+   */
+  def byteStringAsMB(str: String): Long = {
+    JavaUtils.byteStringAsMB(str)
+  }
+
+  /**
+   * Convert a passed byte string (e.g. 50b, 100kb, or 250mb, 500gb) to gigabytes for
+   * internal use.
+   *
+   * If no suffix is provided, the passed number is assumed to be in gigabytes.
+   */
+  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 megabytes.
    */
diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
index fb97e650ff95..570baa3c9ac6 100644
--- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
@@ -33,6 +33,7 @@ import org.scalatest.FunSuite
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.Path
 
+import org.apache.spark.network.util.ByteUnit
 import org.apache.spark.SparkConf
 
 class UtilsSuite extends FunSuite with ResetSystemProperties {
@@ -63,6 +64,10 @@ class UtilsSuite extends FunSuite with ResetSystemProperties {
     assert(Utils.timeStringAsMs("1d") === TimeUnit.DAYS.toMillis(1))
     
     // Test invalid strings
+    intercept[NumberFormatException] {
+      Utils.timeStringAsMs("600l")
+    }
+    
     intercept[NumberFormatException] {
       Utils.timeStringAsMs("This breaks 600s")
     }
@@ -79,6 +84,64 @@ class UtilsSuite extends FunSuite with ResetSystemProperties {
       Utils.timeStringAsMs("This 123s breaks")
     }
   }
+
+  test("Test byteString conversion") {
+    // Test zero
+    assert(Utils.byteStringAsBytes("0") === 0)
+
+    assert(Utils.byteStringAsGB("1") === 1)
+    assert(Utils.byteStringAsGB("1g") === 1)
+    assert(Utils.byteStringAsGB("1000m") === 1)
+    assert(Utils.byteStringAsGB("1000000k") === 1)
+    assert(Utils.byteStringAsGB("1k") === ByteUnit.KB.toGB(1))
+    assert(Utils.byteStringAsGB("1m") === ByteUnit.MB.toGB(1))
+    assert(Utils.byteStringAsGB("1t") === ByteUnit.TB.toGB(1))
+    assert(Utils.byteStringAsGB("1p") === ByteUnit.PB.toGB(1))
+    
+    assert(Utils.byteStringAsMB("1") === 1)
+    assert(Utils.byteStringAsMB("1m") === 1)
+    assert(Utils.byteStringAsMB("1000k") === 1)
+    assert(Utils.byteStringAsMB("1k") === ByteUnit.KB.toMB(1))
+    assert(Utils.byteStringAsMB("1g") === ByteUnit.GB.toMB(1))
+    assert(Utils.byteStringAsMB("1t") === ByteUnit.TB.toMB(1))
+    assert(Utils.byteStringAsMB("1p") === ByteUnit.PB.toMB(1))
+
+    assert(Utils.byteStringAsKB("1") === 1)
+    assert(Utils.byteStringAsKB("1k") === 1)
+    assert(Utils.byteStringAsKB("1m") === ByteUnit.MB.toKB(1))
+    assert(Utils.byteStringAsKB("1g") === ByteUnit.GB.toKB(1))
+    assert(Utils.byteStringAsKB("1t") === ByteUnit.TB.toKB(1))
+    assert(Utils.byteStringAsKB("1p") === ByteUnit.PB.toKB(1))
+    
+    assert(Utils.byteStringAsBytes("1") === 1)
+    assert(Utils.byteStringAsBytes("1k") === ByteUnit.KB.toBytes(1))
+    assert(Utils.byteStringAsBytes("1m") === ByteUnit.MB.toBytes(1))
+    assert(Utils.byteStringAsBytes("1g") === ByteUnit.GB.toBytes(1))
+    assert(Utils.byteStringAsBytes("1t") === ByteUnit.TB.toBytes(1))
+    assert(Utils.byteStringAsBytes("1p") === ByteUnit.PB.toBytes(1))
+
+    // Test invalid strings
+    intercept[NumberFormatException] {
+      Utils.byteStringAsBytes("500ub")
+    }
+    
+    // Test invalid strings
+    intercept[NumberFormatException] {
+      Utils.byteStringAsBytes("This breaks 600b")
+    }
+
+    intercept[NumberFormatException] {
+      Utils.byteStringAsBytes("This breaks 600")
+    }
+
+    intercept[NumberFormatException] {
+      Utils.byteStringAsBytes("600gb This breaks")
+    }
+
+    intercept[NumberFormatException] {
+      Utils.byteStringAsBytes("This 123mb breaks")
+    }
+  }
   
   test("bytesToString") {
     assert(Utils.bytesToString(10) === "10.0 B")
diff --git a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
index 78970cd735e3..f659d9d66546 100644
--- a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
+++ b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
@@ -22,14 +22,12 @@
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.concurrent.TimeUnit;
-
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import com.google.common.base.Charsets;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableMap;
-import com.sun.javafx.css.SizeUnits;
 import io.netty.buffer.Unpooled;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -142,11 +140,11 @@ private static boolean isSymlink(File file) throws IOException {
   private static ImmutableMap byteSuffixes =
     ImmutableMap.builder()
       .put("b", ByteUnit.BYTE)
-      .put("kb", ByteUnit.KB)
-      .put("mb", ByteUnit.MB)
-      .put("gb", ByteUnit.GB)
-      .put("tb", ByteUnit.TB)
-      .put("pb", ByteUnit.PB)
+      .put("k", ByteUnit.KB)
+      .put("m", ByteUnit.MB)
+      .put("g", ByteUnit.GB)
+      .put("t", ByteUnit.TB)
+      .put("p", ByteUnit.PB)
       .build();
 
   /**
@@ -273,6 +271,4 @@ public static long byteStringAsMB(String str) {
   public static long byteStringAsGB(String str) {
     return parseByteString(str, ByteUnit.GB);
   }
-
-  
 }

From a9f4fcf4fab340795dd711e610e94d32febc0127 Mon Sep 17 00:00:00 2001
From: Ilya Ganelin 
Date: Sat, 18 Apr 2015 10:00:21 -0700
Subject: [PATCH 16/45] [SPARK-5932] Added unit tests for unit conversion

---
 core/src/test/scala/org/apache/spark/util/UtilsSuite.scala  | 6 +++---
 .../main/java/org/apache/spark/network/util/ByteUnit.java   | 4 +---
 .../main/java/org/apache/spark/network/util/JavaUtils.java  | 3 ++-
 3 files changed, 6 insertions(+), 7 deletions(-)

diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
index 570baa3c9ac6..4d096a642d07 100644
--- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
@@ -93,15 +93,15 @@ class UtilsSuite extends FunSuite with ResetSystemProperties {
     assert(Utils.byteStringAsGB("1g") === 1)
     assert(Utils.byteStringAsGB("1000m") === 1)
     assert(Utils.byteStringAsGB("1000000k") === 1)
-    assert(Utils.byteStringAsGB("1k") === ByteUnit.KB.toGB(1))
-    assert(Utils.byteStringAsGB("1m") === ByteUnit.MB.toGB(1))
+    assert(Utils.byteStringAsGB("1k") === 0)
+    assert(Utils.byteStringAsGB("1023m") === 1)
     assert(Utils.byteStringAsGB("1t") === ByteUnit.TB.toGB(1))
     assert(Utils.byteStringAsGB("1p") === ByteUnit.PB.toGB(1))
     
     assert(Utils.byteStringAsMB("1") === 1)
     assert(Utils.byteStringAsMB("1m") === 1)
     assert(Utils.byteStringAsMB("1000k") === 1)
-    assert(Utils.byteStringAsMB("1k") === ByteUnit.KB.toMB(1))
+    assert(Utils.byteStringAsMB("3645k") === 3)
     assert(Utils.byteStringAsMB("1g") === ByteUnit.GB.toMB(1))
     assert(Utils.byteStringAsMB("1t") === ByteUnit.TB.toMB(1))
     assert(Utils.byteStringAsMB("1p") === ByteUnit.PB.toMB(1))
diff --git a/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java b/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
index 6b97b346bee5..1f9fefe32afb 100644
--- a/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
+++ b/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
@@ -86,9 +86,7 @@ public double toBytes(double d) {
     }
 
     @Override
-    public double convert(double d, ByteUnit u) {
-      return u.toGB(d);
-    }
+    public double convert(double d, ByteUnit u) { return u.toGB(d); }
   },
 
   /**
diff --git a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
index f659d9d66546..e7a28ee81fce 100644
--- a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
+++ b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
@@ -222,7 +222,8 @@ private static long parseByteString(String str, ByteUnit unit) {
       }
 
       // If suffix is valid use that, otherwise none was provided and use the default passed
-      return (long) unit.convert(val, suffix != null ? byteSuffixes.get(suffix) : unit);
+      return new Double(
+        unit.convert(val, suffix != null ? byteSuffixes.get(suffix) : unit)).longValue();
     } catch (NumberFormatException e) {
       String timeError = "Size must be specified as bytes (b), " +
         "kilobytes (kb), megabytes (mb), gigabytes (gb), terabytes (tb), or petabytes(pb). " +

From 851d6917d8cbd05eb90eae763622c7dbc5743d67 Mon Sep 17 00:00:00 2001
From: Ilya Ganelin 
Date: Sat, 18 Apr 2015 10:14:24 -0700
Subject: [PATCH 17/45] [SPARK-5932] Updated memoryStringToMb to use new
 interfaces

---
 .../scala/org/apache/spark/util/Utils.scala   | 15 +++--------
 .../org/apache/spark/util/UtilsSuite.scala    | 12 ++++++---
 .../apache/spark/network/util/ByteUnit.java   | 25 ++++++++-----------
 3 files changed, 21 insertions(+), 31 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala
index ff5231e2e997..f316944e14e0 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -1080,18 +1080,9 @@ private[spark] object Utils extends Logging {
    * Convert a Java memory parameter passed to -Xmx (such as 300m or 1g) to a number of megabytes.
    */
   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) / 1048576.0).toInt
   }
 
   /**
diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
index 4d096a642d07..daca003151c4 100644
--- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
@@ -91,16 +91,20 @@ class UtilsSuite extends FunSuite with ResetSystemProperties {
 
     assert(Utils.byteStringAsGB("1") === 1)
     assert(Utils.byteStringAsGB("1g") === 1)
-    assert(Utils.byteStringAsGB("1000m") === 1)
-    assert(Utils.byteStringAsGB("1000000k") === 1)
+    assert(Utils.byteStringAsGB("1023m") === 0)
+    assert(Utils.byteStringAsGB("1024m") === 1)
+    assert(Utils.byteStringAsGB("1048575k") === 0)
+    assert(Utils.byteStringAsGB("1048576k") === 1)
     assert(Utils.byteStringAsGB("1k") === 0)
-    assert(Utils.byteStringAsGB("1023m") === 1)
     assert(Utils.byteStringAsGB("1t") === ByteUnit.TB.toGB(1))
     assert(Utils.byteStringAsGB("1p") === ByteUnit.PB.toGB(1))
     
     assert(Utils.byteStringAsMB("1") === 1)
     assert(Utils.byteStringAsMB("1m") === 1)
-    assert(Utils.byteStringAsMB("1000k") === 1)
+    assert(Utils.byteStringAsMB("1048575b") === 0)
+    assert(Utils.byteStringAsMB("1048576b") === 1)
+    assert(Utils.byteStringAsMB("1023k") === 0)
+    assert(Utils.byteStringAsMB("1024k") === 1)
     assert(Utils.byteStringAsMB("3645k") === 3)
     assert(Utils.byteStringAsMB("1g") === ByteUnit.GB.toMB(1))
     assert(Utils.byteStringAsMB("1t") === ByteUnit.TB.toMB(1))
diff --git a/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java b/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
index 1f9fefe32afb..21144432f243 100644
--- a/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
+++ b/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
@@ -42,8 +42,7 @@ public double convert(double d, ByteUnit u) {
 
   /**
    * 
-   * Kilobyte (kB)
-   * 10^3 Byte = 1.000 Byte
+   * Kilobyte (kB) = 1024 Byte
    */
   KB {
     @Override
@@ -59,8 +58,7 @@ public double convert(double d, ByteUnit u) {
 
   /**
    * 
-   * Megabyte (MB)
-   * 10^6 Byte = 1.000.000 Byte
+   * Megabyte (MB) = 1024 * 1024 Byte
    */
   MB {
     @Override
@@ -76,8 +74,7 @@ public double convert(double d, ByteUnit u) {
 
   /**
    * 
-   * Gigabyte (GB)
-   * 10^9 Byte = 1.000.000.000 Byte
+   * Gigabyte (GB) = 1024 * 1024 * 1024 Byte
    */
   GB {
     @Override
@@ -91,8 +88,7 @@ public double toBytes(double d) {
 
   /**
    * 
-   * Terabyte (TB)
-   * 10^12 Byte = 1.000.000.000.000 Byte
+   * Terabyte (TB) = 1024 * 1024 * 1024 * 1024 Byte
    */
   TB {
     @Override
@@ -108,8 +104,7 @@ public double convert(double d, ByteUnit u) {
 
   /**
    * 
-   * Petabyte (PB)
-   * 10^15 Byte = 1.000.000.000.000.000 Byte
+   * Petabyte (PB) = 1024 * 1024 * 1024 * 1024 * 1024 Byte
    */
   PB {
     @Override
@@ -123,11 +118,11 @@ public double convert(double d, ByteUnit u) {
     }
   };
 
-  static final double C_KB = Math.pow(10d, 3d);
-  static final double C_MB = Math.pow(10d, 6d);
-  static final double C_GB = Math.pow(10d, 9d);
-  static final double C_TB = Math.pow(10d, 12d);
-  static final double C_PB = Math.pow(10d, 15d);
+  static final double C_KB = 1024d;
+  static final double C_MB = Math.pow(1024d, 2d);
+  static final double C_GB = Math.pow(1024d, 3d);
+  static final double C_TB = Math.pow(1024d, 4d);
+  static final double C_PB = Math.pow(1024d, 5d);
 
   private static final double MAX = Double.MAX_VALUE;
 

From 475370af4eef847d54a94bd6f9e628224297e905 Mon Sep 17 00:00:00 2001
From: Ilya Ganelin 
Date: Sat, 18 Apr 2015 12:53:35 -0700
Subject: [PATCH 18/45] [SPARK-5932] Simplified ByteUnit code, switched to
 using longs. Updated docs to clarify that we use kibi, mebi etc instead of
 kilo, mega

---
 .../scala/org/apache/spark/util/Utils.scala   |  28 +--
 .../org/apache/spark/util/UtilsSuite.scala    |  52 +++---
 .../apache/spark/network/util/ByteUnit.java   | 165 +++++-------------
 .../apache/spark/network/util/JavaUtils.java  |  24 +--
 4 files changed, 100 insertions(+), 169 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala
index f316944e14e0..c2a731ae9da6 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -1037,7 +1037,7 @@ private[spark] object Utils extends Logging {
   }
 
   /**
-   * Convert a passed byte string (e.g. 50b, 100kb, or 250mb) to bytes for
+   * 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.
@@ -1047,37 +1047,37 @@ private[spark] object Utils extends Logging {
   }
 
   /**
-   * Convert a passed byte string (e.g. 50b, 100kb, or 250mb) to kilobytes for
+   * 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 kilobytes.
+   * If no suffix is provided, the passed number is assumed to be in kibibytes.
    */
-  def byteStringAsKB(str: String): Long = {
-    JavaUtils.byteStringAsKB(str)
+  def byteStringAsKb(str: String): Long = {
+    JavaUtils.byteStringAsKb(str)
   }
 
   /**
-   * Convert a passed byte string (e.g. 50b, 100kb, or 250mb) to megabytes for
+   * 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 megabytes.
+   * If no suffix is provided, the passed number is assumed to be in mebibytes.
    */
-  def byteStringAsMB(str: String): Long = {
-    JavaUtils.byteStringAsMB(str)
+  def byteStringAsMb(str: String): Long = {
+    JavaUtils.byteStringAsMb(str)
   }
 
   /**
-   * Convert a passed byte string (e.g. 50b, 100kb, or 250mb, 500gb) to gigabytes for
+   * 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 gigabytes.
+   * If no suffix is provided, the passed number is assumed to be in gibibytes.
    */
-  def byteStringAsGB(str: String): Long = {
-    JavaUtils.byteStringAsGB(str)
+  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 megabytes.
+   * Convert a Java memory parameter passed to -Xmx (such as 300m or 1g) to a number of mebibytes.
    */
   def memoryStringToMb(str: String): Int = {
     // Convert to bytes, rather than directly to MB, because when no units are specified the unit
diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
index daca003151c4..aacb4ac5156b 100644
--- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
@@ -89,33 +89,33 @@ class UtilsSuite extends FunSuite with ResetSystemProperties {
     // Test zero
     assert(Utils.byteStringAsBytes("0") === 0)
 
-    assert(Utils.byteStringAsGB("1") === 1)
-    assert(Utils.byteStringAsGB("1g") === 1)
-    assert(Utils.byteStringAsGB("1023m") === 0)
-    assert(Utils.byteStringAsGB("1024m") === 1)
-    assert(Utils.byteStringAsGB("1048575k") === 0)
-    assert(Utils.byteStringAsGB("1048576k") === 1)
-    assert(Utils.byteStringAsGB("1k") === 0)
-    assert(Utils.byteStringAsGB("1t") === ByteUnit.TB.toGB(1))
-    assert(Utils.byteStringAsGB("1p") === ByteUnit.PB.toGB(1))
+    assert(Utils.byteStringAsGb("1") === 1)
+    assert(Utils.byteStringAsGb("1g") === 1)
+    assert(Utils.byteStringAsGb("1023m") === 0)
+    assert(Utils.byteStringAsGb("1024m") === 1)
+    assert(Utils.byteStringAsGb("1048575k") === 0)
+    assert(Utils.byteStringAsGb("1048576k") === 1)
+    assert(Utils.byteStringAsGb("1k") === 0)
+    assert(Utils.byteStringAsGb("1t") === ByteUnit.TB.toGB(1))
+    assert(Utils.byteStringAsGb("1p") === ByteUnit.PB.toGB(1))
     
-    assert(Utils.byteStringAsMB("1") === 1)
-    assert(Utils.byteStringAsMB("1m") === 1)
-    assert(Utils.byteStringAsMB("1048575b") === 0)
-    assert(Utils.byteStringAsMB("1048576b") === 1)
-    assert(Utils.byteStringAsMB("1023k") === 0)
-    assert(Utils.byteStringAsMB("1024k") === 1)
-    assert(Utils.byteStringAsMB("3645k") === 3)
-    assert(Utils.byteStringAsMB("1g") === ByteUnit.GB.toMB(1))
-    assert(Utils.byteStringAsMB("1t") === ByteUnit.TB.toMB(1))
-    assert(Utils.byteStringAsMB("1p") === ByteUnit.PB.toMB(1))
-
-    assert(Utils.byteStringAsKB("1") === 1)
-    assert(Utils.byteStringAsKB("1k") === 1)
-    assert(Utils.byteStringAsKB("1m") === ByteUnit.MB.toKB(1))
-    assert(Utils.byteStringAsKB("1g") === ByteUnit.GB.toKB(1))
-    assert(Utils.byteStringAsKB("1t") === ByteUnit.TB.toKB(1))
-    assert(Utils.byteStringAsKB("1p") === ByteUnit.PB.toKB(1))
+    assert(Utils.byteStringAsMb("1") === 1)
+    assert(Utils.byteStringAsMb("1m") === 1)
+    assert(Utils.byteStringAsMb("1048575b") === 0)
+    assert(Utils.byteStringAsMb("1048576b") === 1)
+    assert(Utils.byteStringAsMb("1023k") === 0)
+    assert(Utils.byteStringAsMb("1024k") === 1)
+    assert(Utils.byteStringAsMb("3645k") === 3)
+    assert(Utils.byteStringAsMb("1g") === ByteUnit.GB.toMB(1))
+    assert(Utils.byteStringAsMb("1t") === ByteUnit.TB.toMB(1))
+    assert(Utils.byteStringAsMb("1p") === ByteUnit.PB.toMB(1))
+
+    assert(Utils.byteStringAsKb("1") === 1)
+    assert(Utils.byteStringAsKb("1k") === 1)
+    assert(Utils.byteStringAsKb("1m") === ByteUnit.MB.toKB(1))
+    assert(Utils.byteStringAsKb("1g") === ByteUnit.GB.toKB(1))
+    assert(Utils.byteStringAsKb("1t") === ByteUnit.TB.toKB(1))
+    assert(Utils.byteStringAsKb("1p") === ByteUnit.PB.toKB(1))
     
     assert(Utils.byteStringAsBytes("1") === 1)
     assert(Utils.byteStringAsBytes("1k") === ByteUnit.KB.toBytes(1))
diff --git a/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java b/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
index 21144432f243..7658a72663cf 100644
--- a/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
+++ b/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
@@ -16,150 +16,81 @@
  */
 package org.apache.spark.network.util;
 
-/**
- * Code based on https://github.com/fabian-barney/Utils
- * Copyright 2011 Fabian Barney
- *
- * @author Fabian Barney
- */
 public enum ByteUnit {
-  /**
-   * 
-   * Byte (B)
-   * 1 Byte
-   */
+  /** Byte (B) */
   BYTE {
-    @Override
-    public double toBytes(double d) {
-      return d;
-    }
-
-    @Override
-    public double convert(double d, ByteUnit u) {
-      return u.toBytes(d);
-    }
+    public long toBytes(long d) { return d; }
+    
+    public long convert(long d, ByteUnit u) { return u.toBytes(d); }
   },
 
-  /**
-   * 
-   * Kilobyte (kB) = 1024 Byte
-   */
+  /** Kibibyte (KiB) = 1024 Byte */
   KB {
-    @Override
-    public double toBytes(double d) {
-      return safeMulti(d, C_KB);
-    }
-
-    @Override
-    public double convert(double d, ByteUnit u) {
-      return u.toKB(d);
-    }
+    public long toBytes(long d) { return x(d, C_KB); }
+    
+    public long convert(long d, ByteUnit u) { return u.toKB(d); }
   },
 
-  /**
-   * 
-   * Megabyte (MB) = 1024 * 1024 Byte
-   */
+  /** Mebibyte (MiB) = (1024^2) Byte */
   MB {
-    @Override
-    public double toBytes(double d) {
-      return safeMulti(d, C_MB);
-    }
-
-    @Override
-    public double convert(double d, ByteUnit u) {
-      return u.toMB(d);
-    }
+    public long toBytes(long d) { return x(d, C_MB); }
+    
+    public long convert(long d, ByteUnit u) { return u.toMB(d); }
   },
 
-  /**
-   * 
-   * Gigabyte (GB) = 1024 * 1024 * 1024 Byte
-   */
+  /** Gibibyte (GiB) = (1024^3) Byte */
   GB {
-    @Override
-    public double toBytes(double d) {
-      return safeMulti(d, C_GB);
+    public long toBytes(long d) { return x(d, C_GB);
     }
 
-    @Override
-    public double convert(double d, ByteUnit u) { return u.toGB(d); }
+    public long convert(long d, ByteUnit u) { return u.toGB(d); }
   },
 
-  /**
-   * 
-   * Terabyte (TB) = 1024 * 1024 * 1024 * 1024 Byte
-   */
+  /** Tebibyte (TiB) = (1024^4) Byte */
   TB {
-    @Override
-    public double toBytes(double d) {
-      return safeMulti(d, C_TB);
-    }
-
-    @Override
-    public double convert(double d, ByteUnit u) {
-      return u.toTB(d);
-    }
+    public long toBytes(long d) { return x(d, C_TB); }
+    
+    public long convert(long d, ByteUnit u) { return u.toTB(d); }
   },
 
-  /**
-   * 
-   * Petabyte (PB) = 1024 * 1024 * 1024 * 1024 * 1024 Byte
-   */
+  /** Pebibyte (PB) = (1024^5) Byte */
   PB {
-    @Override
-    public double toBytes(double d) {
-      return safeMulti(d, C_PB);
-    }
-
-    @Override
-    public double convert(double d, ByteUnit u) {
-      return u.toPB(d);
-    }
+    public long toBytes(long d) { return x(d, C_PB); }
+    
+    public long convert(long d, ByteUnit u) { return u.toPB(d); }
   };
 
-  static final double C_KB = 1024d;
-  static final double C_MB = Math.pow(1024d, 2d);
-  static final double C_GB = Math.pow(1024d, 3d);
-  static final double C_TB = Math.pow(1024d, 4d);
-  static final double C_PB = Math.pow(1024d, 5d);
-
-  private static final double MAX = Double.MAX_VALUE;
+  static final long C_KB = 1024l;
+  static final long C_MB = (long) Math.pow(1024l, 2l);
+  static final long C_GB = (long) Math.pow(1024l, 3l);
+  static final long C_TB = (long) Math.pow(1024l, 4l);
+  static final long C_PB = (long) Math.pow(1024l, 5l);
 
-  static double safeMulti(double d, double multi) {
-    double limit = MAX / multi;
+  static final long MAX = Long.MAX_VALUE;
 
-    if (d > limit) {
-      return Double.MAX_VALUE;
-    }
-    if (d < -limit) {
-      return Double.MIN_VALUE;
-    }
-
-    return d * multi;
-  }
-
-  public abstract double toBytes(double d);
-
-  public final double toKB(double d) {
-    return toBytes(d) / C_KB;
+  /**
+   * Scale d by m, checking for overflow.
+   * This has a short name to make above code more readable.
+   */
+  static long x(long d, long m) {
+    long over = MAX / d;
+    if (d >  over) return Long.MAX_VALUE;
+    if (d < -over) return Long.MIN_VALUE;
+    return d * m;
   }
+  
+  public long toBytes(long d) { throw new AbstractMethodError(); }
+  public long convert(long d, ByteUnit u) { throw new AbstractMethodError(); }
+  
+  public long toKB(long d) { return toBytes(d) / C_KB; }
 
-  public final double toMB(double d) {
-    return toBytes(d) / C_MB;
-  }
+  public long toMB(long d) { return toBytes(d) / C_MB; }
 
-  public final double toGB(double d) {
-    return toBytes(d) / C_GB;
-  }
+  public long toGB(long d) { return toBytes(d) / C_GB; }
 
-  public final double toTB(double d) {
-    return toBytes(d) / C_TB;
-  }
+  public long toTB(long d) { return toBytes(d) / C_TB; }
 
-  public final double toPB(double d) {
-    return toBytes(d) / C_PB;
-  }
+  public long toPB(long d) { return toBytes(d) / C_PB; }
 
-  public abstract double convert(double d, ByteUnit u);
+  
 }
diff --git a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
index e7a28ee81fce..f0284b64abd0 100644
--- a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
+++ b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
@@ -226,15 +226,15 @@ private static long parseByteString(String str, ByteUnit unit) {
         unit.convert(val, suffix != null ? byteSuffixes.get(suffix) : unit)).longValue();
     } catch (NumberFormatException e) {
       String timeError = "Size must be specified as bytes (b), " +
-        "kilobytes (kb), megabytes (mb), gigabytes (gb), terabytes (tb), or petabytes(pb). " +
-        "E.g. 50b, 100kb, or 250mb.";
+        "kibibytes (kb), mebibytes (m), gibibytes (g), tebibytes (t), or pebibytes(p). " +
+        "E.g. 50b, 100k, or 250m.";
 
       throw new NumberFormatException(timeError + "\n" + e.getMessage());
     }
   }
 
   /**
-   * Convert a passed byte string (e.g. 50b, 100kb, or 250mb) to bytes for
+   * 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.
@@ -244,32 +244,32 @@ public static long byteStringAsBytes(String str) {
   }
 
   /**
-   * Convert a passed byte string (e.g. 50b, 100kb, or 250mb) to kilobytes for
+   * 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 kilobytes.
+   * If no suffix is provided, the passed number is assumed to be in kibibytes.
    */
-  public static long byteStringAsKB(String str) {
+  public static long byteStringAsKb(String str) {
     return parseByteString(str, ByteUnit.KB);
   }
   
   /**
-   * Convert a passed byte string (e.g. 50b, 100kb, or 250mb) to megabytes for
+   * 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 megabytes.
+   * If no suffix is provided, the passed number is assumed to be in mebibytes.
    */
-  public static long byteStringAsMB(String str) {
+  public static long byteStringAsMb(String str) {
     return parseByteString(str, ByteUnit.MB);
   }
 
   /**
-   * Convert a passed byte string (e.g. 50b, 100kb, or 250mb) to gigabytes for
+   * Convert a passed byte string (e.g. 50b, 100k, or 250m) to gibibytes for
    * internal use.
    *
-   * If no suffix is provided, the passed number is assumed to be in gigabytes.
+   * If no suffix is provided, the passed number is assumed to be in gibibytes.
    */
-  public static long byteStringAsGB(String str) {
+  public static long byteStringAsGb(String str) {
     return parseByteString(str, ByteUnit.GB);
   }
 }

From 0cdff352881a2a6c0927c38c52b9574c96b14ce7 Mon Sep 17 00:00:00 2001
From: Ilya Ganelin 
Date: Sat, 18 Apr 2015 13:20:43 -0700
Subject: [PATCH 19/45] [SPARK-5932] Updated to use bibibytes in method names.
 Updated spark.kryoserializer.buffer.mb and spark.reducer.maxMbInFlight

---
 .../scala/org/apache/spark/SparkConf.scala    | 83 ++++++++++++++++++-
 .../spark/serializer/KryoSerializer.scala     |  4 +-
 .../hash/BlockStoreShuffleFetcher.scala       |  3 +-
 .../org/apache/spark/DistributedSuite.scala   |  2 +-
 .../KryoSerializerResizableOutputSuite.scala  |  4 +-
 .../BlockManagerReplicationSuite.scala        |  2 +-
 .../spark/storage/BlockManagerSuite.scala     |  2 +-
 .../org/apache/spark/util/UtilsSuite.scala    | 28 +++----
 docs/configuration.md                         | 12 +--
 docs/tuning.md                                |  2 +-
 .../spark/examples/mllib/MovieLensALS.scala   |  2 +-
 .../apache/spark/network/util/ByteUnit.java   | 55 ++++++------
 .../apache/spark/network/util/JavaUtils.java  | 16 ++--
 13 files changed, 146 insertions(+), 69 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
index e3a649d75545..45d823298b31 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -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.byteStringAsMb(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.byteStringAsMb(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.reducer.maxMbInFlight", "1.4",
+        "Please use spark.reducer.maxSizeInFlight instead."),
+      DeprecatedConfig("spark.kryoserializer.buffer.mb", "1.4",
+        "Please use spark.kryoserializer.buffer instead."))
+
+    
     Map(configs.map { cfg => (cfg.key -> cfg) }:_*)
   }
 
@@ -431,8 +504,12 @@ private[spark] object SparkConf extends Logging {
     "spark.yarn.am.waitTime" -> Seq(
       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"))
-    )
+        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"))
+  )
 
   /**
    * A view of `configsWithAlternatives` that makes it more efficient to look up deprecated
diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
index 579fb6624e69..e64dfe27fff5 100644
--- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
@@ -49,9 +49,9 @@ class KryoSerializer(conf: SparkConf)
   with Logging
   with Serializable {
 
-  private val bufferSizeMb = conf.getDouble("spark.kryoserializer.buffer.mb", 0.064)
+  private val bufferSizeMb = conf.getSizeAsMb("spark.kryoserializer.buffer", "64k")
   if (bufferSizeMb >= 2048) {
-    throw new IllegalArgumentException("spark.kryoserializer.buffer.mb must be less than " +
+    throw new IllegalArgumentException("spark.kryoserializer.buffer must be less than " +
       s"2048 mb, got: + $bufferSizeMb mb.")
   }
   private val bufferSize = (bufferSizeMb * 1024 * 1024).toInt
diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala
index 7a2c5ae32d98..c1640754898e 100644
--- a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala
+++ b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala
@@ -79,7 +79,8 @@ private[hash] object BlockStoreShuffleFetcher extends Logging {
       blockManager,
       blocksByAddress,
       serializer,
-      SparkEnv.get.conf.getLong("spark.reducer.maxMbInFlight", 48) * 1024 * 1024)
+      // Note: we use getSizeAsMb to assume Mb when no suffix is provided
+      SparkEnv.get.conf.getSizeAsMb("spark.reducer.maxSizeInFlight", "48m") * 1024 * 1024) 
     val itr = blockFetcherItr.flatMap(unpackBlock)
 
     val completionIter = CompletionIterator[T, Iterator[T]](itr, {
diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala
index 97ea3578aa8b..96a9c207ad02 100644
--- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala
+++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala
@@ -77,7 +77,7 @@ class DistributedSuite extends FunSuite with Matchers with LocalSparkContext {
   }
 
   test("groupByKey where map output sizes exceed maxMbInFlight") {
-    val conf = new SparkConf().set("spark.reducer.maxMbInFlight", "1")
+    val conf = new SparkConf().set("spark.reducer.maxSizeInFlight", "1m")
     sc = new SparkContext(clusterUrl, "test", conf)
     // This data should be around 20 MB, so even with 4 mappers and 2 reducers, each map output
     // file should be about 2.5 MB
diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala
index 967c9e9899c9..f6738dee49a8 100644
--- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala
+++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala
@@ -33,7 +33,7 @@ class KryoSerializerResizableOutputSuite extends FunSuite {
   test("kryo without resizable output buffer should fail on large array") {
     val conf = new SparkConf(false)
     conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
-    conf.set("spark.kryoserializer.buffer.mb", "1")
+    conf.set("spark.kryoserializer.buffer", "1m")
     conf.set("spark.kryoserializer.buffer.max.mb", "1")
     val sc = new SparkContext("local", "test", conf)
     intercept[SparkException](sc.parallelize(x).collect())
@@ -43,7 +43,7 @@ class KryoSerializerResizableOutputSuite extends FunSuite {
   test("kryo with resizable output buffer should succeed on large array") {
     val conf = new SparkConf(false)
     conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
-    conf.set("spark.kryoserializer.buffer.mb", "1")
+    conf.set("spark.kryoserializer.buffer", "1m")
     conf.set("spark.kryoserializer.buffer.max.mb", "2")
     val sc = new SparkContext("local", "test", conf)
     assert(sc.parallelize(x).collect() === x)
diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala
index ffa5162a3184..f647200402ec 100644
--- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala
@@ -50,7 +50,7 @@ class BlockManagerReplicationSuite extends FunSuite with Matchers with BeforeAnd
   val allStores = new ArrayBuffer[BlockManager]
 
   // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test
-  conf.set("spark.kryoserializer.buffer.mb", "1")
+  conf.set("spark.kryoserializer.buffer", "1m")
   val serializer = new KryoSerializer(conf)
 
   // Implicitly convert strings to BlockIds for test clarity.
diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
index 545722b050ee..c21f38e81962 100644
--- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
@@ -55,7 +55,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach
   val shuffleManager = new HashShuffleManager(conf)
 
   // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test
-  conf.set("spark.kryoserializer.buffer.mb", "1")
+  conf.set("spark.kryoserializer.buffer", "1m")
   val serializer = new KryoSerializer(conf)
 
   // Implicitly convert strings to BlockIds for test clarity.
diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
index aacb4ac5156b..e8c64651b9ee 100644
--- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
@@ -96,8 +96,8 @@ class UtilsSuite extends FunSuite with ResetSystemProperties {
     assert(Utils.byteStringAsGb("1048575k") === 0)
     assert(Utils.byteStringAsGb("1048576k") === 1)
     assert(Utils.byteStringAsGb("1k") === 0)
-    assert(Utils.byteStringAsGb("1t") === ByteUnit.TB.toGB(1))
-    assert(Utils.byteStringAsGb("1p") === ByteUnit.PB.toGB(1))
+    assert(Utils.byteStringAsGb("1t") === ByteUnit.TiB.toGiB(1))
+    assert(Utils.byteStringAsGb("1p") === ByteUnit.PiB.toGiB(1))
     
     assert(Utils.byteStringAsMb("1") === 1)
     assert(Utils.byteStringAsMb("1m") === 1)
@@ -106,23 +106,23 @@ class UtilsSuite extends FunSuite with ResetSystemProperties {
     assert(Utils.byteStringAsMb("1023k") === 0)
     assert(Utils.byteStringAsMb("1024k") === 1)
     assert(Utils.byteStringAsMb("3645k") === 3)
-    assert(Utils.byteStringAsMb("1g") === ByteUnit.GB.toMB(1))
-    assert(Utils.byteStringAsMb("1t") === ByteUnit.TB.toMB(1))
-    assert(Utils.byteStringAsMb("1p") === ByteUnit.PB.toMB(1))
+    assert(Utils.byteStringAsMb("1g") === ByteUnit.GiB.toMiB(1))
+    assert(Utils.byteStringAsMb("1t") === ByteUnit.TiB.toMiB(1))
+    assert(Utils.byteStringAsMb("1p") === ByteUnit.PiB.toMiB(1))
 
     assert(Utils.byteStringAsKb("1") === 1)
     assert(Utils.byteStringAsKb("1k") === 1)
-    assert(Utils.byteStringAsKb("1m") === ByteUnit.MB.toKB(1))
-    assert(Utils.byteStringAsKb("1g") === ByteUnit.GB.toKB(1))
-    assert(Utils.byteStringAsKb("1t") === ByteUnit.TB.toKB(1))
-    assert(Utils.byteStringAsKb("1p") === ByteUnit.PB.toKB(1))
+    assert(Utils.byteStringAsKb("1m") === ByteUnit.MiB.toKiB(1))
+    assert(Utils.byteStringAsKb("1g") === ByteUnit.GiB.toKiB(1))
+    assert(Utils.byteStringAsKb("1t") === ByteUnit.TiB.toKiB(1))
+    assert(Utils.byteStringAsKb("1p") === ByteUnit.PiB.toKiB(1))
     
     assert(Utils.byteStringAsBytes("1") === 1)
-    assert(Utils.byteStringAsBytes("1k") === ByteUnit.KB.toBytes(1))
-    assert(Utils.byteStringAsBytes("1m") === ByteUnit.MB.toBytes(1))
-    assert(Utils.byteStringAsBytes("1g") === ByteUnit.GB.toBytes(1))
-    assert(Utils.byteStringAsBytes("1t") === ByteUnit.TB.toBytes(1))
-    assert(Utils.byteStringAsBytes("1p") === ByteUnit.PB.toBytes(1))
+    assert(Utils.byteStringAsBytes("1k") === ByteUnit.KiB.toBytes(1))
+    assert(Utils.byteStringAsBytes("1m") === ByteUnit.MiB.toBytes(1))
+    assert(Utils.byteStringAsBytes("1g") === ByteUnit.GiB.toBytes(1))
+    assert(Utils.byteStringAsBytes("1t") === ByteUnit.TiB.toBytes(1))
+    assert(Utils.byteStringAsBytes("1p") === ByteUnit.PiB.toBytes(1))
 
     // Test invalid strings
     intercept[NumberFormatException] {
diff --git a/docs/configuration.md b/docs/configuration.md
index d9e9e67026cb..992baf49ca47 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -366,10 +366,10 @@ Apart from these, the following properties are also available, and may be useful
 
-  
-  
+  
+  
@@ -650,10 +650,10 @@ Apart from these, the following properties are also available, and may be useful
   
-  
-  
+  
+  
diff --git a/docs/tuning.md b/docs/tuning.md
index cbd227868b24..1cb223e74f38 100644
--- a/docs/tuning.md
+++ b/docs/tuning.md
@@ -60,7 +60,7 @@ val sc = new SparkContext(conf)
 The [Kryo documentation](https://github.com/EsotericSoftware/kryo) describes more advanced
 registration options, such as adding custom serialization code.
 
-If your objects are large, you may also need to increase the `spark.kryoserializer.buffer.mb`
+If your objects are large, you may also need to increase the `spark.kryoserializer.buffer`
 config property. The default is 2, but this value needs to be large enough to hold the *largest*
 object you will serialize.
 
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala
index 0bc36ea65e1a..99588b0984ab 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala
@@ -100,7 +100,7 @@ object MovieLensALS {
     val conf = new SparkConf().setAppName(s"MovieLensALS with $params")
     if (params.kryo) {
       conf.registerKryoClasses(Array(classOf[mutable.BitSet], classOf[Rating]))
-        .set("spark.kryoserializer.buffer.mb", "8")
+        .set("spark.kryoserializer.buffer", "8m")
     }
     val sc = new SparkContext(conf)
 
diff --git a/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java b/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
index 7658a72663cf..e7de87062eb6 100644
--- a/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
+++ b/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
@@ -25,46 +25,46 @@ public enum ByteUnit {
   },
 
   /** Kibibyte (KiB) = 1024 Byte */
-  KB {
-    public long toBytes(long d) { return x(d, C_KB); }
+  KiB {
+    public long toBytes(long d) { return x(d, C_KiB); }
     
-    public long convert(long d, ByteUnit u) { return u.toKB(d); }
+    public long convert(long d, ByteUnit u) { return u.toKiB(d); }
   },
 
   /** Mebibyte (MiB) = (1024^2) Byte */
-  MB {
-    public long toBytes(long d) { return x(d, C_MB); }
+  MiB {
+    public long toBytes(long d) { return x(d, C_MiB); }
     
-    public long convert(long d, ByteUnit u) { return u.toMB(d); }
+    public long convert(long d, ByteUnit u) { return u.toMiB(d); }
   },
 
   /** Gibibyte (GiB) = (1024^3) Byte */
-  GB {
-    public long toBytes(long d) { return x(d, C_GB);
+  GiB {
+    public long toBytes(long d) { return x(d, C_GiB);
     }
 
-    public long convert(long d, ByteUnit u) { return u.toGB(d); }
+    public long convert(long d, ByteUnit u) { return u.toGiB(d); }
   },
 
   /** Tebibyte (TiB) = (1024^4) Byte */
-  TB {
-    public long toBytes(long d) { return x(d, C_TB); }
+  TiB {
+    public long toBytes(long d) { return x(d, C_TiB); }
     
-    public long convert(long d, ByteUnit u) { return u.toTB(d); }
+    public long convert(long d, ByteUnit u) { return u.toTiB(d); }
   },
 
   /** Pebibyte (PB) = (1024^5) Byte */
-  PB {
-    public long toBytes(long d) { return x(d, C_PB); }
+  PiB {
+    public long toBytes(long d) { return x(d, C_PiB); }
     
-    public long convert(long d, ByteUnit u) { return u.toPB(d); }
+    public long convert(long d, ByteUnit u) { return u.toPiB(d); }
   };
 
-  static final long C_KB = 1024l;
-  static final long C_MB = (long) Math.pow(1024l, 2l);
-  static final long C_GB = (long) Math.pow(1024l, 3l);
-  static final long C_TB = (long) Math.pow(1024l, 4l);
-  static final long C_PB = (long) Math.pow(1024l, 5l);
+  static final long C_KiB = 1024l;
+  static final long C_MiB = (long) Math.pow(1024l, 2l);
+  static final long C_GiB = (long) Math.pow(1024l, 3l);
+  static final long C_TiB = (long) Math.pow(1024l, 4l);
+  static final long C_PiB = (long) Math.pow(1024l, 5l);
 
   static final long MAX = Long.MAX_VALUE;
 
@@ -78,19 +78,18 @@ static long x(long d, long m) {
     if (d < -over) return Long.MIN_VALUE;
     return d * m;
   }
+
+  public long convert(long d, ByteUnit u) { throw new AbstractMethodError(); }
   
   public long toBytes(long d) { throw new AbstractMethodError(); }
-  public long convert(long d, ByteUnit u) { throw new AbstractMethodError(); }
   
-  public long toKB(long d) { return toBytes(d) / C_KB; }
+  public long toKiB(long d) { return toBytes(d) / C_KiB; }
 
-  public long toMB(long d) { return toBytes(d) / C_MB; }
+  public long toMiB(long d) { return toBytes(d) / C_MiB; }
 
-  public long toGB(long d) { return toBytes(d) / C_GB; }
+  public long toGiB(long d) { return toBytes(d) / C_GiB; }
 
-  public long toTB(long d) { return toBytes(d) / C_TB; }
+  public long toTiB(long d) { return toBytes(d) / C_TiB; }
 
-  public long toPB(long d) { return toBytes(d) / C_PB; }
-
-  
+  public long toPiB(long d) { return toBytes(d) / C_PiB; }
 }
diff --git a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
index f0284b64abd0..25ab9e973b48 100644
--- a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
+++ b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
@@ -140,11 +140,11 @@ private static boolean isSymlink(File file) throws IOException {
   private static ImmutableMap byteSuffixes =
     ImmutableMap.builder()
       .put("b", ByteUnit.BYTE)
-      .put("k", ByteUnit.KB)
-      .put("m", ByteUnit.MB)
-      .put("g", ByteUnit.GB)
-      .put("t", ByteUnit.TB)
-      .put("p", ByteUnit.PB)
+      .put("k", ByteUnit.KiB)
+      .put("m", ByteUnit.MiB)
+      .put("g", ByteUnit.GiB)
+      .put("t", ByteUnit.TiB)
+      .put("p", ByteUnit.PiB)
       .build();
 
   /**
@@ -250,7 +250,7 @@ public static long byteStringAsBytes(String str) {
    * If no suffix is provided, the passed number is assumed to be in kibibytes.
    */
   public static long byteStringAsKb(String str) {
-    return parseByteString(str, ByteUnit.KB);
+    return parseByteString(str, ByteUnit.KiB);
   }
   
   /**
@@ -260,7 +260,7 @@ public static long byteStringAsKb(String str) {
    * If no suffix is provided, the passed number is assumed to be in mebibytes.
    */
   public static long byteStringAsMb(String str) {
-    return parseByteString(str, ByteUnit.MB);
+    return parseByteString(str, ByteUnit.MiB);
   }
 
   /**
@@ -270,6 +270,6 @@ public static long byteStringAsMb(String str) {
    * If no suffix is provided, the passed number is assumed to be in gibibytes.
    */
   public static long byteStringAsGb(String str) {
-    return parseByteString(str, ByteUnit.GB);
+    return parseByteString(str, ByteUnit.GiB);
   }
 }

From b809a78c6de3f35df86b23785f9575ad61295d23 Mon Sep 17 00:00:00 2001
From: Ilya Ganelin 
Date: Sat, 18 Apr 2015 13:26:09 -0700
Subject: [PATCH 20/45] [SPARK-5932] Updated spark.kryoserializer.buffer.max

---
 core/src/main/scala/org/apache/spark/SparkConf.scala      | 8 ++++++--
 .../org/apache/spark/serializer/KryoSerializer.scala      | 6 +++---
 .../serializer/KryoSerializerResizableOutputSuite.scala   | 4 ++--
 .../org/apache/spark/serializer/KryoSerializerSuite.scala | 2 +-
 docs/configuration.md                                     | 6 +++---
 5 files changed, 15 insertions(+), 11 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
index 45d823298b31..a7e4e8a06821 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -478,7 +478,9 @@ private[spark] object SparkConf extends Logging {
       DeprecatedConfig("spark.reducer.maxMbInFlight", "1.4",
         "Please use spark.reducer.maxSizeInFlight instead."),
       DeprecatedConfig("spark.kryoserializer.buffer.mb", "1.4",
-        "Please use spark.kryoserializer.buffer instead."))
+        "Please use spark.kryoserializer.buffer instead."),
+      DeprecatedConfig("spark.kryoserializer.buffer.max.mb", "1.4",
+        "Please use spark.kryoserializer.buffer.max instead."))
 
     
     Map(configs.map { cfg => (cfg.key -> cfg) }:_*)
@@ -508,7 +510,9 @@ private[spark] object SparkConf extends Logging {
     "spark.reducer.maxSizeInFlight" -> Seq(
       AlternateConfig("spark.reducer.maxMbInFlight", "1.4")),
     "spark.kryoserializer.buffer" -> Seq(
-      AlternateConfig("spark.kryoserializer.buffer.mb", "1.4"))
+      AlternateConfig("spark.kryoserializer.buffer.mb", "1.4")),
+    "spark.kryoserializer.buffer.max" -> Seq(
+      AlternateConfig("spark.kryoserializer.buffer.max.mb", "1.4"))
   )
 
   /**
diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
index e64dfe27fff5..a2e63e66548e 100644
--- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
@@ -56,9 +56,9 @@ class KryoSerializer(conf: SparkConf)
   }
   private val bufferSize = (bufferSizeMb * 1024 * 1024).toInt
 
-  val maxBufferSizeMb = conf.getInt("spark.kryoserializer.buffer.max.mb", 64)
+  val maxBufferSizeMb = conf.getSizeAsMb("spark.kryoserializer.buffer.max", "64m")
   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 +173,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)
   }
diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala
index f6738dee49a8..da98d0918473 100644
--- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala
+++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala
@@ -34,7 +34,7 @@ class KryoSerializerResizableOutputSuite extends FunSuite {
     val conf = new SparkConf(false)
     conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
     conf.set("spark.kryoserializer.buffer", "1m")
-    conf.set("spark.kryoserializer.buffer.max.mb", "1")
+    conf.set("spark.kryoserializer.buffer.max", "1m")
     val sc = new SparkContext("local", "test", conf)
     intercept[SparkException](sc.parallelize(x).collect())
     LocalSparkContext.stop(sc)
@@ -44,7 +44,7 @@ class KryoSerializerResizableOutputSuite extends FunSuite {
     val conf = new SparkConf(false)
     conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
     conf.set("spark.kryoserializer.buffer", "1m")
-    conf.set("spark.kryoserializer.buffer.max.mb", "2")
+    conf.set("spark.kryoserializer.buffer.max", "2m")
     val sc = new SparkContext("local", "test", conf)
     assert(sc.parallelize(x).collect() === x)
     LocalSparkContext.stop(sc)
diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
index b070a54aa989..1b13559e77cb 100644
--- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
@@ -269,7 +269,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext {
 
   test("serialization buffer overflow reporting") {
     import org.apache.spark.SparkException
-    val kryoBufferMaxProperty = "spark.kryoserializer.buffer.max.mb"
+    val kryoBufferMaxProperty = "spark.kryoserializer.buffer.max"
 
     val largeObject = (1 to 1000000).toArray
 
diff --git a/docs/configuration.md b/docs/configuration.md
index 992baf49ca47..d44e1623ae0b 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -641,10 +641,10 @@ Apart from these, the following properties are also available, and may be useful
   
 
-  
-  
+  
+  

From eba4de6d4de436eb801852c66e9fe20fff664ca4 Mon Sep 17 00:00:00 2001
From: Ilya Ganelin 
Date: Sat, 18 Apr 2015 13:29:43 -0700
Subject: [PATCH 21/45] [SPARK-5932] Updated spark.shuffle.file.buffer.kb

---
 core/src/main/scala/org/apache/spark/SparkConf.scala   | 10 ++++++++--
 .../apache/spark/shuffle/FileShuffleBlockManager.scala |  2 +-
 .../spark/util/collection/ExternalAppendOnlyMap.scala  |  2 +-
 .../apache/spark/util/collection/ExternalSorter.scala  |  2 +-
 4 files changed, 11 insertions(+), 5 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
index a7e4e8a06821..2a3e15a7734f 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -480,7 +480,11 @@ private[spark] object SparkConf extends Logging {
       DeprecatedConfig("spark.kryoserializer.buffer.mb", "1.4",
         "Please use spark.kryoserializer.buffer instead."),
       DeprecatedConfig("spark.kryoserializer.buffer.max.mb", "1.4",
-        "Please use spark.kryoserializer.buffer.max instead."))
+        "Please use spark.kryoserializer.buffer.max instead."),
+      DeprecatedConfig("spark.shuffle.file.buffer.kb", "1.4",
+        "Please use spark.shuffle.file.buffer instead."))
+
+    
 
     
     Map(configs.map { cfg => (cfg.key -> cfg) }:_*)
@@ -512,7 +516,9 @@ private[spark] object SparkConf extends Logging {
     "spark.kryoserializer.buffer" -> Seq(
       AlternateConfig("spark.kryoserializer.buffer.mb", "1.4")),
     "spark.kryoserializer.buffer.max" -> Seq(
-      AlternateConfig("spark.kryoserializer.buffer.max.mb", "1.4"))
+      AlternateConfig("spark.kryoserializer.buffer.max.mb", "1.4")),
+    "spark.shuffle.file.buffer" -> Seq(
+      AlternateConfig("spark.shuffle.file.buffer.kb", "1.4"))
   )
 
   /**
diff --git a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala
index 5be3ed771e53..0e0b34b60982 100644
--- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala
@@ -78,7 +78,7 @@ class FileShuffleBlockManager(conf: SparkConf)
   private val consolidateShuffleFiles =
     conf.getBoolean("spark.shuffle.consolidateFiles", false)
 
-  private val bufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 32) * 1024
+  private val bufferSize = conf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 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 9ff4744593d4..a1498814d8f4 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
@@ -90,7 +90,7 @@ class ExternalAppendOnlyMap[K, V, C](
   // Number of bytes spilled in total
   private var _diskBytesSpilled = 0L
 
-  private val fileBufferSize = sparkConf.getInt("spark.shuffle.file.buffer.kb", 32) * 1024
+  private val fileBufferSize = sparkConf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 1024
 
   // Write metrics for current spill
   private var curWriteMetrics: ShuffleWriteMetrics = _
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 035f3767ff55..3599abf8f085 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
@@ -97,7 +97,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", 32) * 1024
+  private val fileBufferSize = conf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 1024
   private val transferToEnabled = conf.getBoolean("spark.file.transferTo", true)
 
   // Size of object batches when reading/writing from serializers.

From 1fbd435e16721d2eeeaafd834d81a2116376fa2a Mon Sep 17 00:00:00 2001
From: Ilya Ganelin 
Date: Sat, 18 Apr 2015 13:32:35 -0700
Subject: [PATCH 22/45] [SPARK-5932] Updated spark.broadcast.blockSize

---
 core/src/main/scala/org/apache/spark/SparkConf.scala            | 1 -
 .../scala/org/apache/spark/broadcast/TorrentBroadcast.scala     | 2 +-
 2 files changed, 1 insertion(+), 2 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
index 2a3e15a7734f..62e62deda52a 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -483,7 +483,6 @@ private[spark] object SparkConf extends Logging {
         "Please use spark.kryoserializer.buffer.max instead."),
       DeprecatedConfig("spark.shuffle.file.buffer.kb", "1.4",
         "Please use spark.shuffle.file.buffer instead."))
-
     
 
     
diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
index 23b02e60338f..5febed14b06b 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
@@ -74,7 +74,7 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long)
     } else {
       None
     }
-    blockSize = conf.getInt("spark.broadcast.blockSize", 4096) * 1024
+    blockSize = conf.getSizeAsKb("spark.broadcast.blockSize", "4m").toInt * 1024
   }
   setConf(SparkEnv.get.conf)
 

From 2d156810ed6f365fe460064676c54fed852433e0 Mon Sep 17 00:00:00 2001
From: Ilya Ganelin 
Date: Sat, 18 Apr 2015 13:35:54 -0700
Subject: [PATCH 23/45] [SPARK-5932] Updated
 spark.executor.logs.rolling.size.maxBytes

---
 core/src/main/scala/org/apache/spark/SparkConf.scala  | 11 +++++++----
 .../spark/util/logging/RollingFileAppender.scala      |  2 +-
 2 files changed, 8 insertions(+), 5 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
index 62e62deda52a..b793836fef9e 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -482,10 +482,11 @@ private[spark] object SparkConf extends Logging {
       DeprecatedConfig("spark.kryoserializer.buffer.max.mb", "1.4",
         "Please use spark.kryoserializer.buffer.max instead."),
       DeprecatedConfig("spark.shuffle.file.buffer.kb", "1.4",
-        "Please use spark.shuffle.file.buffer instead."))
-    
+        "Please use spark.shuffle.file.buffer instead."),
+      DeprecatedConfig("spark.executor.logs.rolling.size.maxBytes", "1.4",
+        "Please use spark.executor.logs.rolling.maxSize instead."))
+
 
-    
     Map(configs.map { cfg => (cfg.key -> cfg) }:_*)
   }
 
@@ -517,7 +518,9 @@ private[spark] object SparkConf extends Logging {
     "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"))
+      AlternateConfig("spark.shuffle.file.buffer.kb", "1.4")),
+    "spark.executor.logs.rolling.maxSize" -> Seq(
+      AlternateConfig("spark.executor.logs.rolling.size.maxBytes", "1.4"))
   )
 
   /**
diff --git a/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala b/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala
index e57942167634..7138b4b8e453 100644
--- a/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala
+++ b/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala
@@ -138,7 +138,7 @@ private[spark] object RollingFileAppender {
   val STRATEGY_DEFAULT = ""
   val INTERVAL_PROPERTY = "spark.executor.logs.rolling.time.interval"
   val INTERVAL_DEFAULT = "daily"
-  val SIZE_PROPERTY = "spark.executor.logs.rolling.size.maxBytes"
+  val SIZE_PROPERTY = "spark.executor.logs.rolling.maxSize"
   val SIZE_DEFAULT = (1024 * 1024).toString
   val RETAINED_FILES_PROPERTY = "spark.executor.logs.rolling.maxRetainedFiles"
   val DEFAULT_BUFFER_SIZE = 8192

From ae7e9f6a927fb6bbc92277bca15f54f209aaf7b1 Mon Sep 17 00:00:00 2001
From: Ilya Ganelin 
Date: Sat, 18 Apr 2015 13:39:07 -0700
Subject: [PATCH 24/45] [SPARK-5932] Updated
 spark.io.compression.snappy.block.size

---
 core/src/main/scala/org/apache/spark/SparkConf.scala  | 11 +++++++----
 .../scala/org/apache/spark/io/CompressionCodec.scala  |  4 ++--
 2 files changed, 9 insertions(+), 6 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
index b793836fef9e..c1a56d215339 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -484,9 +484,10 @@ private[spark] object SparkConf extends Logging {
       DeprecatedConfig("spark.shuffle.file.buffer.kb", "1.4",
         "Please use spark.shuffle.file.buffer instead."),
       DeprecatedConfig("spark.executor.logs.rolling.size.maxBytes", "1.4",
-        "Please use spark.executor.logs.rolling.maxSize instead."))
-
-
+        "Please use spark.executor.logs.rolling.maxSize instead."),
+      DeprecatedConfig("spark.io.compression.snappy.block.size", "1.4",
+        "Please use spark.io.compression.snappy.blockSize instead."))
+    
     Map(configs.map { cfg => (cfg.key -> cfg) }:_*)
   }
 
@@ -520,7 +521,9 @@ private[spark] object SparkConf extends Logging {
     "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"))
+      AlternateConfig("spark.executor.logs.rolling.size.maxBytes", "1.4")),
+    "spark.io.compression.snappy.blockSize" -> Seq(
+      AlternateConfig("spark.io.compression.snappy.block.size", "1.4"))
   )
 
   /**
diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
index 0709b6d689e8..4379c097a6b1 100644
--- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
+++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
@@ -137,7 +137,7 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec {
 /**
  * :: DeveloperApi ::
  * Snappy implementation of [[org.apache.spark.io.CompressionCodec]].
- * Block size can be configured by `spark.io.compression.snappy.block.size`.
+ * Block size can be configured by `spark.io.compression.snappy.blockSize`.
  *
  * Note: The wire protocol for this codec is not guaranteed to be compatible across versions
  *       of Spark. This is intended for use as an internal compression utility within a single Spark
@@ -153,7 +153,7 @@ class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec {
   }
 
   override def compressedOutputStream(s: OutputStream): OutputStream = {
-    val blockSize = conf.getInt("spark.io.compression.snappy.block.size", 32768)
+    val blockSize = conf.getSizeAsBytes("spark.io.compression.snappy.blockSize", "32k").toInt
     new SnappyOutputStream(s, blockSize)
   }
 

From afc9a38f45e5b101ed9c689eb7fff55261384909 Mon Sep 17 00:00:00 2001
From: Ilya Ganelin 
Date: Sat, 18 Apr 2015 13:45:25 -0700
Subject: [PATCH 25/45] [SPARK-5932] Updated spark.broadcast.blockSize and
 spark.storage.memoryMapThreshold

---
 core/src/main/scala/org/apache/spark/SparkConf.scala      | 2 +-
 .../org/apache/spark/broadcast/TorrentBroadcast.scala     | 1 +
 .../main/scala/org/apache/spark/storage/DiskStore.scala   | 3 +--
 .../org/apache/spark/storage/BlockManagerSuite.scala      | 4 ++--
 docs/configuration.md                                     | 8 ++++----
 5 files changed, 9 insertions(+), 9 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
index c1a56d215339..9b1c5c8ec5b2 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -220,7 +220,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging {
     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.
    */
diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
index 5febed14b06b..a0c9b5e63c74 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
@@ -74,6 +74,7 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long)
     } else {
       None
     }
+    // Note: use getSizeAsKb (not bytes) to maintain compatiblity if no units are provided
     blockSize = conf.getSizeAsKb("spark.broadcast.blockSize", "4m").toInt * 1024
   }
   setConf(SparkEnv.get.conf)
diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala
index 4b232ae7d318..1f4595628216 100644
--- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala
+++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala
@@ -31,8 +31,7 @@ import org.apache.spark.util.Utils
 private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBlockManager)
   extends BlockStore(blockManager) with Logging {
 
-  val minMemoryMapBytes = blockManager.conf.getLong(
-    "spark.storage.memoryMapThreshold", 2 * 1024L * 1024L)
+  val minMemoryMapBytes = blockManager.conf.getSizeAsBytes("spark.storage.memoryMapThreshold", "2m")
 
   override def getSize(blockId: BlockId): Long = {
     diskManager.getFile(blockId.name).length
diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
index c21f38e81962..b63afacfb0d3 100644
--- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
@@ -814,14 +814,14 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach
     // be nice to refactor classes involved in disk storage in a way that
     // allows for easier testing.
     val blockManager = mock(classOf[BlockManager])
-    when(blockManager.conf).thenReturn(conf.clone.set(confKey, 0.toString))
+    when(blockManager.conf).thenReturn(conf.clone.set(confKey, "0m"))
     val diskBlockManager = new DiskBlockManager(blockManager, conf)
 
     val diskStoreMapped = new DiskStore(blockManager, diskBlockManager)
     diskStoreMapped.putBytes(blockId, byteBuffer, StorageLevel.DISK_ONLY)
     val mapped = diskStoreMapped.getBytes(blockId).get
 
-    when(blockManager.conf).thenReturn(conf.clone.set(confKey, (1000 * 1000).toString))
+    when(blockManager.conf).thenReturn(conf.clone.set(confKey, "1m"))
     val diskStoreNotMapped = new DiskStore(blockManager, diskBlockManager)
     diskStoreNotMapped.putBytes(blockId, byteBuffer, StorageLevel.DISK_ONLY)
     val notMapped = diskStoreNotMapped.getBytes(blockId).get
diff --git a/docs/configuration.md b/docs/configuration.md
index d44e1623ae0b..7276b418d0a4 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -698,9 +698,9 @@ Apart from these, the following properties are also available, and may be useful
 
-  
+  
@@ -816,9 +816,9 @@ Apart from these, the following properties are also available, and may be useful
 
-  
+  

From 7a6c847c1401258cf7f1474add21f968853a036e Mon Sep 17 00:00:00 2001
From: Ilya Ganelin 
Date: Sat, 18 Apr 2015 13:48:58 -0700
Subject: [PATCH 26/45] [SPARK-5932] Updated spark.shuffle.file.buffer

---
 core/src/main/scala/org/apache/spark/SparkConf.scala        | 2 +-
 .../org/apache/spark/shuffle/FileShuffleBlockManager.scala  | 1 +
 .../spark/util/collection/ExternalAppendOnlyMap.scala       | 3 ++-
 .../org/apache/spark/util/collection/ExternalSorter.scala   | 2 ++
 docs/configuration.md                                       | 6 +++---
 5 files changed, 9 insertions(+), 5 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
index 9b1c5c8ec5b2..356e4860654f 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -525,7 +525,7 @@ private[spark] object SparkConf extends Logging {
     "spark.io.compression.snappy.blockSize" -> Seq(
       AlternateConfig("spark.io.compression.snappy.block.size", "1.4"))
   )
-
+  
   /**
    * A view of `configsWithAlternatives` that makes it more efficient to look up deprecated
    * config keys.
diff --git a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala
index 0e0b34b60982..ddd285d5465a 100644
--- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala
@@ -78,6 +78,7 @@ class FileShuffleBlockManager(conf: SparkConf)
   private val consolidateShuffleFiles =
     conf.getBoolean("spark.shuffle.consolidateFiles", false)
 
+  // Use getSizeAsKb (not bytes) to maintain backwards compatibility of on units are provided 
   private val bufferSize = conf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 1024
 
   /**
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 a1498814d8f4..514126f8baa9 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
@@ -89,7 +89,8 @@ class ExternalAppendOnlyMap[K, V, C](
 
   // Number of bytes spilled in total
   private var _diskBytesSpilled = 0L
-
+  
+  // Use getSizeAsKb (not bytes) to maintain backwards compatibility of on units are provided
   private val fileBufferSize = sparkConf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 1024
 
   // Write metrics for current spill
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 3599abf8f085..2bf02a2a4ff6 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
@@ -97,6 +97,8 @@ private[spark] class ExternalSorter[K, V, C](
 
   private val conf = SparkEnv.get.conf
   private val spillingEnabled = conf.getBoolean("spark.shuffle.spill", true)
+  
+  // Use getSizeAsKb (not bytes) to maintain backwards compatibility of on units are provided
   private val fileBufferSize = conf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 1024
   private val transferToEnabled = conf.getBoolean("spark.file.transferTo", true)
 
diff --git a/docs/configuration.md b/docs/configuration.md
index 7276b418d0a4..3339d4d2f9ab 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -403,10 +403,10 @@ Apart from these, the following properties are also available, and may be useful
   
 
-  
-  
+  
+  

From 5d29f909ea65a26486bec1e057d9666d3cd04fa9 Mon Sep 17 00:00:00 2001
From: Ilya Ganelin 
Date: Sat, 18 Apr 2015 13:56:41 -0700
Subject: [PATCH 27/45] [SPARK-5932] Finished documentation updates

---
 .../scala/org/apache/spark/SparkConf.scala    |  8 ++++--
 .../apache/spark/io/CompressionCodec.scala    |  4 +--
 docs/configuration.md                         | 28 +++++++++++++------
 3 files changed, 27 insertions(+), 13 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
index 356e4860654f..72121fe43ed9 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -486,7 +486,9 @@ private[spark] object SparkConf extends Logging {
       DeprecatedConfig("spark.executor.logs.rolling.size.maxBytes", "1.4",
         "Please use spark.executor.logs.rolling.maxSize instead."),
       DeprecatedConfig("spark.io.compression.snappy.block.size", "1.4",
-        "Please use spark.io.compression.snappy.blockSize instead."))
+        "Please use spark.io.compression.snappy.blockSize instead."),
+    DeprecatedConfig("spark.io.compression.lz4.block.size", "1.4",
+      "Please use spark.io.compression.lz4.blockSize instead."))
     
     Map(configs.map { cfg => (cfg.key -> cfg) }:_*)
   }
@@ -523,7 +525,9 @@ private[spark] object SparkConf extends Logging {
     "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"))
+      AlternateConfig("spark.io.compression.snappy.block.size", "1.4")),
+    "spark.io.compression.lz4.blockSize" -> Seq(
+      AlternateConfig("spark.io.compression.lz4.block.size", "1.4"))
   )
   
   /**
diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
index 4379c097a6b1..0756cdb2ed8e 100644
--- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
+++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
@@ -97,7 +97,7 @@ private[spark] object CompressionCodec {
 /**
  * :: DeveloperApi ::
  * LZ4 implementation of [[org.apache.spark.io.CompressionCodec]].
- * Block size can be configured by `spark.io.compression.lz4.block.size`.
+ * Block size can be configured by `spark.io.compression.lz4.blockSize`.
  *
  * Note: The wire protocol for this codec is not guaranteed to be compatible across versions
  *       of Spark. This is intended for use as an internal compression utility within a single Spark
@@ -107,7 +107,7 @@ private[spark] object CompressionCodec {
 class LZ4CompressionCodec(conf: SparkConf) extends CompressionCodec {
 
   override def compressedOutputStream(s: OutputStream): OutputStream = {
-    val blockSize = conf.getInt("spark.io.compression.lz4.block.size", 32768)
+    val blockSize = conf.getSizeAsBytes("spark.io.compression.lz4.blockSize", "32k").toInt
     new LZ4BlockOutputStream(s, blockSize)
   }
 
diff --git a/docs/configuration.md b/docs/configuration.md
index 3339d4d2f9ab..676c88780c0a 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 (kibibytes = 1024 bytes)
+    1m (mebibytes = 1024 kibibytes)
+    1g (gibibytes = 1024 mebibytes)
+    1t (tebibytes = 1024 gibibytes)
+    1p (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
   
 
 
-  
+  
@@ -582,18 +592,18 @@ Apart from these, the following properties are also available, and may be useful
   
-  
-  
+  
+  
-  
-  
+  
+  

From 928469edb7c9f330c89d0d1cb2d65ff89488a883 Mon Sep 17 00:00:00 2001
From: Ilya Ganelin 
Date: Sat, 18 Apr 2015 13:59:52 -0700
Subject: [PATCH 28/45] [SPARK-5932] Converted some longs to ints

---
 .../scala/org/apache/spark/serializer/KryoSerializer.scala     | 2 +-
 .../org/apache/spark/shuffle/FileShuffleBlockManager.scala     | 2 +-
 .../apache/spark/util/collection/ExternalAppendOnlyMap.scala   | 3 ++-
 .../org/apache/spark/util/collection/ExternalSorter.scala      | 2 +-
 4 files changed, 5 insertions(+), 4 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
index a2e63e66548e..595dc60580e8 100644
--- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
@@ -56,7 +56,7 @@ class KryoSerializer(conf: SparkConf)
   }
   private val bufferSize = (bufferSizeMb * 1024 * 1024).toInt
 
-  val maxBufferSizeMb = conf.getSizeAsMb("spark.kryoserializer.buffer.max", "64m")
+  val maxBufferSizeMb = conf.getSizeAsMb("spark.kryoserializer.buffer.max", "64m").toInt
   if (maxBufferSizeMb >= 2048) {
     throw new IllegalArgumentException("spark.kryoserializer.buffer.max must be less than " +
       s"2048 mb, got: + $maxBufferSizeMb mb.")
diff --git a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala
index ddd285d5465a..3a5bb70e2373 100644
--- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala
@@ -79,7 +79,7 @@ class FileShuffleBlockManager(conf: SparkConf)
     conf.getBoolean("spark.shuffle.consolidateFiles", false)
 
   // Use getSizeAsKb (not bytes) to maintain backwards compatibility of on units are provided 
-  private val bufferSize = conf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 1024
+  private val bufferSize = conf.getSizeAsKb("spark.shuffle.file.buffer", "32k").toInt * 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 514126f8baa9..90279e6f1cc0 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
@@ -91,7 +91,8 @@ class ExternalAppendOnlyMap[K, V, C](
   private var _diskBytesSpilled = 0L
   
   // Use getSizeAsKb (not bytes) to maintain backwards compatibility of on units are provided
-  private val fileBufferSize = sparkConf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 1024
+  private val fileBufferSize = 
+    sparkConf.getSizeAsKb("spark.shuffle.file.buffer", "32k").toInt * 1024
 
   // Write metrics for current spill
   private var curWriteMetrics: ShuffleWriteMetrics = _
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 2bf02a2a4ff6..8bdce652953b 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
@@ -99,7 +99,7 @@ private[spark] class ExternalSorter[K, V, C](
   private val spillingEnabled = conf.getBoolean("spark.shuffle.spill", true)
   
   // Use getSizeAsKb (not bytes) to maintain backwards compatibility of on units are provided
-  private val fileBufferSize = conf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 1024
+  private val fileBufferSize = conf.getSizeAsKb("spark.shuffle.file.buffer", "32k").toInt * 1024
   private val transferToEnabled = conf.getBoolean("spark.file.transferTo", true)
 
   // Size of object batches when reading/writing from serializers.

From 35a7fa71c8b2d83ecd8e7b060355cc5b538eeb9d Mon Sep 17 00:00:00 2001
From: Ilya Ganelin 
Date: Sat, 18 Apr 2015 16:51:21 -0700
Subject: [PATCH 29/45] Minor formatting

---
 core/src/main/scala/org/apache/spark/SparkConf.scala          | 4 ++--
 .../scala/org/apache/spark/storage/BlockManagerSuite.scala    | 2 +-
 2 files changed, 3 insertions(+), 3 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
index 72121fe43ed9..9dd0a2952e9f 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -487,8 +487,8 @@ private[spark] object SparkConf extends Logging {
         "Please use spark.executor.logs.rolling.maxSize instead."),
       DeprecatedConfig("spark.io.compression.snappy.block.size", "1.4",
         "Please use spark.io.compression.snappy.blockSize instead."),
-    DeprecatedConfig("spark.io.compression.lz4.block.size", "1.4",
-      "Please use spark.io.compression.lz4.blockSize instead."))
+      DeprecatedConfig("spark.io.compression.lz4.block.size", "1.4",
+        "Please use spark.io.compression.lz4.blockSize instead."))
     
     Map(configs.map { cfg => (cfg.key -> cfg) }:_*)
   }
diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
index b63afacfb0d3..1f260fed165b 100644
--- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
@@ -814,7 +814,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach
     // be nice to refactor classes involved in disk storage in a way that
     // allows for easier testing.
     val blockManager = mock(classOf[BlockManager])
-    when(blockManager.conf).thenReturn(conf.clone.set(confKey, "0m"))
+    when(blockManager.conf).thenReturn(conf.clone.set(confKey, "0"))
     val diskBlockManager = new DiskBlockManager(blockManager, conf)
 
     val diskStoreMapped = new DiskStore(blockManager, diskBlockManager)

From f15f20974df9e07ed51199b648a1ccd6e41ceb74 Mon Sep 17 00:00:00 2001
From: Ilya Ganelin 
Date: Sat, 18 Apr 2015 17:07:41 -0700
Subject: [PATCH 30/45] Fixed conversion of kryo buffer size

---
 .../main/scala/org/apache/spark/serializer/KryoSerializer.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
index 595dc60580e8..485404202f1e 100644
--- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
@@ -49,7 +49,7 @@ class KryoSerializer(conf: SparkConf)
   with Logging
   with Serializable {
 
-  private val bufferSizeMb = conf.getSizeAsMb("spark.kryoserializer.buffer", "64k")
+  private val bufferSizeMb = conf.getSizeAsKb("spark.kryoserializer.buffer", "64k").toDouble/1000.0d
   if (bufferSizeMb >= 2048) {
     throw new IllegalArgumentException("spark.kryoserializer.buffer must be less than " +
       s"2048 mb, got: + $bufferSizeMb mb.")

From f32bc01879fcff360914c137ed212fb210e9b10a Mon Sep 17 00:00:00 2001
From: Ilya Ganelin 
Date: Sat, 18 Apr 2015 23:12:09 -0700
Subject: [PATCH 31/45] [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

---
 .../scala/org/apache/spark/SparkConf.scala    |  4 ++--
 .../org/apache/spark/SparkConfSuite.scala     | 22 ++++++++++++++++++-
 2 files changed, 23 insertions(+), 3 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
index 9dd0a2952e9f..807209f83ac8 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -234,7 +234,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging {
    * @throws NoSuchElementException
    */
   def getSizeAsKb(key: String): Long = {
-    Utils.byteStringAsMb(get(key))
+    Utils.byteStringAsKb(get(key))
   }
 
   /**
@@ -242,7 +242,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging {
    * suffix is provided then Kibibytes are assumed.
    */
   def getSizeAsKb(key: String, defaultValue: String): Long = {
-    Utils.byteStringAsMb(get(key, defaultValue))
+    Utils.byteStringAsKb(get(key, defaultValue))
   }
   
   /**
diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
index 8e6c200c4ba0..52adf02ac25d 100644
--- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
@@ -19,14 +19,34 @@ package org.apache.spark
 
 import java.util.concurrent.{TimeUnit, Executors}
 
+import org.apache.spark.network.util.ByteUnit
+
 import scala.util.{Try, Random}
 
 import org.scalatest.FunSuite
 import org.apache.spark.serializer.{KryoRegistrator, KryoSerializer}
-import org.apache.spark.util.ResetSystemProperties
+import org.apache.spark.util.{Utils, ResetSystemProperties}
 import com.esotericsoftware.kryo.Kryo
 
 class SparkConfSuite extends FunSuite with LocalSparkContext with ResetSystemProperties {
+  test("Test byteString conversion") {
+    val conf = new SparkConf()
+    // Simply exercise the API, we don't need a complete conversion test since that's handled in
+    // UtilsSuite.scala
+    assert(conf.getSizeAsBytes("fake","1k") === ByteUnit.KiB.toBytes(1))
+    assert(conf.getSizeAsKb("fake","1k") === ByteUnit.KiB.toKiB(1))
+    assert(conf.getSizeAsMb("fake","1k") === ByteUnit.KiB.toMiB(1))
+    assert(conf.getSizeAsGb("fake","1k") === ByteUnit.KiB.toGiB(1))
+  }
+
+  test("Test timeString conversion") {
+    val conf = new SparkConf()
+    // Simply exercise the API, we don't need a complete conversion test since that's handled in
+    // UtilsSuite.scala
+    assert(conf.getTimeAsMs("fake","1ms") === TimeUnit.MILLISECONDS.toMillis(1))
+    assert(conf.getTimeAsSeconds("fake","1000ms") === TimeUnit.MILLISECONDS.toSeconds(1000))
+  }
+
   test("loading from system properties") {
     System.setProperty("spark.test.testProperty", "2")
     val conf = new SparkConf()

From 69e2f20348d290a637a4cce45857637a59015cd7 Mon Sep 17 00:00:00 2001
From: Ilya Ganelin 
Date: Mon, 20 Apr 2015 20:32:48 -0700
Subject: [PATCH 32/45] Updates to code

---
 .../scala/org/apache/spark/SparkConf.scala    | 16 +---
 .../spark/serializer/KryoSerializer.scala     |  4 +-
 .../hash/BlockStoreShuffleFetcher.scala       |  2 +-
 .../scala/org/apache/spark/util/Utils.scala   | 14 ++--
 docs/configuration.md                         | 10 +--
 .../apache/spark/network/util/ByteUnit.java   | 77 +++++--------------
 .../apache/spark/network/util/JavaUtils.java  | 36 ++++-----
 7 files changed, 53 insertions(+), 106 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
index 807209f83ac8..35f250c29166 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -474,21 +474,7 @@ 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."),
-      DeprecatedConfig("spark.reducer.maxMbInFlight", "1.4",
-        "Please use spark.reducer.maxSizeInFlight instead."),
-      DeprecatedConfig("spark.kryoserializer.buffer.mb", "1.4",
-        "Please use spark.kryoserializer.buffer instead."),
-      DeprecatedConfig("spark.kryoserializer.buffer.max.mb", "1.4",
-        "Please use spark.kryoserializer.buffer.max instead."),
-      DeprecatedConfig("spark.shuffle.file.buffer.kb", "1.4",
-        "Please use spark.shuffle.file.buffer instead."),
-      DeprecatedConfig("spark.executor.logs.rolling.size.maxBytes", "1.4",
-        "Please use spark.executor.logs.rolling.maxSize instead."),
-      DeprecatedConfig("spark.io.compression.snappy.block.size", "1.4",
-        "Please use spark.io.compression.snappy.blockSize instead."),
-      DeprecatedConfig("spark.io.compression.lz4.block.size", "1.4",
-        "Please use spark.io.compression.lz4.blockSize instead."))
+        "Please use spark.{driver,executor}.userClassPathFirst instead."))
     
     Map(configs.map { cfg => (cfg.key -> cfg) }:_*)
   }
diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
index 485404202f1e..4221a9a0b90f 100644
--- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
@@ -49,7 +49,9 @@ class KryoSerializer(conf: SparkConf)
   with Logging
   with Serializable {
 
-  private val bufferSizeMb = conf.getSizeAsKb("spark.kryoserializer.buffer", "64k").toDouble/1000.0d
+  private val bufferSizeMb = 
+    conf.getSizeAsMb("spark.kryoserializer.buffer", "64k").toDouble / 1000.0d
+  
   if (bufferSizeMb >= 2048) {
     throw new IllegalArgumentException("spark.kryoserializer.buffer must be less than " +
       s"2048 mb, got: + $bufferSizeMb mb.")
diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala
index c1640754898e..80374adc4429 100644
--- a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala
+++ b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala
@@ -79,7 +79,7 @@ private[hash] object BlockStoreShuffleFetcher extends Logging {
       blockManager,
       blocksByAddress,
       serializer,
-      // Note: we use getSizeAsMb to assume Mb when no suffix is provided
+      // Note: we use getSizeAsMb when no suffix is provided for backwards compatibility
       SparkEnv.get.conf.getSizeAsMb("spark.reducer.maxSizeInFlight", "48m") * 1024 * 1024) 
     val itr = blockFetcherItr.flatMap(unpackBlock)
 
diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala
index c2a731ae9da6..5a36560586f0 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -1037,8 +1037,7 @@ private[spark] object Utils extends Logging {
   }
 
   /**
-   * Convert a passed byte string (e.g. 50b, 100k, or 250m) to bytes for
-   * internal use.
+   * 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.
    */
@@ -1047,8 +1046,7 @@ private[spark] object Utils extends Logging {
   }
 
   /**
-   * Convert a passed byte string (e.g. 50b, 100k, or 250m) to kibibytes for
-   * internal use.
+   * 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.
    */
@@ -1057,8 +1055,7 @@ private[spark] object Utils extends Logging {
   }
 
   /**
-   * Convert a passed byte string (e.g. 50b, 100k, or 250m) to mebibytes for
-   * internal use.
+   * 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.
    */
@@ -1067,8 +1064,7 @@ private[spark] object Utils extends Logging {
   }
 
   /**
-   * Convert a passed byte string (e.g. 50b, 100k, or 250m, 500g) to gibibytes for
-   * internal use.
+   * 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.
    */
@@ -1082,7 +1078,7 @@ private[spark] object Utils extends Logging {
   def memoryStringToMb(str: String): Int = {
     // Convert to bytes, rather than directly to MB, because when no units are specified the unit
     // is assumed to be bytes
-    (JavaUtils.byteStringAsBytes(str) / 1048576.0).toInt
+    (JavaUtils.byteStringAsBytes(str) / 1024.0d / 1024.0d).toInt
   }
 
   /**
diff --git a/docs/configuration.md b/docs/configuration.md
index 676c88780c0a..2500ba8de44c 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -53,11 +53,11 @@ Properties that specify a byte size should be configured with a unit of size.
 The following format is accepted:
 
     1b (bytes)
-    1k (kibibytes = 1024 bytes)
-    1m (mebibytes = 1024 kibibytes)
-    1g (gibibytes = 1024 mebibytes)
-    1t (tebibytes = 1024 gibibytes)
-    1p (pebibytes = 1024 tebibytes)
+    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
diff --git a/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java b/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
index e7de87062eb6..830b9a959128 100644
--- a/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
+++ b/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
@@ -17,55 +17,28 @@
 package org.apache.spark.network.util;
 
 public enum ByteUnit {
-  /** Byte (B) */
-  BYTE {
-    public long toBytes(long d) { return d; }
-    
-    public long convert(long d, ByteUnit u) { return u.toBytes(d); }
-  },
-
-  /** Kibibyte (KiB) = 1024 Byte */
-  KiB {
-    public long toBytes(long d) { return x(d, C_KiB); }
-    
-    public long convert(long d, ByteUnit u) { return u.toKiB(d); }
-  },
-
-  /** Mebibyte (MiB) = (1024^2) Byte */
-  MiB {
-    public long toBytes(long d) { return x(d, C_MiB); }
-    
-    public long convert(long d, ByteUnit u) { return u.toMiB(d); }
-  },
-
-  /** Gibibyte (GiB) = (1024^3) Byte */
-  GiB {
-    public long toBytes(long d) { return x(d, C_GiB);
-    }
-
-    public long convert(long d, ByteUnit u) { return u.toGiB(d); }
-  },
-
-  /** Tebibyte (TiB) = (1024^4) Byte */
-  TiB {
-    public long toBytes(long d) { return x(d, C_TiB); }
-    
-    public long convert(long d, ByteUnit u) { return u.toTiB(d); }
-  },
-
-  /** Pebibyte (PB) = (1024^5) Byte */
-  PiB {
-    public long toBytes(long d) { return x(d, C_PiB); }
-    
-    public long convert(long d, ByteUnit u) { return u.toPiB(d); }
-  };
+  BYTE (1),
+  KiB (1024l),
+  MiB ((long) Math.pow(1024l, 2l)),
+  GiB ((long) Math.pow(1024l, 3l)),
+  TiB ((long) Math.pow(1024l, 4l)),
+  PiB ((long) Math.pow(1024l, 5l));
+
+  private ByteUnit(long multiplier) {
+    this.multiplier = multiplier;
+  }
 
-  static final long C_KiB = 1024l;
-  static final long C_MiB = (long) Math.pow(1024l, 2l);
-  static final long C_GiB = (long) Math.pow(1024l, 3l);
-  static final long C_TiB = (long) Math.pow(1024l, 4l);
-  static final long C_PiB = (long) Math.pow(1024l, 5l);
+  public long convert(long d, ByteUnit u) { return toBytes(d) / u.multiplier; }
 
+  public long toBytes(long d) { return multiplier * d; }
+  public long toKiB(long d) { return convert(d, KiB); }
+  public long toMiB(long d) { return convert(d, MiB); }
+  public long toGiB(long d) { return convert(d, GiB); }
+  public long toTiB(long d) { return convert(d, TiB); }
+  public long toPiB(long d) { return convert(d, PiB); }
+  
+  long multiplier = 0;
+  
   static final long MAX = Long.MAX_VALUE;
 
   /**
@@ -78,18 +51,8 @@ static long x(long d, long m) {
     if (d < -over) return Long.MIN_VALUE;
     return d * m;
   }
-
-  public long convert(long d, ByteUnit u) { throw new AbstractMethodError(); }
   
-  public long toBytes(long d) { throw new AbstractMethodError(); }
   
-  public long toKiB(long d) { return toBytes(d) / C_KiB; }
-
-  public long toMiB(long d) { return toBytes(d) / C_MiB; }
-
-  public long toGiB(long d) { return toBytes(d) / C_GiB; }
 
-  public long toTiB(long d) { return toBytes(d) / C_TiB; }
 
-  public long toPiB(long d) { return toBytes(d) / C_PiB; }
 }
diff --git a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
index 25ab9e973b48..6bac184cd534 100644
--- a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
+++ b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
@@ -126,7 +126,7 @@ private static boolean isSymlink(File file) throws IOException {
     return !fileInCanonicalDir.getCanonicalFile().equals(fileInCanonicalDir.getAbsoluteFile());
   }
 
-  private static ImmutableMap timeSuffixes = 
+  private static final ImmutableMap timeSuffixes = 
     ImmutableMap.builder()
       .put("us", TimeUnit.MICROSECONDS)
       .put("ms", TimeUnit.MILLISECONDS)
@@ -137,14 +137,19 @@ private static boolean isSymlink(File file) throws IOException {
       .put("d", TimeUnit.DAYS)
       .build();
 
-  private static ImmutableMap byteSuffixes =
+  private static final ImmutableMap byteSuffixes =
     ImmutableMap.builder()
       .put("b", ByteUnit.BYTE)
       .put("k", ByteUnit.KiB)
+      .put("kb", ByteUnit.KiB)
       .put("m", ByteUnit.MiB)
+      .put("mb", ByteUnit.MiB)
       .put("g", ByteUnit.GiB)
+      .put("gb", ByteUnit.GiB)
       .put("t", ByteUnit.TiB)
+      .put("tb", ByteUnit.TiB)
       .put("p", ByteUnit.PiB)
+      .put("pb", ByteUnit.PiB)
       .build();
 
   /**
@@ -155,16 +160,14 @@ private static long parseTimeString(String str, TimeUnit unit) {
     String lower = str.toLowerCase().trim();
     
     try {
-      String suffix;
-      long val;
       Matcher m = Pattern.compile("(-?[0-9]+)([a-z]+)?").matcher(lower);
-      if (m.matches()) {
-        val = Long.parseLong(m.group(1));
-        suffix = m.group(2);
-      } else {
+      if (!m.matches()) {
         throw new NumberFormatException("Failed to parse time string: " + str);
       }
       
+      long val = Long.parseLong(m.group(1));
+      String suffix = m.group(2);
+      
       // Check for invalid suffixes
       if (suffix != null && !timeSuffixes.containsKey(suffix)) {
         throw new NumberFormatException("Invalid suffix: \"" + suffix + "\"");
@@ -206,27 +209,24 @@ private static long parseByteString(String str, ByteUnit unit) {
     String lower = str.toLowerCase().trim();
 
     try {
-      String suffix;
-      long val;
       Matcher m = Pattern.compile("([0-9]+)([a-z]+)?").matcher(lower);
-      if (m.matches()) {
-        val = Long.parseLong(m.group(1));
-        suffix = m.group(2);
-      } else {
+      if (!m.matches()) {
         throw new NumberFormatException("Failed to parse byte string: " + str);
       }
-
+      
+      long val = Long.parseLong(m.group(1));
+      String suffix = m.group(2);
+      
       // Check for invalid suffixes
       if (suffix != null && !byteSuffixes.containsKey(suffix)) {
         throw new NumberFormatException("Invalid suffix: \"" + suffix + "\"");
       }
 
       // If suffix is valid use that, otherwise none was provided and use the default passed
-      return new Double(
-        unit.convert(val, suffix != null ? byteSuffixes.get(suffix) : unit)).longValue();
+      return unit.convert(val, suffix != null ? byteSuffixes.get(suffix) : unit);
     } catch (NumberFormatException e) {
       String timeError = "Size must be specified as bytes (b), " +
-        "kibibytes (kb), mebibytes (m), gibibytes (g), tebibytes (t), or pebibytes(p). " +
+        "kibibytes (k), mebibytes (m), gibibytes (g), tebibytes (t), or pebibytes(p). " +
         "E.g. 50b, 100k, or 250m.";
 
       throw new NumberFormatException(timeError + "\n" + e.getMessage());

From 54b78b4dea60376045ac48ad46c8f4ae50b41328 Mon Sep 17 00:00:00 2001
From: Ilya Ganelin 
Date: Mon, 20 Apr 2015 21:24:55 -0700
Subject: [PATCH 33/45] Simplified byteUnit class

---
 .../org/apache/spark/network/util/ByteUnit.java    | 14 ++++++++++++--
 .../org/apache/spark/network/util/JavaUtils.java   |  2 +-
 2 files changed, 13 insertions(+), 3 deletions(-)

diff --git a/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java b/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
index 830b9a959128..85f9ff6cf03d 100644
--- a/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
+++ b/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
@@ -28,9 +28,18 @@ private ByteUnit(long multiplier) {
     this.multiplier = multiplier;
   }
 
-  public long convert(long d, ByteUnit u) { return toBytes(d) / u.multiplier; }
+  // Interpret the provided number (d) with suffix (u) as this unit type.
+  // E.g. KiB.interpret(1, MiB) interprets 1MiB as its KiB representation = 1024k
+  public long interpret(long d, ByteUnit u) {
+    return u.toBytes(d) / multiplier;  
+  }
+  
+  // Convert the provided number (d) interpreted as this unit type to unit type (u). 
+  public long convert(long d, ByteUnit u) {
+    return toBytes(d) / u.multiplier;
+  }
 
-  public long toBytes(long d) { return multiplier * d; }
+  public long toBytes(long d) { return x(d, multiplier); }
   public long toKiB(long d) { return convert(d, KiB); }
   public long toMiB(long d) { return convert(d, MiB); }
   public long toGiB(long d) { return convert(d, GiB); }
@@ -46,6 +55,7 @@ private ByteUnit(long multiplier) {
    * This has a short name to make above code more readable.
    */
   static long x(long d, long m) {
+    if (d == 0) { return 0; }
     long over = MAX / d;
     if (d >  over) return Long.MAX_VALUE;
     if (d < -over) return Long.MIN_VALUE;
diff --git a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
index 6bac184cd534..5190eb13a9f7 100644
--- a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
+++ b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
@@ -223,7 +223,7 @@ private static long parseByteString(String str, ByteUnit unit) {
       }
 
       // If suffix is valid use that, otherwise none was provided and use the default passed
-      return unit.convert(val, suffix != null ? byteSuffixes.get(suffix) : unit);
+      return unit.interpret(val, suffix != null ? byteSuffixes.get(suffix) : unit);
     } catch (NumberFormatException e) {
       String timeError = "Size must be specified as bytes (b), " +
         "kibibytes (k), mebibytes (m), gibibytes (g), tebibytes (t), or pebibytes(p). " +

From c7803cd8bae1fbc69fbee9bdfc2ca540a8516b9c Mon Sep 17 00:00:00 2001
From: Ilya Ganelin 
Date: Mon, 20 Apr 2015 21:27:09 -0700
Subject: [PATCH 34/45] Empty lines

---
 .../main/java/org/apache/spark/network/util/ByteUnit.java    | 5 -----
 1 file changed, 5 deletions(-)

diff --git a/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java b/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
index 85f9ff6cf03d..07a587def23c 100644
--- a/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
+++ b/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
@@ -47,7 +47,6 @@ public long convert(long d, ByteUnit u) {
   public long toPiB(long d) { return convert(d, PiB); }
   
   long multiplier = 0;
-  
   static final long MAX = Long.MAX_VALUE;
 
   /**
@@ -61,8 +60,4 @@ static long x(long d, long m) {
     if (d < -over) return Long.MIN_VALUE;
     return d * m;
   }
-  
-  
-
-
 }

From d3d09b62db1a6480d14ec11ab3ce3359adf25cf5 Mon Sep 17 00:00:00 2001
From: Ilya Ganelin 
Date: Tue, 21 Apr 2015 09:27:46 -0700
Subject: [PATCH 35/45] [SPARK-5932] Fixing error in KryoSerializer

---
 .../scala/org/apache/spark/serializer/KryoSerializer.scala     | 3 +--
 core/src/test/scala/org/apache/spark/util/UtilsSuite.scala     | 1 +
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
index 4221a9a0b90f..d90d4cd4bc30 100644
--- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
@@ -49,8 +49,7 @@ class KryoSerializer(conf: SparkConf)
   with Logging
   with Serializable {
 
-  private val bufferSizeMb = 
-    conf.getSizeAsMb("spark.kryoserializer.buffer", "64k").toDouble / 1000.0d
+  private val bufferSizeMb = conf.getSizeAsMb("spark.kryoserializer.buffer", "64k")
   
   if (bufferSizeMb >= 2048) {
     throw new IllegalArgumentException("spark.kryoserializer.buffer must be less than " +
diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
index e8c64651b9ee..e5adfe750b29 100644
--- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
@@ -106,6 +106,7 @@ class UtilsSuite extends FunSuite with ResetSystemProperties {
     assert(Utils.byteStringAsMb("1023k") === 0)
     assert(Utils.byteStringAsMb("1024k") === 1)
     assert(Utils.byteStringAsMb("3645k") === 3)
+    assert(Utils.byteStringAsMb("1024gb") === 1048576)
     assert(Utils.byteStringAsMb("1g") === ByteUnit.GiB.toMiB(1))
     assert(Utils.byteStringAsMb("1t") === ByteUnit.TiB.toMiB(1))
     assert(Utils.byteStringAsMb("1p") === ByteUnit.PiB.toMiB(1))

From 84a25819ebfc57638f73403a4dc81fde173c9039 Mon Sep 17 00:00:00 2001
From: Ilya Ganelin 
Date: Tue, 21 Apr 2015 13:52:20 -0700
Subject: [PATCH 36/45] Added smoother handling of fractional values for size
 parameters. This now throws an exception and added a warning for old
 spark.kryoserializer.buffer

---
 .../scala/org/apache/spark/SparkConf.scala    |  7 ++++-
 .../spark/serializer/KryoSerializer.scala     |  8 ++---
 .../org/apache/spark/util/UtilsSuite.scala    | 10 ++++++
 .../apache/spark/network/util/JavaUtils.java  | 31 ++++++++++++-------
 4 files changed, 39 insertions(+), 17 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
index 49e3daeaef5a..72ed55fbaebd 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -474,7 +474,12 @@ 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) }:_*)
   }
diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
index d90d4cd4bc30..754832b8a4ca 100644
--- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
@@ -49,13 +49,13 @@ class KryoSerializer(conf: SparkConf)
   with Logging
   with Serializable {
 
-  private val bufferSizeMb = conf.getSizeAsMb("spark.kryoserializer.buffer", "64k")
+  private val bufferSizeKb = conf.getSizeAsKb("spark.kryoserializer.buffer", "64k")
   
-  if (bufferSizeMb >= 2048) {
+  if (bufferSizeKb >= 2048) {
     throw new IllegalArgumentException("spark.kryoserializer.buffer must be less than " +
-      s"2048 mb, got: + $bufferSizeMb mb.")
+      s"2048 mb, got: + $bufferSizeKb mb.")
   }
-  private val bufferSize = (bufferSizeMb * 1024 * 1024).toInt
+  private val bufferSize = (bufferSizeKb * 1024).toInt
 
   val maxBufferSizeMb = conf.getSizeAsMb("spark.kryoserializer.buffer.max", "64m").toInt
   if (maxBufferSizeMb >= 2048) {
diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
index e5adfe750b29..4639d2160f34 100644
--- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
@@ -125,6 +125,16 @@ class UtilsSuite extends FunSuite with ResetSystemProperties {
     assert(Utils.byteStringAsBytes("1t") === ByteUnit.TiB.toBytes(1))
     assert(Utils.byteStringAsBytes("1p") === ByteUnit.PiB.toBytes(1))
 
+    // Test fractional string
+    intercept[NumberFormatException] {
+      Utils.byteStringAsMb("0.064")
+    }
+    
+    // Test fractional string
+    intercept[NumberFormatException] {
+      Utils.byteStringAsMb("0.064m")
+    }
+    
     // Test invalid strings
     intercept[NumberFormatException] {
       Utils.byteStringAsBytes("500ub")
diff --git a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
index 5190eb13a9f7..7a1b208a4e8d 100644
--- a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
+++ b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
@@ -210,20 +210,27 @@ private static long parseByteString(String str, ByteUnit unit) {
 
     try {
       Matcher m = Pattern.compile("([0-9]+)([a-z]+)?").matcher(lower);
-      if (!m.matches()) {
-        throw new NumberFormatException("Failed to parse byte string: " + str);
-      }
+      Matcher fractionMatcher = Pattern.compile("([0-9]*\\.[0-9]*)([a-z]+)?").matcher(lower);
       
-      long val = Long.parseLong(m.group(1));
-      String suffix = m.group(2);
-      
-      // Check for invalid suffixes
-      if (suffix != null && !byteSuffixes.containsKey(suffix)) {
-        throw new NumberFormatException("Invalid suffix: \"" + suffix + "\"");
-      }
+      if(m.matches()) {
+        long val = Long.parseLong(m.group(1));
+        String suffix = m.group(2);
 
-      // If suffix is valid use that, otherwise none was provided and use the default passed
-      return unit.interpret(val, suffix != null ? byteSuffixes.get(suffix) : unit);
+        // Check for invalid suffixes
+        if (suffix != null && !byteSuffixes.containsKey(suffix)) {
+          throw new NumberFormatException("Invalid suffix: \"" + suffix + "\"");
+        }
+
+        // If suffix is valid use that, otherwise none was provided and use the default passed
+        return unit.interpret(val, suffix != null ? byteSuffixes.get(suffix) : unit);  
+      } else if (fractionMatcher.matches()) {
+        double val = Long.parseLong(m.group(1));
+
+        throw new NumberFormatException("Fractional values are not supported. Input was: " + val);
+      } else {
+        throw new NumberFormatException("Failed to parse byte string: " + str);  
+      }
+      
     } catch (NumberFormatException e) {
       String timeError = "Size must be specified as bytes (b), " +
         "kibibytes (k), mebibytes (m), gibibytes (g), tebibytes (t), or pebibytes(p). " +

From 8b43748145e442c7dabbe9b2d75f49f40f78ab8e Mon Sep 17 00:00:00 2001
From: Ilya Ganelin 
Date: Tue, 21 Apr 2015 15:10:58 -0700
Subject: [PATCH 37/45] Fixed error in pattern matching for doubles

---
 .../src/main/java/org/apache/spark/network/util/JavaUtils.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
index 7a1b208a4e8d..2a36d25ca48e 100644
--- a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
+++ b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
@@ -224,7 +224,7 @@ private static long parseByteString(String str, ByteUnit unit) {
         // If suffix is valid use that, otherwise none was provided and use the default passed
         return unit.interpret(val, suffix != null ? byteSuffixes.get(suffix) : unit);  
       } else if (fractionMatcher.matches()) {
-        double val = Long.parseLong(m.group(1));
+        double val = Double.parseDouble(fractionMatcher.group(1));
 
         throw new NumberFormatException("Fractional values are not supported. Input was: " + val);
       } else {

From 3dfae96ee8f594e4136c6916326ef7e0fe70b4be Mon Sep 17 00:00:00 2001
From: Ilya Ganelin 
Date: Wed, 22 Apr 2015 12:11:38 -0700
Subject: [PATCH 38/45] Fixed some nits. Added automatic conversion of old
 paramter for kryoserializer.mb to new values.

---
 .../scala/org/apache/spark/SparkConf.scala    |  5 +--
 .../scala/org/apache/spark/util/Utils.scala   |  2 +-
 .../apache/spark/network/util/ByteUnit.java   | 32 ++++++++-----------
 .../apache/spark/network/util/JavaUtils.java  |  2 +-
 4 files changed, 18 insertions(+), 23 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
index 72ed55fbaebd..a8fc90ad2050 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -507,8 +507,9 @@ private[spark] object SparkConf extends Logging {
         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")),
+    "spark.kryoserializer.buffer" ->
+        Seq(AlternateConfig("spark.kryoserializer.buffer.mb", "1.4", 
+          translation = s => s"${s.toDouble * 1000}k")),
     "spark.kryoserializer.buffer.max" -> Seq(
       AlternateConfig("spark.kryoserializer.buffer.max.mb", "1.4")),
     "spark.shuffle.file.buffer" -> Seq(
diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala
index cfcfaf2de418..2c60a9f07e01 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -1082,7 +1082,7 @@ private[spark] object Utils extends Logging {
   def memoryStringToMb(str: String): Int = {
     // 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.0d / 1024.0d).toInt
+    (JavaUtils.byteStringAsBytes(str) / 1024 / 1024).toInt
   }
 
   /**
diff --git a/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java b/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
index 07a587def23c..c943863793cd 100644
--- a/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
+++ b/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
@@ -18,11 +18,11 @@
 
 public enum ByteUnit {
   BYTE (1),
-  KiB (1024l),
-  MiB ((long) Math.pow(1024l, 2l)),
-  GiB ((long) Math.pow(1024l, 3l)),
-  TiB ((long) Math.pow(1024l, 4l)),
-  PiB ((long) Math.pow(1024l, 5l));
+  KiB (1024L),
+  MiB ((long) Math.pow(1024L, 2L)),
+  GiB ((long) Math.pow(1024L, 3L)),
+  TiB ((long) Math.pow(1024L, 4L)),
+  PiB ((long) Math.pow(1024L, 5L));
 
   private ByteUnit(long multiplier) {
     this.multiplier = multiplier;
@@ -39,25 +39,19 @@ public long convert(long d, ByteUnit u) {
     return toBytes(d) / u.multiplier;
   }
 
-  public long toBytes(long d) { return x(d, multiplier); }
+  public long toBytes(long d) {
+    if (d == 0) { return 0; }
+    long over = MAX / d;
+    if (d >  over) return Long.MAX_VALUE;
+    if (d < -over) return Long.MIN_VALUE;
+    return d * multiplier; 
+  }
   public long toKiB(long d) { return convert(d, KiB); }
   public long toMiB(long d) { return convert(d, MiB); }
   public long toGiB(long d) { return convert(d, GiB); }
   public long toTiB(long d) { return convert(d, TiB); }
   public long toPiB(long d) { return convert(d, PiB); }
   
-  long multiplier = 0;
+  private long multiplier = 0;
   static final long MAX = Long.MAX_VALUE;
-
-  /**
-   * Scale d by m, checking for overflow.
-   * This has a short name to make above code more readable.
-   */
-  static long x(long d, long m) {
-    if (d == 0) { return 0; }
-    long over = MAX / d;
-    if (d >  over) return Long.MAX_VALUE;
-    if (d < -over) return Long.MIN_VALUE;
-    return d * m;
-  }
 }
diff --git a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
index 2a36d25ca48e..c10eb34c06f9 100644
--- a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
+++ b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
@@ -212,7 +212,7 @@ private static long parseByteString(String str, ByteUnit unit) {
       Matcher m = Pattern.compile("([0-9]+)([a-z]+)?").matcher(lower);
       Matcher fractionMatcher = Pattern.compile("([0-9]*\\.[0-9]*)([a-z]+)?").matcher(lower);
       
-      if(m.matches()) {
+      if (m.matches()) {
         long val = Long.parseLong(m.group(1));
         String suffix = m.group(2);
 

From 22413b19e9323f5df45cfb6d1758baa709382184 Mon Sep 17 00:00:00 2001
From: Ilya Ganelin 
Date: Wed, 22 Apr 2015 14:22:09 -0700
Subject: [PATCH 39/45] Made MAX private

---
 .../src/main/java/org/apache/spark/network/util/ByteUnit.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java b/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
index c943863793cd..6d561a1af0c7 100644
--- a/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
+++ b/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
@@ -53,5 +53,5 @@ public long toBytes(long d) {
   public long toPiB(long d) { return convert(d, PiB); }
   
   private long multiplier = 0;
-  static final long MAX = Long.MAX_VALUE;
+  private static final long MAX = Long.MAX_VALUE;
 }

From 9ee779c20937ce1b52abd8fcc61dae65a68cb4f6 Mon Sep 17 00:00:00 2001
From: Ilya Ganelin 
Date: Wed, 22 Apr 2015 15:14:28 -0700
Subject: [PATCH 40/45] Simplified fraction matches

---
 .../main/java/org/apache/spark/network/util/JavaUtils.java | 7 +++----
 1 file changed, 3 insertions(+), 4 deletions(-)

diff --git a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
index c10eb34c06f9..2a0dc44699b1 100644
--- a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
+++ b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
@@ -210,7 +210,7 @@ private static long parseByteString(String str, ByteUnit unit) {
 
     try {
       Matcher m = Pattern.compile("([0-9]+)([a-z]+)?").matcher(lower);
-      Matcher fractionMatcher = Pattern.compile("([0-9]*\\.[0-9]*)([a-z]+)?").matcher(lower);
+      Matcher fractionMatcher = Pattern.compile("([0-9]+\\.[0-9]+)([a-z]+)?").matcher(lower);
       
       if (m.matches()) {
         long val = Long.parseLong(m.group(1));
@@ -224,9 +224,8 @@ private static long parseByteString(String str, ByteUnit unit) {
         // If suffix is valid use that, otherwise none was provided and use the default passed
         return unit.interpret(val, suffix != null ? byteSuffixes.get(suffix) : unit);  
       } else if (fractionMatcher.matches()) {
-        double val = Double.parseDouble(fractionMatcher.group(1));
-
-        throw new NumberFormatException("Fractional values are not supported. Input was: " + val);
+        throw new NumberFormatException("Fractional values are not supported. Input was: " 
+          + fractionMatcher.group(1);
       } else {
         throw new NumberFormatException("Failed to parse byte string: " + str);  
       }

From 852a407b469a12345a3253c62871e186b2ddb06c Mon Sep 17 00:00:00 2001
From: Ilya Ganelin 
Date: Thu, 23 Apr 2015 14:49:10 -0700
Subject: [PATCH 41/45] [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

---
 .../org/apache/spark/util/UtilsSuite.scala    | 28 ++++++++++
 .../apache/spark/network/util/ByteUnit.java   | 56 +++++++++++--------
 .../apache/spark/network/util/JavaUtils.java  |  4 +-
 3 files changed, 63 insertions(+), 25 deletions(-)

diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
index f3607f9d5c84..2dea58519289 100644
--- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
@@ -126,6 +126,27 @@ class UtilsSuite extends FunSuite with ResetSystemProperties {
     assert(Utils.byteStringAsBytes("1t") === ByteUnit.TiB.toBytes(1))
     assert(Utils.byteStringAsBytes("1p") === ByteUnit.PiB.toBytes(1))
 
+    // Overflow handling, 1073741824p exceeds Long.MAX_VALUE if converted straight to Bytes
+    // This demonstrates that we can have e.g 1024^3 PB without overflowing. 
+    assert(Utils.byteStringAsGb("1073741824p") === ByteUnit.PiB.toGiB(1073741824))                                  
+    assert(Utils.byteStringAsMb("1073741824p") === ByteUnit.PiB.toMiB(1073741824))
+    
+    // Run this to confirm it doesn't throw an exception
+    assert(Utils.byteStringAsBytes("9223372036854775807") === 9223372036854775807L) 
+    assert(ByteUnit.PiB.toPiB(9223372036854775807L) === 9223372036854775807L)
+    
+    // Test overflow exception
+    intercept[IllegalArgumentException] {
+      // This value exceeds Long.MAX when converted to bytes 
+      Utils.byteStringAsBytes("9223372036854775808")
+    }
+
+    // Test overflow exception
+    intercept[IllegalArgumentException] {
+      // This value exceeds Long.MAX when converted to TB
+      ByteUnit.PiB.toTiB(9223372036854775807L)
+    }
+    
     // Test fractional string
     intercept[NumberFormatException] {
       Utils.byteStringAsMb("0.064")
@@ -157,6 +178,13 @@ class UtilsSuite extends FunSuite with ResetSystemProperties {
     intercept[NumberFormatException] {
       Utils.byteStringAsBytes("This 123mb breaks")
     }
+    
+//    // Test overflow
+//    intercept[NumberFormatException] {
+//      
+//
+//      ByteUnit.convertTo(0x7fffffffffffffffL, ByteUnit.KiB)
+//    }
   }
   
   test("bytesToString") {
diff --git a/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java b/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
index 6d561a1af0c7..57561223b690 100644
--- a/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
+++ b/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
@@ -17,41 +17,51 @@
 package org.apache.spark.network.util;
 
 public enum ByteUnit {
-  BYTE (1),
-  KiB (1024L),
-  MiB ((long) Math.pow(1024L, 2L)),
-  GiB ((long) Math.pow(1024L, 3L)),
-  TiB ((long) Math.pow(1024L, 4L)),
-  PiB ((long) Math.pow(1024L, 5L));
+  BYTE (1, "Bytes"),
+  KiB (1024, "KiB"),
+  MiB (Math.pow(1024, 2), "MiB"),
+  GiB (Math.pow(1024, 3), "GiB"),
+  TiB (Math.pow(1024, 4), "TiB"),
+  PiB (Math.pow(1024, 5), "PiB");
 
-  private ByteUnit(long multiplier) {
+  private ByteUnit(double multiplier, String name) {
     this.multiplier = multiplier;
+    this.name = name;
   }
 
   // Interpret the provided number (d) with suffix (u) as this unit type.
   // E.g. KiB.interpret(1, MiB) interprets 1MiB as its KiB representation = 1024k
-  public long interpret(long d, ByteUnit u) {
-    return u.toBytes(d) / multiplier;  
+  public long convertFrom(long d, ByteUnit u) {
+    double converted = u.toBytes(d) / multiplier;
+    if (converted > Long.MAX_VALUE)
+      throw new IllegalArgumentException("Converted value (" + converted + ") " +
+        "exceeds Long.MAX_VALUE for " + name + ". Try a larger suffix (e.g. MiB instead of KiB)");
+    return (long) converted;  
   }
   
   // Convert the provided number (d) interpreted as this unit type to unit type (u). 
-  public long convert(long d, ByteUnit u) {
-    return toBytes(d) / u.multiplier;
+  public long convertTo(long d, ByteUnit u) {
+    double converted = toBytes(d) / u.multiplier;
+    if (converted > Long.MAX_VALUE)
+      throw new IllegalArgumentException("Converted value (" + converted + ") " +
+        "exceeds Long.MAX_VALUE for " + u.name + ". Try a larger suffix (e.g. MiB instead of KiB)");
+
+    return (long) converted;
   }
 
-  public long toBytes(long d) {
-    if (d == 0) { return 0; }
-    long over = MAX / d;
-    if (d >  over) return Long.MAX_VALUE;
-    if (d < -over) return Long.MIN_VALUE;
+  public double toBytes(long d) {
+    if (d < 0) {
+      throw new IllegalArgumentException("Negative size value. Size must be positive: " + d);
+    }
     return d * multiplier; 
   }
-  public long toKiB(long d) { return convert(d, KiB); }
-  public long toMiB(long d) { return convert(d, MiB); }
-  public long toGiB(long d) { return convert(d, GiB); }
-  public long toTiB(long d) { return convert(d, TiB); }
-  public long toPiB(long d) { return convert(d, PiB); }
   
-  private long multiplier = 0;
-  private static final long MAX = Long.MAX_VALUE;
+  public long toKiB(long d) { return convertTo(d, KiB); }
+  public long toMiB(long d) { return convertTo(d, MiB); }
+  public long toGiB(long d) { return convertTo(d, GiB); }
+  public long toTiB(long d) { return convertTo(d, TiB); }
+  public long toPiB(long d) { return convertTo(d, PiB); }
+  
+  private double multiplier = 0;
+  private String name = "";
 }
diff --git a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
index 2a0dc44699b1..6b514aaa1290 100644
--- a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
+++ b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
@@ -222,10 +222,10 @@ private static long parseByteString(String str, ByteUnit unit) {
         }
 
         // If suffix is valid use that, otherwise none was provided and use the default passed
-        return unit.interpret(val, suffix != null ? byteSuffixes.get(suffix) : unit);  
+        return unit.convertFrom(val, suffix != null ? byteSuffixes.get(suffix) : unit);  
       } else if (fractionMatcher.matches()) {
         throw new NumberFormatException("Fractional values are not supported. Input was: " 
-          + fractionMatcher.group(1);
+          + fractionMatcher.group(1));
       } else {
         throw new NumberFormatException("Failed to parse byte string: " + str);  
       }

From fc85733a9489ba008df8aadc61971136d58b79be Mon Sep 17 00:00:00 2001
From: Ilya Ganelin 
Date: Thu, 23 Apr 2015 17:31:22 -0700
Subject: [PATCH 42/45] Got rid of floating point math

---
 .../apache/spark/network/util/ByteUnit.java   | 41 +++++++++----------
 1 file changed, 20 insertions(+), 21 deletions(-)

diff --git a/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java b/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
index 57561223b690..89df54f58cd5 100644
--- a/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
+++ b/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
@@ -17,36 +17,36 @@
 package org.apache.spark.network.util;
 
 public enum ByteUnit {
-  BYTE (1, "Bytes"),
-  KiB (1024, "KiB"),
-  MiB (Math.pow(1024, 2), "MiB"),
-  GiB (Math.pow(1024, 3), "GiB"),
-  TiB (Math.pow(1024, 4), "TiB"),
-  PiB (Math.pow(1024, 5), "PiB");
+  BYTE (1),
+  KiB (1024L),
+  MiB ((long) Math.pow(1024L, 2L)),
+  GiB ((long) Math.pow(1024L, 3L)),
+  TiB ((long) Math.pow(1024L, 4L)),
+  PiB ((long) Math.pow(1024L, 5L));
 
-  private ByteUnit(double multiplier, String name) {
+  private ByteUnit(long multiplier) {
     this.multiplier = multiplier;
-    this.name = name;
   }
 
   // Interpret the provided number (d) with suffix (u) as this unit type.
   // E.g. KiB.interpret(1, MiB) interprets 1MiB as its KiB representation = 1024k
   public long convertFrom(long d, ByteUnit u) {
-    double converted = u.toBytes(d) / multiplier;
-    if (converted > Long.MAX_VALUE)
-      throw new IllegalArgumentException("Converted value (" + converted + ") " +
-        "exceeds Long.MAX_VALUE for " + name + ". Try a larger suffix (e.g. MiB instead of KiB)");
-    return (long) converted;  
+    return u.convertTo(d, this);
   }
   
   // Convert the provided number (d) interpreted as this unit type to unit type (u). 
   public long convertTo(long d, ByteUnit u) {
-    double converted = toBytes(d) / u.multiplier;
-    if (converted > Long.MAX_VALUE)
-      throw new IllegalArgumentException("Converted value (" + converted + ") " +
-        "exceeds Long.MAX_VALUE for " + u.name + ". Try a larger suffix (e.g. MiB instead of KiB)");
-
-    return (long) converted;
+    if (multiplier > u.multiplier) {
+      long ratio = multiplier / u.multiplier;
+      if (Long.MAX_VALUE / ratio < d) {
+        throw new IllegalArgumentException("Conversion of" + d + "exceeds Long.MAX_VALUE in "
+          + name() + ". Try a larger suffix (e.g. MiB instead of KiB)");
+      }
+      return d * ratio;
+    } else {
+      // Perform operations in this order to avoid potential overflow when computing d * multiplier
+      return d / (u.multiplier / multiplier);
+    }
   }
 
   public double toBytes(long d) {
@@ -62,6 +62,5 @@ public double toBytes(long d) {
   public long toTiB(long d) { return convertTo(d, TiB); }
   public long toPiB(long d) { return convertTo(d, PiB); }
   
-  private double multiplier = 0;
-  private String name = "";
+  private final long multiplier;
 }

From 2ab886b034df177eb19fbcccd05b675c76d0369a Mon Sep 17 00:00:00 2001
From: Ilya Ganelin 
Date: Thu, 23 Apr 2015 17:39:22 -0700
Subject: [PATCH 43/45] Scala style

---
 .../src/main/java/org/apache/spark/network/util/ByteUnit.java  | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)

diff --git a/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java b/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
index 89df54f58cd5..0216d0098638 100644
--- a/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
+++ b/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
@@ -44,7 +44,8 @@ public long convertTo(long d, ByteUnit u) {
       }
       return d * ratio;
     } else {
-      // Perform operations in this order to avoid potential overflow when computing d * multiplier
+      // Perform operations in this order to avoid potential overflow 
+      // when computing d * multiplier
       return d / (u.multiplier / multiplier);
     }
   }

From 49a8720c32ac191248e23428092192c5bf9faed9 Mon Sep 17 00:00:00 2001
From: Ilya Ganelin 
Date: Thu, 23 Apr 2015 17:48:45 -0700
Subject: [PATCH 44/45] Whitespace fix

---
 core/src/test/scala/org/apache/spark/util/UtilsSuite.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
index 2dea58519289..84e07c285e18 100644
--- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
@@ -128,7 +128,7 @@ class UtilsSuite extends FunSuite with ResetSystemProperties {
 
     // Overflow handling, 1073741824p exceeds Long.MAX_VALUE if converted straight to Bytes
     // This demonstrates that we can have e.g 1024^3 PB without overflowing. 
-    assert(Utils.byteStringAsGb("1073741824p") === ByteUnit.PiB.toGiB(1073741824))                                  
+    assert(Utils.byteStringAsGb("1073741824p") === ByteUnit.PiB.toGiB(1073741824))
     assert(Utils.byteStringAsMb("1073741824p") === ByteUnit.PiB.toMiB(1073741824))
     
     // Run this to confirm it doesn't throw an exception

From 11f699948b70ed6dadb4653e2c55013fd60a074d Mon Sep 17 00:00:00 2001
From: Ilya Ganelin 
Date: Fri, 24 Apr 2015 10:28:16 -0700
Subject: [PATCH 45/45] Nit fixes

---
 .../test/scala/org/apache/spark/util/UtilsSuite.scala    | 9 +--------
 .../java/org/apache/spark/network/util/ByteUnit.java     | 4 ++--
 2 files changed, 3 insertions(+), 10 deletions(-)

diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
index 84e07c285e18..62a3cbcdf69e 100644
--- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
@@ -174,17 +174,10 @@ class UtilsSuite extends FunSuite with ResetSystemProperties {
     intercept[NumberFormatException] {
       Utils.byteStringAsBytes("600gb This breaks")
     }
-
+    
     intercept[NumberFormatException] {
       Utils.byteStringAsBytes("This 123mb breaks")
     }
-    
-//    // Test overflow
-//    intercept[NumberFormatException] {
-//      
-//
-//      ByteUnit.convertTo(0x7fffffffffffffffL, ByteUnit.KiB)
-//    }
   }
   
   test("bytesToString") {
diff --git a/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java b/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
index 0216d0098638..36d655017fb0 100644
--- a/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
+++ b/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java
@@ -39,8 +39,8 @@ public long convertTo(long d, ByteUnit u) {
     if (multiplier > u.multiplier) {
       long ratio = multiplier / u.multiplier;
       if (Long.MAX_VALUE / ratio < d) {
-        throw new IllegalArgumentException("Conversion of" + d + "exceeds Long.MAX_VALUE in "
-          + name() + ". Try a larger suffix (e.g. MiB instead of KiB)");
+        throw new IllegalArgumentException("Conversion of " + d + " exceeds Long.MAX_VALUE in "
+          + name() + ". Try a larger unit (e.g. MiB instead of KiB)");
       }
       return d * ratio;
     } else {
Property NameDefaultMeaning
spark.reducer.maxMbInFlight48spark.reducer.maxSizeInFlight48m - 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.
spark.kryoserializer.buffer.mb0.064spark.kryoserializer.buffer64k - 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 per core on each worker. This buffer will grow up to spark.kryoserializer.buffer.max.mb if needed.
spark.kryoserializer.buffer.max.mb64spark.kryoserializer.buffer.max64m - 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.
Property NameDefaultMeaning
spark.broadcast.blockSize40964m - Size of each piece of a block in kilobytes for TorrentBroadcastFactory. + Size of each piece of a block for TorrentBroadcastFactory. Too large a value decreases parallelism during broadcast (makes it slower); however, if it is too small, BlockManager might take a performance hit.
spark.storage.memoryMapThreshold20971522m - 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.
spark.shuffle.file.buffer.kb32spark.shuffle.file.buffer32k - 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.
spark.executor.logs.rolling.size.maxBytesspark.executor.logs.rolling.maxSize (none) 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 spark.executor.logs.rolling.maxRetainedFiles + Rolling is disabled by default. See spark.executor.logs.rolling.maxRetainedFiles for automatic cleaning of old logs.
spark.io.compression.lz4.block.size32768spark.io.compression.lz4.blockSize32k - 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.
spark.io.compression.snappy.block.size32768spark.io.compression.snappy.blockSize32k - 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.