From 59bf9e1d590e49f451aaa7feeb5165572f205358 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Fri, 27 Mar 2015 14:15:26 -0700 Subject: [PATCH 01/58] [SPARK-5931] Updated Utils and JavaUtils classes to add helper methods to handle time strings. Updated time strings in a few places to properly parse time --- .../org/apache/spark/HeartbeatReceiver.scala | 26 +++++------ .../spark/network/nio/ConnectionManager.scala | 9 ++-- .../org/apache/spark/util/AkkaUtils.scala | 5 ++- .../scala/org/apache/spark/util/Utils.scala | 43 ++++++++++++++++--- .../apache/spark/network/util/JavaUtils.java | 34 +++++++++++++++ .../spark/network/util/TransportConf.java | 6 ++- 6 files changed, 97 insertions(+), 26 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index 548dcb93c3358..90378eb52ad8b 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -25,11 +25,11 @@ import akka.actor.{Actor, Cancellable} import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId import org.apache.spark.scheduler.{SlaveLost, TaskScheduler} -import org.apache.spark.util.ActorLogReceive +import org.apache.spark.util.{Utils, ActorLogReceive} /** * A heartbeat from executors to the driver. This is a shared message used by several internal - * components to convey liveness or execution information for in-progress tasks. It will also + * components to convey liveness or execution information for in-progress tasks. It will also * expire the hosts that have not heartbeated for more than spark.network.timeout. */ private[spark] case class Heartbeat( @@ -37,8 +37,8 @@ private[spark] case class Heartbeat( taskMetrics: Array[(Long, TaskMetrics)], // taskId -> TaskMetrics blockManagerId: BlockManagerId) -private[spark] case object ExpireDeadHosts - +private[spark] case object ExpireDeadHosts + private[spark] case class HeartbeatResponse(reregisterBlockManager: Boolean) /** @@ -52,24 +52,24 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, scheduler: TaskSchedule // "spark.network.timeout" uses "seconds", while `spark.storage.blockManagerSlaveTimeoutMs` uses // "milliseconds" - private val executorTimeoutMs = sc.conf.getOption("spark.network.timeout").map(_.toLong * 1000). - getOrElse(sc.conf.getLong("spark.storage.blockManagerSlaveTimeoutMs", 120000)) + private val executorTimeoutMs = Utils.timeStringToMs(sc.conf.get("spark.network.timeout", + sc.conf.get("spark.storage.blockManagerSlaveTimeoutMs", "120s"))) // "spark.network.timeoutInterval" uses "seconds", while // "spark.storage.blockManagerTimeoutIntervalMs" uses "milliseconds" - private val checkTimeoutIntervalMs = - sc.conf.getOption("spark.network.timeoutInterval").map(_.toLong * 1000). - getOrElse(sc.conf.getLong("spark.storage.blockManagerTimeoutIntervalMs", 60000)) - + private val checkTimeoutIntervalMs = Utils.timeStringToMs( + sc.conf.get("spark.network.timeoutInterval", + sc.conf.get("spark.storage.blockManagerTimeoutIntervalMs", "60s"))) + private var timeoutCheckingTask: Cancellable = null - + override def preStart(): Unit = { import context.dispatcher timeoutCheckingTask = context.system.scheduler.schedule(0.seconds, checkTimeoutIntervalMs.milliseconds, self, ExpireDeadHosts) super.preStart() } - + override def receiveWithLogging: PartialFunction[Any, Unit] = { case Heartbeat(executorId, taskMetrics, blockManagerId) => val unknownExecutor = !scheduler.executorHeartbeatReceived( @@ -97,7 +97,7 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, scheduler: TaskSchedule } } } - + override def postStop(): Unit = { if (timeoutCheckingTask != null) { timeoutCheckingTask.cancel() diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala index 741fe3e1ea750..f8346b2f404e6 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala @@ -82,14 +82,15 @@ private[nio] class ConnectionManager( new HashedWheelTimer(Utils.namedThreadFactory("AckTimeoutMonitor")) private val ackTimeout = - conf.getInt("spark.core.connection.ack.wait.timeout", conf.getInt("spark.network.timeout", 120)) + Utils.timeStringToS(conf.get("spark.core.connection.ack.wait.timeout", + conf.get("spark.network.timeout", "120s"))) // Get the thread counts from the Spark Configuration. - // + // // Even though the ThreadPoolExecutor constructor takes both a minimum and maximum value, // we only query for the minimum value because we are using LinkedBlockingDeque. - // - // The JavaDoc for ThreadPoolExecutor points out that when using a LinkedBlockingDeque (which is + // + // The JavaDoc for ThreadPoolExecutor points out that when using a LinkedBlockingDeque (which is // an unbounded queue) no more than corePoolSize threads will ever be created, so only the "min" // parameter is necessary. private val handlerThreadCount = conf.getInt("spark.core.connection.handler.threads.min", 20) diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 48a6ede05e17b..9db72854b6155 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -66,7 +66,8 @@ private[spark] object AkkaUtils extends Logging { val akkaThreads = conf.getInt("spark.akka.threads", 4) val akkaBatchSize = conf.getInt("spark.akka.batchSize", 15) - val akkaTimeout = conf.getInt("spark.akka.timeout", conf.getInt("spark.network.timeout", 120)) + val akkaTimeoutS = Utils.timeStringToS(conf.get("spark.akka.timeout", + conf.get("spark.network.timeout", "120s"))) val akkaFrameSize = maxFrameSizeBytes(conf) val akkaLogLifecycleEvents = conf.getBoolean("spark.akka.logLifecycleEvents", false) val lifecycleEvents = if (akkaLogLifecycleEvents) "on" else "off" @@ -109,7 +110,7 @@ private[spark] object AkkaUtils extends Logging { |akka.remote.netty.tcp.hostname = "$host" |akka.remote.netty.tcp.port = $port |akka.remote.netty.tcp.tcp-nodelay = on - |akka.remote.netty.tcp.connection-timeout = $akkaTimeout s + |akka.remote.netty.tcp.connection-timeout = $akkaTimeoutS s |akka.remote.netty.tcp.maximum-frame-size = ${akkaFrameSize}B |akka.remote.netty.tcp.execution-pool-size = $akkaThreads |akka.actor.default-dispatcher.throughput = $akkaBatchSize 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 0b5a914e7dbbf..aba2cc492c995 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1010,6 +1010,39 @@ private[spark] object Utils extends Logging { ) } + /** + * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use + */ + def timeStringToUs(str: String): Long = { + val lower = str.toLowerCase.trim() + if (lower.endsWith("s")) { + lower.substring(0, lower.length-1).toLong * 1000 * 1000 + } else if (lower.endsWith("ms")) { + lower.substring(0, lower.length-2).toLong * 1000 + } else if (lower.endsWith("us")) { + lower.substring(0, lower.length-2).toLong + } else {// Invalid suffix, force correct formatting + throw new IllegalArgumentException("Time must be specified as seconds (s), " + + "milliseconds (ms), or microseconds (us) e.g. 50s, 100ms, or 250us.") + } + } + + /** + * Convert a time parameter such as (50s, 100ms, or 250us) to milliseconds for internal use. + * Note: may round in some cases + */ + def timeStringToMs(str : String) : Long = { + timeStringToUs(str)/1000 + } + + /** + * Convert a time parameter such as (50s, 100ms, or 250us) to seconds for internal use. + * Note: may round in some cases + */ + def timeStringToS(str : String) : Long = { + timeStringToUs(str)/1000/1000 + } + /** * Convert a Java memory parameter passed to -Xmx (such as 300m or 1g) to a number of megabytes. */ @@ -1146,7 +1179,7 @@ private[spark] object Utils extends Logging { /** * Execute a block of code that evaluates to Unit, forwarding any uncaught exceptions to the * default UncaughtExceptionHandler - * + * * NOTE: This method is to be called by the spark-started JVM process. */ def tryOrExit(block: => Unit) { @@ -1159,11 +1192,11 @@ private[spark] object Utils extends Logging { } /** - * Execute a block of code that evaluates to Unit, stop SparkContext is there is any uncaught + * Execute a block of code that evaluates to Unit, stop SparkContext is there is any uncaught * exception - * - * NOTE: This method is to be called by the driver-side components to avoid stopping the - * user-started JVM process completely; in contrast, tryOrExit is to be called in the + * + * NOTE: This method is to be called by the driver-side components to avoid stopping the + * user-started JVM process completely; in contrast, tryOrExit is to be called in the * spark-started JVM process . */ def tryOrStopSparkContext(sc: SparkContext)(block: => Unit) { 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 73da9b7346f4d..854b78ba8118c 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 @@ -121,4 +121,38 @@ private static boolean isSymlink(File file) throws IOException { } return !fileInCanonicalDir.getCanonicalFile().equals(fileInCanonicalDir.getAbsoluteFile()); } + + /** + * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use + */ + public static long timeStringToUs(String str) throws IllegalArgumentException { + String lower = str.toLowerCase().trim(); + if (lower.endsWith("s")) { + return Long.parseLong(lower.substring(0, lower.length()-1)) * 1000 * 1000; + } else if (lower.endsWith("ms")) { + return Long.parseLong(lower.substring(0, lower.length()-2)) * 1000; + } else if (lower.endsWith("us")) { + return Long.parseLong(lower.substring(0, lower.length()-2)); + } else {// Invalid suffix, force correct formatting + throw new IllegalArgumentException("Time must be specified as seconds (s), " + + "milliseconds (ms), or microseconds (us) e.g. 50s, 100ms, or 250us."); + } + } + + /** + * Convert a time parameter such as (50s, 100ms, or 250us) to milliseconds for internal use. + * Note: may round in some cases + */ + public static long timeStringToMs(String str) throws IllegalArgumentException { + return timeStringToUs(str)/1000; + } + + /** + * Convert a time parameter such as (50s, 100ms, or 250us) to seconds for internal use. + * Note: may round in some cases + */ + public static long timeStringToS(String str) throws IllegalArgumentException { + return timeStringToUs(str)/1000/1000; + } + } diff --git a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java index 2eaf3b71d9a49..a334d759381bc 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -37,8 +37,10 @@ public boolean preferDirectBufs() { /** Connect timeout in milliseconds. Default 120 secs. */ public int connectionTimeoutMs() { - int defaultTimeout = conf.getInt("spark.network.timeout", 120); - return conf.getInt("spark.shuffle.io.connectionTimeout", defaultTimeout) * 1000; + long defaultTimeout = JavaUtils.timeStringToMs( + conf.get("spark.shuffle.io.connectionTimeout", + conf.get("spark.network.timeout", "120s"))); + return (int) defaultTimeout; } /** Number of concurrent connections between two nodes for fetching data. */ From 404f8c3762d7e23fdf3ef8d158d68f5b5aa38360 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Fri, 27 Mar 2015 14:25:15 -0700 Subject: [PATCH 02/58] Updated usage of spark.core.connection.ack.wait.timeout --- .../org/apache/spark/executor/Executor.scala | 8 +-- .../scala/org/apache/spark/util/Utils.scala | 6 +- .../network/nio/ConnectionManagerSuite.scala | 2 +- .../BlockManagerReplicationSuite.scala | 2 +- docs/configuration.md | 58 +++++++++---------- 5 files changed, 38 insertions(+), 38 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index bf3135ef081c1..ded7f7ae3b018 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -442,14 +442,14 @@ private[spark] class Executor( * This thread stops running when the executor is stopped. */ private def startDriverHeartbeater(): Unit = { - val interval = conf.getInt("spark.executor.heartbeatInterval", 10000) + val intervalMs = Utils.timeStringToMs(conf.get("spark.executor.heartbeatInterval", "10s")) val thread = new Thread() { override def run() { - // Sleep a random interval so the heartbeats don't end up in sync - Thread.sleep(interval + (math.random * interval).asInstanceOf[Int]) + // Sleep a random intervalMs so the heartbeats don't end up in sync + Thread.sleep(intervalMs + (math.random * intervalMs).asInstanceOf[Int]) while (!isStopped) { reportHeartBeat() - Thread.sleep(interval) + Thread.sleep(intervalMs) } } } 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 aba2cc492c995..96efb3667598b 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -611,9 +611,9 @@ private[spark] object Utils extends Logging { } Utils.setupSecureURLConnection(uc, securityMgr) - val timeout = conf.getInt("spark.files.fetchTimeout", 60) * 1000 - uc.setConnectTimeout(timeout) - uc.setReadTimeout(timeout) + val timeoutMs = Utils.timeStringToMs(conf.get("spark.files.fetchTimeout", "60s")).toInt + uc.setConnectTimeout(timeoutMs) + uc.setReadTimeout(timeoutMs) uc.connect() val in = uc.getInputStream() downloadFile(url, in, targetFile, fileOverwrite) diff --git a/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala b/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala index 716f875d30b8a..e8916c2f886fc 100644 --- a/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala @@ -261,7 +261,7 @@ class ConnectionManagerSuite extends FunSuite { val clientConf = new SparkConf clientConf.set("spark.authenticate", "false") val ackTimeout = 30 - clientConf.set("spark.core.connection.ack.wait.timeout", s"${ackTimeout}") + clientConf.set("spark.core.connection.ack.wait.timeout", s"${ackTimeout}s") val clientSecurityManager = new SecurityManager(clientConf) val manager = new ConnectionManager(0, clientConf, clientSecurityManager) 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 c2903c8597997..d406d08a2d3db 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -79,7 +79,7 @@ class BlockManagerReplicationSuite extends FunSuite with Matchers with BeforeAnd conf.set("spark.storage.unrollMemoryThreshold", "512") // to make a replication attempt to inactive store fail fast - conf.set("spark.core.connection.ack.wait.timeout", "1") + conf.set("spark.core.connection.ack.wait.timeout", "1s") // to make cached peers refresh frequently conf.set("spark.storage.cachedPeersTtl", "10") diff --git a/docs/configuration.md b/docs/configuration.md index 7fe11475212b3..c8486c626ec55 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -120,7 +120,7 @@ of the most common options to set are: Amount of memory to use for the driver process, i.e. where SparkContext is initialized. (e.g. 512m, 2g). - +
Note: In client mode, this config must not be set through the SparkConf directly in your application, because the driver JVM has already started at that point. Instead, please set this through the --driver-memory command line option @@ -188,7 +188,7 @@ Apart from these, the following properties are also available, and may be useful
Note: In client mode, this config must not be set through the SparkConf directly in your application, because the driver JVM has already started at that point. - Instead, please set this through the --driver-class-path command line option or in + Instead, please set this through the --driver-class-path command line option or in your default properties file. @@ -197,10 +197,10 @@ Apart from these, the following properties are also available, and may be useful (none) A string of extra JVM options to pass to the driver. For instance, GC settings or other logging. - +
Note: In client mode, this config must not be set through the SparkConf directly in your application, because the driver JVM has already started at that point. - Instead, please set this through the --driver-java-options command line option or in + Instead, please set this through the --driver-java-options command line option or in your default properties file. @@ -209,10 +209,10 @@ Apart from these, the following properties are also available, and may be useful (none) Set a special library path to use when launching the driver JVM. - +
Note: In client mode, this config must not be set through the SparkConf directly in your application, because the driver JVM has already started at that point. - Instead, please set this through the --driver-library-path command line option or in + Instead, please set this through the --driver-library-path command line option or in your default properties file. @@ -223,7 +223,7 @@ Apart from these, the following properties are also available, and may be useful (Experimental) Whether to give user-added jars precedence over Spark's own jars when loading classes in the the driver. This feature can be used to mitigate conflicts between Spark's dependencies and user dependencies. It is currently an experimental feature. - + This is used in cluster mode only. @@ -231,8 +231,8 @@ Apart from these, the following properties are also available, and may be useful spark.executor.extraClassPath (none) - Extra classpath entries to append to the classpath of executors. This exists primarily for - backwards-compatibility with older versions of Spark. Users typically should not need to set + Extra classpath entries to append to the classpath of executors. This exists primarily for + backwards-compatibility with older versions of Spark. Users typically should not need to set this option. @@ -240,9 +240,9 @@ Apart from these, the following properties are also available, and may be useful spark.executor.extraJavaOptions (none) - A string of extra JVM options to pass to executors. For instance, GC settings or other logging. - Note that it is illegal to set Spark properties or heap size settings with this option. Spark - properties should be set using a SparkConf object or the spark-defaults.conf file used with the + A string of extra JVM options to pass to executors. For instance, GC settings or other logging. + Note that it is illegal to set Spark properties or heap size settings with this option. Spark + properties should be set using a SparkConf object or the spark-defaults.conf file used with the spark-submit script. Heap size settings can be set with spark.executor.memory. @@ -732,17 +732,17 @@ Apart from these, the following properties are also available, and may be useful spark.executor.heartbeatInterval - 10000 - Interval (milliseconds) between each executor's heartbeats to the driver. Heartbeats let + 10s + Interval between each executor's heartbeats to the driver. Heartbeats let the driver know that the executor is still alive and update it with metrics for in-progress tasks. spark.files.fetchTimeout - 60 + 60s Communication timeout to use when fetching files added through SparkContext.addFile() from - the driver, in seconds. + the driver. @@ -855,14 +855,14 @@ Apart from these, the following properties are also available, and may be useful spark.akka.heartbeat.interval 1000 - This is set to a larger value to disable the transport failure detector that comes built in to - Akka. It can be enabled again, if you plan to use this feature (Not recommended). A larger - interval value in seconds reduces network overhead and a smaller value ( ~ 1 s) might be more - informative for Akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses` - if you need to. A likely positive use case for using failure detector would be: a sensistive - failure detector can help evict rogue executors quickly. However this is usually not the case - as GC pauses and network lags are expected in a real Spark cluster. Apart from that enabling - this leads to a lot of exchanges of heart beats between nodes leading to flooding the network + This is set to a larger value to disable the transport failure detector that comes built in to + Akka. It can be enabled again, if you plan to use this feature (Not recommended). A larger + interval value in seconds reduces network overhead and a smaller value ( ~ 1 s) might be more + informative for Akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses` + if you need to. A likely positive use case for using failure detector would be: a sensistive + failure detector can help evict rogue executors quickly. However this is usually not the case + as GC pauses and network lags are expected in a real Spark cluster. Apart from that enabling + this leads to a lot of exchanges of heart beats between nodes leading to flooding the network with those. @@ -871,7 +871,7 @@ Apart from these, the following properties are also available, and may be useful 6000 This is set to a larger value to disable the transport failure detector that comes built in to Akka. - It can be enabled again, if you plan to use this feature (Not recommended). Acceptable heart + It can be enabled again, if you plan to use this feature (Not recommended). Acceptable heart beat pause in seconds for Akka. This can be used to control sensitivity to GC pauses. Tune this along with `spark.akka.heartbeat.interval` if you need to. @@ -938,9 +938,9 @@ Apart from these, the following properties are also available, and may be useful spark.network.timeout - 120 + 120s - Default timeout for all network interactions, in seconds. This config will be used in + Default timeout for all network interactions. This config will be used in place of spark.core.connection.ack.wait.timeout, spark.akka.timeout, spark.storage.blockManagerSlaveTimeoutMs or spark.shuffle.io.connectionTimeout, if they are not configured. @@ -1215,9 +1215,9 @@ Apart from these, the following properties are also available, and may be useful spark.core.connection.ack.wait.timeout - 60 + 60s - Number of seconds for the connection to wait for ack to occur before timing + How long for the connection to wait for ack to occur before timing out and giving up. To avoid unwilling timeout caused by long pause like GC, you can set larger value. From 7db6d2a73ea4de5bb52bcdf8e4af7edcc06a10dd Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Fri, 27 Mar 2015 14:27:57 -0700 Subject: [PATCH 03/58] Updated usage of spark.akka.timeout --- docs/configuration.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index c8486c626ec55..35a168225c21f 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -886,9 +886,9 @@ Apart from these, the following properties are also available, and may be useful spark.akka.timeout - 100 + 100s - Communication timeout between Spark nodes, in seconds. + Communication timeout between Spark nodes. From 4933fda30ce468d72e646536489468e295800775 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Fri, 27 Mar 2015 14:30:05 -0700 Subject: [PATCH 04/58] Updated usage of spark.storage.blockManagerSlaveTimeout --- core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala | 4 +--- docs/configuration.md | 2 +- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index 90378eb52ad8b..6a8b96a184e2a 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -50,10 +50,8 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, scheduler: TaskSchedule // executor ID -> timestamp of when the last heartbeat from this executor was received private val executorLastSeen = new mutable.HashMap[String, Long] - // "spark.network.timeout" uses "seconds", while `spark.storage.blockManagerSlaveTimeoutMs` uses - // "milliseconds" private val executorTimeoutMs = Utils.timeStringToMs(sc.conf.get("spark.network.timeout", - sc.conf.get("spark.storage.blockManagerSlaveTimeoutMs", "120s"))) + sc.conf.get("spark.storage.blockManagerSlaveTimeout", "120s"))) // "spark.network.timeoutInterval" uses "seconds", while // "spark.storage.blockManagerTimeoutIntervalMs" uses "milliseconds" diff --git a/docs/configuration.md b/docs/configuration.md index 35a168225c21f..a7f0b96edd1d1 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -942,7 +942,7 @@ Apart from these, the following properties are also available, and may be useful Default timeout for all network interactions. This config will be used in place of spark.core.connection.ack.wait.timeout, spark.akka.timeout, - spark.storage.blockManagerSlaveTimeoutMs or + spark.storage.blockManagerSlaveTimeout or spark.shuffle.io.connectionTimeout, if they are not configured. From c9f5cad6a35cc2f405d874a5ca62fa3f92ef0a8b Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Fri, 27 Mar 2015 14:39:15 -0700 Subject: [PATCH 05/58] Updated spark.shuffle.io.retryWait --- docs/configuration.md | 6 +++--- .../java/org/apache/spark/network/util/TransportConf.java | 4 +++- .../spark/network/shuffle/RetryingBlockFetcherSuite.java | 2 +- 3 files changed, 7 insertions(+), 5 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index a7f0b96edd1d1..1fa1929897d00 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -429,10 +429,10 @@ Apart from these, the following properties are also available, and may be useful spark.shuffle.io.retryWait - 5 + 5s - (Netty only) Seconds to wait between retries of fetches. The maximum delay caused by retrying - is simply maxRetries * retryWait, by default 15 seconds. + (Netty only) How long to wait between retries of fetches. The maximum delay caused by retrying + is simply maxRetries * retryWait, by default 5 seconds. diff --git a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java index a334d759381bc..5ccef0950cb76 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -82,7 +82,9 @@ public int numConnectionsPerPeer() { * Time (in milliseconds) that we will wait in order to perform a retry after an IOException. * Only relevant if maxIORetries > 0. */ - public int ioRetryWaitTimeMs() { return conf.getInt("spark.shuffle.io.retryWait", 5) * 1000; } + public int ioRetryWaitTimeMs() { + return (int)JavaUtils.timeStringToMs(conf.get("spark.shuffle.io.retryWait", "5s")); + } /** * Minimum size of a block that we should start using memory map rather than reading in through diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java index 1ad0d72ae5ec5..51d47511e7d1b 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java @@ -54,7 +54,7 @@ public class RetryingBlockFetcherSuite { @Before public void beforeEach() { System.setProperty("spark.shuffle.io.maxRetries", "2"); - System.setProperty("spark.shuffle.io.retryWait", "0"); + System.setProperty("spark.shuffle.io.retryWait", "0s"); } @After From 21ef3dd6165022a1edc396c50817641d3321b1f7 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Fri, 27 Mar 2015 14:40:52 -0700 Subject: [PATCH 06/58] updated spark.shuffle.sasl.timeout --- .../java/org/apache/spark/network/util/TransportConf.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java index 5ccef0950cb76..99e3c8458e8c0 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -70,7 +70,9 @@ public int numConnectionsPerPeer() { public int sendBuf() { return conf.getInt("spark.shuffle.io.sendBuffer", -1); } /** Timeout for a single round trip of SASL token exchange, in milliseconds. */ - public int saslRTTimeoutMs() { return conf.getInt("spark.shuffle.sasl.timeout", 30) * 1000; } + public int saslRTTimeoutMs() { + return (int) JavaUtils.timeStringToMs(conf.get("spark.shuffle.sasl.timeout", "30s")); + } /** * Max number of times we will try IO exceptions (such as connection timeouts) per request. @@ -83,7 +85,7 @@ public int numConnectionsPerPeer() { * Only relevant if maxIORetries > 0. */ public int ioRetryWaitTimeMs() { - return (int)JavaUtils.timeStringToMs(conf.get("spark.shuffle.io.retryWait", "5s")); + return (int) JavaUtils.timeStringToMs(conf.get("spark.shuffle.io.retryWait", "5s")); } /** From 064ebd67faf655f03d0e02763f53f1df60b3e891 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Fri, 27 Mar 2015 14:50:51 -0700 Subject: [PATCH 07/58] Updated usage of spark.cleaner.ttl --- .../org/apache/spark/util/MetadataCleaner.scala | 2 +- docs/configuration.md | 2 +- .../spark/streaming/StreamingContextSuite.scala | 17 +++++++++-------- 3 files changed, 11 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala index 375ed430bde45..3e2ee097980b5 100644 --- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala @@ -76,7 +76,7 @@ private[spark] object MetadataCleanerType extends Enumeration { // initialization of StreamingContext. It's okay for users trying to configure stuff themselves. private[spark] object MetadataCleaner { def getDelaySeconds(conf: SparkConf): Int = { - conf.getInt("spark.cleaner.ttl", -1) + Utils.timeStringToS(conf.get("spark.cleaner.ttl", "-1s")).toInt } def getDelaySeconds( diff --git a/docs/configuration.md b/docs/configuration.md index 1fa1929897d00..72774750ca0a8 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -706,7 +706,7 @@ Apart from these, the following properties are also available, and may be useful spark.cleaner.ttl (infinite) - Duration (seconds) of how long Spark will remember any metadata (stages generated, tasks + Duration of how long Spark will remember any metadata (stages generated, tasks generated, etc.). Periodic cleanups will ensure that metadata older than this duration will be forgotten. This is useful for running Spark for many hours / days (for example, running 24/7 in case of Spark Streaming applications). Note that any RDD that persists in memory for more than diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index 2e5005ef6ff14..b105da6b1d156 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -73,9 +73,9 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("from conf with settings") { val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) - myConf.set("spark.cleaner.ttl", "10") + myConf.set("spark.cleaner.ttl", "10s") ssc = new StreamingContext(myConf, batchDuration) - assert(ssc.conf.getInt("spark.cleaner.ttl", -1) === 10) + assert(Utils.timeStringToS(ssc.conf.get("spark.cleaner.ttl", "-1s")) === 10) } test("from existing SparkContext") { @@ -85,24 +85,25 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("from existing SparkContext with settings") { val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) - myConf.set("spark.cleaner.ttl", "10") + myConf.set("spark.cleaner.ttl", "10s") ssc = new StreamingContext(myConf, batchDuration) - assert(ssc.conf.getInt("spark.cleaner.ttl", -1) === 10) + assert(Utils.timeStringToS(ssc.conf.get("spark.cleaner.ttl", "-1s")) === 10) } test("from checkpoint") { val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) - myConf.set("spark.cleaner.ttl", "10") + myConf.set("spark.cleaner.ttl", "10s") val ssc1 = new StreamingContext(myConf, batchDuration) addInputStream(ssc1).register() ssc1.start() val cp = new Checkpoint(ssc1, Time(1000)) - assert(cp.sparkConfPairs.toMap.getOrElse("spark.cleaner.ttl", "-1") === "10") + assert( + Utils.timeStringToS(cp.sparkConfPairs.toMap.getOrElse("spark.cleaner.ttl", "-1s")) === 10) ssc1.stop() val newCp = Utils.deserialize[Checkpoint](Utils.serialize(cp)) - assert(newCp.createSparkConf().getInt("spark.cleaner.ttl", -1) === 10) + assert(Utils.timeStringToS(newCp.createSparkConf().get("spark.cleaner.ttl", "-1s")) === 10) ssc = new StreamingContext(null, newCp, null) - assert(ssc.conf.getInt("spark.cleaner.ttl", -1) === 10) + assert(Utils.timeStringToS(ssc.conf.get("spark.cleaner.ttl", "-1s")) === 10) } test("start and stop state check") { From 7320c87289ded3f61607070a4098d00dc5ee17be Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Fri, 27 Mar 2015 14:56:23 -0700 Subject: [PATCH 08/58] updated spark.akka.heartbeat.interval --- core/src/main/scala/org/apache/spark/util/AkkaUtils.scala | 5 +++-- docs/configuration.md | 4 ++-- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 9db72854b6155..2e15866d7906c 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -80,7 +80,8 @@ private[spark] object AkkaUtils extends Logging { val logAkkaConfig = if (conf.getBoolean("spark.akka.logAkkaConfig", false)) "on" else "off" val akkaHeartBeatPauses = conf.getInt("spark.akka.heartbeat.pauses", 6000) - val akkaHeartBeatInterval = conf.getInt("spark.akka.heartbeat.interval", 1000) + val akkaHeartBeatIntervalS = + Utils.timeStringToS(conf.get("spark.akka.heartbeat.interval", "1000s")) val secretKey = securityManager.getSecretKey() val isAuthOn = securityManager.isAuthenticationEnabled() @@ -103,7 +104,7 @@ private[spark] object AkkaUtils extends Logging { |akka.jvm-exit-on-fatal-error = off |akka.remote.require-cookie = "$requireCookie" |akka.remote.secure-cookie = "$secureCookie" - |akka.remote.transport-failure-detector.heartbeat-interval = $akkaHeartBeatInterval s + |akka.remote.transport-failure-detector.heartbeat-interval = $akkaHeartBeatIntervalS s |akka.remote.transport-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPauses s |akka.actor.provider = "akka.remote.RemoteActorRefProvider" |akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport" diff --git a/docs/configuration.md b/docs/configuration.md index 72774750ca0a8..0e5def3349157 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -853,11 +853,11 @@ Apart from these, the following properties are also available, and may be useful spark.akka.heartbeat.interval - 1000 + 1000s This is set to a larger value to disable the transport failure detector that comes built in to Akka. It can be enabled again, if you plan to use this feature (Not recommended). A larger - interval value in seconds reduces network overhead and a smaller value ( ~ 1 s) might be more + interval value in seconds reduces network overhead and a smaller value ( ~ 1s) might be more informative for Akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses` if you need to. A likely positive use case for using failure detector would be: a sensistive failure detector can help evict rogue executors quickly. However this is usually not the case From 272c2159195948ccae9b399126f8e85015e1aaae Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Fri, 27 Mar 2015 15:00:57 -0700 Subject: [PATCH 09/58] Updated spark.locality.wait --- .../scala/org/apache/spark/scheduler/TaskSetManager.scala | 8 ++++---- .../org/apache/spark/scheduler/TaskSetManagerSuite.scala | 4 ++-- docs/configuration.md | 4 ++-- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index d509881c74fef..72bf0e4bc10ac 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -848,14 +848,14 @@ private[spark] class TaskSetManager( } private def getLocalityWait(level: TaskLocality.TaskLocality): Long = { - val defaultWait = conf.get("spark.locality.wait", "3000") + val defaultWait = conf.get("spark.locality.wait", "3000ms") level match { case TaskLocality.PROCESS_LOCAL => - conf.get("spark.locality.wait.process", defaultWait).toLong + Utils.timeStringToMs(conf.get("spark.locality.wait.process", defaultWait)) case TaskLocality.NODE_LOCAL => - conf.get("spark.locality.wait.node", defaultWait).toLong + Utils.timeStringToMs(conf.get("spark.locality.wait.node", defaultWait)) case TaskLocality.RACK_LOCAL => - conf.get("spark.locality.wait.rack", defaultWait).toLong + Utils.timeStringToMs(conf.get("spark.locality.wait.rack", defaultWait)) case _ => 0L } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 12330d8f63c40..b8d372770b959 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -27,7 +27,7 @@ import org.scalatest.FunSuite import org.apache.spark._ import org.apache.spark.executor.TaskMetrics -import org.apache.spark.util.ManualClock +import org.apache.spark.util.{Utils, ManualClock} class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler) extends DAGScheduler(sc) { @@ -152,7 +152,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { private val conf = new SparkConf - val LOCALITY_WAIT = conf.getLong("spark.locality.wait", 3000) + val LOCALITY_WAIT = Utils.timeStringToMs(conf.get("spark.locality.wait", "3000ms")) val MAX_TASK_FAILURES = 4 override def beforeEach() { diff --git a/docs/configuration.md b/docs/configuration.md index 0e5def3349157..1f90ea862dd38 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -989,9 +989,9 @@ Apart from these, the following properties are also available, and may be useful spark.locality.wait - 3000 + 3000ms - Number of milliseconds to wait to launch a data-local task before giving up and launching it + How long to wait to launch a data-local task before giving up and launching it on a less-local node. The same wait will be used to step through multiple locality levels (process-local, node-local, rack-local and then any). It is also possible to customize the waiting time for each level by setting spark.locality.wait.node, etc. From 3352d34d925650fdba34b05fc3521e15d18fcd11 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Fri, 27 Mar 2015 15:02:59 -0700 Subject: [PATCH 10/58] Updated spark.scheduler.maxRegisteredResourcesWaitingTime --- .../scheduler/cluster/CoarseGrainedSchedulerBackend.scala | 8 ++++---- docs/configuration.md | 5 ++--- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 5d258d9da4d1a..c2c6c1c813c60 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -57,8 +57,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste math.min(1, conf.getDouble("spark.scheduler.minRegisteredResourcesRatio", 0)) // Submit tasks after maxRegisteredWaitingTime milliseconds // if minRegisteredRatio has not yet been reached - val maxRegisteredWaitingTime = - conf.getInt("spark.scheduler.maxRegisteredResourcesWaitingTime", 30000) + val maxRegisteredWaitingTimeMs = + Utils.timeStringToMs(conf.get("spark.scheduler.maxRegisteredResourcesWaitingTime", "30000ms")) val createTime = System.currentTimeMillis() private val executorDataMap = new HashMap[String, ExecutorData] @@ -297,9 +297,9 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste s"reached minRegisteredResourcesRatio: $minRegisteredRatio") return true } - if ((System.currentTimeMillis() - createTime) >= maxRegisteredWaitingTime) { + if ((System.currentTimeMillis() - createTime) >= maxRegisteredWaitingTimeMs) { logInfo("SchedulerBackend is ready for scheduling beginning after waiting " + - s"maxRegisteredResourcesWaitingTime: $maxRegisteredWaitingTime(ms)") + s"maxRegisteredResourcesWaitingTime: $maxRegisteredWaitingTimeMs(ms)") return true } false diff --git a/docs/configuration.md b/docs/configuration.md index 1f90ea862dd38..9c4081160c75e 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1024,10 +1024,9 @@ Apart from these, the following properties are also available, and may be useful spark.scheduler.maxRegisteredResourcesWaitingTime - 30000 + 30000ms - Maximum amount of time to wait for resources to register before scheduling begins - (in milliseconds). + Maximum amount of time to wait for resources to register before scheduling begins. From 3f1cfc84e669df7c2198ed21d776fd391977f34f Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Fri, 27 Mar 2015 15:05:04 -0700 Subject: [PATCH 11/58] Updated spark.scheduler.revive.interval --- .../scheduler/cluster/CoarseGrainedSchedulerBackend.scala | 5 +++-- docs/configuration.md | 5 ++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index c2c6c1c813c60..0cb9b9cfdca20 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -80,9 +80,10 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) // Periodically revive offers to allow delay scheduling to work - val reviveInterval = conf.getLong("spark.scheduler.revive.interval", 1000) + val reviveIntervalMs = Utils.timeStringToMs( + conf.get("spark.scheduler.revive.interval", "1000ms")) import context.dispatcher - context.system.scheduler.schedule(0.millis, reviveInterval.millis, self, ReviveOffers) + context.system.scheduler.schedule(0.millis, reviveIntervalMs.millis, self, ReviveOffers) } def receiveWithLogging: PartialFunction[Any, Unit] = { diff --git a/docs/configuration.md b/docs/configuration.md index 9c4081160c75e..2b45005ed0741 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1053,10 +1053,9 @@ Apart from these, the following properties are also available, and may be useful spark.scheduler.revive.interval - 1000 + 1000ms - The interval length for the scheduler to revive the worker resource offers to run tasks - (in milliseconds). + The interval length for the scheduler to revive the worker resource offers to run tasks. From 6d1518e8870dc91fec86d75be0cac11d3e1dc2b4 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Fri, 27 Mar 2015 15:07:57 -0700 Subject: [PATCH 12/58] Upated spark.speculation.interval --- .../apache/spark/scheduler/TaskSchedulerImpl.scala | 11 ++++++----- docs/configuration.md | 4 ++-- 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 076b36e86c0ce..d931c20fe27e5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -62,10 +62,11 @@ private[spark] class TaskSchedulerImpl( val conf = sc.conf // How often to check for speculative tasks - val SPECULATION_INTERVAL = conf.getLong("spark.speculation.interval", 100) + val SPECULATION_INTERVAL_MS = + Utils.timeStringToMs(conf.get("spark.speculation.interval", "100ms")) // Threshold above which we warn user initial TaskSet may be starved - val STARVATION_TIMEOUT = conf.getLong("spark.starvation.timeout", 15000) + val STARVATION_TIMEOUT_MS = Utils.timeStringToMs(conf.get("spark.starvation.timeout", "15000ms")) // CPUs to request per task val CPUS_PER_TASK = conf.getInt("spark.task.cpus", 1) @@ -143,8 +144,8 @@ private[spark] class TaskSchedulerImpl( if (!isLocal && conf.getBoolean("spark.speculation", false)) { logInfo("Starting speculative execution thread") import sc.env.actorSystem.dispatcher - sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL milliseconds, - SPECULATION_INTERVAL milliseconds) { + sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL_MS milliseconds, + SPECULATION_INTERVAL_MS milliseconds) { Utils.tryOrStopSparkContext(sc) { checkSpeculatableTasks() } } } @@ -173,7 +174,7 @@ private[spark] class TaskSchedulerImpl( this.cancel() } } - }, STARVATION_TIMEOUT, STARVATION_TIMEOUT) + }, STARVATION_TIMEOUT_MS, STARVATION_TIMEOUT_MS) } hasReceivedTask = true } diff --git a/docs/configuration.md b/docs/configuration.md index 2b45005ed0741..0889daa2ca9fe 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1068,9 +1068,9 @@ Apart from these, the following properties are also available, and may be useful spark.speculation.interval - 100 + 100ms - How often Spark will check for tasks to speculate, in milliseconds. + How often Spark will check for tasks to speculate. From 2fcc91c7cd062ca67c5b9e1565d13533e7013c2f Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Fri, 27 Mar 2015 15:13:51 -0700 Subject: [PATCH 13/58] Updated spark.dynamicAllocation.executorIdleTimeout --- .../scala/org/apache/spark/ExecutorAllocationManager.scala | 6 +++--- .../org/apache/spark/ExecutorAllocationManagerSuite.scala | 2 +- docs/configuration.md | 6 +++--- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 21c6e6ffa6666..44a11cb15f688 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -20,7 +20,7 @@ package org.apache.spark import scala.collection.mutable import org.apache.spark.scheduler._ -import org.apache.spark.util.{SystemClock, Clock} +import org.apache.spark.util.{Utils, SystemClock, Clock} /** * An agent that dynamically allocates and removes executors based on the workload. @@ -86,8 +86,8 @@ private[spark] class ExecutorAllocationManager( "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", schedulerBacklogTimeout) // How long an executor must be idle for before it is removed (seconds) - private val executorIdleTimeout = conf.getLong( - "spark.dynamicAllocation.executorIdleTimeout", 600) + private val executorIdleTimeout = Utils.timeStringToS(conf.get( + "spark.dynamicAllocation.executorIdleTimeout", "600s")) // During testing, the methods to actually kill and add executors are mocked out private val testing = conf.getBoolean("spark.dynamicAllocation.testing", false) diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index abfcee75728dc..f18fab5d0e808 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -686,7 +686,7 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { .set("spark.dynamicAllocation.schedulerBacklogTimeout", schedulerBacklogTimeout.toString) .set("spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", sustainedSchedulerBacklogTimeout.toString) - .set("spark.dynamicAllocation.executorIdleTimeout", executorIdleTimeout.toString) + .set("spark.dynamicAllocation.executorIdleTimeout", s"${executorIdleTimeout.toString}s") .set("spark.dynamicAllocation.testing", "true") new SparkContext(conf) } diff --git a/docs/configuration.md b/docs/configuration.md index 0889daa2ca9fe..1ab3c57bdfae1 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1125,10 +1125,10 @@ Apart from these, the following properties are also available, and may be useful spark.dynamicAllocation.executorIdleTimeout - 600 + 600s - If dynamic allocation is enabled and an executor has been idle for more than this duration - (in seconds), the executor will be removed. For more detail, see this + If dynamic allocation is enabled and an executor has been idle for more than this duration, + the executor will be removed. For more detail, see this description. From 5181597688a6d6f0d9be311784582d033c908225 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Fri, 27 Mar 2015 15:17:58 -0700 Subject: [PATCH 14/58] Updated spark.dynamicAllocation.schedulerBacklogTimeout --- .../spark/ExecutorAllocationManager.scala | 46 +++++++++---------- .../ExecutorAllocationManagerSuite.scala | 5 +- docs/configuration.md | 6 +-- docs/job-scheduling.md | 2 +- 4 files changed, 30 insertions(+), 29 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 44a11cb15f688..c86c7c0a2040a 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -52,14 +52,14 @@ import org.apache.spark.util.{Utils, SystemClock, Clock} * spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors * spark.dynamicAllocation.initialExecutors - Number of executors to start with * - * spark.dynamicAllocation.schedulerBacklogTimeout (M) - + * spark.dynamicAllocation.schedulerBacklogTimeoutS (M) - * If there are backlogged tasks for this duration, add new executors * - * spark.dynamicAllocation.sustainedSchedulerBacklogTimeout (N) - + * spark.dynamicAllocation.sustainedSchedulerBacklogTimeoutS (N) - * If the backlog is sustained for this duration, add more executors * This is used only after the initial backlog timeout is exceeded * - * spark.dynamicAllocation.executorIdleTimeout (K) - + * spark.dynamicAllocation.executorIdleTimeoutS (K) - * If an executor has been idle for this duration, remove it */ private[spark] class ExecutorAllocationManager( @@ -78,16 +78,16 @@ private[spark] class ExecutorAllocationManager( Integer.MAX_VALUE) // How long there must be backlogged tasks for before an addition is triggered (seconds) - private val schedulerBacklogTimeout = conf.getLong( - "spark.dynamicAllocation.schedulerBacklogTimeout", 5) + private val schedulerBacklogTimeoutS = Utils.timeStringToS(conf.get( + "spark.dynamicAllocation.schedulerBacklogTimeoutS", "5s")) - // Same as above, but used only after `schedulerBacklogTimeout` is exceeded - private val sustainedSchedulerBacklogTimeout = conf.getLong( - "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", schedulerBacklogTimeout) + // Same as above, but used only after `schedulerBacklogTimeoutS` is exceeded + private val sustainedSchedulerBacklogTimeoutS = Utils.timeStringToS(conf.get( + "spark.dynamicAllocation.sustainedSchedulerBacklogTimeoutS", s"${schedulerBacklogTimeoutS}s")) // How long an executor must be idle for before it is removed (seconds) - private val executorIdleTimeout = Utils.timeStringToS(conf.get( - "spark.dynamicAllocation.executorIdleTimeout", "600s")) + private val executorIdleTimeoutS = Utils.timeStringToS(conf.get( + "spark.dynamicAllocation.executorIdleTimeoutS", "600s")) // During testing, the methods to actually kill and add executors are mocked out private val testing = conf.getBoolean("spark.dynamicAllocation.testing", false) @@ -144,15 +144,15 @@ private[spark] class ExecutorAllocationManager( throw new SparkException(s"spark.dynamicAllocation.minExecutors ($minNumExecutors) must " + s"be less than or equal to spark.dynamicAllocation.maxExecutors ($maxNumExecutors)!") } - if (schedulerBacklogTimeout <= 0) { - throw new SparkException("spark.dynamicAllocation.schedulerBacklogTimeout must be > 0!") + if (schedulerBacklogTimeoutS <= 0) { + throw new SparkException("spark.dynamicAllocation.schedulerBacklogTimeoutS must be > 0!") } - if (sustainedSchedulerBacklogTimeout <= 0) { + if (sustainedSchedulerBacklogTimeoutS <= 0) { throw new SparkException( - "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout must be > 0!") + "spark.dynamicAllocation.sustainedSchedulerBacklogTimeoutS must be > 0!") } - if (executorIdleTimeout <= 0) { - throw new SparkException("spark.dynamicAllocation.executorIdleTimeout must be > 0!") + if (executorIdleTimeoutS <= 0) { + throw new SparkException("spark.dynamicAllocation.executorIdleTimeoutS must be > 0!") } // Require external shuffle service for dynamic allocation // Otherwise, we may lose shuffle files when killing executors @@ -264,8 +264,8 @@ private[spark] class ExecutorAllocationManager( } else if (addTime != NOT_SET && now >= addTime) { val delta = addExecutors(maxNeeded) logDebug(s"Starting timer to add more executors (to " + - s"expire in $sustainedSchedulerBacklogTimeout seconds)") - addTime += sustainedSchedulerBacklogTimeout * 1000 + s"expire in $sustainedSchedulerBacklogTimeoutS seconds)") + addTime += sustainedSchedulerBacklogTimeoutS * 1000 delta } else { 0 @@ -353,7 +353,7 @@ private[spark] class ExecutorAllocationManager( val removeRequestAcknowledged = testing || client.killExecutor(executorId) if (removeRequestAcknowledged) { logInfo(s"Removing executor $executorId because it has been idle for " + - s"$executorIdleTimeout seconds (new desired total will be ${numExistingExecutors - 1})") + s"$executorIdleTimeoutS seconds (new desired total will be ${numExistingExecutors - 1})") executorsPendingToRemove.add(executorId) true } else { @@ -409,8 +409,8 @@ private[spark] class ExecutorAllocationManager( private def onSchedulerBacklogged(): Unit = synchronized { if (addTime == NOT_SET) { logDebug(s"Starting timer to add executors because pending tasks " + - s"are building up (to expire in $schedulerBacklogTimeout seconds)") - addTime = clock.getTimeMillis + schedulerBacklogTimeout * 1000 + s"are building up (to expire in $schedulerBacklogTimeoutS seconds)") + addTime = clock.getTimeMillis + schedulerBacklogTimeoutS * 1000 } } @@ -433,8 +433,8 @@ private[spark] class ExecutorAllocationManager( if (executorIds.contains(executorId)) { if (!removeTimes.contains(executorId) && !executorsPendingToRemove.contains(executorId)) { logDebug(s"Starting idle timer for $executorId because there are no more tasks " + - s"scheduled to run on the executor (to expire in $executorIdleTimeout seconds)") - removeTimes(executorId) = clock.getTimeMillis + executorIdleTimeout * 1000 + s"scheduled to run on the executor (to expire in $executorIdleTimeoutS seconds)") + removeTimes(executorId) = clock.getTimeMillis + executorIdleTimeoutS * 1000 } } else { logWarning(s"Attempted to mark unknown executor $executorId idle") diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index f18fab5d0e808..630c8513617d6 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -683,9 +683,10 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { .set("spark.dynamicAllocation.enabled", "true") .set("spark.dynamicAllocation.minExecutors", minExecutors.toString) .set("spark.dynamicAllocation.maxExecutors", maxExecutors.toString) - .set("spark.dynamicAllocation.schedulerBacklogTimeout", schedulerBacklogTimeout.toString) + .set("spark.dynamicAllocation.schedulerBacklogTimeout", + s"${schedulerBacklogTimeout.toString}s") .set("spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", - sustainedSchedulerBacklogTimeout.toString) + s"${sustainedSchedulerBacklogTimeout.toString}s") .set("spark.dynamicAllocation.executorIdleTimeout", s"${executorIdleTimeout.toString}s") .set("spark.dynamicAllocation.testing", "true") new SparkContext(conf) diff --git a/docs/configuration.md b/docs/configuration.md index 1ab3c57bdfae1..e41c90a61258e 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1127,7 +1127,7 @@ Apart from these, the following properties are also available, and may be useful spark.dynamicAllocation.executorIdleTimeout 600s - If dynamic allocation is enabled and an executor has been idle for more than this duration, + If dynamic allocation is enabled and an executor has been idle for more than this duration, the executor will be removed. For more detail, see this description. @@ -1155,10 +1155,10 @@ Apart from these, the following properties are also available, and may be useful spark.dynamicAllocation.schedulerBacklogTimeout - 5 + 5s If dynamic allocation is enabled and there have been pending tasks backlogged for more than - this duration (in seconds), new executors will be requested. For more detail, see this + this duration, new executors will be requested. For more detail, see this description. diff --git a/docs/job-scheduling.md b/docs/job-scheduling.md index 963e88a3e1d8f..8d9c2ba2041b2 100644 --- a/docs/job-scheduling.md +++ b/docs/job-scheduling.md @@ -32,7 +32,7 @@ Resource allocation can be configured as follows, based on the cluster type: * **Standalone mode:** By default, applications submitted to the standalone mode cluster will run in FIFO (first-in-first-out) order, and each application will try to use all available nodes. You can limit the number of nodes an application uses by setting the `spark.cores.max` configuration property in it, - or change the default for applications that don't set this setting through `spark.deploy.defaultCores`. + or change the default for applications that don't set this setting through `spark.deploy.defaultCores`. Finally, in addition to controlling cores, each application's `spark.executor.memory` setting controls its memory use. * **Mesos:** To use static partitioning on Mesos, set the `spark.mesos.coarse` configuration property to `true`, From c6a00954f33994c35e9d76cb0222b35ce46bab69 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Fri, 27 Mar 2015 15:22:49 -0700 Subject: [PATCH 15/58] Updated spark.core.connection.auth.wait.timeout --- docs/configuration.md | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index e41c90a61258e..079eef0307d49 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1222,10 +1222,9 @@ Apart from these, the following properties are also available, and may be useful spark.core.connection.auth.wait.timeout - 30 + 30s - Number of seconds for the connection to wait for authentication to occur before timing - out and giving up. + How long for the connection for authentication to occur before timing out and giving up. From cde9bffef6ded083a9dfa80486687ddd07e85640 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Fri, 27 Mar 2015 15:25:44 -0700 Subject: [PATCH 16/58] Updated spark.streaming.blockInterval --- docs/configuration.md | 6 ++-- docs/streaming-programming-guide.md | 28 +++++++++---------- .../streaming/receiver/BlockGenerator.scala | 11 ++++---- .../spark/streaming/ReceiverSuite.scala | 4 +-- 4 files changed, 25 insertions(+), 24 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 079eef0307d49..dd5fab4c19558 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1344,10 +1344,10 @@ Apart from these, the following properties are also available, and may be useful Property NameDefaultMeaning spark.streaming.blockInterval - 200 + 200ms - Interval (milliseconds) at which data received by Spark Streaming receivers is chunked - into blocks of data before storing them in Spark. Minimum recommended - 50 ms. See the + Interval at which data received by Spark Streaming receivers is chunked + into blocks of data before storing them in Spark. Minimum recommended - 50ms. See the performance tuning section in the Spark Streaming programing guide for more details. diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 6d6229625f3f9..12b96af2a7e3f 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -1151,7 +1151,7 @@ joinedStream = stream1.join(stream2) {% endhighlight %} -Here, in each batch interval, the RDD generated by `stream1` will be joined with the RDD generated by `stream2`. You can also do `leftOuterJoin`, `rightOuterJoin`, `fullOuterJoin`. Furthermore, it is often very useful to do joins over windows of the streams. That is pretty easy as well. +Here, in each batch interval, the RDD generated by `stream1` will be joined with the RDD generated by `stream2`. You can also do `leftOuterJoin`, `rightOuterJoin`, `fullOuterJoin`. Furthermore, it is often very useful to do joins over windows of the streams. That is pretty easy as well.
@@ -1195,7 +1195,7 @@ JavaPairRDD dataset = ... JavaPairDStream windowedStream = stream.window(Durations.seconds(20)); JavaPairDStream joinedStream = windowedStream.transform( new Function>, JavaRDD>>() { - @Override + @Override public JavaRDD> call(JavaRDD> rdd) { return rdd.join(dataset); } @@ -1464,7 +1464,7 @@ words.foreachRDD { rdd => wordsDataFrame.registerTempTable("words") // Do word count on DataFrame using SQL and print it - val wordCountsDataFrame = + val wordCountsDataFrame = sqlContext.sql("select word, count(*) as total from words group by word") wordCountsDataFrame.show() } @@ -1506,7 +1506,7 @@ public class JavaRow implements java.io.Serializable { /** DataFrame operations inside your streaming program */ -JavaDStream words = ... +JavaDStream words = ... words.foreachRDD( new Function2, Time, Void>() { @@ -1979,11 +1979,11 @@ which is determined by the [configuration parameter](configuration.html#spark-st blocks of data before storing inside Spark's memory. The number of blocks in each batch determines the number of tasks that will be used to process those the received data in a map-like transformation. The number of tasks per receiver per batch will be -approximately (batch interval / block interval). For example, block interval of 200 ms will +approximately (batch interval / block interval). For example, block interval of 200ms will create 10 tasks per 2 second batches. Too low the number of tasks (that is, less than the number of cores per machine), then it will be inefficient as all available cores will not be used to process the data. To increase the number of tasks for a given batch interval, reduce the -block interval. However, the recommended minimum value of block interval is about 50 ms, +block interval. However, the recommended minimum value of block interval is about 50ms, below which the task launching overheads may be a problem. An alternative to receiving data with multiple input streams / receivers is to explicitly repartition @@ -2006,7 +2006,7 @@ documentation), or set the `spark.default.parallelism` {:.no_toc} The overheads of data serialization can be reduce by tuning the serialization formats. In case of streaming, there are two types of data that are being serialized. -* **Input data**: By default, the input data received through Receivers is stored in the executors' memory with [StorageLevel.MEMORY_AND_DISK_SER_2](api/scala/index.html#org.apache.spark.storage.StorageLevel$). That is, the data is serialized into bytes to reduce GC overheads, and replicated for tolerating executor failures. Also, the data is kept first in memory, and spilled over to disk only if the memory is unsufficient to hold all the input data necessary for the streaming computation. This serialization obviously has overheads -- the receiver must deserialize the received data and re-serialize it using Spark's serialization format. +* **Input data**: By default, the input data received through Receivers is stored in the executors' memory with [StorageLevel.MEMORY_AND_DISK_SER_2](api/scala/index.html#org.apache.spark.storage.StorageLevel$). That is, the data is serialized into bytes to reduce GC overheads, and replicated for tolerating executor failures. Also, the data is kept first in memory, and spilled over to disk only if the memory is unsufficient to hold all the input data necessary for the streaming computation. This serialization obviously has overheads -- the receiver must deserialize the received data and re-serialize it using Spark's serialization format. * **Persisted RDDs generated by Streaming Operations**: RDDs generated by streaming computations may be persisted in memory. For example, window operation persist data in memory as they would be processed multiple times. However, unlike Spark, by default RDDs are persisted with [StorageLevel.MEMORY_ONLY_SER](api/scala/index.html#org.apache.spark.storage.StorageLevel$) (i.e. serialized) to minimize GC overheads. @@ -2068,15 +2068,15 @@ in the [Tuning Guide](tuning.html#memory-tuning). It is strongly recommended tha The amount of cluster memory required by a Spark Streaming application depends heavily on the type of transformations used. For example, if you want to use a window operation on last 10 minutes of data, then your cluster should have sufficient memory to hold 10 minutes of worth of data in memory. Or if you want to use `updateStateByKey` with a large number of keys, then the necessary memory will be high. On the contrary, if you want to do a simple map-filter-store operation, then necessary memory will be low. -In general, since the data received through receivers are stored with StorageLevel.MEMORY_AND_DISK_SER_2, the data that does not fit in memory will spill over to the disk. This may reduce the performance of the streaming application, and hence it is advised to provide sufficient memory as required by your streaming application. Its best to try and see the memory usage on a small scale and estimate accordingly. +In general, since the data received through receivers are stored with StorageLevel.MEMORY_AND_DISK_SER_2, the data that does not fit in memory will spill over to the disk. This may reduce the performance of the streaming application, and hence it is advised to provide sufficient memory as required by your streaming application. Its best to try and see the memory usage on a small scale and estimate accordingly. -Another aspect of memory tuning is garbage collection. For a streaming application that require low latency, it is undesirable to have large pauses caused by JVM Garbage Collection. +Another aspect of memory tuning is garbage collection. For a streaming application that require low latency, it is undesirable to have large pauses caused by JVM Garbage Collection. There are a few parameters that can help you tune the memory usage and GC overheads. * **Persistence Level of DStreams**: As mentioned earlier in the [Data Serialization](#data-serialization) section, the input data and RDDs are by default persisted as serialized bytes. This reduces both, the memory usage and GC overheads, compared to deserialized persistence. Enabling Kryo serialization further reduces serialized sizes and memory usage. Further reduction in memory usage can be achieved with compression (see the Spark configuration `spark.rdd.compress`), at the cost of CPU time. -* **Clearing old data**: By default, all input data and persisted RDDs generated by DStream transformations are automatically cleared. Spark Streaming decides when to clear the data based on the transformations that are used. For example, if you are using window operation of 10 minutes, then Spark Streaming will keep around last 10 minutes of data, and actively throw away older data. +* **Clearing old data**: By default, all input data and persisted RDDs generated by DStream transformations are automatically cleared. Spark Streaming decides when to clear the data based on the transformations that are used. For example, if you are using window operation of 10 minutes, then Spark Streaming will keep around last 10 minutes of data, and actively throw away older data. Data can be retained for longer duration (e.g. interactively querying older data) by setting `streamingContext.remember`. * **CMS Garbage Collector**: Use of the concurrent mark-and-sweep GC is strongly recommended for keeping GC-related pauses consistently low. Even though concurrent GC is known to reduce the @@ -2093,7 +2093,7 @@ consistent batch processing times. Make sure you set the CMS GC on both the driv # Fault-tolerance Semantics In this section, we will discuss the behavior of Spark Streaming applications in the event -of failures. +of failures. ## Background {:.no_toc} @@ -2188,7 +2188,7 @@ lost. This will affect the results of the stateful transformations. To avoid this loss of past received data, Spark 1.2 introduced _write ahead logs_ which saves the received data to fault-tolerant storage. With the [write ahead logs -enabled](#deploying-applications) and reliable receivers, there is zero data loss. In terms of semantics, it provides at-least once guarantee. +enabled](#deploying-applications) and reliable receivers, there is zero data loss. In terms of semantics, it provides at-least once guarantee. The following table summarizes the semantics under failures: @@ -2238,7 +2238,7 @@ In Spark 1.3, we have introduced a new Kafka Direct API, which can ensure that a ## Semantics of output operations {:.no_toc} -Output operations (like `foreachRDD`) have _at-least once_ semantics, that is, +Output operations (like `foreachRDD`) have _at-least once_ semantics, that is, the transformed data may get written to an external entity more than once in the event of a worker failure. While this is acceptable for saving to file systems using the `saveAs***Files` operations (as the file will simply get overwritten with the same data), @@ -2249,7 +2249,7 @@ additional effort may be necessary to achieve exactly-once semantics. There are - *Transactional updates*: All updates are made transactionally so that updates are made exactly once atomically. One way to do this would be the following. - Use the batch time (available in `foreachRDD`) and the partition index of the transformed RDD to create an identifier. This identifier uniquely identifies a blob data in the streaming application. - - Update external system with this blob transactionally (that is, exactly once, atomically) using the identifier. That is, if the identifier is not already committed, commit the partition data and the identifier atomically. Else if this was already committed, skip the update. + - Update external system with this blob transactionally (that is, exactly once, atomically) using the identifier. That is, if the identifier is not already committed, commit the partition data and the identifier atomically. Else if this was already committed, skip the update. *************************************************************************************************** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala index 42514d8b47dcf..c993a7f4385f3 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala @@ -24,7 +24,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.{Logging, SparkConf} import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.util.RecurringTimer -import org.apache.spark.util.SystemClock +import org.apache.spark.util.{Utils, SystemClock} /** Listener object for BlockGenerator events */ private[streaming] trait BlockGeneratorListener { @@ -79,9 +79,10 @@ private[streaming] class BlockGenerator( private case class Block(id: StreamBlockId, buffer: ArrayBuffer[Any]) private val clock = new SystemClock() - private val blockInterval = conf.getLong("spark.streaming.blockInterval", 200) + private val blockIntervalMs = + Utils.timeStringToMs(conf.get("spark.streaming.blockIntervalMs", "200ms")) private val blockIntervalTimer = - new RecurringTimer(clock, blockInterval, updateCurrentBuffer, "BlockGenerator") + new RecurringTimer(clock, blockIntervalMs, updateCurrentBuffer, "BlockGenerator") private val blockQueueSize = conf.getInt("spark.streaming.blockQueueSize", 10) private val blocksForPushing = new ArrayBlockingQueue[Block](blockQueueSize) private val blockPushingThread = new Thread() { override def run() { keepPushingBlocks() } } @@ -132,7 +133,7 @@ private[streaming] class BlockGenerator( val newBlockBuffer = currentBuffer currentBuffer = new ArrayBuffer[Any] if (newBlockBuffer.size > 0) { - val blockId = StreamBlockId(receiverId, time - blockInterval) + val blockId = StreamBlockId(receiverId, time - blockIntervalMs) val newBlock = new Block(blockId, newBlockBuffer) listener.onGenerateBlock(blockId) blocksForPushing.put(newBlock) // put is blocking when queue is full @@ -177,7 +178,7 @@ private[streaming] class BlockGenerator( logError(message, t) listener.onError(message, t) } - + private def pushBlock(block: Block) { listener.onPushBlock(block.id, block.buffer) logInfo("Pushed block " + block.id) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala index aa20ad0b5374e..cbbfdf1f980e7 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala @@ -132,7 +132,7 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { test("block generator") { val blockGeneratorListener = new FakeBlockGeneratorListener val blockInterval = 200 - val conf = new SparkConf().set("spark.streaming.blockInterval", blockInterval.toString) + val conf = new SparkConf().set("spark.streaming.blockInterval", s"${blockInterval.toString}ms") val blockGenerator = new BlockGenerator(blockGeneratorListener, 1, conf) val expectedBlocks = 5 val waitTime = expectedBlocks * blockInterval + (blockInterval / 2) @@ -159,7 +159,7 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { val blockGeneratorListener = new FakeBlockGeneratorListener val blockInterval = 100 val maxRate = 100 - val conf = new SparkConf().set("spark.streaming.blockInterval", blockInterval.toString). + val conf = new SparkConf().set("spark.streaming.blockInterval", s"${blockInterval.toString}ms"). set("spark.streaming.receiver.maxRate", maxRate.toString) val blockGenerator = new BlockGenerator(blockGeneratorListener, 1, conf) val expectedBlocks = 20 From 42477aadc1f39911d325a0c703fae2ad55a6c42a Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Fri, 27 Mar 2015 15:28:32 -0700 Subject: [PATCH 17/58] Updated configuration doc with note on specifying time properties --- docs/configuration.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/configuration.md b/docs/configuration.md index dd5fab4c19558..391d178f794b5 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -38,6 +38,8 @@ val sc = new SparkContext(conf) Note that we can have more than 1 thread in local mode, and in cases like spark streaming, we may actually require one to prevent any sort of starvation issues. +Properties that specify some time duration must be configured with a unit, e.g. `5s`, `10ms`, or `20us`. + ## 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 From 9a29d8d3f502f43c237859e9f75f5d6c76662915 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Fri, 27 Mar 2015 16:16:58 -0700 Subject: [PATCH 18/58] Fixed misuse of time in streaming context test --- .../apache/spark/streaming/scheduler/JobGenerator.scala | 8 +++----- .../apache/spark/streaming/StreamingContextSuite.scala | 4 ++-- 2 files changed, 5 insertions(+), 7 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index 4946806d2ee95..4447dae6c68f3 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -24,7 +24,7 @@ import akka.actor.{ActorRef, Props, Actor} import org.apache.spark.{SparkEnv, Logging} import org.apache.spark.streaming.{Checkpoint, CheckpointWriter, Time} import org.apache.spark.streaming.util.RecurringTimer -import org.apache.spark.util.{Clock, ManualClock} +import org.apache.spark.util.{Utils, Clock, ManualClock} /** Event classes for JobGenerator */ private[scheduler] sealed trait JobGeneratorEvent @@ -104,10 +104,8 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { if (processReceivedData) { logInfo("Stopping JobGenerator gracefully") val timeWhenStopStarted = System.currentTimeMillis() - val stopTimeout = conf.getLong( - "spark.streaming.gracefulStopTimeout", - 10 * ssc.graph.batchDuration.milliseconds - ) + val stopTimeout = Utils.timeStringToMs(conf.get( + "spark.streaming.gracefulStopTimeout", s"${10 * ssc.graph.batchDuration.milliseconds}ms")) val pollTime = 100 // To prevent graceful stop to get stuck permanently diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index b105da6b1d156..09d9fc43a946c 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -177,7 +177,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("stop gracefully") { val conf = new SparkConf().setMaster(master).setAppName(appName) - conf.set("spark.cleaner.ttl", "3600") + conf.set("spark.cleaner.ttl", "3600ms") sc = new SparkContext(conf) for (i <- 1 to 4) { logInfo("==================================\n\n\n") @@ -208,7 +208,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("stop slow receiver gracefully") { val conf = new SparkConf().setMaster(master).setAppName(appName) - conf.set("spark.streaming.gracefulStopTimeout", "20000") + conf.set("spark.streaming.gracefulStopTimeout", "20000ms") sc = new SparkContext(conf) logInfo("==================================\n\n\n") ssc = new StreamingContext(sc, Milliseconds(100)) From 34f87c26dd936305fec3d00d6a8cc1e6bc48734c Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Fri, 27 Mar 2015 22:27:58 -0700 Subject: [PATCH 19/58] Update Utils.scala Fixed ordering to fix overlap of s with ms and us. --- core/src/main/scala/org/apache/spark/util/Utils.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 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 96efb3667598b..f94815b79cd45 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1015,13 +1015,13 @@ private[spark] object Utils extends Logging { */ def timeStringToUs(str: String): Long = { val lower = str.toLowerCase.trim() - if (lower.endsWith("s")) { - lower.substring(0, lower.length-1).toLong * 1000 * 1000 - } else if (lower.endsWith("ms")) { + if (lower.endsWith("ms")) { lower.substring(0, lower.length-2).toLong * 1000 } else if (lower.endsWith("us")) { lower.substring(0, lower.length-2).toLong - } else {// Invalid suffix, force correct formatting + } else if (lower.endsWith("s")) { + lower.substring(0, lower.length-1).toLong * 1000 * 1000 + } else { // Invalid suffix, force correct formatting throw new IllegalArgumentException("Time must be specified as seconds (s), " + "milliseconds (ms), or microseconds (us) e.g. 50s, 100ms, or 250us.") } From 8f741e19f66b011decb3cca285606a733a08990b Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Fri, 27 Mar 2015 22:28:50 -0700 Subject: [PATCH 20/58] Update JavaUtils.java Fixed overlap of (s) with (ms) and (us) in cascading if. --- .../main/java/org/apache/spark/network/util/JavaUtils.java | 6 +++--- 1 file changed, 3 insertions(+), 3 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 854b78ba8118c..7d8a2e48f2eec 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 @@ -127,12 +127,12 @@ private static boolean isSymlink(File file) throws IOException { */ public static long timeStringToUs(String str) throws IllegalArgumentException { String lower = str.toLowerCase().trim(); - if (lower.endsWith("s")) { - return Long.parseLong(lower.substring(0, lower.length()-1)) * 1000 * 1000; - } else if (lower.endsWith("ms")) { + if (lower.endsWith("ms")) { return Long.parseLong(lower.substring(0, lower.length()-2)) * 1000; } else if (lower.endsWith("us")) { return Long.parseLong(lower.substring(0, lower.length()-2)); + } else if (lower.endsWith("s")) { + return Long.parseLong(lower.substring(0, lower.length()-1)) * 1000 * 1000; } else {// Invalid suffix, force correct formatting throw new IllegalArgumentException("Time must be specified as seconds (s), " + "milliseconds (ms), or microseconds (us) e.g. 50s, 100ms, or 250us."); From 9e2547c3216c335f2b6d12d95c6a853ef8868c21 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Mon, 30 Mar 2015 08:58:01 -0700 Subject: [PATCH 21/58] Reverting doc changes --- docs/configuration.md | 117 ++++++++++++++-------------- docs/job-scheduling.md | 2 +- docs/streaming-programming-guide.md | 28 +++---- 3 files changed, 74 insertions(+), 73 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 391d178f794b5..7fe11475212b3 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -38,8 +38,6 @@ val sc = new SparkContext(conf) Note that we can have more than 1 thread in local mode, and in cases like spark streaming, we may actually require one to prevent any sort of starvation issues. -Properties that specify some time duration must be configured with a unit, e.g. `5s`, `10ms`, or `20us`. - ## 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 @@ -122,7 +120,7 @@ of the most common options to set are: Amount of memory to use for the driver process, i.e. where SparkContext is initialized. (e.g. 512m, 2g). - +
Note: In client mode, this config must not be set through the SparkConf directly in your application, because the driver JVM has already started at that point. Instead, please set this through the --driver-memory command line option @@ -190,7 +188,7 @@ Apart from these, the following properties are also available, and may be useful
Note: In client mode, this config must not be set through the SparkConf directly in your application, because the driver JVM has already started at that point. - Instead, please set this through the --driver-class-path command line option or in + Instead, please set this through the --driver-class-path command line option or in your default properties file. @@ -199,10 +197,10 @@ Apart from these, the following properties are also available, and may be useful (none) A string of extra JVM options to pass to the driver. For instance, GC settings or other logging. - +
Note: In client mode, this config must not be set through the SparkConf directly in your application, because the driver JVM has already started at that point. - Instead, please set this through the --driver-java-options command line option or in + Instead, please set this through the --driver-java-options command line option or in your default properties file. @@ -211,10 +209,10 @@ Apart from these, the following properties are also available, and may be useful (none) Set a special library path to use when launching the driver JVM. - +
Note: In client mode, this config must not be set through the SparkConf directly in your application, because the driver JVM has already started at that point. - Instead, please set this through the --driver-library-path command line option or in + Instead, please set this through the --driver-library-path command line option or in your default properties file. @@ -225,7 +223,7 @@ Apart from these, the following properties are also available, and may be useful (Experimental) Whether to give user-added jars precedence over Spark's own jars when loading classes in the the driver. This feature can be used to mitigate conflicts between Spark's dependencies and user dependencies. It is currently an experimental feature. - + This is used in cluster mode only. @@ -233,8 +231,8 @@ Apart from these, the following properties are also available, and may be useful spark.executor.extraClassPath (none) - Extra classpath entries to append to the classpath of executors. This exists primarily for - backwards-compatibility with older versions of Spark. Users typically should not need to set + Extra classpath entries to append to the classpath of executors. This exists primarily for + backwards-compatibility with older versions of Spark. Users typically should not need to set this option. @@ -242,9 +240,9 @@ Apart from these, the following properties are also available, and may be useful spark.executor.extraJavaOptions (none) - A string of extra JVM options to pass to executors. For instance, GC settings or other logging. - Note that it is illegal to set Spark properties or heap size settings with this option. Spark - properties should be set using a SparkConf object or the spark-defaults.conf file used with the + A string of extra JVM options to pass to executors. For instance, GC settings or other logging. + Note that it is illegal to set Spark properties or heap size settings with this option. Spark + properties should be set using a SparkConf object or the spark-defaults.conf file used with the spark-submit script. Heap size settings can be set with spark.executor.memory. @@ -431,10 +429,10 @@ Apart from these, the following properties are also available, and may be useful spark.shuffle.io.retryWait - 5s + 5 - (Netty only) How long to wait between retries of fetches. The maximum delay caused by retrying - is simply maxRetries * retryWait, by default 5 seconds. + (Netty only) Seconds to wait between retries of fetches. The maximum delay caused by retrying + is simply maxRetries * retryWait, by default 15 seconds. @@ -708,7 +706,7 @@ Apart from these, the following properties are also available, and may be useful spark.cleaner.ttl (infinite) - Duration of how long Spark will remember any metadata (stages generated, tasks + Duration (seconds) of how long Spark will remember any metadata (stages generated, tasks generated, etc.). Periodic cleanups will ensure that metadata older than this duration will be forgotten. This is useful for running Spark for many hours / days (for example, running 24/7 in case of Spark Streaming applications). Note that any RDD that persists in memory for more than @@ -734,17 +732,17 @@ Apart from these, the following properties are also available, and may be useful spark.executor.heartbeatInterval - 10s - Interval between each executor's heartbeats to the driver. Heartbeats let + 10000 + Interval (milliseconds) between each executor's heartbeats to the driver. Heartbeats let the driver know that the executor is still alive and update it with metrics for in-progress tasks. spark.files.fetchTimeout - 60s + 60 Communication timeout to use when fetching files added through SparkContext.addFile() from - the driver. + the driver, in seconds. @@ -855,16 +853,16 @@ Apart from these, the following properties are also available, and may be useful spark.akka.heartbeat.interval - 1000s - - This is set to a larger value to disable the transport failure detector that comes built in to - Akka. It can be enabled again, if you plan to use this feature (Not recommended). A larger - interval value in seconds reduces network overhead and a smaller value ( ~ 1s) might be more - informative for Akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses` - if you need to. A likely positive use case for using failure detector would be: a sensistive - failure detector can help evict rogue executors quickly. However this is usually not the case - as GC pauses and network lags are expected in a real Spark cluster. Apart from that enabling - this leads to a lot of exchanges of heart beats between nodes leading to flooding the network + 1000 + + This is set to a larger value to disable the transport failure detector that comes built in to + Akka. It can be enabled again, if you plan to use this feature (Not recommended). A larger + interval value in seconds reduces network overhead and a smaller value ( ~ 1 s) might be more + informative for Akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses` + if you need to. A likely positive use case for using failure detector would be: a sensistive + failure detector can help evict rogue executors quickly. However this is usually not the case + as GC pauses and network lags are expected in a real Spark cluster. Apart from that enabling + this leads to a lot of exchanges of heart beats between nodes leading to flooding the network with those. @@ -873,7 +871,7 @@ Apart from these, the following properties are also available, and may be useful 6000 This is set to a larger value to disable the transport failure detector that comes built in to Akka. - It can be enabled again, if you plan to use this feature (Not recommended). Acceptable heart + It can be enabled again, if you plan to use this feature (Not recommended). Acceptable heart beat pause in seconds for Akka. This can be used to control sensitivity to GC pauses. Tune this along with `spark.akka.heartbeat.interval` if you need to. @@ -888,9 +886,9 @@ Apart from these, the following properties are also available, and may be useful spark.akka.timeout - 100s + 100 - Communication timeout between Spark nodes. + Communication timeout between Spark nodes, in seconds. @@ -940,11 +938,11 @@ Apart from these, the following properties are also available, and may be useful spark.network.timeout - 120s + 120 - Default timeout for all network interactions. This config will be used in + Default timeout for all network interactions, in seconds. This config will be used in place of spark.core.connection.ack.wait.timeout, spark.akka.timeout, - spark.storage.blockManagerSlaveTimeout or + spark.storage.blockManagerSlaveTimeoutMs or spark.shuffle.io.connectionTimeout, if they are not configured. @@ -991,9 +989,9 @@ Apart from these, the following properties are also available, and may be useful spark.locality.wait - 3000ms + 3000 - How long to wait to launch a data-local task before giving up and launching it + Number of milliseconds to wait to launch a data-local task before giving up and launching it on a less-local node. The same wait will be used to step through multiple locality levels (process-local, node-local, rack-local and then any). It is also possible to customize the waiting time for each level by setting spark.locality.wait.node, etc. @@ -1026,9 +1024,10 @@ Apart from these, the following properties are also available, and may be useful spark.scheduler.maxRegisteredResourcesWaitingTime - 30000ms + 30000 - Maximum amount of time to wait for resources to register before scheduling begins. + Maximum amount of time to wait for resources to register before scheduling begins + (in milliseconds). @@ -1055,9 +1054,10 @@ Apart from these, the following properties are also available, and may be useful spark.scheduler.revive.interval - 1000ms + 1000 - The interval length for the scheduler to revive the worker resource offers to run tasks. + The interval length for the scheduler to revive the worker resource offers to run tasks + (in milliseconds). @@ -1070,9 +1070,9 @@ Apart from these, the following properties are also available, and may be useful spark.speculation.interval - 100ms + 100 - How often Spark will check for tasks to speculate. + How often Spark will check for tasks to speculate, in milliseconds. @@ -1127,10 +1127,10 @@ Apart from these, the following properties are also available, and may be useful spark.dynamicAllocation.executorIdleTimeout - 600s + 600 - If dynamic allocation is enabled and an executor has been idle for more than this duration, - the executor will be removed. For more detail, see this + If dynamic allocation is enabled and an executor has been idle for more than this duration + (in seconds), the executor will be removed. For more detail, see this description. @@ -1157,10 +1157,10 @@ Apart from these, the following properties are also available, and may be useful spark.dynamicAllocation.schedulerBacklogTimeout - 5s + 5 If dynamic allocation is enabled and there have been pending tasks backlogged for more than - this duration, new executors will be requested. For more detail, see this + this duration (in seconds), new executors will be requested. For more detail, see this description. @@ -1215,18 +1215,19 @@ Apart from these, the following properties are also available, and may be useful spark.core.connection.ack.wait.timeout - 60s + 60 - How long for the connection to wait for ack to occur before timing + Number of seconds for the connection to wait for ack to occur before timing out and giving up. To avoid unwilling timeout caused by long pause like GC, you can set larger value. spark.core.connection.auth.wait.timeout - 30s + 30 - How long for the connection for authentication to occur before timing out and giving up. + Number of seconds for the connection to wait for authentication to occur before timing + out and giving up. @@ -1346,10 +1347,10 @@ Apart from these, the following properties are also available, and may be useful Property NameDefaultMeaning spark.streaming.blockInterval - 200ms + 200 - Interval at which data received by Spark Streaming receivers is chunked - into blocks of data before storing them in Spark. Minimum recommended - 50ms. See the + Interval (milliseconds) at which data received by Spark Streaming receivers is chunked + into blocks of data before storing them in Spark. Minimum recommended - 50 ms. See the performance tuning section in the Spark Streaming programing guide for more details. diff --git a/docs/job-scheduling.md b/docs/job-scheduling.md index 8d9c2ba2041b2..963e88a3e1d8f 100644 --- a/docs/job-scheduling.md +++ b/docs/job-scheduling.md @@ -32,7 +32,7 @@ Resource allocation can be configured as follows, based on the cluster type: * **Standalone mode:** By default, applications submitted to the standalone mode cluster will run in FIFO (first-in-first-out) order, and each application will try to use all available nodes. You can limit the number of nodes an application uses by setting the `spark.cores.max` configuration property in it, - or change the default for applications that don't set this setting through `spark.deploy.defaultCores`. + or change the default for applications that don't set this setting through `spark.deploy.defaultCores`. Finally, in addition to controlling cores, each application's `spark.executor.memory` setting controls its memory use. * **Mesos:** To use static partitioning on Mesos, set the `spark.mesos.coarse` configuration property to `true`, diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 12b96af2a7e3f..6d6229625f3f9 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -1151,7 +1151,7 @@ joinedStream = stream1.join(stream2) {% endhighlight %}
-Here, in each batch interval, the RDD generated by `stream1` will be joined with the RDD generated by `stream2`. You can also do `leftOuterJoin`, `rightOuterJoin`, `fullOuterJoin`. Furthermore, it is often very useful to do joins over windows of the streams. That is pretty easy as well. +Here, in each batch interval, the RDD generated by `stream1` will be joined with the RDD generated by `stream2`. You can also do `leftOuterJoin`, `rightOuterJoin`, `fullOuterJoin`. Furthermore, it is often very useful to do joins over windows of the streams. That is pretty easy as well.
@@ -1195,7 +1195,7 @@ JavaPairRDD dataset = ... JavaPairDStream windowedStream = stream.window(Durations.seconds(20)); JavaPairDStream joinedStream = windowedStream.transform( new Function>, JavaRDD>>() { - @Override + @Override public JavaRDD> call(JavaRDD> rdd) { return rdd.join(dataset); } @@ -1464,7 +1464,7 @@ words.foreachRDD { rdd => wordsDataFrame.registerTempTable("words") // Do word count on DataFrame using SQL and print it - val wordCountsDataFrame = + val wordCountsDataFrame = sqlContext.sql("select word, count(*) as total from words group by word") wordCountsDataFrame.show() } @@ -1506,7 +1506,7 @@ public class JavaRow implements java.io.Serializable { /** DataFrame operations inside your streaming program */ -JavaDStream words = ... +JavaDStream words = ... words.foreachRDD( new Function2, Time, Void>() { @@ -1979,11 +1979,11 @@ which is determined by the [configuration parameter](configuration.html#spark-st blocks of data before storing inside Spark's memory. The number of blocks in each batch determines the number of tasks that will be used to process those the received data in a map-like transformation. The number of tasks per receiver per batch will be -approximately (batch interval / block interval). For example, block interval of 200ms will +approximately (batch interval / block interval). For example, block interval of 200 ms will create 10 tasks per 2 second batches. Too low the number of tasks (that is, less than the number of cores per machine), then it will be inefficient as all available cores will not be used to process the data. To increase the number of tasks for a given batch interval, reduce the -block interval. However, the recommended minimum value of block interval is about 50ms, +block interval. However, the recommended minimum value of block interval is about 50 ms, below which the task launching overheads may be a problem. An alternative to receiving data with multiple input streams / receivers is to explicitly repartition @@ -2006,7 +2006,7 @@ documentation), or set the `spark.default.parallelism` {:.no_toc} The overheads of data serialization can be reduce by tuning the serialization formats. In case of streaming, there are two types of data that are being serialized. -* **Input data**: By default, the input data received through Receivers is stored in the executors' memory with [StorageLevel.MEMORY_AND_DISK_SER_2](api/scala/index.html#org.apache.spark.storage.StorageLevel$). That is, the data is serialized into bytes to reduce GC overheads, and replicated for tolerating executor failures. Also, the data is kept first in memory, and spilled over to disk only if the memory is unsufficient to hold all the input data necessary for the streaming computation. This serialization obviously has overheads -- the receiver must deserialize the received data and re-serialize it using Spark's serialization format. +* **Input data**: By default, the input data received through Receivers is stored in the executors' memory with [StorageLevel.MEMORY_AND_DISK_SER_2](api/scala/index.html#org.apache.spark.storage.StorageLevel$). That is, the data is serialized into bytes to reduce GC overheads, and replicated for tolerating executor failures. Also, the data is kept first in memory, and spilled over to disk only if the memory is unsufficient to hold all the input data necessary for the streaming computation. This serialization obviously has overheads -- the receiver must deserialize the received data and re-serialize it using Spark's serialization format. * **Persisted RDDs generated by Streaming Operations**: RDDs generated by streaming computations may be persisted in memory. For example, window operation persist data in memory as they would be processed multiple times. However, unlike Spark, by default RDDs are persisted with [StorageLevel.MEMORY_ONLY_SER](api/scala/index.html#org.apache.spark.storage.StorageLevel$) (i.e. serialized) to minimize GC overheads. @@ -2068,15 +2068,15 @@ in the [Tuning Guide](tuning.html#memory-tuning). It is strongly recommended tha The amount of cluster memory required by a Spark Streaming application depends heavily on the type of transformations used. For example, if you want to use a window operation on last 10 minutes of data, then your cluster should have sufficient memory to hold 10 minutes of worth of data in memory. Or if you want to use `updateStateByKey` with a large number of keys, then the necessary memory will be high. On the contrary, if you want to do a simple map-filter-store operation, then necessary memory will be low. -In general, since the data received through receivers are stored with StorageLevel.MEMORY_AND_DISK_SER_2, the data that does not fit in memory will spill over to the disk. This may reduce the performance of the streaming application, and hence it is advised to provide sufficient memory as required by your streaming application. Its best to try and see the memory usage on a small scale and estimate accordingly. +In general, since the data received through receivers are stored with StorageLevel.MEMORY_AND_DISK_SER_2, the data that does not fit in memory will spill over to the disk. This may reduce the performance of the streaming application, and hence it is advised to provide sufficient memory as required by your streaming application. Its best to try and see the memory usage on a small scale and estimate accordingly. -Another aspect of memory tuning is garbage collection. For a streaming application that require low latency, it is undesirable to have large pauses caused by JVM Garbage Collection. +Another aspect of memory tuning is garbage collection. For a streaming application that require low latency, it is undesirable to have large pauses caused by JVM Garbage Collection. There are a few parameters that can help you tune the memory usage and GC overheads. * **Persistence Level of DStreams**: As mentioned earlier in the [Data Serialization](#data-serialization) section, the input data and RDDs are by default persisted as serialized bytes. This reduces both, the memory usage and GC overheads, compared to deserialized persistence. Enabling Kryo serialization further reduces serialized sizes and memory usage. Further reduction in memory usage can be achieved with compression (see the Spark configuration `spark.rdd.compress`), at the cost of CPU time. -* **Clearing old data**: By default, all input data and persisted RDDs generated by DStream transformations are automatically cleared. Spark Streaming decides when to clear the data based on the transformations that are used. For example, if you are using window operation of 10 minutes, then Spark Streaming will keep around last 10 minutes of data, and actively throw away older data. +* **Clearing old data**: By default, all input data and persisted RDDs generated by DStream transformations are automatically cleared. Spark Streaming decides when to clear the data based on the transformations that are used. For example, if you are using window operation of 10 minutes, then Spark Streaming will keep around last 10 minutes of data, and actively throw away older data. Data can be retained for longer duration (e.g. interactively querying older data) by setting `streamingContext.remember`. * **CMS Garbage Collector**: Use of the concurrent mark-and-sweep GC is strongly recommended for keeping GC-related pauses consistently low. Even though concurrent GC is known to reduce the @@ -2093,7 +2093,7 @@ consistent batch processing times. Make sure you set the CMS GC on both the driv # Fault-tolerance Semantics In this section, we will discuss the behavior of Spark Streaming applications in the event -of failures. +of failures. ## Background {:.no_toc} @@ -2188,7 +2188,7 @@ lost. This will affect the results of the stateful transformations. To avoid this loss of past received data, Spark 1.2 introduced _write ahead logs_ which saves the received data to fault-tolerant storage. With the [write ahead logs -enabled](#deploying-applications) and reliable receivers, there is zero data loss. In terms of semantics, it provides at-least once guarantee. +enabled](#deploying-applications) and reliable receivers, there is zero data loss. In terms of semantics, it provides at-least once guarantee. The following table summarizes the semantics under failures: @@ -2238,7 +2238,7 @@ In Spark 1.3, we have introduced a new Kafka Direct API, which can ensure that a ## Semantics of output operations {:.no_toc} -Output operations (like `foreachRDD`) have _at-least once_ semantics, that is, +Output operations (like `foreachRDD`) have _at-least once_ semantics, that is, the transformed data may get written to an external entity more than once in the event of a worker failure. While this is acceptable for saving to file systems using the `saveAs***Files` operations (as the file will simply get overwritten with the same data), @@ -2249,7 +2249,7 @@ additional effort may be necessary to achieve exactly-once semantics. There are - *Transactional updates*: All updates are made transactionally so that updates are made exactly once atomically. One way to do this would be the following. - Use the batch time (available in `foreachRDD`) and the partition index of the transformed RDD to create an identifier. This identifier uniquely identifies a blob data in the streaming application. - - Update external system with this blob transactionally (that is, exactly once, atomically) using the identifier. That is, if the identifier is not already committed, commit the partition data and the identifier atomically. Else if this was already committed, skip the update. + - Update external system with this blob transactionally (that is, exactly once, atomically) using the identifier. That is, if the identifier is not already committed, commit the partition data and the identifier atomically. Else if this was already committed, skip the update. *************************************************************************************************** From 5232a36f660022f87b3a8c68a8aa4b514d57fbb1 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Mon, 30 Mar 2015 09:30:34 -0700 Subject: [PATCH 22/58] [SPARK-5931] Changed default behavior of time string conversion. --- .../spark/ExecutorAllocationManager.scala | 6 +- .../org/apache/spark/HeartbeatReceiver.scala | 4 +- .../org/apache/spark/executor/Executor.scala | 2 +- .../spark/network/nio/ConnectionManager.scala | 2 +- .../spark/scheduler/TaskSchedulerImpl.scala | 4 +- .../spark/scheduler/TaskSetManager.scala | 6 +- .../CoarseGrainedSchedulerBackend.scala | 4 +- .../org/apache/spark/util/AkkaUtils.scala | 4 +- .../apache/spark/util/MetadataCleaner.scala | 2 +- .../scala/org/apache/spark/util/Utils.scala | 62 +++++++++++++------ .../spark/scheduler/TaskSetManagerSuite.scala | 2 +- .../streaming/receiver/BlockGenerator.scala | 2 +- .../streaming/scheduler/JobGenerator.scala | 2 +- .../streaming/StreamingContextSuite.scala | 10 +-- 14 files changed, 67 insertions(+), 45 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index c86c7c0a2040a..8d8de52e83e21 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -78,15 +78,15 @@ private[spark] class ExecutorAllocationManager( Integer.MAX_VALUE) // How long there must be backlogged tasks for before an addition is triggered (seconds) - private val schedulerBacklogTimeoutS = Utils.timeStringToS(conf.get( + private val schedulerBacklogTimeoutS = Utils.timeStringAsS(conf.get( "spark.dynamicAllocation.schedulerBacklogTimeoutS", "5s")) // Same as above, but used only after `schedulerBacklogTimeoutS` is exceeded - private val sustainedSchedulerBacklogTimeoutS = Utils.timeStringToS(conf.get( + private val sustainedSchedulerBacklogTimeoutS = Utils.timeStringAsS(conf.get( "spark.dynamicAllocation.sustainedSchedulerBacklogTimeoutS", s"${schedulerBacklogTimeoutS}s")) // How long an executor must be idle for before it is removed (seconds) - private val executorIdleTimeoutS = Utils.timeStringToS(conf.get( + private val executorIdleTimeoutS = Utils.timeStringAsS(conf.get( "spark.dynamicAllocation.executorIdleTimeoutS", "600s")) // During testing, the methods to actually kill and add executors are mocked out diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index 6a8b96a184e2a..4ff5ef8692329 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -50,12 +50,12 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, scheduler: TaskSchedule // executor ID -> timestamp of when the last heartbeat from this executor was received private val executorLastSeen = new mutable.HashMap[String, Long] - private val executorTimeoutMs = Utils.timeStringToMs(sc.conf.get("spark.network.timeout", + private val executorTimeoutMs = Utils.timeStringAsMs(sc.conf.get("spark.network.timeout", sc.conf.get("spark.storage.blockManagerSlaveTimeout", "120s"))) // "spark.network.timeoutInterval" uses "seconds", while // "spark.storage.blockManagerTimeoutIntervalMs" uses "milliseconds" - private val checkTimeoutIntervalMs = Utils.timeStringToMs( + private val checkTimeoutIntervalMs = Utils.timeStringAsMs( sc.conf.get("spark.network.timeoutInterval", sc.conf.get("spark.storage.blockManagerTimeoutIntervalMs", "60s"))) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index ded7f7ae3b018..730abc66501e8 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -442,7 +442,7 @@ private[spark] class Executor( * This thread stops running when the executor is stopped. */ private def startDriverHeartbeater(): Unit = { - val intervalMs = Utils.timeStringToMs(conf.get("spark.executor.heartbeatInterval", "10s")) + val intervalMs = Utils.timeStringAsMs(conf.get("spark.executor.heartbeatInterval", "10s")) val thread = new Thread() { override def run() { // Sleep a random intervalMs so the heartbeats don't end up in sync diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala index f8346b2f404e6..ce217bdc546ba 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala @@ -82,7 +82,7 @@ private[nio] class ConnectionManager( new HashedWheelTimer(Utils.namedThreadFactory("AckTimeoutMonitor")) private val ackTimeout = - Utils.timeStringToS(conf.get("spark.core.connection.ack.wait.timeout", + Utils.timeStringAsS(conf.get("spark.core.connection.ack.wait.timeout", conf.get("spark.network.timeout", "120s"))) // Get the thread counts from the Spark Configuration. diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index d931c20fe27e5..ca76f79d18432 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -63,10 +63,10 @@ private[spark] class TaskSchedulerImpl( // How often to check for speculative tasks val SPECULATION_INTERVAL_MS = - Utils.timeStringToMs(conf.get("spark.speculation.interval", "100ms")) + Utils.timeStringAsMs(conf.get("spark.speculation.interval", "100ms")) // Threshold above which we warn user initial TaskSet may be starved - val STARVATION_TIMEOUT_MS = Utils.timeStringToMs(conf.get("spark.starvation.timeout", "15000ms")) + val STARVATION_TIMEOUT_MS = Utils.timeStringAsMs(conf.get("spark.starvation.timeout", "15000ms")) // CPUs to request per task val CPUS_PER_TASK = conf.getInt("spark.task.cpus", 1) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 72bf0e4bc10ac..ad4f38ac06934 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -851,11 +851,11 @@ private[spark] class TaskSetManager( val defaultWait = conf.get("spark.locality.wait", "3000ms") level match { case TaskLocality.PROCESS_LOCAL => - Utils.timeStringToMs(conf.get("spark.locality.wait.process", defaultWait)) + Utils.timeStringAsMs(conf.get("spark.locality.wait.process", defaultWait)) case TaskLocality.NODE_LOCAL => - Utils.timeStringToMs(conf.get("spark.locality.wait.node", defaultWait)) + Utils.timeStringAsMs(conf.get("spark.locality.wait.node", defaultWait)) case TaskLocality.RACK_LOCAL => - Utils.timeStringToMs(conf.get("spark.locality.wait.rack", defaultWait)) + Utils.timeStringAsMs(conf.get("spark.locality.wait.rack", defaultWait)) case _ => 0L } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 0cb9b9cfdca20..5ac4282d14bfb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -58,7 +58,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste // Submit tasks after maxRegisteredWaitingTime milliseconds // if minRegisteredRatio has not yet been reached val maxRegisteredWaitingTimeMs = - Utils.timeStringToMs(conf.get("spark.scheduler.maxRegisteredResourcesWaitingTime", "30000ms")) + Utils.timeStringAsMs(conf.get("spark.scheduler.maxRegisteredResourcesWaitingTime", "30000ms")) val createTime = System.currentTimeMillis() private val executorDataMap = new HashMap[String, ExecutorData] @@ -80,7 +80,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) // Periodically revive offers to allow delay scheduling to work - val reviveIntervalMs = Utils.timeStringToMs( + val reviveIntervalMs = Utils.timeStringAsMs( conf.get("spark.scheduler.revive.interval", "1000ms")) import context.dispatcher context.system.scheduler.schedule(0.millis, reviveIntervalMs.millis, self, ReviveOffers) diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 2e15866d7906c..26adb24556051 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -66,7 +66,7 @@ private[spark] object AkkaUtils extends Logging { val akkaThreads = conf.getInt("spark.akka.threads", 4) val akkaBatchSize = conf.getInt("spark.akka.batchSize", 15) - val akkaTimeoutS = Utils.timeStringToS(conf.get("spark.akka.timeout", + val akkaTimeoutS = Utils.timeStringAsS(conf.get("spark.akka.timeout", conf.get("spark.network.timeout", "120s"))) val akkaFrameSize = maxFrameSizeBytes(conf) val akkaLogLifecycleEvents = conf.getBoolean("spark.akka.logLifecycleEvents", false) @@ -81,7 +81,7 @@ private[spark] object AkkaUtils extends Logging { val akkaHeartBeatPauses = conf.getInt("spark.akka.heartbeat.pauses", 6000) val akkaHeartBeatIntervalS = - Utils.timeStringToS(conf.get("spark.akka.heartbeat.interval", "1000s")) + Utils.timeStringAsS(conf.get("spark.akka.heartbeat.interval", "1000s")) val secretKey = securityManager.getSecretKey() val isAuthOn = securityManager.isAuthenticationEnabled() diff --git a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala index 3e2ee097980b5..f0a9245a2cbfc 100644 --- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala @@ -76,7 +76,7 @@ private[spark] object MetadataCleanerType extends Enumeration { // initialization of StreamingContext. It's okay for users trying to configure stuff themselves. private[spark] object MetadataCleaner { def getDelaySeconds(conf: SparkConf): Int = { - Utils.timeStringToS(conf.get("spark.cleaner.ttl", "-1s")).toInt + Utils.timeStringAsS(conf.get("spark.cleaner.ttl", "-1s")).toInt } def getDelaySeconds( 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 f94815b79cd45..d543934e0d3ba 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -611,7 +611,7 @@ private[spark] object Utils extends Logging { } Utils.setupSecureURLConnection(uc, securityMgr) - val timeoutMs = Utils.timeStringToMs(conf.get("spark.files.fetchTimeout", "60s")).toInt + val timeoutMs = Utils.timeStringAsMs(conf.get("spark.files.fetchTimeout", "60s")).toInt uc.setConnectTimeout(timeoutMs) uc.setReadTimeout(timeoutMs) uc.connect() @@ -1010,36 +1010,58 @@ private[spark] object Utils extends Logging { ) } + /** Check whether a time-suffix was provided for the time string. */ + private def hasTimeSuffix(str: String) : Boolean = { + val lower = str.toLowerCase.trim() + lower.endsWith("ms") || lower.endsWith("us") || lower.endsWith("s") + } + + val timeError = "Time must be specified as seconds (s), " + + "milliseconds (ms), or microseconds (us) e.g. 50s, 100ms, or 250us." + /** - * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use + * Convert a passed time string (e.g. 50s, 100ms, or 250us) to a microsecond count for + * internal use. If no suffix is provided a direct conversion is attempted. */ - def timeStringToUs(str: String): Long = { - val lower = str.toLowerCase.trim() - if (lower.endsWith("ms")) { - lower.substring(0, lower.length-2).toLong * 1000 - } else if (lower.endsWith("us")) { - lower.substring(0, lower.length-2).toLong - } else if (lower.endsWith("s")) { - lower.substring(0, lower.length-1).toLong * 1000 * 1000 - } else { // Invalid suffix, force correct formatting - throw new IllegalArgumentException("Time must be specified as seconds (s), " + - "milliseconds (ms), or microseconds (us) e.g. 50s, 100ms, or 250us.") + private def timeStringToUs(str: String) : Long = { + try { + val lower = str.toLowerCase.trim() + if (lower.endsWith("ms")) { + lower.substring(0, lower.length - 2).toLong * 1000 + } else if (lower.endsWith("us")) { + lower.substring(0, lower.length - 2).toLong + } else if (lower.endsWith("s")) { + lower.substring(0, lower.length - 1).toLong * 1000 * 1000 + } else { + // Invalid suffix, force correct formatting + lower.toLong + } + } catch { + case e: NumberFormatException => throw new NumberFormatException(timeError) } } /** - * Convert a time parameter such as (50s, 100ms, or 250us) to milliseconds for internal use. - * Note: may round in some cases + * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If + * no suffix is provided, the passed number is assumed to be in us. + */ + def timeStringAsUs(str: String): Long = { + timeStringToUs(str) + } + + /** + * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If + * no suffix is provided, the passed number is assumed to be in ms. */ - def timeStringToMs(str : String) : Long = { - timeStringToUs(str)/1000 + def timeStringAsMs(str : String) : Long = { + timeStringToUs(str)/1000 } /** - * Convert a time parameter such as (50s, 100ms, or 250us) to seconds for internal use. - * Note: may round in some cases + * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If + * no suffix is provided, the passed number is assumed to be in seconds. */ - def timeStringToS(str : String) : Long = { + def timeStringAsS(str : String) : Long = { timeStringToUs(str)/1000/1000 } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index b8d372770b959..0786d5289144c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -152,7 +152,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { private val conf = new SparkConf - val LOCALITY_WAIT = Utils.timeStringToMs(conf.get("spark.locality.wait", "3000ms")) + val LOCALITY_WAIT = Utils.timeStringAsMs(conf.get("spark.locality.wait", "3000ms")) val MAX_TASK_FAILURES = 4 override def beforeEach() { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala index c993a7f4385f3..a4754ccb5d47c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala @@ -80,7 +80,7 @@ private[streaming] class BlockGenerator( private val clock = new SystemClock() private val blockIntervalMs = - Utils.timeStringToMs(conf.get("spark.streaming.blockIntervalMs", "200ms")) + Utils.timeStringAsMs(conf.get("spark.streaming.blockIntervalMs", "200ms")) private val blockIntervalTimer = new RecurringTimer(clock, blockIntervalMs, updateCurrentBuffer, "BlockGenerator") private val blockQueueSize = conf.getInt("spark.streaming.blockQueueSize", 10) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index 4447dae6c68f3..facb42c712591 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -104,7 +104,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { if (processReceivedData) { logInfo("Stopping JobGenerator gracefully") val timeWhenStopStarted = System.currentTimeMillis() - val stopTimeout = Utils.timeStringToMs(conf.get( + val stopTimeout = Utils.timeStringAsMs(conf.get( "spark.streaming.gracefulStopTimeout", s"${10 * ssc.graph.batchDuration.milliseconds}ms")) val pollTime = 100 diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index 09d9fc43a946c..560067064f8a1 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -75,7 +75,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) myConf.set("spark.cleaner.ttl", "10s") ssc = new StreamingContext(myConf, batchDuration) - assert(Utils.timeStringToS(ssc.conf.get("spark.cleaner.ttl", "-1s")) === 10) + assert(Utils.timeStringAsS(ssc.conf.get("spark.cleaner.ttl", "-1s")) === 10) } test("from existing SparkContext") { @@ -87,7 +87,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) myConf.set("spark.cleaner.ttl", "10s") ssc = new StreamingContext(myConf, batchDuration) - assert(Utils.timeStringToS(ssc.conf.get("spark.cleaner.ttl", "-1s")) === 10) + assert(Utils.timeStringAsS(ssc.conf.get("spark.cleaner.ttl", "-1s")) === 10) } test("from checkpoint") { @@ -98,12 +98,12 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w ssc1.start() val cp = new Checkpoint(ssc1, Time(1000)) assert( - Utils.timeStringToS(cp.sparkConfPairs.toMap.getOrElse("spark.cleaner.ttl", "-1s")) === 10) + Utils.timeStringAsS(cp.sparkConfPairs.toMap.getOrElse("spark.cleaner.ttl", "-1s")) === 10) ssc1.stop() val newCp = Utils.deserialize[Checkpoint](Utils.serialize(cp)) - assert(Utils.timeStringToS(newCp.createSparkConf().get("spark.cleaner.ttl", "-1s")) === 10) + assert(Utils.timeStringAsS(newCp.createSparkConf().get("spark.cleaner.ttl", "-1s")) === 10) ssc = new StreamingContext(null, newCp, null) - assert(Utils.timeStringToS(ssc.conf.get("spark.cleaner.ttl", "-1s")) === 10) + assert(Utils.timeStringAsS(ssc.conf.get("spark.cleaner.ttl", "-1s")) === 10) } test("start and stop state check") { From 3a12dd8c85d936fb74298f92f8a55951998ce450 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Mon, 30 Mar 2015 09:38:25 -0700 Subject: [PATCH 23/58] Updated host revceiver --- .../spark/ExecutorAllocationManager.scala | 20 +++++++++---------- .../org/apache/spark/HeartbeatReceiver.scala | 6 ++---- 2 files changed, 12 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 8d8de52e83e21..a88e1cdd2ab57 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -20,7 +20,7 @@ package org.apache.spark import scala.collection.mutable import org.apache.spark.scheduler._ -import org.apache.spark.util.{Utils, SystemClock, Clock} +import org.apache.spark.util.{Clock, SystemClock, Utils} /** * An agent that dynamically allocates and removes executors based on the workload. @@ -52,14 +52,14 @@ import org.apache.spark.util.{Utils, SystemClock, Clock} * spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors * spark.dynamicAllocation.initialExecutors - Number of executors to start with * - * spark.dynamicAllocation.schedulerBacklogTimeoutS (M) - + * spark.dynamicAllocation.schedulerBacklogTimeout (M) - * If there are backlogged tasks for this duration, add new executors * - * spark.dynamicAllocation.sustainedSchedulerBacklogTimeoutS (N) - + * spark.dynamicAllocation.sustainedSchedulerBacklogTimeout (N) - * If the backlog is sustained for this duration, add more executors * This is used only after the initial backlog timeout is exceeded * - * spark.dynamicAllocation.executorIdleTimeoutS (K) - + * spark.dynamicAllocation.executorIdleTimeout (K) - * If an executor has been idle for this duration, remove it */ private[spark] class ExecutorAllocationManager( @@ -79,15 +79,15 @@ private[spark] class ExecutorAllocationManager( // How long there must be backlogged tasks for before an addition is triggered (seconds) private val schedulerBacklogTimeoutS = Utils.timeStringAsS(conf.get( - "spark.dynamicAllocation.schedulerBacklogTimeoutS", "5s")) + "spark.dynamicAllocation.schedulerBacklogTimeout", "5s")) // Same as above, but used only after `schedulerBacklogTimeoutS` is exceeded private val sustainedSchedulerBacklogTimeoutS = Utils.timeStringAsS(conf.get( - "spark.dynamicAllocation.sustainedSchedulerBacklogTimeoutS", s"${schedulerBacklogTimeoutS}s")) + "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", s"${schedulerBacklogTimeoutS}s")) // How long an executor must be idle for before it is removed (seconds) private val executorIdleTimeoutS = Utils.timeStringAsS(conf.get( - "spark.dynamicAllocation.executorIdleTimeoutS", "600s")) + "spark.dynamicAllocation.executorIdleTimeout", "600s")) // During testing, the methods to actually kill and add executors are mocked out private val testing = conf.getBoolean("spark.dynamicAllocation.testing", false) @@ -145,14 +145,14 @@ private[spark] class ExecutorAllocationManager( s"be less than or equal to spark.dynamicAllocation.maxExecutors ($maxNumExecutors)!") } if (schedulerBacklogTimeoutS <= 0) { - throw new SparkException("spark.dynamicAllocation.schedulerBacklogTimeoutS must be > 0!") + throw new SparkException("spark.dynamicAllocation.schedulerBacklogTimeout must be > 0!") } if (sustainedSchedulerBacklogTimeoutS <= 0) { throw new SparkException( - "spark.dynamicAllocation.sustainedSchedulerBacklogTimeoutS must be > 0!") + "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout must be > 0!") } if (executorIdleTimeoutS <= 0) { - throw new SparkException("spark.dynamicAllocation.executorIdleTimeoutS must be > 0!") + throw new SparkException("spark.dynamicAllocation.executorIdleTimeout must be > 0!") } // Require external shuffle service for dynamic allocation // Otherwise, we may lose shuffle files when killing executors diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index 4ff5ef8692329..4252d9e6697cd 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -25,7 +25,7 @@ import akka.actor.{Actor, Cancellable} import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId import org.apache.spark.scheduler.{SlaveLost, TaskScheduler} -import org.apache.spark.util.{Utils, ActorLogReceive} +import org.apache.spark.util.{ActorLogReceive, Utils} /** * A heartbeat from executors to the driver. This is a shared message used by several internal @@ -38,7 +38,6 @@ private[spark] case class Heartbeat( blockManagerId: BlockManagerId) private[spark] case object ExpireDeadHosts - private[spark] case class HeartbeatResponse(reregisterBlockManager: Boolean) /** @@ -58,9 +57,8 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, scheduler: TaskSchedule private val checkTimeoutIntervalMs = Utils.timeStringAsMs( sc.conf.get("spark.network.timeoutInterval", sc.conf.get("spark.storage.blockManagerTimeoutIntervalMs", "60s"))) - + private var timeoutCheckingTask: Cancellable = null - override def preStart(): Unit = { import context.dispatcher timeoutCheckingTask = context.system.scheduler.schedule(0.seconds, From 68f4e937a17d0f70440a2fb6c415f73de215033e Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Mon, 30 Mar 2015 10:03:32 -0700 Subject: [PATCH 24/58] Updated more files to clean up usage of default time strings --- .../org/apache/spark/HeartbeatReceiver.scala | 13 ++++++----- .../spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../spark/scheduler/TaskSetManager.scala | 2 +- .../CoarseGrainedSchedulerBackend.scala | 2 +- .../scala/org/apache/spark/util/Utils.scala | 2 +- .../spark/scheduler/TaskSetManagerSuite.scala | 22 +++++++++---------- docs/configuration.md | 7 ++++-- 7 files changed, 28 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index 4252d9e6697cd..c189c4991d7e7 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -49,15 +49,18 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, scheduler: TaskSchedule // executor ID -> timestamp of when the last heartbeat from this executor was received private val executorLastSeen = new mutable.HashMap[String, Long] - private val executorTimeoutMs = Utils.timeStringAsMs(sc.conf.get("spark.network.timeout", - sc.conf.get("spark.storage.blockManagerSlaveTimeout", "120s"))) + // "spark.network.timeout" uses "seconds", while `spark.storage.blockManagerSlaveTimeoutMs` uses + // "milliseconds" + private val networkTimeoutS = sc.conf.get("spark.network.timeout","120s") + private val executorTimeoutMs = Utils.timeStringAsMs( + sc.conf.get("spark.storage.blockManagerSlaveTimeout", networkTimeoutS)) // "spark.network.timeoutInterval" uses "seconds", while // "spark.storage.blockManagerTimeoutIntervalMs" uses "milliseconds" + private val networkTimeoutIntervalS = sc.conf.get("spark.network.timeoutInterval","60s") private val checkTimeoutIntervalMs = Utils.timeStringAsMs( - sc.conf.get("spark.network.timeoutInterval", - sc.conf.get("spark.storage.blockManagerTimeoutIntervalMs", "60s"))) - + sc.conf.get("spark.storage.blockManagerTimeoutIntervalMs", networkTimeoutIntervalS)) + private var timeoutCheckingTask: Cancellable = null override def preStart(): Unit = { import context.dispatcher diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index ca76f79d18432..2768ce20c4073 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -66,7 +66,7 @@ private[spark] class TaskSchedulerImpl( Utils.timeStringAsMs(conf.get("spark.speculation.interval", "100ms")) // Threshold above which we warn user initial TaskSet may be starved - val STARVATION_TIMEOUT_MS = Utils.timeStringAsMs(conf.get("spark.starvation.timeout", "15000ms")) + val STARVATION_TIMEOUT_MS = Utils.timeStringAsMs(conf.get("spark.starvation.timeout", "15s")) // CPUs to request per task val CPUS_PER_TASK = conf.getInt("spark.task.cpus", 1) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index ad4f38ac06934..870130e7d9c93 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -848,7 +848,7 @@ private[spark] class TaskSetManager( } private def getLocalityWait(level: TaskLocality.TaskLocality): Long = { - val defaultWait = conf.get("spark.locality.wait", "3000ms") + val defaultWait = conf.get("spark.locality.wait", "3s") level match { case TaskLocality.PROCESS_LOCAL => Utils.timeStringAsMs(conf.get("spark.locality.wait.process", defaultWait)) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 5ac4282d14bfb..d5e07619b8a43 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -58,7 +58,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste // Submit tasks after maxRegisteredWaitingTime milliseconds // if minRegisteredRatio has not yet been reached val maxRegisteredWaitingTimeMs = - Utils.timeStringAsMs(conf.get("spark.scheduler.maxRegisteredResourcesWaitingTime", "30000ms")) + Utils.timeStringAsMs(conf.get("spark.scheduler.maxRegisteredResourcesWaitingTime", "30s")) val createTime = System.currentTimeMillis() private val executorDataMap = new HashMap[String, ExecutorData] 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 d543934e0d3ba..784068b991ea9 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -611,7 +611,7 @@ private[spark] object Utils extends Logging { } Utils.setupSecureURLConnection(uc, securityMgr) - val timeoutMs = Utils.timeStringAsMs(conf.get("spark.files.fetchTimeout", "60s")).toInt + val timeoutMs = Utils.timeStringAsS(conf.get("spark.files.fetchTimeout","60s")).toInt*1000 uc.setConnectTimeout(timeoutMs) uc.setReadTimeout(timeoutMs) uc.connect() diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 0786d5289144c..44bb6ecbaff57 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -152,7 +152,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { private val conf = new SparkConf - val LOCALITY_WAIT = Utils.timeStringAsMs(conf.get("spark.locality.wait", "3000ms")) + val LOCALITY_WAIT_MS = Utils.timeStringAsMs(conf.get("spark.locality.wait", "3000ms")) val MAX_TASK_FAILURES = 4 override def beforeEach() { @@ -240,7 +240,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) == None) - clock.advance(LOCALITY_WAIT) + clock.advance(LOCALITY_WAIT_MS) // Offer host1, exec1 again, at NODE_LOCAL level: the node local (task 2) should // get chosen before the noPref task assert(manager.resourceOffer("exec1", "host1", NODE_LOCAL).get.index == 2) @@ -251,7 +251,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // Offer host2, exec3 again, at NODE_LOCAL level: we should get noPref task // after failing to find a node_Local task assert(manager.resourceOffer("exec2", "host2", NODE_LOCAL) == None) - clock.advance(LOCALITY_WAIT) + clock.advance(LOCALITY_WAIT_MS) assert(manager.resourceOffer("exec2", "host2", NO_PREF).get.index == 3) } @@ -292,7 +292,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // Offer host1 again: nothing should get chosen assert(manager.resourceOffer("exec1", "host1", ANY) === None) - clock.advance(LOCALITY_WAIT) + clock.advance(LOCALITY_WAIT_MS) // Offer host1 again: second task (on host2) should get chosen assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 1) @@ -306,7 +306,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // Now that we've launched a local task, we should no longer launch the task for host3 assert(manager.resourceOffer("exec2", "host2", ANY) === None) - clock.advance(LOCALITY_WAIT) + clock.advance(LOCALITY_WAIT_MS) // After another delay, we can go ahead and launch that task non-locally assert(manager.resourceOffer("exec2", "host2", ANY).get.index === 3) @@ -338,7 +338,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // nothing should be chosen assert(manager.resourceOffer("exec1", "host1", ANY) === None) - clock.advance(LOCALITY_WAIT * 2) + clock.advance(LOCALITY_WAIT_MS * 2) // task 1 and 2 would be scheduled as nonLocal task assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 1) @@ -527,7 +527,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY))) // Set allowed locality to ANY - clock.advance(LOCALITY_WAIT * 3) + clock.advance(LOCALITY_WAIT_MS * 3) // Offer host3 // No task is scheduled if we restrict locality to RACK_LOCAL assert(manager.resourceOffer("execC", "host3", RACK_LOCAL) === None) @@ -619,12 +619,12 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(manager.resourceOffer("execA", "host1", NO_PREF).get.index == 1) manager.speculatableTasks += 1 - clock.advance(LOCALITY_WAIT) + clock.advance(LOCALITY_WAIT_MS) // schedule the nonPref task assert(manager.resourceOffer("execA", "host1", NO_PREF).get.index === 2) // schedule the speculative task assert(manager.resourceOffer("execB", "host2", NO_PREF).get.index === 1) - clock.advance(LOCALITY_WAIT * 3) + clock.advance(LOCALITY_WAIT_MS * 3) // schedule non-local tasks assert(manager.resourceOffer("execB", "host2", ANY).get.index === 3) } @@ -710,13 +710,13 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // Valid locality should contain PROCESS_LOCAL, NODE_LOCAL and ANY assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, ANY))) assert(manager.resourceOffer("execA", "host1", ANY) !== None) - clock.advance(LOCALITY_WAIT * 4) + clock.advance(LOCALITY_WAIT_MS * 4) assert(manager.resourceOffer("execB.2", "host2", ANY) !== None) sched.removeExecutor("execA") sched.removeExecutor("execB.2") manager.executorLost("execA", "host1") manager.executorLost("execB.2", "host2") - clock.advance(LOCALITY_WAIT * 4) + clock.advance(LOCALITY_WAIT_MS * 4) sched.addExecutor("execC", "host3") manager.executorAdded() // Prior to the fix, this line resulted in an ArrayIndexOutOfBoundsException: diff --git a/docs/configuration.md b/docs/configuration.md index 7fe11475212b3..036a648f25938 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -35,8 +35,11 @@ val conf = new SparkConf() val sc = new SparkContext(conf) {% endhighlight %} -Note that we can have more than 1 thread in local mode, and in cases like spark streaming, we may actually -require one to prevent any sort of starvation issues. +Note that we can have more than 1 thread in local mode, and in cases like spark streaming, we may +actually require one to prevent any sort of starvation issues. + +Properties that specify some time duration shoud be configured with a unit, e.g. `5s`, `10ms`, or +`20us`. If no units are provided, the default unit specified in the documentation is assumed. ## Dynamically Loading Spark Properties In some cases, you may want to avoid hard-coding certain configurations in a `SparkConf`. For From 70ac2138d8c1c23bc3bbd4fa91607686b2c329cc Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Mon, 30 Mar 2015 10:59:44 -0700 Subject: [PATCH 25/58] Fixed remaining usages to be consistent. Updated Java-side time conversion --- .../org/apache/spark/HeartbeatReceiver.scala | 9 ++-- .../scala/org/apache/spark/util/Utils.scala | 4 ++ .../apache/spark/network/util/JavaUtils.java | 46 ++++++++++++------- .../spark/network/util/TransportConf.java | 13 +++--- .../streaming/receiver/BlockGenerator.scala | 4 +- .../streaming/scheduler/JobGenerator.scala | 2 +- 6 files changed, 49 insertions(+), 29 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index c189c4991d7e7..64aaefa0a8eb5 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -51,15 +51,16 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, scheduler: TaskSchedule // "spark.network.timeout" uses "seconds", while `spark.storage.blockManagerSlaveTimeoutMs` uses // "milliseconds" - private val networkTimeoutS = sc.conf.get("spark.network.timeout","120s") + private val networkTimeoutS = Utils.timeStringAsS(sc.conf.get("spark.network.timeout","120s")) private val executorTimeoutMs = Utils.timeStringAsMs( - sc.conf.get("spark.storage.blockManagerSlaveTimeout", networkTimeoutS)) + sc.conf.get("spark.storage.blockManagerSlaveTimeout", s"${networkTimeoutS}s")) // "spark.network.timeoutInterval" uses "seconds", while // "spark.storage.blockManagerTimeoutIntervalMs" uses "milliseconds" - private val networkTimeoutIntervalS = sc.conf.get("spark.network.timeoutInterval","60s") + private val networkTimeoutIntervalS = + Utils.timeStringAsS(sc.conf.get("spark.network.timeoutInterval","60s")) private val checkTimeoutIntervalMs = Utils.timeStringAsMs( - sc.conf.get("spark.storage.blockManagerTimeoutIntervalMs", networkTimeoutIntervalS)) + sc.conf.get("spark.storage.blockManagerTimeoutIntervalMs", s"${networkTimeoutIntervalS}s")) private var timeoutCheckingTask: Cancellable = null override def preStart(): Unit = { 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 784068b991ea9..17a624628d4bd 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1023,6 +1023,7 @@ private[spark] object Utils extends Logging { * Convert a passed time string (e.g. 50s, 100ms, or 250us) to a microsecond count for * internal use. If no suffix is provided a direct conversion is attempted. */ + @throws(classOf[NumberFormatException]) private def timeStringToUs(str: String) : Long = { try { val lower = str.toLowerCase.trim() @@ -1045,6 +1046,7 @@ private[spark] object Utils extends Logging { * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If * no suffix is provided, the passed number is assumed to be in us. */ + @throws(classOf[NumberFormatException]) def timeStringAsUs(str: String): Long = { timeStringToUs(str) } @@ -1053,6 +1055,7 @@ private[spark] object Utils extends Logging { * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If * no suffix is provided, the passed number is assumed to be in ms. */ + @throws(classOf[NumberFormatException]) def timeStringAsMs(str : String) : Long = { timeStringToUs(str)/1000 } @@ -1061,6 +1064,7 @@ private[spark] object Utils extends Logging { * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If * no suffix is provided, the passed number is assumed to be in seconds. */ + @throws(classOf[NumberFormatException]) def timeStringAsS(str : String) : Long = { timeStringToUs(str)/1000/1000 } 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 7d8a2e48f2eec..0591297bcf537 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 @@ -123,35 +123,49 @@ private static boolean isSymlink(File file) throws IOException { } /** - * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use + * Convert a passed time string (e.g. 50s, 100ms, or 250us) to a microsecond count for + * internal use. If no suffix is provided a direct conversion is attempted. */ - public static long timeStringToUs(String str) throws IllegalArgumentException { + private static long timeStringToUs(String str) throws NumberFormatException { String lower = str.toLowerCase().trim(); - if (lower.endsWith("ms")) { - return Long.parseLong(lower.substring(0, lower.length()-2)) * 1000; - } else if (lower.endsWith("us")) { - return Long.parseLong(lower.substring(0, lower.length()-2)); - } else if (lower.endsWith("s")) { - return Long.parseLong(lower.substring(0, lower.length()-1)) * 1000 * 1000; - } else {// Invalid suffix, force correct formatting - throw new IllegalArgumentException("Time must be specified as seconds (s), " + + try { + if (lower.endsWith("ms")) { + return Long.parseLong(lower.substring(0, lower.length() - 2)) * 1000; + } else if (lower.endsWith("us")) { + return Long.parseLong(lower.substring(0, lower.length() - 2)); + } else if (lower.endsWith("s")) { + return Long.parseLong(lower.substring(0, lower.length() - 1)) * 1000 * 1000; + } else {// Invalid suffix, force correct formatting + return Long.parseLong(lower); + } + } catch(NumberFormatException e) { + throw new NumberFormatException("Time must be specified as seconds (s), " + "milliseconds (ms), or microseconds (us) e.g. 50s, 100ms, or 250us."); } + + } + + /** + * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If + * no suffix is provided, the passed number is assumed to be in us. + */ + public static long timeStringAsUs(String str) throws NumberFormatException { + return timeStringToUs(str); } /** - * Convert a time parameter such as (50s, 100ms, or 250us) to milliseconds for internal use. - * Note: may round in some cases + * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If + * no suffix is provided, the passed number is assumed to be in ms. */ - public static long timeStringToMs(String str) throws IllegalArgumentException { + public static long timeStringAsMs(String str) throws NumberFormatException { return timeStringToUs(str)/1000; } /** - * Convert a time parameter such as (50s, 100ms, or 250us) to seconds for internal use. - * Note: may round in some cases + * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If + * no suffix is provided, the passed number is assumed to be in seconds. */ - public static long timeStringToS(String str) throws IllegalArgumentException { + public static long timeStringAsS(String str) throws NumberFormatException { return timeStringToUs(str)/1000/1000; } diff --git a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java index 99e3c8458e8c0..9b1c6e8aeb4c0 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -37,10 +37,11 @@ public boolean preferDirectBufs() { /** Connect timeout in milliseconds. Default 120 secs. */ public int connectionTimeoutMs() { - long defaultTimeout = JavaUtils.timeStringToMs( - conf.get("spark.shuffle.io.connectionTimeout", - conf.get("spark.network.timeout", "120s"))); - return (int) defaultTimeout; + long defaultNetworkTimeoutS = JavaUtils.timeStringAsS( + conf.get("spark.network.timeout","120s")); + long defaultTimeoutS = JavaUtils.timeStringAsS( + conf.get("spark.shuffle.io.connectionTimeout", defaultNetworkTimeoutS + "s")); + return (int) defaultTimeoutS * 1000; } /** Number of concurrent connections between two nodes for fetching data. */ @@ -71,7 +72,7 @@ public int numConnectionsPerPeer() { /** Timeout for a single round trip of SASL token exchange, in milliseconds. */ public int saslRTTimeoutMs() { - return (int) JavaUtils.timeStringToMs(conf.get("spark.shuffle.sasl.timeout", "30s")); + return (int) JavaUtils.timeStringAsS(conf.get("spark.shuffle.sasl.timeout", "30s")) * 1000; } /** @@ -85,7 +86,7 @@ public int saslRTTimeoutMs() { * Only relevant if maxIORetries > 0. */ public int ioRetryWaitTimeMs() { - return (int) JavaUtils.timeStringToMs(conf.get("spark.shuffle.io.retryWait", "5s")); + return (int) JavaUtils.timeStringAsS(conf.get("spark.shuffle.io.retryWait", "5s")) * 1000; } /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala index a4754ccb5d47c..1b9531900be3b 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala @@ -24,7 +24,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.{Logging, SparkConf} import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.util.RecurringTimer -import org.apache.spark.util.{Utils, SystemClock} +import org.apache.spark.util.{SystemClock, Utils} /** Listener object for BlockGenerator events */ private[streaming] trait BlockGeneratorListener { @@ -80,7 +80,7 @@ private[streaming] class BlockGenerator( private val clock = new SystemClock() private val blockIntervalMs = - Utils.timeStringAsMs(conf.get("spark.streaming.blockIntervalMs", "200ms")) + Utils.timeStringAsMs(conf.get("spark.streaming.blockInterval", "200ms")) private val blockIntervalTimer = new RecurringTimer(clock, blockIntervalMs, updateCurrentBuffer, "BlockGenerator") private val blockQueueSize = conf.getInt("spark.streaming.blockQueueSize", 10) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index facb42c712591..e74c34bd293dd 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -24,7 +24,7 @@ import akka.actor.{ActorRef, Props, Actor} import org.apache.spark.{SparkEnv, Logging} import org.apache.spark.streaming.{Checkpoint, CheckpointWriter, Time} import org.apache.spark.streaming.util.RecurringTimer -import org.apache.spark.util.{Utils, Clock, ManualClock} +import org.apache.spark.util.{Clock, ManualClock, Utils} /** Event classes for JobGenerator */ private[scheduler] sealed trait JobGeneratorEvent From 647b5ac0ba7dc846abcf46245d0d1a9f512decc0 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Tue, 31 Mar 2015 00:38:46 -0700 Subject: [PATCH 26/58] Udpated time conversion to use map iterator instead of if fall through --- .../scala/org/apache/spark/util/Utils.scala | 57 ++++++++++--------- 1 file changed, 29 insertions(+), 28 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 17a624628d4bd..ec92b5b80cc55 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -22,7 +22,7 @@ import java.lang.management.ManagementFactory import java.net._ import java.nio.ByteBuffer import java.util.{Properties, Locale, Random, UUID} -import java.util.concurrent.{ThreadFactory, ConcurrentHashMap, Executors, ThreadPoolExecutor} +import java.util.concurrent._ import javax.net.ssl.HttpsURLConnection import scala.collection.JavaConversions._ @@ -1010,35 +1010,36 @@ private[spark] object Utils extends Logging { ) } - /** Check whether a time-suffix was provided for the time string. */ - private def hasTimeSuffix(str: String) : Boolean = { - val lower = str.toLowerCase.trim() - lower.endsWith("ms") || lower.endsWith("us") || lower.endsWith("s") - } - - val timeError = "Time must be specified as seconds (s), " + - "milliseconds (ms), or microseconds (us) e.g. 50s, 100ms, or 250us." - + val timeSuffixes = Map ( + "us" -> TimeUnit.MICROSECONDS, + "ms" -> TimeUnit.MILLISECONDS, + "s" -> TimeUnit.SECONDS, + "min" -> TimeUnit.MINUTES, + "h" -> TimeUnit.HOURS, + "d" -> TimeUnit.DAYS + ) /** * Convert a passed time string (e.g. 50s, 100ms, or 250us) to a microsecond count for * internal use. If no suffix is provided a direct conversion is attempted. */ @throws(classOf[NumberFormatException]) - private def timeStringToUs(str: String) : Long = { + private def parseTimeString(str: String) : (Option[TimeUnit], Long) = { + val timeError = "Time must be specified as seconds (s), " + + "milliseconds (ms), or microseconds (us) e.g. 50s, 100ms, or 250us." + try { val lower = str.toLowerCase.trim() - if (lower.endsWith("ms")) { - lower.substring(0, lower.length - 2).toLong * 1000 - } else if (lower.endsWith("us")) { - lower.substring(0, lower.length - 2).toLong - } else if (lower.endsWith("s")) { - lower.substring(0, lower.length - 1).toLong * 1000 * 1000 - } else { - // Invalid suffix, force correct formatting - lower.toLong - } + var suffix: String = "" + timeSuffixes.foreach(s => { + if(lower.endsWith(s._1)) + suffix = s._1 + }) + + (timeSuffixes.get(suffix), str.substring(0, str.length - suffix.length).toLong) } catch { - case e: NumberFormatException => throw new NumberFormatException(timeError) + + case e: NumberFormatException => throw new NumberFormatException(timeError + "\n" + + e.toString) } } @@ -1046,27 +1047,27 @@ private[spark] object Utils extends Logging { * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If * no suffix is provided, the passed number is assumed to be in us. */ - @throws(classOf[NumberFormatException]) def timeStringAsUs(str: String): Long = { - timeStringToUs(str) + val parsed = parseTimeString(str) + parsed._1.getOrElse(TimeUnit.MICROSECONDS).toMicros(parsed._2) } /** * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If * no suffix is provided, the passed number is assumed to be in ms. */ - @throws(classOf[NumberFormatException]) def timeStringAsMs(str : String) : Long = { - timeStringToUs(str)/1000 + val parsed = parseTimeString(str) + parsed._1.getOrElse(TimeUnit.MILLISECONDS).toMicros(parsed._2) } /** * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If * no suffix is provided, the passed number is assumed to be in seconds. */ - @throws(classOf[NumberFormatException]) def timeStringAsS(str : String) : Long = { - timeStringToUs(str)/1000/1000 + val parsed = parseTimeString(str) + parsed._1.getOrElse(TimeUnit.SECONDS).toMicros(parsed._2) } /** From 1c0c07c5f59cf9e163bde6d04a8c6dedb10517bf Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Tue, 31 Mar 2015 00:50:59 -0700 Subject: [PATCH 27/58] Updated Java code to add day, minutes, and hours --- core/src/main/scala/org/apache/spark/util/Utils.scala | 7 ++++--- .../java/org/apache/spark/network/util/JavaUtils.java | 11 +++++++++-- 2 files changed, 13 insertions(+), 5 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 ec92b5b80cc55..8a71ef2085c99 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1022,17 +1022,18 @@ private[spark] object Utils extends Logging { * Convert a passed time string (e.g. 50s, 100ms, or 250us) to a microsecond count for * internal use. If no suffix is provided a direct conversion is attempted. */ - @throws(classOf[NumberFormatException]) private def parseTimeString(str: String) : (Option[TimeUnit], Long) = { val timeError = "Time must be specified as seconds (s), " + - "milliseconds (ms), or microseconds (us) e.g. 50s, 100ms, or 250us." + "milliseconds (ms), microseconds (us), minutes (min) hour (h), or day(d). " + + "E.g. 50s, 100ms, or 250us." try { val lower = str.toLowerCase.trim() var suffix: String = "" timeSuffixes.foreach(s => { - if(lower.endsWith(s._1)) + if(lower.endsWith(s._1)) { suffix = s._1 + } }) (timeSuffixes.get(suffix), str.substring(0, str.length - suffix.length).toLong) 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 0591297bcf537..7954c35a8e672 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 @@ -135,12 +135,19 @@ private static long timeStringToUs(String str) throws NumberFormatException { return Long.parseLong(lower.substring(0, lower.length() - 2)); } else if (lower.endsWith("s")) { return Long.parseLong(lower.substring(0, lower.length() - 1)) * 1000 * 1000; - } else {// Invalid suffix, force correct formatting + } else if (lower.endsWith("min")) { + return Long.parseLong(lower.substring(0, lower.length() - 1)) * 1000 * 1000 * 60; + } else if (lower.endsWith("h")) { + return Long.parseLong(lower.substring(0, lower.length() - 1)) * 1000 * 1000 * 60 * 60; + } else if (lower.endsWith("d")) { + return Long.parseLong(lower.substring(0, lower.length() - 1)) * 1000 * 1000 * 60 * 60 * 24; + } else {// No suffix, default selected by calling function return Long.parseLong(lower); } } catch(NumberFormatException e) { throw new NumberFormatException("Time must be specified as seconds (s), " + - "milliseconds (ms), or microseconds (us) e.g. 50s, 100ms, or 250us."); + "milliseconds (ms), microseconds (us), minutes (min) hour (h), or day(d). " + + "E.g. 50s, 100ms, or 250us.\n" + e.toString()); } } From 8613631f2f2eb6249de4b44c0489caa8ecdf7940 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Tue, 31 Mar 2015 08:37:54 -0700 Subject: [PATCH 28/58] Whitespace --- .../main/scala/org/apache/spark/HeartbeatReceiver.scala | 9 +++++---- .../org/apache/spark/network/nio/ConnectionManager.scala | 6 +++--- .../apache/spark/streaming/receiver/BlockGenerator.scala | 2 +- 3 files changed, 9 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index 64aaefa0a8eb5..b20089801ebbf 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -29,7 +29,7 @@ import org.apache.spark.util.{ActorLogReceive, Utils} /** * A heartbeat from executors to the driver. This is a shared message used by several internal - * components to convey liveness or execution information for in-progress tasks. It will also + * components to convey liveness or execution information for in-progress tasks. It will also * expire the hosts that have not heartbeated for more than spark.network.timeout. */ private[spark] case class Heartbeat( @@ -38,6 +38,7 @@ private[spark] case class Heartbeat( blockManagerId: BlockManagerId) private[spark] case object ExpireDeadHosts + private[spark] case class HeartbeatResponse(reregisterBlockManager: Boolean) /** @@ -61,7 +62,7 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, scheduler: TaskSchedule Utils.timeStringAsS(sc.conf.get("spark.network.timeoutInterval","60s")) private val checkTimeoutIntervalMs = Utils.timeStringAsMs( sc.conf.get("spark.storage.blockManagerTimeoutIntervalMs", s"${networkTimeoutIntervalS}s")) - + private var timeoutCheckingTask: Cancellable = null override def preStart(): Unit = { import context.dispatcher @@ -69,7 +70,7 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, scheduler: TaskSchedule checkTimeoutIntervalMs.milliseconds, self, ExpireDeadHosts) super.preStart() } - + override def receiveWithLogging: PartialFunction[Any, Unit] = { case Heartbeat(executorId, taskMetrics, blockManagerId) => val unknownExecutor = !scheduler.executorHeartbeatReceived( @@ -97,7 +98,7 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, scheduler: TaskSchedule } } } - + override def postStop(): Unit = { if (timeoutCheckingTask != null) { timeoutCheckingTask.cancel() diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala index ce217bdc546ba..35255c9235d36 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala @@ -86,11 +86,11 @@ private[nio] class ConnectionManager( conf.get("spark.network.timeout", "120s"))) // Get the thread counts from the Spark Configuration. - // + // // Even though the ThreadPoolExecutor constructor takes both a minimum and maximum value, // we only query for the minimum value because we are using LinkedBlockingDeque. - // - // The JavaDoc for ThreadPoolExecutor points out that when using a LinkedBlockingDeque (which is + // + // The JavaDoc for ThreadPoolExecutor points out that when using a LinkedBlockingDeque (which is // an unbounded queue) no more than corePoolSize threads will ever be created, so only the "min" // parameter is necessary. private val handlerThreadCount = conf.getInt("spark.core.connection.handler.threads.min", 20) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala index 1b9531900be3b..70ceae8398ef7 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala @@ -178,7 +178,7 @@ private[streaming] class BlockGenerator( logError(message, t) listener.onError(message, t) } - + private def pushBlock(block: Block) { listener.onPushBlock(block.id, block.buffer) logInfo("Pushed block " + block.id) From bac9edfe3feb285dc33b2212534841cfe859515f Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Tue, 31 Mar 2015 08:39:41 -0700 Subject: [PATCH 29/58] More whitespace --- core/src/main/scala/org/apache/spark/util/Utils.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 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 8a71ef2085c99..391684b545841 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1207,7 +1207,7 @@ private[spark] object Utils extends Logging { /** * Execute a block of code that evaluates to Unit, forwarding any uncaught exceptions to the * default UncaughtExceptionHandler - * + * * NOTE: This method is to be called by the spark-started JVM process. */ def tryOrExit(block: => Unit) { @@ -1220,11 +1220,11 @@ private[spark] object Utils extends Logging { } /** - * Execute a block of code that evaluates to Unit, stop SparkContext is there is any uncaught + * Execute a block of code that evaluates to Unit, stop SparkContext is there is any uncaught * exception - * - * NOTE: This method is to be called by the driver-side components to avoid stopping the - * user-started JVM process completely; in contrast, tryOrExit is to be called in the + * + * NOTE: This method is to be called by the driver-side components to avoid stopping the + * user-started JVM process completely; in contrast, tryOrExit is to be called in the * spark-started JVM process . */ def tryOrStopSparkContext(sc: SparkContext)(block: => Unit) { From 1858197cbb7f0fb75337d25092a525a2aa70f785 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Tue, 31 Mar 2015 09:29:07 -0700 Subject: [PATCH 30/58] Fixed bug where all time was being converted to us instead of the appropriate units --- .../main/scala/org/apache/spark/HeartbeatReceiver.scala | 8 ++++---- core/src/main/scala/org/apache/spark/util/Utils.scala | 6 +++--- .../java/org/apache/spark/network/util/JavaUtils.java | 2 +- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index b20089801ebbf..b704111f0434c 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -37,8 +37,8 @@ private[spark] case class Heartbeat( taskMetrics: Array[(Long, TaskMetrics)], // taskId -> TaskMetrics blockManagerId: BlockManagerId) -private[spark] case object ExpireDeadHosts - +private[spark] case object ExpireDeadHosts + private[spark] case class HeartbeatResponse(reregisterBlockManager: Boolean) /** @@ -54,14 +54,14 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, scheduler: TaskSchedule // "milliseconds" private val networkTimeoutS = Utils.timeStringAsS(sc.conf.get("spark.network.timeout","120s")) private val executorTimeoutMs = Utils.timeStringAsMs( - sc.conf.get("spark.storage.blockManagerSlaveTimeout", s"${networkTimeoutS}s")) + sc.conf.get("spark.storage.blockManagerSlaveTimeoutMs", s"${networkTimeoutS}us")) // "spark.network.timeoutInterval" uses "seconds", while // "spark.storage.blockManagerTimeoutIntervalMs" uses "milliseconds" private val networkTimeoutIntervalS = Utils.timeStringAsS(sc.conf.get("spark.network.timeoutInterval","60s")) private val checkTimeoutIntervalMs = Utils.timeStringAsMs( - sc.conf.get("spark.storage.blockManagerTimeoutIntervalMs", s"${networkTimeoutIntervalS}s")) + sc.conf.get("spark.storage.blockManagerTimeoutIntervalMs", s"${networkTimeoutIntervalS}us")) private var timeoutCheckingTask: Cancellable = null override def preStart(): Unit = { 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 391684b545841..f6fff4a458cd5 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1022,7 +1022,7 @@ private[spark] object Utils extends Logging { * Convert a passed time string (e.g. 50s, 100ms, or 250us) to a microsecond count for * internal use. If no suffix is provided a direct conversion is attempted. */ - private def parseTimeString(str: String) : (Option[TimeUnit], Long) = { + def parseTimeString(str: String) : (Option[TimeUnit], Long) = { val timeError = "Time must be specified as seconds (s), " + "milliseconds (ms), microseconds (us), minutes (min) hour (h), or day(d). " + "E.g. 50s, 100ms, or 250us." @@ -1059,7 +1059,7 @@ private[spark] object Utils extends Logging { */ def timeStringAsMs(str : String) : Long = { val parsed = parseTimeString(str) - parsed._1.getOrElse(TimeUnit.MILLISECONDS).toMicros(parsed._2) + parsed._1.getOrElse(TimeUnit.MILLISECONDS).toMillis(parsed._2) } /** @@ -1068,7 +1068,7 @@ private[spark] object Utils extends Logging { */ def timeStringAsS(str : String) : Long = { val parsed = parseTimeString(str) - parsed._1.getOrElse(TimeUnit.SECONDS).toMicros(parsed._2) + parsed._1.getOrElse(TimeUnit.SECONDS).toSeconds(parsed._2) } /** 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 7954c35a8e672..7d6418c14bc6d 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 @@ -136,7 +136,7 @@ private static long timeStringToUs(String str) throws NumberFormatException { } else if (lower.endsWith("s")) { return Long.parseLong(lower.substring(0, lower.length() - 1)) * 1000 * 1000; } else if (lower.endsWith("min")) { - return Long.parseLong(lower.substring(0, lower.length() - 1)) * 1000 * 1000 * 60; + return Long.parseLong(lower.substring(0, lower.length() - 3)) * 1000 * 1000 * 60; } else if (lower.endsWith("h")) { return Long.parseLong(lower.substring(0, lower.length() - 1)) * 1000 * 1000 * 60 * 60; } else if (lower.endsWith("d")) { From 3b126e1cd0da475f1c4c284e656d2125356244a5 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Tue, 31 Mar 2015 11:26:01 -0700 Subject: [PATCH 31/58] Fixed conversion to US from seconds --- core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index b704111f0434c..e88791208c7cd 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -54,14 +54,14 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, scheduler: TaskSchedule // "milliseconds" private val networkTimeoutS = Utils.timeStringAsS(sc.conf.get("spark.network.timeout","120s")) private val executorTimeoutMs = Utils.timeStringAsMs( - sc.conf.get("spark.storage.blockManagerSlaveTimeoutMs", s"${networkTimeoutS}us")) + sc.conf.get("spark.storage.blockManagerSlaveTimeoutMs", s"${networkTimeoutS}s")) // "spark.network.timeoutInterval" uses "seconds", while // "spark.storage.blockManagerTimeoutIntervalMs" uses "milliseconds" private val networkTimeoutIntervalS = Utils.timeStringAsS(sc.conf.get("spark.network.timeoutInterval","60s")) private val checkTimeoutIntervalMs = Utils.timeStringAsMs( - sc.conf.get("spark.storage.blockManagerTimeoutIntervalMs", s"${networkTimeoutIntervalS}us")) + sc.conf.get("spark.storage.blockManagerTimeoutIntervalMs", s"${networkTimeoutIntervalS}s")) private var timeoutCheckingTask: Cancellable = null override def preStart(): Unit = { From 39164f984800e7474215b1803b594c12def13ae6 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Tue, 31 Mar 2015 14:08:19 -0700 Subject: [PATCH 32/58] [SPARK-5931] Updated Java conversion to be similar to scala conversion. Updated conversions to clean up code a little using TimeUnit.convert. Added Unit tests --- .../org/apache/spark/HeartbeatReceiver.scala | 10 ++-- .../apache/spark/util/MetadataCleaner.scala | 2 +- .../scala/org/apache/spark/util/Utils.scala | 22 ++++--- .../spark/scheduler/TaskSetManagerSuite.scala | 5 +- .../org/apache/spark/util/UtilsSuite.scala | 27 ++++++++- .../apache/spark/network/util/JavaUtils.java | 57 +++++++++++-------- .../spark/network/util/TransportConf.java | 2 +- .../streaming/StreamingContextSuite.scala | 10 ++-- 8 files changed, 82 insertions(+), 53 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index e88791208c7cd..e46da05f8a0b9 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -52,18 +52,18 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, scheduler: TaskSchedule // "spark.network.timeout" uses "seconds", while `spark.storage.blockManagerSlaveTimeoutMs` uses // "milliseconds" - private val networkTimeoutS = Utils.timeStringAsS(sc.conf.get("spark.network.timeout","120s")) private val executorTimeoutMs = Utils.timeStringAsMs( - sc.conf.get("spark.storage.blockManagerSlaveTimeoutMs", s"${networkTimeoutS}s")) + sc.conf.get("spark.storage.blockManagerSlaveTimeoutMs", + s"${Utils.timeStringAsS(sc.conf.get("spark.network.timeout", "120s"))}s")) // "spark.network.timeoutInterval" uses "seconds", while // "spark.storage.blockManagerTimeoutIntervalMs" uses "milliseconds" - private val networkTimeoutIntervalS = - Utils.timeStringAsS(sc.conf.get("spark.network.timeoutInterval","60s")) private val checkTimeoutIntervalMs = Utils.timeStringAsMs( - sc.conf.get("spark.storage.blockManagerTimeoutIntervalMs", s"${networkTimeoutIntervalS}s")) + sc.conf.get("spark.storage.blockManagerTimeoutIntervalMs", + s"${Utils.timeStringAsS(sc.conf.get("spark.network.timeoutInterval", "60s"))}s")) private var timeoutCheckingTask: Cancellable = null + override def preStart(): Unit = { import context.dispatcher timeoutCheckingTask = context.system.scheduler.schedule(0.seconds, diff --git a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala index f0a9245a2cbfc..4ca71946b66b2 100644 --- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala @@ -76,7 +76,7 @@ private[spark] object MetadataCleanerType extends Enumeration { // initialization of StreamingContext. It's okay for users trying to configure stuff themselves. private[spark] object MetadataCleaner { def getDelaySeconds(conf: SparkConf): Int = { - Utils.timeStringAsS(conf.get("spark.cleaner.ttl", "-1s")).toInt + Utils.timeStringAsS(conf.get("spark.cleaner.ttl", "-1")).toInt } def getDelaySeconds( 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 f6fff4a458cd5..76eb63a1257bc 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -611,7 +611,7 @@ private[spark] object Utils extends Logging { } Utils.setupSecureURLConnection(uc, securityMgr) - val timeoutMs = Utils.timeStringAsS(conf.get("spark.files.fetchTimeout","60s")).toInt*1000 + val timeoutMs = Utils.timeStringAsMs(conf.get("spark.files.fetchTimeout", "60s")).toInt uc.setConnectTimeout(timeoutMs) uc.setReadTimeout(timeoutMs) uc.connect() @@ -1010,7 +1010,7 @@ private[spark] object Utils extends Logging { ) } - val timeSuffixes = Map ( + private val timeSuffixes = Map( "us" -> TimeUnit.MICROSECONDS, "ms" -> TimeUnit.MILLISECONDS, "s" -> TimeUnit.SECONDS, @@ -1018,11 +1018,12 @@ private[spark] object Utils extends Logging { "h" -> TimeUnit.HOURS, "d" -> TimeUnit.DAYS ) + /** * Convert a passed time string (e.g. 50s, 100ms, or 250us) to a microsecond count for * internal use. If no suffix is provided a direct conversion is attempted. */ - def parseTimeString(str: String) : (Option[TimeUnit], Long) = { + private def parseTimeString(str: String, unit:TimeUnit) : Long = { val timeError = "Time must be specified as seconds (s), " + "milliseconds (ms), microseconds (us), minutes (min) hour (h), or day(d). " + "E.g. 50s, 100ms, or 250us." @@ -1031,14 +1032,14 @@ private[spark] object Utils extends Logging { val lower = str.toLowerCase.trim() var suffix: String = "" timeSuffixes.foreach(s => { - if(lower.endsWith(s._1)) { + if (lower.endsWith(s._1)) { suffix = s._1 } }) - (timeSuffixes.get(suffix), str.substring(0, str.length - suffix.length).toLong) + unit.convert(str.substring(0, str.length - suffix.length).toLong, + timeSuffixes.getOrElse(suffix, unit)) } catch { - case e: NumberFormatException => throw new NumberFormatException(timeError + "\n" + e.toString) } @@ -1049,8 +1050,7 @@ private[spark] object Utils extends Logging { * no suffix is provided, the passed number is assumed to be in us. */ def timeStringAsUs(str: String): Long = { - val parsed = parseTimeString(str) - parsed._1.getOrElse(TimeUnit.MICROSECONDS).toMicros(parsed._2) + parseTimeString(str, TimeUnit.MICROSECONDS) } /** @@ -1058,8 +1058,7 @@ private[spark] object Utils extends Logging { * no suffix is provided, the passed number is assumed to be in ms. */ def timeStringAsMs(str : String) : Long = { - val parsed = parseTimeString(str) - parsed._1.getOrElse(TimeUnit.MILLISECONDS).toMillis(parsed._2) + parseTimeString(str, TimeUnit.MILLISECONDS) } /** @@ -1067,8 +1066,7 @@ private[spark] object Utils extends Logging { * no suffix is provided, the passed number is assumed to be in seconds. */ def timeStringAsS(str : String) : Long = { - val parsed = parseTimeString(str) - parsed._1.getOrElse(TimeUnit.SECONDS).toSeconds(parsed._2) + parseTimeString(str, TimeUnit.SECONDS) } /** diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 44bb6ecbaff57..192db6ffa1956 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark.scheduler -import java.io.{ObjectInputStream, ObjectOutputStream, IOException} import java.util.Random import scala.collection.mutable.ArrayBuffer @@ -27,7 +26,7 @@ import org.scalatest.FunSuite import org.apache.spark._ import org.apache.spark.executor.TaskMetrics -import org.apache.spark.util.{Utils, ManualClock} +import org.apache.spark.util.{ManualClock, Utils} class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler) extends DAGScheduler(sc) { @@ -152,7 +151,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { private val conf = new SparkConf - val LOCALITY_WAIT_MS = Utils.timeStringAsMs(conf.get("spark.locality.wait", "3000ms")) + val LOCALITY_WAIT_MS = Utils.timeStringAsMs(conf.get("spark.locality.wait", "3s")) val MAX_TASK_FAILURES = 4 override def beforeEach() { 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 5d93086082189..9acafaf84f8d7 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -35,7 +35,32 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkConf class UtilsSuite extends FunSuite with ResetSystemProperties { - + test("timeConversion") { + assert(Utils.timeStringAsS("1") === 1) + assert(Utils.timeStringAsS("1s") === 1) + assert(Utils.timeStringAsS("1000ms") === 1) + assert(Utils.timeStringAsS("1000000us") === 1) + assert(Utils.timeStringAsS("1min") === 60) + assert(Utils.timeStringAsS("1h") === 60 * 60) + assert(Utils.timeStringAsS("1d") === 60 * 60 * 24) + + assert(Utils.timeStringAsMs("1") === 1) + assert(Utils.timeStringAsMs("1ms") === 1) + assert(Utils.timeStringAsMs("1000us") === 1) + assert(Utils.timeStringAsMs("1s") === 1000) + assert(Utils.timeStringAsMs("1min") === 1000 * 60) + assert(Utils.timeStringAsMs("1h") === 1000 * 60 * 60) + assert(Utils.timeStringAsMs("1d") === 1000 * 60 * 60 * 24) + + assert(Utils.timeStringAsUs("1") === 1) + assert(Utils.timeStringAsUs("1us") === 1) + assert(Utils.timeStringAsUs("1ms") === 1000) + assert(Utils.timeStringAsUs("1s") === 1000 * 1000) + assert(Utils.timeStringAsUs("1min") === 1000 * 1000 * 60) + assert(Utils.timeStringAsUs("1h") === 1000L * 1000L * 60 * 60) + assert(Utils.timeStringAsUs("1d") === 1000L * 1000L * 60 * 60 * 24) + } + test("bytesToString") { assert(Utils.bytesToString(10) === "10.0 B") assert(Utils.bytesToString(1500) === "1500.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 7d6418c14bc6d..16cc5fc7eed7f 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 @@ -21,9 +21,11 @@ import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.concurrent.TimeUnit; import com.google.common.base.Charsets; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; import io.netty.buffer.Unpooled; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -122,34 +124,39 @@ private static boolean isSymlink(File file) throws IOException { return !fileInCanonicalDir.getCanonicalFile().equals(fileInCanonicalDir.getAbsoluteFile()); } + private static ImmutableMap timeSuffixes = + new ImmutableMap.Builder() + .put("us",TimeUnit.MICROSECONDS) + .put("ms",TimeUnit.MILLISECONDS) + .put("s", TimeUnit.SECONDS) + .put("min", TimeUnit.MINUTES) + .put("h", TimeUnit.HOURS) + .put("d", TimeUnit.DAYS) + .build(); + /** - * Convert a passed time string (e.g. 50s, 100ms, or 250us) to a microsecond count for + * 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. */ - private static long timeStringToUs(String str) throws NumberFormatException { + private static long parseTimeString(String str, TimeUnit unit) throws NumberFormatException { + String timeError = "Time must be specified as seconds (s), " + + "milliseconds (ms), microseconds (us), minutes (min) hour (h), or day(d). " + + "E.g. 50s, 100ms, or 250us."; String lower = str.toLowerCase().trim(); + try { - if (lower.endsWith("ms")) { - return Long.parseLong(lower.substring(0, lower.length() - 2)) * 1000; - } else if (lower.endsWith("us")) { - return Long.parseLong(lower.substring(0, lower.length() - 2)); - } else if (lower.endsWith("s")) { - return Long.parseLong(lower.substring(0, lower.length() - 1)) * 1000 * 1000; - } else if (lower.endsWith("min")) { - return Long.parseLong(lower.substring(0, lower.length() - 3)) * 1000 * 1000 * 60; - } else if (lower.endsWith("h")) { - return Long.parseLong(lower.substring(0, lower.length() - 1)) * 1000 * 1000 * 60 * 60; - } else if (lower.endsWith("d")) { - return Long.parseLong(lower.substring(0, lower.length() - 1)) * 1000 * 1000 * 60 * 60 * 24; - } else {// No suffix, default selected by calling function - return Long.parseLong(lower); + String suffix = ""; + for (String tail: timeSuffixes.keySet()) { + if (lower.endsWith(tail)) { + suffix = tail; + } } + + return unit.convert(Long.parseLong(str.substring(0, str.length() - suffix.length())), + timeSuffixes.getOrDefault(suffix, unit)); } catch(NumberFormatException e) { - throw new NumberFormatException("Time must be specified as seconds (s), " + - "milliseconds (ms), microseconds (us), minutes (min) hour (h), or day(d). " + - "E.g. 50s, 100ms, or 250us.\n" + e.toString()); + throw new NumberFormatException(timeError + "\n" + e.toString()); } - } /** @@ -157,23 +164,23 @@ private static long timeStringToUs(String str) throws NumberFormatException { * no suffix is provided, the passed number is assumed to be in us. */ public static long timeStringAsUs(String str) throws NumberFormatException { - return timeStringToUs(str); + return parseTimeString(str, TimeUnit.MICROSECONDS); } /** - * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If + * Convert a time parameter such as (50s, 100ms, or 250us) to milliseconds for internal use. If * no suffix is provided, the passed number is assumed to be in ms. */ public static long timeStringAsMs(String str) throws NumberFormatException { - return timeStringToUs(str)/1000; + return parseTimeString(str, TimeUnit.MILLISECONDS); } /** - * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If + * Convert a time parameter such as (50s, 100ms, or 250us) to seconds for internal use. If * no suffix is provided, the passed number is assumed to be in seconds. */ public static long timeStringAsS(String str) throws NumberFormatException { - return timeStringToUs(str)/1000/1000; + return parseTimeString(str, TimeUnit.SECONDS); } } diff --git a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java index 9b1c6e8aeb4c0..b0f95c830483d 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -38,7 +38,7 @@ public boolean preferDirectBufs() { /** Connect timeout in milliseconds. Default 120 secs. */ public int connectionTimeoutMs() { long defaultNetworkTimeoutS = JavaUtils.timeStringAsS( - conf.get("spark.network.timeout","120s")); + conf.get("spark.network.timeout", "120s")); long defaultTimeoutS = JavaUtils.timeStringAsS( conf.get("spark.shuffle.io.connectionTimeout", defaultNetworkTimeoutS + "s")); return (int) defaultTimeoutS * 1000; diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index 560067064f8a1..061caeef03749 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -75,7 +75,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) myConf.set("spark.cleaner.ttl", "10s") ssc = new StreamingContext(myConf, batchDuration) - assert(Utils.timeStringAsS(ssc.conf.get("spark.cleaner.ttl", "-1s")) === 10) + assert(Utils.timeStringAsS(ssc.conf.get("spark.cleaner.ttl", "-1")) === 10) } test("from existing SparkContext") { @@ -87,7 +87,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) myConf.set("spark.cleaner.ttl", "10s") ssc = new StreamingContext(myConf, batchDuration) - assert(Utils.timeStringAsS(ssc.conf.get("spark.cleaner.ttl", "-1s")) === 10) + assert(Utils.timeStringAsS(ssc.conf.get("spark.cleaner.ttl", "-1")) === 10) } test("from checkpoint") { @@ -98,12 +98,12 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w ssc1.start() val cp = new Checkpoint(ssc1, Time(1000)) assert( - Utils.timeStringAsS(cp.sparkConfPairs.toMap.getOrElse("spark.cleaner.ttl", "-1s")) === 10) + Utils.timeStringAsS(cp.sparkConfPairs.toMap.getOrElse("spark.cleaner.ttl", "-1")) === 10) ssc1.stop() val newCp = Utils.deserialize[Checkpoint](Utils.serialize(cp)) - assert(Utils.timeStringAsS(newCp.createSparkConf().get("spark.cleaner.ttl", "-1s")) === 10) + assert(Utils.timeStringAsS(newCp.createSparkConf().get("spark.cleaner.ttl", "-1")) === 10) ssc = new StreamingContext(null, newCp, null) - assert(Utils.timeStringAsS(ssc.conf.get("spark.cleaner.ttl", "-1s")) === 10) + assert(Utils.timeStringAsS(ssc.conf.get("spark.cleaner.ttl", "-1")) === 10) } test("start and stop state check") { From b2fc9652c519254f889135196b809e59658637f7 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Tue, 31 Mar 2015 14:17:41 -0700 Subject: [PATCH 33/58] replaced get or default since it's not present in this version of java --- .../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 16cc5fc7eed7f..960f087de4540 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 @@ -153,7 +153,7 @@ private static long parseTimeString(String str, TimeUnit unit) throws NumberForm } return unit.convert(Long.parseLong(str.substring(0, str.length() - suffix.length())), - timeSuffixes.getOrDefault(suffix, unit)); + timeSuffixes.containsKey(suffix) ? timeSuffixes.get(suffix) : unit); } catch(NumberFormatException e) { throw new NumberFormatException(timeError + "\n" + e.toString()); } From dd0a68027ab2924128638d903d5ad2d8dcb35bdf Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Tue, 31 Mar 2015 16:28:57 -0700 Subject: [PATCH 34/58] Updated scala code to call into java --- .../scala/org/apache/spark/util/Utils.scala | 43 +++---------------- .../apache/spark/network/util/JavaUtils.java | 28 ++++++------ 2 files changed, 19 insertions(+), 52 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 76eb63a1257bc..9073dba89d848 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -25,6 +25,8 @@ import java.util.{Properties, Locale, Random, UUID} import java.util.concurrent._ import javax.net.ssl.HttpsURLConnection +import org.apache.spark.network.util.JavaUtils + import scala.collection.JavaConversions._ import scala.collection.Map import scala.collection.mutable.ArrayBuffer @@ -1010,47 +1012,12 @@ private[spark] object Utils extends Logging { ) } - private val timeSuffixes = Map( - "us" -> TimeUnit.MICROSECONDS, - "ms" -> TimeUnit.MILLISECONDS, - "s" -> TimeUnit.SECONDS, - "min" -> TimeUnit.MINUTES, - "h" -> TimeUnit.HOURS, - "d" -> TimeUnit.DAYS - ) - - /** - * Convert a passed time string (e.g. 50s, 100ms, or 250us) to a microsecond count for - * internal use. If no suffix is provided a direct conversion is attempted. - */ - private def parseTimeString(str: String, unit:TimeUnit) : Long = { - val timeError = "Time must be specified as seconds (s), " + - "milliseconds (ms), microseconds (us), minutes (min) hour (h), or day(d). " + - "E.g. 50s, 100ms, or 250us." - - try { - val lower = str.toLowerCase.trim() - var suffix: String = "" - timeSuffixes.foreach(s => { - if (lower.endsWith(s._1)) { - suffix = s._1 - } - }) - - unit.convert(str.substring(0, str.length - suffix.length).toLong, - timeSuffixes.getOrElse(suffix, unit)) - } catch { - case e: NumberFormatException => throw new NumberFormatException(timeError + "\n" + - e.toString) - } - } - /** * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If * no suffix is provided, the passed number is assumed to be in us. */ def timeStringAsUs(str: String): Long = { - parseTimeString(str, TimeUnit.MICROSECONDS) + JavaUtils.parseTimeString(str, TimeUnit.MICROSECONDS) } /** @@ -1058,7 +1025,7 @@ private[spark] object Utils extends Logging { * no suffix is provided, the passed number is assumed to be in ms. */ def timeStringAsMs(str : String) : Long = { - parseTimeString(str, TimeUnit.MILLISECONDS) + JavaUtils.parseTimeString(str, TimeUnit.MILLISECONDS) } /** @@ -1066,7 +1033,7 @@ private[spark] object Utils extends Logging { * no suffix is provided, the passed number is assumed to be in seconds. */ def timeStringAsS(str : String) : Long = { - parseTimeString(str, TimeUnit.SECONDS) + JavaUtils.parseTimeString(str, TimeUnit.SECONDS) } /** 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 960f087de4540..a64ffbc774edf 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 @@ -125,23 +125,23 @@ private static boolean isSymlink(File file) throws IOException { } private static ImmutableMap timeSuffixes = - new ImmutableMap.Builder() - .put("us",TimeUnit.MICROSECONDS) - .put("ms",TimeUnit.MILLISECONDS) - .put("s", TimeUnit.SECONDS) - .put("min", TimeUnit.MINUTES) - .put("h", TimeUnit.HOURS) - .put("d", TimeUnit.DAYS) - .build(); - + ImmutableMap.builder() + .put("us",TimeUnit.MICROSECONDS) + .put("ms",TimeUnit.MILLISECONDS) + .put("s", TimeUnit.SECONDS) + .put("min", TimeUnit.MINUTES) + .put("h", TimeUnit.HOURS) + .put("d", TimeUnit.DAYS) + .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. */ - private static long parseTimeString(String str, TimeUnit unit) throws NumberFormatException { + public static long parseTimeString(String str, TimeUnit unit) throws NumberFormatException { String timeError = "Time must be specified as seconds (s), " + - "milliseconds (ms), microseconds (us), minutes (min) hour (h), or day(d). " + - "E.g. 50s, 100ms, or 250us."; + "milliseconds (ms), microseconds (us), minutes (min) hour (h), or day(d). " + + "E.g. 50s, 100ms, or 250us."; String lower = str.toLowerCase().trim(); try { @@ -153,8 +153,8 @@ private static long parseTimeString(String str, TimeUnit unit) throws NumberForm } return unit.convert(Long.parseLong(str.substring(0, str.length() - suffix.length())), - timeSuffixes.containsKey(suffix) ? timeSuffixes.get(suffix) : unit); - } catch(NumberFormatException e) { + timeSuffixes.containsKey(suffix) ? timeSuffixes.get(suffix) : unit); + } catch (NumberFormatException e) { throw new NumberFormatException(timeError + "\n" + e.toString()); } } From bf779b0e348e599e6b98ceb267a4b30a7ee8ed5a Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Tue, 31 Mar 2015 17:24:22 -0700 Subject: [PATCH 35/58] Special handling of overlapping usffixes for java --- .../java/org/apache/spark/network/util/JavaUtils.java | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 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 a64ffbc774edf..94abd8f1803a0 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,8 +126,8 @@ private static boolean isSymlink(File file) throws IOException { private static ImmutableMap timeSuffixes = ImmutableMap.builder() - .put("us",TimeUnit.MICROSECONDS) - .put("ms",TimeUnit.MILLISECONDS) + .put("us", TimeUnit.MICROSECONDS) + .put("ms", TimeUnit.MILLISECONDS) .put("s", TimeUnit.SECONDS) .put("min", TimeUnit.MINUTES) .put("h", TimeUnit.HOURS) @@ -148,7 +148,12 @@ public static long parseTimeString(String str, TimeUnit unit) throws NumberForma String suffix = ""; for (String tail: timeSuffixes.keySet()) { if (lower.endsWith(tail)) { - suffix = tail; + // Handle special case for overlapping suffixes + if(!tail.equals("s") || + (tail.equals("s") && !lower.endsWith("us") && !lower.endsWith("ms"))) + { + suffix = tail; + } } } From 76cfa2799ceae93534710c28545465c0a2b970f2 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Tue, 31 Mar 2015 23:26:32 -0700 Subject: [PATCH 36/58] [SPARK-5931] Minor nit fixes' --- core/src/main/scala/org/apache/spark/util/Utils.scala | 6 +++--- .../main/java/org/apache/spark/network/util/JavaUtils.java | 6 +++--- 2 files changed, 6 insertions(+), 6 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 9073dba89d848..029442ea653b0 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1017,7 +1017,7 @@ private[spark] object Utils extends Logging { * no suffix is provided, the passed number is assumed to be in us. */ def timeStringAsUs(str: String): Long = { - JavaUtils.parseTimeString(str, TimeUnit.MICROSECONDS) + JavaUtils.timeStringAsUs(str) } /** @@ -1025,7 +1025,7 @@ private[spark] object Utils extends Logging { * no suffix is provided, the passed number is assumed to be in ms. */ def timeStringAsMs(str : String) : Long = { - JavaUtils.parseTimeString(str, TimeUnit.MILLISECONDS) + JavaUtils.timeStringAsMs(str) } /** @@ -1033,7 +1033,7 @@ private[spark] object Utils extends Logging { * no suffix is provided, the passed number is assumed to be in seconds. */ def timeStringAsS(str : String) : Long = { - JavaUtils.parseTimeString(str, TimeUnit.SECONDS) + JavaUtils.timeStringAsS(str) } /** 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 94abd8f1803a0..3bc3954ec3598 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 @@ -138,7 +138,7 @@ private static boolean isSymlink(File file) throws IOException { * 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. */ - public static long parseTimeString(String str, TimeUnit unit) throws NumberFormatException { + private static long parseTimeString(String str, TimeUnit unit) throws NumberFormatException { String timeError = "Time must be specified as seconds (s), " + "milliseconds (ms), microseconds (us), minutes (min) hour (h), or day(d). " + "E.g. 50s, 100ms, or 250us."; @@ -149,7 +149,7 @@ public static long parseTimeString(String str, TimeUnit unit) throws NumberForma for (String tail: timeSuffixes.keySet()) { if (lower.endsWith(tail)) { // Handle special case for overlapping suffixes - if(!tail.equals("s") || + if (!tail.equals("s") || (tail.equals("s") && !lower.endsWith("us") && !lower.endsWith("ms"))) { suffix = tail; @@ -160,7 +160,7 @@ public static long parseTimeString(String str, TimeUnit unit) throws NumberForma return unit.convert(Long.parseLong(str.substring(0, str.length() - suffix.length())), timeSuffixes.containsKey(suffix) ? timeSuffixes.get(suffix) : unit); } catch (NumberFormatException e) { - throw new NumberFormatException(timeError + "\n" + e.toString()); + throw new NumberFormatException(timeError + "\n" + e.getMessage()); } } From 638777293c1049d0b68020868a6aec4b4573d58d Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Mon, 6 Apr 2015 09:16:14 -0700 Subject: [PATCH 37/58] Updated suffix handling to handle overlap of units more gracefully --- .../org/apache/spark/HeartbeatReceiver.scala | 9 ++++++--- .../cluster/CoarseGrainedSchedulerBackend.scala | 2 +- .../scala/org/apache/spark/util/AkkaUtils.scala | 4 ++-- .../network/nio/ConnectionManagerSuite.scala | 8 ++++---- .../apache/spark/network/util/JavaUtils.java | 17 +++++++---------- .../spark/network/util/TransportConf.java | 4 ++-- .../shuffle/RetryingBlockFetcherSuite.java | 2 +- .../streaming/scheduler/JobGenerator.scala | 6 +++--- .../apache/spark/streaming/ReceiverSuite.scala | 6 +++--- .../spark/streaming/StreamingContextSuite.scala | 4 ++-- 10 files changed, 31 insertions(+), 31 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index 532b8cf8399ef..fad6d9a5b9283 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -25,7 +25,7 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.rpc.{ThreadSafeRpcEndpoint, RpcEnv, RpcCallContext} import org.apache.spark.storage.BlockManagerId import org.apache.spark.scheduler.{SlaveLost, TaskScheduler} -import org.apache.spark.util.{Utils} +import org.apache.spark.util.Utils /** * A heartbeat from executors to the driver. This is a shared message used by several internal @@ -62,15 +62,18 @@ private[spark] class HeartbeatReceiver(sc: SparkContext) // "spark.network.timeout" uses "seconds", while `spark.storage.blockManagerSlaveTimeoutMs` uses // "milliseconds" + private val networkTimeoutS = Utils.timeStringAsS(sc.conf.get("spark.network.timeout", "120s")) private val executorTimeoutMs = Utils.timeStringAsMs( sc.conf.get("spark.storage.blockManagerSlaveTimeoutMs", - s"${Utils.timeStringAsS(sc.conf.get("spark.network.timeout", "120s"))}s")) + s"${networkTimeoutS}s")) // "spark.network.timeoutInterval" uses "seconds", while // "spark.storage.blockManagerTimeoutIntervalMs" uses "milliseconds" + private val timeoutIntervalS = Utils.timeStringAsS(sc.conf + .get("spark.network.timeoutInterval", "60s")) private val checkTimeoutIntervalMs = Utils.timeStringAsMs( sc.conf.get("spark.storage.blockManagerTimeoutIntervalMs", - s"${Utils.timeStringAsS(sc.conf.get("spark.network.timeoutInterval", "60s"))}s")) + s"${timeoutIntervalS}s")) private var timeoutCheckingTask: ScheduledFuture[_] = null diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 7c062277d857f..d55d4d885e882 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -78,7 +78,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp override def onStart() { // Periodically revive offers to allow delay scheduling to work val reviveIntervalMs = Utils.timeStringAsMs( - conf.get("spark.scheduler.revive.interval", "1000ms")) + conf.get("spark.scheduler.revive.interval", "1s")) reviveThread.scheduleAtFixedRate(new Runnable { override def run(): Unit = Utils.tryLogNonFatalError { diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index d6a1a3ec9b11d..61254d6d7363b 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -79,7 +79,7 @@ private[spark] object AkkaUtils extends Logging { val logAkkaConfig = if (conf.getBoolean("spark.akka.logAkkaConfig", false)) "on" else "off" - val akkaHeartBeatPauses = conf.getInt("spark.akka.heartbeat.pauses", 6000) + val akkaHeartBeatPausesS = Utils.timeStringAsS(conf.get("spark.akka.heartbeat.pauses", "6000s")) val akkaHeartBeatIntervalS = Utils.timeStringAsS(conf.get("spark.akka.heartbeat.interval", "1000s")) @@ -105,7 +105,7 @@ private[spark] object AkkaUtils extends Logging { |akka.remote.require-cookie = "$requireCookie" |akka.remote.secure-cookie = "$secureCookie" |akka.remote.transport-failure-detector.heartbeat-interval = $akkaHeartBeatIntervalS s - |akka.remote.transport-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPauses s + |akka.remote.transport-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPausesS s |akka.actor.provider = "akka.remote.RemoteActorRefProvider" |akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport" |akka.remote.netty.tcp.hostname = "$host" diff --git a/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala b/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala index e8916c2f886fc..02424c59d6831 100644 --- a/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala @@ -260,8 +260,8 @@ class ConnectionManagerSuite extends FunSuite { test("sendMessageReliably timeout") { val clientConf = new SparkConf clientConf.set("spark.authenticate", "false") - val ackTimeout = 30 - clientConf.set("spark.core.connection.ack.wait.timeout", s"${ackTimeout}s") + val ackTimeoutS = 30 + clientConf.set("spark.core.connection.ack.wait.timeout", s"${ackTimeoutS}s") val clientSecurityManager = new SecurityManager(clientConf) val manager = new ConnectionManager(0, clientConf, clientSecurityManager) @@ -272,7 +272,7 @@ class ConnectionManagerSuite extends FunSuite { val managerServer = new ConnectionManager(0, serverConf, serverSecurityManager) managerServer.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { // sleep 60 sec > ack timeout for simulating server slow down or hang up - Thread.sleep(ackTimeout * 3 * 1000) + Thread.sleep(ackTimeoutS * 3 * 1000) None }) @@ -287,7 +287,7 @@ class ConnectionManagerSuite extends FunSuite { // Otherwise TimeoutExcepton is thrown from Await.result. // We expect TimeoutException is not thrown. intercept[IOException] { - Await.result(future, (ackTimeout * 2) second) + Await.result(future, (ackTimeoutS * 2) second) } manager.stop() 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 3bc3954ec3598..c3ea991c2790c 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 @@ -139,27 +139,24 @@ private static boolean isSymlink(File file) throws IOException { * internal use. If no suffix is provided a direct conversion is attempted. */ private static long parseTimeString(String str, TimeUnit unit) throws NumberFormatException { - String timeError = "Time must be specified as seconds (s), " + - "milliseconds (ms), microseconds (us), minutes (min) hour (h), or day(d). " + - "E.g. 50s, 100ms, or 250us."; String lower = str.toLowerCase().trim(); try { String suffix = ""; for (String tail: timeSuffixes.keySet()) { - if (lower.endsWith(tail)) { - // Handle special case for overlapping suffixes - if (!tail.equals("s") || - (tail.equals("s") && !lower.endsWith("us") && !lower.endsWith("ms"))) - { - suffix = tail; - } + char prevChar = lower.charAt(lower.length() - tail.length()-1); + if (lower.endsWith(tail) && Character.isDigit(prevChar)) { + suffix = tail; } } return unit.convert(Long.parseLong(str.substring(0, str.length() - suffix.length())), timeSuffixes.containsKey(suffix) ? timeSuffixes.get(suffix) : unit); } catch (NumberFormatException e) { + String timeError = "Time must be specified as seconds (s), " + + "milliseconds (ms), microseconds (us), minutes (min) hour (h), or day(d). " + + "E.g. 50s, 100ms, or 250us."; + throw new NumberFormatException(timeError + "\n" + e.getMessage()); } } diff --git a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java index b0f95c830483d..b70b6867820c1 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -72,7 +72,7 @@ public int numConnectionsPerPeer() { /** Timeout for a single round trip of SASL token exchange, in milliseconds. */ public int saslRTTimeoutMs() { - return (int) JavaUtils.timeStringAsS(conf.get("spark.shuffle.sasl.timeout", "30s")) * 1000; + return (int) JavaUtils.timeStringAsMs(conf.get("spark.shuffle.sasl.timeout", "30s")); } /** @@ -86,7 +86,7 @@ public int saslRTTimeoutMs() { * Only relevant if maxIORetries > 0. */ public int ioRetryWaitTimeMs() { - return (int) JavaUtils.timeStringAsS(conf.get("spark.shuffle.io.retryWait", "5s")) * 1000; + return (int) JavaUtils.timeStringAsMs(conf.get("spark.shuffle.io.retryWait", "5s")); } /** diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java index 51d47511e7d1b..1ad0d72ae5ec5 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java @@ -54,7 +54,7 @@ public class RetryingBlockFetcherSuite { @Before public void beforeEach() { System.setProperty("spark.shuffle.io.maxRetries", "2"); - System.setProperty("spark.shuffle.io.retryWait", "0s"); + System.setProperty("spark.shuffle.io.retryWait", "0"); } @After diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index e74c34bd293dd..12086a6b06e10 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -104,15 +104,15 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { if (processReceivedData) { logInfo("Stopping JobGenerator gracefully") val timeWhenStopStarted = System.currentTimeMillis() - val stopTimeout = Utils.timeStringAsMs(conf.get( + val stopTimeoutMs = Utils.timeStringAsMs(conf.get( "spark.streaming.gracefulStopTimeout", s"${10 * ssc.graph.batchDuration.milliseconds}ms")) val pollTime = 100 // To prevent graceful stop to get stuck permanently def hasTimedOut: Boolean = { - val timedOut = (System.currentTimeMillis() - timeWhenStopStarted) > stopTimeout + val timedOut = (System.currentTimeMillis() - timeWhenStopStarted) > stopTimeoutMs if (timedOut) { - logWarning("Timed out while stopping the job generator (timeout = " + stopTimeout + ")") + logWarning("Timed out while stopping the job generator (timeout = " + stopTimeoutMs + ")") } timedOut } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala index cbbfdf1f980e7..ffd02e79acaeb 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala @@ -131,11 +131,11 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { test("block generator") { val blockGeneratorListener = new FakeBlockGeneratorListener - val blockInterval = 200 - val conf = new SparkConf().set("spark.streaming.blockInterval", s"${blockInterval.toString}ms") + val blockIntervalMs = 200 + val conf = new SparkConf().set("spark.streaming.blockInterval", s"${blockIntervalMs.toString}ms") val blockGenerator = new BlockGenerator(blockGeneratorListener, 1, conf) val expectedBlocks = 5 - val waitTime = expectedBlocks * blockInterval + (blockInterval / 2) + val waitTime = expectedBlocks * blockIntervalMs + (blockIntervalMs / 2) val generatedData = new ArrayBuffer[Int] // Generate blocks diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index 061caeef03749..fddbcc8b700c7 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -177,7 +177,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("stop gracefully") { val conf = new SparkConf().setMaster(master).setAppName(appName) - conf.set("spark.cleaner.ttl", "3600ms") + conf.set("spark.cleaner.ttl", "3600s") sc = new SparkContext(conf) for (i <- 1 to 4) { logInfo("==================================\n\n\n") @@ -208,7 +208,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("stop slow receiver gracefully") { val conf = new SparkConf().setMaster(master).setAppName(appName) - conf.set("spark.streaming.gracefulStopTimeout", "20000ms") + conf.set("spark.streaming.gracefulStopTimeout", "20000s") sc = new SparkContext(conf) logInfo("==================================\n\n\n") ssc = new StreamingContext(sc, Milliseconds(100)) From 19c31afc8feb621452ac5b6a3cb156466a7da5c4 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Mon, 6 Apr 2015 10:18:22 -0700 Subject: [PATCH 38/58] Added cleaner computation of time conversions in tests --- .../org/apache/spark/util/UtilsSuite.scala | 26 ++++++++++--------- 1 file changed, 14 insertions(+), 12 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 9acafaf84f8d7..d23ada60b9e2f 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.util +import java.util.concurrent.TimeUnit + import scala.util.Random import java.io.{File, ByteArrayOutputStream, ByteArrayInputStream, FileOutputStream} @@ -40,25 +42,25 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { assert(Utils.timeStringAsS("1s") === 1) assert(Utils.timeStringAsS("1000ms") === 1) assert(Utils.timeStringAsS("1000000us") === 1) - assert(Utils.timeStringAsS("1min") === 60) - assert(Utils.timeStringAsS("1h") === 60 * 60) - assert(Utils.timeStringAsS("1d") === 60 * 60 * 24) + assert(Utils.timeStringAsS("1min") === TimeUnit.MINUTES.toSeconds(1)) + assert(Utils.timeStringAsS("1h") === TimeUnit.HOURS.toSeconds(1)) + assert(Utils.timeStringAsS("1d") === TimeUnit.DAYS.toSeconds(1)) assert(Utils.timeStringAsMs("1") === 1) assert(Utils.timeStringAsMs("1ms") === 1) assert(Utils.timeStringAsMs("1000us") === 1) - assert(Utils.timeStringAsMs("1s") === 1000) - assert(Utils.timeStringAsMs("1min") === 1000 * 60) - assert(Utils.timeStringAsMs("1h") === 1000 * 60 * 60) - assert(Utils.timeStringAsMs("1d") === 1000 * 60 * 60 * 24) + assert(Utils.timeStringAsMs("1s") === TimeUnit.SECONDS.toMillis(1)) + assert(Utils.timeStringAsMs("1min") === TimeUnit.MINUTES.toMillis(1)) + assert(Utils.timeStringAsMs("1h") === TimeUnit.HOURS.toMillis(1)) + assert(Utils.timeStringAsMs("1d") === TimeUnit.DAYS.toMillis(1)) assert(Utils.timeStringAsUs("1") === 1) assert(Utils.timeStringAsUs("1us") === 1) - assert(Utils.timeStringAsUs("1ms") === 1000) - assert(Utils.timeStringAsUs("1s") === 1000 * 1000) - assert(Utils.timeStringAsUs("1min") === 1000 * 1000 * 60) - assert(Utils.timeStringAsUs("1h") === 1000L * 1000L * 60 * 60) - assert(Utils.timeStringAsUs("1d") === 1000L * 1000L * 60 * 60 * 24) + assert(Utils.timeStringAsUs("1ms") === TimeUnit.MILLISECONDS.toMillis(1)) + assert(Utils.timeStringAsUs("1s") === TimeUnit.SECONDS.toSeconds(1)) + assert(Utils.timeStringAsUs("1min") === TimeUnit.MINUTES.toMicros(1)) + assert(Utils.timeStringAsUs("1h") === TimeUnit.HOURS.toMicros(1)) + assert(Utils.timeStringAsUs("1d") === TimeUnit.DAYS.toMicros(1)) } test("bytesToString") { From ff40bfe6aaf17b17d334060800025d1442437ca5 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Mon, 6 Apr 2015 10:23:27 -0700 Subject: [PATCH 39/58] Updated tests to fix small bugs --- core/src/test/scala/org/apache/spark/util/UtilsSuite.scala | 4 ++-- .../main/java/org/apache/spark/network/util/JavaUtils.java | 4 ++-- 2 files changed, 4 insertions(+), 4 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 d23ada60b9e2f..fbc96c7909387 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -56,8 +56,8 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { assert(Utils.timeStringAsUs("1") === 1) assert(Utils.timeStringAsUs("1us") === 1) - assert(Utils.timeStringAsUs("1ms") === TimeUnit.MILLISECONDS.toMillis(1)) - assert(Utils.timeStringAsUs("1s") === TimeUnit.SECONDS.toSeconds(1)) + assert(Utils.timeStringAsUs("1ms") === TimeUnit.MILLISECONDS.toMicros(1)) + assert(Utils.timeStringAsUs("1s") === TimeUnit.SECONDS.toMicros(1)) assert(Utils.timeStringAsUs("1min") === TimeUnit.MINUTES.toMicros(1)) assert(Utils.timeStringAsUs("1h") === TimeUnit.HOURS.toMicros(1)) assert(Utils.timeStringAsUs("1d") === TimeUnit.DAYS.toMicros(1)) 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 c3ea991c2790c..12ac98fa65847 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 @@ -144,8 +144,8 @@ private static long parseTimeString(String str, TimeUnit unit) throws NumberForm try { String suffix = ""; for (String tail: timeSuffixes.keySet()) { - char prevChar = lower.charAt(lower.length() - tail.length()-1); - if (lower.endsWith(tail) && Character.isDigit(prevChar)) { + int charIdx = lower.length() - tail.length()-1; + if (lower.endsWith(tail) && (charIdx >= 0 && Character.isDigit(lower.charAt(charIdx)))) { suffix = tail; } } From 28187bf8abb4a635f066b0d8d9aff19d78c33b9b Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Mon, 6 Apr 2015 11:53:21 -0700 Subject: [PATCH 40/58] Convert straight to seconds --- .../java/org/apache/spark/network/util/TransportConf.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java index b70b6867820c1..8a1bad3804be0 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -39,9 +39,9 @@ public boolean preferDirectBufs() { public int connectionTimeoutMs() { long defaultNetworkTimeoutS = JavaUtils.timeStringAsS( conf.get("spark.network.timeout", "120s")); - long defaultTimeoutS = JavaUtils.timeStringAsS( + long defaultTimeoutMs = JavaUtils.timeStringAsMs( conf.get("spark.shuffle.io.connectionTimeout", defaultNetworkTimeoutS + "s")); - return (int) defaultTimeoutS * 1000; + return (int) defaultTimeoutMs; } /** Number of concurrent connections between two nodes for fetching data. */ From 14653907e118ead01ec3b0fe6640b5fcaba6d199 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Mon, 6 Apr 2015 11:54:25 -0700 Subject: [PATCH 41/58] Nit --- core/src/test/scala/org/apache/spark/util/UtilsSuite.scala | 1 + 1 file changed, 1 insertion(+) 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 fbc96c7909387..e088535369c87 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -37,6 +37,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkConf class UtilsSuite extends FunSuite with ResetSystemProperties { + test("timeConversion") { assert(Utils.timeStringAsS("1") === 1) assert(Utils.timeStringAsS("1s") === 1) From cbf41db9e33e6be592668a2d9b263f1fa40111c8 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Tue, 7 Apr 2015 08:56:14 -0700 Subject: [PATCH 42/58] Got rid of thrown exceptions --- .../java/org/apache/spark/network/util/JavaUtils.java | 8 ++++---- 1 file changed, 4 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 12ac98fa65847..45bb04bc63640 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 @@ -138,7 +138,7 @@ private static boolean isSymlink(File file) throws IOException { * 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. */ - private static long parseTimeString(String str, TimeUnit unit) throws NumberFormatException { + private static long parseTimeString(String str, TimeUnit unit) { String lower = str.toLowerCase().trim(); try { @@ -165,7 +165,7 @@ private static long parseTimeString(String str, TimeUnit unit) throws NumberForm * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If * no suffix is provided, the passed number is assumed to be in us. */ - public static long timeStringAsUs(String str) throws NumberFormatException { + public static long timeStringAsUs(String str) { return parseTimeString(str, TimeUnit.MICROSECONDS); } @@ -173,7 +173,7 @@ public static long timeStringAsUs(String str) throws NumberFormatException { * Convert a time parameter such as (50s, 100ms, or 250us) to milliseconds for internal use. If * no suffix is provided, the passed number is assumed to be in ms. */ - public static long timeStringAsMs(String str) throws NumberFormatException { + public static long timeStringAsMs(String str) { return parseTimeString(str, TimeUnit.MILLISECONDS); } @@ -181,7 +181,7 @@ public static long timeStringAsMs(String str) throws NumberFormatException { * Convert a time parameter such as (50s, 100ms, or 250us) to seconds for internal use. If * no suffix is provided, the passed number is assumed to be in seconds. */ - public static long timeStringAsS(String str) throws NumberFormatException { + public static long timeStringAsS(String str) { return parseTimeString(str, TimeUnit.SECONDS); } From d4efd269f2a8716a46254b87b17a31eb07ec61b4 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 8 Apr 2015 11:13:49 -0700 Subject: [PATCH 43/58] Added time conversion for yarn.scheduler.heartbeat.interval-ms --- .../scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 24a1e02795218..84a0d6745e4ce 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -295,7 +295,7 @@ private[spark] class ApplicationMaster( // we want to be reasonably responsive without causing too many requests to RM. val schedulerInterval = - sparkConf.getLong("spark.yarn.scheduler.heartbeat.interval-ms", 5000) + Utils.timeStringAsMs(sparkConf.get("spark.yarn.scheduler.heartbeat.interval-ms", "5s")) // must be <= expiryInterval / 2. val interval = math.max(0, math.min(expiryInterval / 2, schedulerInterval)) From 4e4867909cb7e7480ac062f92dc2fba0a2028a56 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 8 Apr 2015 12:08:19 -0700 Subject: [PATCH 44/58] Fixed priority order and mixed up conversions in a couple spots --- .../org/apache/spark/HeartbeatReceiver.scala | 19 +++++++++---------- .../org/apache/spark/executor/Executor.scala | 2 +- .../scala/org/apache/spark/util/Utils.scala | 3 ++- .../spark/network/util/TransportConf.java | 8 ++++---- 4 files changed, 16 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index fad6d9a5b9283..90f8448890916 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -62,18 +62,17 @@ private[spark] class HeartbeatReceiver(sc: SparkContext) // "spark.network.timeout" uses "seconds", while `spark.storage.blockManagerSlaveTimeoutMs` uses // "milliseconds" - private val networkTimeoutS = Utils.timeStringAsS(sc.conf.get("spark.network.timeout", "120s")) - private val executorTimeoutMs = Utils.timeStringAsMs( - sc.conf.get("spark.storage.blockManagerSlaveTimeoutMs", - s"${networkTimeoutS}s")) - + private val slaveTimeoutMs = Utils.timeStringAsMs( + sc.conf.get("spark.storage.blockManagerSlaveTimeoutMs", "120s")) + private val executorTimeoutMs = Utils.timeStringAsS(sc.conf.get("spark.network.timeout", + s"${slaveTimeoutMs}ms")) * 1000 + // "spark.network.timeoutInterval" uses "seconds", while // "spark.storage.blockManagerTimeoutIntervalMs" uses "milliseconds" - private val timeoutIntervalS = Utils.timeStringAsS(sc.conf - .get("spark.network.timeoutInterval", "60s")) - private val checkTimeoutIntervalMs = Utils.timeStringAsMs( - sc.conf.get("spark.storage.blockManagerTimeoutIntervalMs", - s"${timeoutIntervalS}s")) + private val timeoutIntervalMs = Utils.timeStringAsMs( + sc.conf.get("spark.storage.blockManagerTimeoutIntervalMs", "60s")) + private val checkTimeoutIntervalMs = Utils.timeStringAsS( + sc.conf.get("spark.network.timeoutInterval", s"${timeoutIntervalMs}ms")) * 1000 private var timeoutCheckingTask: ScheduledFuture[_] = null diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 3d073cee776be..dc994da33b79f 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -439,7 +439,7 @@ private[spark] class Executor( val intervalMs = Utils.timeStringAsMs(conf.get("spark.executor.heartbeatInterval", "10s")) val thread = new Thread() { override def run() { - // Sleep a random intervalMs so the heartbeats don't end up in sync + // Sleep a random interval so the heartbeats don't end up in sync Thread.sleep(intervalMs + (math.random * intervalMs).asInstanceOf[Int]) while (!isStopped) { reportHeartBeat() 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 489b6b3d69377..b33f70a992784 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -613,7 +613,8 @@ private[spark] object Utils extends Logging { } Utils.setupSecureURLConnection(uc, securityMgr) - val timeoutMs = Utils.timeStringAsMs(conf.get("spark.files.fetchTimeout", "60s")).toInt + val timeoutMs = Utils.timeStringAsS( + conf.get("spark.files.fetchTimeout", "60s")).toInt * 1000 uc.setConnectTimeout(timeoutMs) uc.setReadTimeout(timeoutMs) uc.connect() diff --git a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java index 8a1bad3804be0..d28e459757d0c 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -39,8 +39,8 @@ public boolean preferDirectBufs() { public int connectionTimeoutMs() { long defaultNetworkTimeoutS = JavaUtils.timeStringAsS( conf.get("spark.network.timeout", "120s")); - long defaultTimeoutMs = JavaUtils.timeStringAsMs( - conf.get("spark.shuffle.io.connectionTimeout", defaultNetworkTimeoutS + "s")); + long defaultTimeoutMs = JavaUtils.timeStringAsS( + conf.get("spark.shuffle.io.connectionTimeout", defaultNetworkTimeoutS + "s")) * 1000; return (int) defaultTimeoutMs; } @@ -72,7 +72,7 @@ public int numConnectionsPerPeer() { /** Timeout for a single round trip of SASL token exchange, in milliseconds. */ public int saslRTTimeoutMs() { - return (int) JavaUtils.timeStringAsMs(conf.get("spark.shuffle.sasl.timeout", "30s")); + return (int) JavaUtils.timeStringAsS(conf.get("spark.shuffle.sasl.timeout", "30s")) * 1000; } /** @@ -86,7 +86,7 @@ public int saslRTTimeoutMs() { * Only relevant if maxIORetries > 0. */ public int ioRetryWaitTimeMs() { - return (int) JavaUtils.timeStringAsMs(conf.get("spark.shuffle.io.retryWait", "5s")); + return (int) JavaUtils.timeStringAsS(conf.get("spark.shuffle.io.retryWait", "5s")) * 1000; } /** From 1a1122c80d966d650690d1a1e271a9846a9fec08 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 8 Apr 2015 12:16:04 -0700 Subject: [PATCH 45/58] Formatting fixes and added m for use as minute formatter --- .../apache/spark/ExecutorAllocationManager.scala | 6 +++--- .../scala/org/apache/spark/HeartbeatReceiver.scala | 4 ++-- .../spark/network/nio/ConnectionManager.scala | 2 +- .../apache/spark/scheduler/TaskSetManager.scala | 14 ++++++-------- .../scala/org/apache/spark/util/AkkaUtils.scala | 6 +++--- .../org/apache/spark/util/MetadataCleaner.scala | 2 +- .../main/scala/org/apache/spark/util/Utils.scala | 8 ++++---- .../scala/org/apache/spark/util/UtilsSuite.scala | 14 +++++++------- .../org/apache/spark/network/util/JavaUtils.java | 8 +++++--- .../apache/spark/network/util/TransportConf.java | 12 ++++++------ .../spark/streaming/StreamingContextSuite.scala | 10 +++++----- 11 files changed, 43 insertions(+), 43 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 345bc7f6e3395..8eb35a9b39a9f 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -80,15 +80,15 @@ private[spark] class ExecutorAllocationManager( Integer.MAX_VALUE) // How long there must be backlogged tasks for before an addition is triggered (seconds) - private val schedulerBacklogTimeoutS = Utils.timeStringAsS(conf.get( + private val schedulerBacklogTimeoutS = Utils.timeStringAsSec(conf.get( "spark.dynamicAllocation.schedulerBacklogTimeout", "5s")) // Same as above, but used only after `schedulerBacklogTimeoutS` is exceeded - private val sustainedSchedulerBacklogTimeoutS = Utils.timeStringAsS(conf.get( + private val sustainedSchedulerBacklogTimeoutS = Utils.timeStringAsSec(conf.get( "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", s"${schedulerBacklogTimeoutS}s")) // How long an executor must be idle for before it is removed (seconds) - private val executorIdleTimeoutS = Utils.timeStringAsS(conf.get( + private val executorIdleTimeoutS = Utils.timeStringAsSec(conf.get( "spark.dynamicAllocation.executorIdleTimeout", "600s")) // During testing, the methods to actually kill and add executors are mocked out diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index 90f8448890916..abf354ee9b927 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -64,14 +64,14 @@ private[spark] class HeartbeatReceiver(sc: SparkContext) // "milliseconds" private val slaveTimeoutMs = Utils.timeStringAsMs( sc.conf.get("spark.storage.blockManagerSlaveTimeoutMs", "120s")) - private val executorTimeoutMs = Utils.timeStringAsS(sc.conf.get("spark.network.timeout", + private val executorTimeoutMs = Utils.timeStringAsSec(sc.conf.get("spark.network.timeout", s"${slaveTimeoutMs}ms")) * 1000 // "spark.network.timeoutInterval" uses "seconds", while // "spark.storage.blockManagerTimeoutIntervalMs" uses "milliseconds" private val timeoutIntervalMs = Utils.timeStringAsMs( sc.conf.get("spark.storage.blockManagerTimeoutIntervalMs", "60s")) - private val checkTimeoutIntervalMs = Utils.timeStringAsS( + private val checkTimeoutIntervalMs = Utils.timeStringAsSec( sc.conf.get("spark.network.timeoutInterval", s"${timeoutIntervalMs}ms")) * 1000 private var timeoutCheckingTask: ScheduledFuture[_] = null diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala index 35255c9235d36..52e8b87a0cc96 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala @@ -82,7 +82,7 @@ private[nio] class ConnectionManager( new HashedWheelTimer(Utils.namedThreadFactory("AckTimeoutMonitor")) private val ackTimeout = - Utils.timeStringAsS(conf.get("spark.core.connection.ack.wait.timeout", + Utils.timeStringAsSec(conf.get("spark.core.connection.ack.wait.timeout", conf.get("spark.network.timeout", "120s"))) // Get the thread counts from the Spark Configuration. diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 870130e7d9c93..6c302c66ea0fb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -849,15 +849,13 @@ private[spark] class TaskSetManager( private def getLocalityWait(level: TaskLocality.TaskLocality): Long = { val defaultWait = conf.get("spark.locality.wait", "3s") - level match { - case TaskLocality.PROCESS_LOCAL => - Utils.timeStringAsMs(conf.get("spark.locality.wait.process", defaultWait)) - case TaskLocality.NODE_LOCAL => - Utils.timeStringAsMs(conf.get("spark.locality.wait.node", defaultWait)) - case TaskLocality.RACK_LOCAL => - Utils.timeStringAsMs(conf.get("spark.locality.wait.rack", defaultWait)) - case _ => 0L + val localityWaitKey = level match { + case TaskLocality.PROCESS_LOCAL => "spark.locality.wait.process" + case TaskLocality.NODE_LOCAL => "spark.locality.wait.node" + case TaskLocality.RACK_LOCAL => "spark.locality.wait.rack" + case _ => "" } + Utils.timeStringAsMs(conf.get(localityWaitKey, defaultWait)) } /** diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 61254d6d7363b..4481faa39a896 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -66,7 +66,7 @@ private[spark] object AkkaUtils extends Logging { val akkaThreads = conf.getInt("spark.akka.threads", 4) val akkaBatchSize = conf.getInt("spark.akka.batchSize", 15) - val akkaTimeoutS = Utils.timeStringAsS(conf.get("spark.akka.timeout", + val akkaTimeoutS = Utils.timeStringAsSec(conf.get("spark.akka.timeout", conf.get("spark.network.timeout", "120s"))) val akkaFrameSize = maxFrameSizeBytes(conf) val akkaLogLifecycleEvents = conf.getBoolean("spark.akka.logLifecycleEvents", false) @@ -79,9 +79,9 @@ private[spark] object AkkaUtils extends Logging { val logAkkaConfig = if (conf.getBoolean("spark.akka.logAkkaConfig", false)) "on" else "off" - val akkaHeartBeatPausesS = Utils.timeStringAsS(conf.get("spark.akka.heartbeat.pauses", "6000s")) + val akkaHeartBeatPausesS = Utils.timeStringAsSec(conf.get("spark.akka.heartbeat.pauses", "6000s")) val akkaHeartBeatIntervalS = - Utils.timeStringAsS(conf.get("spark.akka.heartbeat.interval", "1000s")) + Utils.timeStringAsSec(conf.get("spark.akka.heartbeat.interval", "1000s")) val secretKey = securityManager.getSecretKey() val isAuthOn = securityManager.isAuthenticationEnabled() diff --git a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala index 4ca71946b66b2..8cc801ffd3521 100644 --- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala @@ -76,7 +76,7 @@ private[spark] object MetadataCleanerType extends Enumeration { // initialization of StreamingContext. It's okay for users trying to configure stuff themselves. private[spark] object MetadataCleaner { def getDelaySeconds(conf: SparkConf): Int = { - Utils.timeStringAsS(conf.get("spark.cleaner.ttl", "-1")).toInt + Utils.timeStringAsSec(conf.get("spark.cleaner.ttl", "-1")).toInt } def getDelaySeconds( 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 b33f70a992784..e2920d6cf6bea 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -613,7 +613,7 @@ private[spark] object Utils extends Logging { } Utils.setupSecureURLConnection(uc, securityMgr) - val timeoutMs = Utils.timeStringAsS( + val timeoutMs = Utils.timeStringAsSec( conf.get("spark.files.fetchTimeout", "60s")).toInt * 1000 uc.setConnectTimeout(timeoutMs) uc.setReadTimeout(timeoutMs) @@ -1025,7 +1025,7 @@ private[spark] object Utils extends Logging { * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If * no suffix is provided, the passed number is assumed to be in ms. */ - def timeStringAsMs(str : String) : Long = { + def timeStringAsMs(str: String) : Long = { JavaUtils.timeStringAsMs(str) } @@ -1033,8 +1033,8 @@ private[spark] object Utils extends Logging { * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If * no suffix is provided, the passed number is assumed to be in seconds. */ - def timeStringAsS(str : String) : Long = { - JavaUtils.timeStringAsS(str) + def timeStringAsSec(str: String) : Long = { + JavaUtils.timeStringAsSec(str) } /** 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 e088535369c87..48bb3588d6c38 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -39,13 +39,13 @@ import org.apache.spark.SparkConf class UtilsSuite extends FunSuite with ResetSystemProperties { test("timeConversion") { - assert(Utils.timeStringAsS("1") === 1) - assert(Utils.timeStringAsS("1s") === 1) - assert(Utils.timeStringAsS("1000ms") === 1) - assert(Utils.timeStringAsS("1000000us") === 1) - assert(Utils.timeStringAsS("1min") === TimeUnit.MINUTES.toSeconds(1)) - assert(Utils.timeStringAsS("1h") === TimeUnit.HOURS.toSeconds(1)) - assert(Utils.timeStringAsS("1d") === TimeUnit.DAYS.toSeconds(1)) + assert(Utils.timeStringAsSec("1") === 1) + assert(Utils.timeStringAsSec("1s") === 1) + assert(Utils.timeStringAsSec("1000ms") === 1) + assert(Utils.timeStringAsSec("1000000us") === 1) + assert(Utils.timeStringAsSec("1min") === TimeUnit.MINUTES.toSeconds(1)) + assert(Utils.timeStringAsSec("1h") === TimeUnit.HOURS.toSeconds(1)) + assert(Utils.timeStringAsSec("1d") === TimeUnit.DAYS.toSeconds(1)) assert(Utils.timeStringAsMs("1") === 1) assert(Utils.timeStringAsMs("1ms") === 1) 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 45bb04bc63640..445e58acecedb 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 @@ -129,6 +129,7 @@ private static boolean isSymlink(File file) throws IOException { .put("us", TimeUnit.MICROSECONDS) .put("ms", TimeUnit.MILLISECONDS) .put("s", TimeUnit.SECONDS) + .put("m", TimeUnit.MINUTES) .put("min", TimeUnit.MINUTES) .put("h", TimeUnit.HOURS) .put("d", TimeUnit.DAYS) @@ -144,9 +145,10 @@ private static long parseTimeString(String str, TimeUnit unit) { try { String suffix = ""; for (String tail: timeSuffixes.keySet()) { - int charIdx = lower.length() - tail.length()-1; + int charIdx = lower.length() - tail.length() - 1; if (lower.endsWith(tail) && (charIdx >= 0 && Character.isDigit(lower.charAt(charIdx)))) { suffix = tail; + break; } } @@ -154,7 +156,7 @@ private static long parseTimeString(String str, TimeUnit unit) { timeSuffixes.containsKey(suffix) ? timeSuffixes.get(suffix) : unit); } catch (NumberFormatException e) { String timeError = "Time must be specified as seconds (s), " + - "milliseconds (ms), microseconds (us), minutes (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()); @@ -181,7 +183,7 @@ public static long timeStringAsMs(String str) { * Convert a time parameter such as (50s, 100ms, or 250us) to seconds for internal use. If * no suffix is provided, the passed number is assumed to be in seconds. */ - public static long timeStringAsS(String str) { + public static long timeStringAsSec(String str) { return parseTimeString(str, TimeUnit.SECONDS); } diff --git a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java index d28e459757d0c..0aef7f1987315 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -37,10 +37,10 @@ public boolean preferDirectBufs() { /** Connect timeout in milliseconds. Default 120 secs. */ public int connectionTimeoutMs() { - long defaultNetworkTimeoutS = JavaUtils.timeStringAsS( - conf.get("spark.network.timeout", "120s")); - long defaultTimeoutMs = JavaUtils.timeStringAsS( - conf.get("spark.shuffle.io.connectionTimeout", defaultNetworkTimeoutS + "s")) * 1000; + long defaultNetworkTimeoutS = JavaUtils.timeStringAsSec( + conf.get("spark.network.timeout", "120s")); + long defaultTimeoutMs = JavaUtils.timeStringAsSec( + conf.get("spark.shuffle.io.connectionTimeout", defaultNetworkTimeoutS + "s")) * 1000; return (int) defaultTimeoutMs; } @@ -72,7 +72,7 @@ public int numConnectionsPerPeer() { /** Timeout for a single round trip of SASL token exchange, in milliseconds. */ public int saslRTTimeoutMs() { - return (int) JavaUtils.timeStringAsS(conf.get("spark.shuffle.sasl.timeout", "30s")) * 1000; + return (int) JavaUtils.timeStringAsSec(conf.get("spark.shuffle.sasl.timeout", "30s")) * 1000; } /** @@ -86,7 +86,7 @@ public int saslRTTimeoutMs() { * Only relevant if maxIORetries > 0. */ public int ioRetryWaitTimeMs() { - return (int) JavaUtils.timeStringAsS(conf.get("spark.shuffle.io.retryWait", "5s")) * 1000; + return (int) JavaUtils.timeStringAsSec(conf.get("spark.shuffle.io.retryWait", "5s")) * 1000; } /** diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index fddbcc8b700c7..beb8bbfacb27a 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -75,7 +75,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) myConf.set("spark.cleaner.ttl", "10s") ssc = new StreamingContext(myConf, batchDuration) - assert(Utils.timeStringAsS(ssc.conf.get("spark.cleaner.ttl", "-1")) === 10) + assert(Utils.timeStringAsSec(ssc.conf.get("spark.cleaner.ttl", "-1")) === 10) } test("from existing SparkContext") { @@ -87,7 +87,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) myConf.set("spark.cleaner.ttl", "10s") ssc = new StreamingContext(myConf, batchDuration) - assert(Utils.timeStringAsS(ssc.conf.get("spark.cleaner.ttl", "-1")) === 10) + assert(Utils.timeStringAsSec(ssc.conf.get("spark.cleaner.ttl", "-1")) === 10) } test("from checkpoint") { @@ -98,12 +98,12 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w ssc1.start() val cp = new Checkpoint(ssc1, Time(1000)) assert( - Utils.timeStringAsS(cp.sparkConfPairs.toMap.getOrElse("spark.cleaner.ttl", "-1")) === 10) + Utils.timeStringAsSec(cp.sparkConfPairs.toMap.getOrElse("spark.cleaner.ttl", "-1")) === 10) ssc1.stop() val newCp = Utils.deserialize[Checkpoint](Utils.serialize(cp)) - assert(Utils.timeStringAsS(newCp.createSparkConf().get("spark.cleaner.ttl", "-1")) === 10) + assert(Utils.timeStringAsSec(newCp.createSparkConf().get("spark.cleaner.ttl", "-1")) === 10) ssc = new StreamingContext(null, newCp, null) - assert(Utils.timeStringAsS(ssc.conf.get("spark.cleaner.ttl", "-1")) === 10) + assert(Utils.timeStringAsSec(ssc.conf.get("spark.cleaner.ttl", "-1")) === 10) } test("start and stop state check") { From cbd2ca68b54df440372fcdea64a0ffa4ddca40c7 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 8 Apr 2015 12:21:22 -0700 Subject: [PATCH 46/58] Formatting error --- core/src/main/scala/org/apache/spark/util/AkkaUtils.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 4481faa39a896..6eda291eea13f 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -79,7 +79,8 @@ private[spark] object AkkaUtils extends Logging { val logAkkaConfig = if (conf.getBoolean("spark.akka.logAkkaConfig", false)) "on" else "off" - val akkaHeartBeatPausesS = Utils.timeStringAsSec(conf.get("spark.akka.heartbeat.pauses", "6000s")) + val akkaHeartBeatPausesS = Utils.timeStringAsSec(conf.get("spark.akka.heartbeat.pauses", + "6000s")) val akkaHeartBeatIntervalS = Utils.timeStringAsSec(conf.get("spark.akka.heartbeat.interval", "1000s")) From 6f651a8ab68c5ceb00a22ce5b68eff5bc51e631b Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 8 Apr 2015 13:57:10 -0700 Subject: [PATCH 47/58] Now using regexes to simplify code in parseTimeString. Introduces getTimeAsSec and getTimeAsMs methods in SparkConf. Updated documentation --- .../spark/ExecutorAllocationManager.scala | 12 ++--- .../org/apache/spark/HeartbeatReceiver.scala | 16 +++---- .../scala/org/apache/spark/SparkConf.scala | 29 ++++++++++++ .../org/apache/spark/executor/Executor.scala | 2 +- .../spark/network/nio/ConnectionManager.scala | 4 +- .../spark/scheduler/TaskSchedulerImpl.scala | 5 +- .../spark/scheduler/TaskSetManager.scala | 2 +- .../CoarseGrainedSchedulerBackend.scala | 5 +- .../org/apache/spark/util/AkkaUtils.scala | 11 ++--- .../apache/spark/util/MetadataCleaner.scala | 2 +- .../scala/org/apache/spark/util/Utils.scala | 4 +- .../spark/scheduler/TaskSetManagerSuite.scala | 2 +- .../org/apache/spark/util/UtilsSuite.scala | 25 ++++++++++ docs/configuration.md | 46 +++++++++++-------- docs/running-on-yarn.md | 2 +- .../apache/spark/network/util/JavaUtils.java | 18 +++++--- .../streaming/receiver/BlockGenerator.scala | 3 +- .../streaming/scheduler/JobGenerator.scala | 4 +- .../streaming/StreamingContextSuite.scala | 8 ++-- .../spark/deploy/yarn/ApplicationMaster.scala | 9 ++-- 20 files changed, 137 insertions(+), 72 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 8eb35a9b39a9f..7977b0e6b5deb 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -80,16 +80,16 @@ private[spark] class ExecutorAllocationManager( Integer.MAX_VALUE) // How long there must be backlogged tasks for before an addition is triggered (seconds) - private val schedulerBacklogTimeoutS = Utils.timeStringAsSec(conf.get( - "spark.dynamicAllocation.schedulerBacklogTimeout", "5s")) + private val schedulerBacklogTimeoutS = conf.getTimeAsSec( + "spark.dynamicAllocation.schedulerBacklogTimeout", "5s") // Same as above, but used only after `schedulerBacklogTimeoutS` is exceeded - private val sustainedSchedulerBacklogTimeoutS = Utils.timeStringAsSec(conf.get( - "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", s"${schedulerBacklogTimeoutS}s")) + private val sustainedSchedulerBacklogTimeoutS = conf.getTimeAsSec( + "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", s"${schedulerBacklogTimeoutS}s") // How long an executor must be idle for before it is removed (seconds) - private val executorIdleTimeoutS = Utils.timeStringAsSec(conf.get( - "spark.dynamicAllocation.executorIdleTimeout", "600s")) + private val executorIdleTimeoutS = conf.getTimeAsSec( + "spark.dynamicAllocation.executorIdleTimeout", "600s") // During testing, the methods to actually kill and add executors are mocked out private val testing = conf.getBoolean("spark.dynamicAllocation.testing", false) diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index abf354ee9b927..e19a3707e02be 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -62,17 +62,17 @@ private[spark] class HeartbeatReceiver(sc: SparkContext) // "spark.network.timeout" uses "seconds", while `spark.storage.blockManagerSlaveTimeoutMs` uses // "milliseconds" - private val slaveTimeoutMs = Utils.timeStringAsMs( - sc.conf.get("spark.storage.blockManagerSlaveTimeoutMs", "120s")) - private val executorTimeoutMs = Utils.timeStringAsSec(sc.conf.get("spark.network.timeout", - s"${slaveTimeoutMs}ms")) * 1000 + private val slaveTimeoutMs = + sc.conf.getTimeAsMs("spark.storage.blockManagerSlaveTimeoutMs", "120s") + private val executorTimeoutMs = + sc.conf.getTimeAsSec("spark.network.timeout", s"${slaveTimeoutMs}ms") * 1000 // "spark.network.timeoutInterval" uses "seconds", while // "spark.storage.blockManagerTimeoutIntervalMs" uses "milliseconds" - private val timeoutIntervalMs = Utils.timeStringAsMs( - sc.conf.get("spark.storage.blockManagerTimeoutIntervalMs", "60s")) - private val checkTimeoutIntervalMs = Utils.timeStringAsSec( - sc.conf.get("spark.network.timeoutInterval", s"${timeoutIntervalMs}ms")) * 1000 + private val timeoutIntervalMs = + sc.conf.getTimeAsMs("spark.storage.blockManagerTimeoutIntervalMs", "60s") + private val checkTimeoutIntervalMs = + sc.conf.getTimeAsSec("spark.network.timeoutInterval", s"${timeoutIntervalMs}ms") * 1000 private var timeoutCheckingTask: ScheduledFuture[_] = null diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 0c123c96b8d7b..c6ad8194625ac 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -174,6 +174,35 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { getOption(key).getOrElse(defaultValue) } + /** Get a time parameter as seconds; throws a NoSuchElementException if it's not set. If no + * suffix is provided then seconds are assumed. + */ + def getTimeAsSec(key: String): Long = { + Utils.timeStringAsSec(getOption(key).getOrElse(throw new NoSuchElementException(key))) + } + + /** Get a time parameter as seconds, falling back to a default if not set. If no + * suffix is provided then seconds are assumed. + */ + def getTimeAsSec(key: String, defaultValue: String): Long = { + Utils.timeStringAsSec(getOption(key).getOrElse(defaultValue)) + } + + /** Get a time parameter as milliseconds; throws a NoSuchElementException if it's not set. If no + * suffix is provided then milliseconds are assumed. + */ + def getTimeAsMs(key: String): Long = { + Utils.timeStringAsMs(getOption(key).getOrElse(throw new NoSuchElementException(key))) + } + + /** Get a time parameter as milliseconds, falling back to a default if not set. If no + * suffix is provided then milliseconds are assumed. + */ + def getTimeAsMs(key: String, defaultValue: String): Long = { + Utils.timeStringAsMs(getOption(key).getOrElse(defaultValue)) + } + + /** Get a parameter as an Option */ def getOption(key: String): Option[String] = { Option(settings.get(key)) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index dc994da33b79f..516f619529c48 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -436,7 +436,7 @@ private[spark] class Executor( * This thread stops running when the executor is stopped. */ private def startDriverHeartbeater(): Unit = { - val intervalMs = Utils.timeStringAsMs(conf.get("spark.executor.heartbeatInterval", "10s")) + val intervalMs = conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s") val thread = new Thread() { override def run() { // Sleep a random interval so the heartbeats don't end up in sync diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala index 52e8b87a0cc96..9f6be1e2fca55 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala @@ -82,8 +82,8 @@ private[nio] class ConnectionManager( new HashedWheelTimer(Utils.namedThreadFactory("AckTimeoutMonitor")) private val ackTimeout = - Utils.timeStringAsSec(conf.get("spark.core.connection.ack.wait.timeout", - conf.get("spark.network.timeout", "120s"))) + conf.getTimeAsSec("spark.core.connection.ack.wait.timeout", + conf.get("spark.network.timeout", "120s")) // Get the thread counts from the Spark Configuration. // diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 2768ce20c4073..2362cc7240039 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -62,11 +62,10 @@ private[spark] class TaskSchedulerImpl( val conf = sc.conf // How often to check for speculative tasks - val SPECULATION_INTERVAL_MS = - Utils.timeStringAsMs(conf.get("spark.speculation.interval", "100ms")) + val SPECULATION_INTERVAL_MS = conf.getTimeAsMs("spark.speculation.interval", "100ms") // Threshold above which we warn user initial TaskSet may be starved - val STARVATION_TIMEOUT_MS = Utils.timeStringAsMs(conf.get("spark.starvation.timeout", "15s")) + val STARVATION_TIMEOUT_MS = conf.getTimeAsMs("spark.starvation.timeout", "15s") // CPUs to request per task val CPUS_PER_TASK = conf.getInt("spark.task.cpus", 1) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 6c302c66ea0fb..ebfb3c8a98bdf 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -855,7 +855,7 @@ private[spark] class TaskSetManager( case TaskLocality.RACK_LOCAL => "spark.locality.wait.rack" case _ => "" } - Utils.timeStringAsMs(conf.get(localityWaitKey, defaultWait)) + conf.getTimeAsMs(localityWaitKey, defaultWait) } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index d55d4d885e882..63987dfb32695 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -53,7 +53,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // Submit tasks after maxRegisteredWaitingTime milliseconds // if minRegisteredRatio has not yet been reached val maxRegisteredWaitingTimeMs = - Utils.timeStringAsMs(conf.get("spark.scheduler.maxRegisteredResourcesWaitingTime", "30s")) + conf.getTimeAsMs("spark.scheduler.maxRegisteredResourcesWaitingTime", "30s") val createTime = System.currentTimeMillis() private val executorDataMap = new HashMap[String, ExecutorData] @@ -77,8 +77,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp override def onStart() { // Periodically revive offers to allow delay scheduling to work - val reviveIntervalMs = Utils.timeStringAsMs( - conf.get("spark.scheduler.revive.interval", "1s")) + val reviveIntervalMs = conf.getTimeAsMs("spark.scheduler.revive.interval", "1s") reviveThread.scheduleAtFixedRate(new Runnable { override def run(): Unit = Utils.tryLogNonFatalError { diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 6eda291eea13f..137f897c9e2ce 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -20,7 +20,6 @@ package org.apache.spark.util import scala.collection.JavaConversions.mapAsJavaMap import scala.concurrent.Await import scala.concurrent.duration.{Duration, FiniteDuration} -import scala.util.Try import akka.actor.{ActorRef, ActorSystem, ExtendedActorSystem} import akka.pattern.ask @@ -66,8 +65,8 @@ private[spark] object AkkaUtils extends Logging { val akkaThreads = conf.getInt("spark.akka.threads", 4) val akkaBatchSize = conf.getInt("spark.akka.batchSize", 15) - val akkaTimeoutS = Utils.timeStringAsSec(conf.get("spark.akka.timeout", - conf.get("spark.network.timeout", "120s"))) + val akkaTimeoutS = conf.getTimeAsSec("spark.akka.timeout", + conf.get("spark.network.timeout", "120s")) val akkaFrameSize = maxFrameSizeBytes(conf) val akkaLogLifecycleEvents = conf.getBoolean("spark.akka.logLifecycleEvents", false) val lifecycleEvents = if (akkaLogLifecycleEvents) "on" else "off" @@ -79,10 +78,10 @@ private[spark] object AkkaUtils extends Logging { val logAkkaConfig = if (conf.getBoolean("spark.akka.logAkkaConfig", false)) "on" else "off" - val akkaHeartBeatPausesS = Utils.timeStringAsSec(conf.get("spark.akka.heartbeat.pauses", - "6000s")) + val akkaHeartBeatPausesS = conf.getTimeAsSec("spark.akka.heartbeat.pauses", + "6000s") val akkaHeartBeatIntervalS = - Utils.timeStringAsSec(conf.get("spark.akka.heartbeat.interval", "1000s")) + conf.getTimeAsSec("spark.akka.heartbeat.interval", "1000s") val secretKey = securityManager.getSecretKey() val isAuthOn = securityManager.isAuthenticationEnabled() diff --git a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala index 8cc801ffd3521..aa8f970d61ba4 100644 --- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala @@ -76,7 +76,7 @@ private[spark] object MetadataCleanerType extends Enumeration { // initialization of StreamingContext. It's okay for users trying to configure stuff themselves. private[spark] object MetadataCleaner { def getDelaySeconds(conf: SparkConf): Int = { - Utils.timeStringAsSec(conf.get("spark.cleaner.ttl", "-1")).toInt + conf.getTimeAsSec("spark.cleaner.ttl", "-1").toInt } def getDelaySeconds( 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 e2920d6cf6bea..1b5f696e06175 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -613,8 +613,8 @@ private[spark] object Utils extends Logging { } Utils.setupSecureURLConnection(uc, securityMgr) - val timeoutMs = Utils.timeStringAsSec( - conf.get("spark.files.fetchTimeout", "60s")).toInt * 1000 + val timeoutMs = + conf.getTimeAsSec("spark.files.fetchTimeout", "60s").toInt * 1000 uc.setConnectTimeout(timeoutMs) uc.setReadTimeout(timeoutMs) uc.connect() diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 192db6ffa1956..49fdf5fc9dd26 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -151,7 +151,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { private val conf = new SparkConf - val LOCALITY_WAIT_MS = Utils.timeStringAsMs(conf.get("spark.locality.wait", "3s")) + val LOCALITY_WAIT_MS = conf.getTimeAsMs("spark.locality.wait", "3s") val MAX_TASK_FAILURES = 4 override def beforeEach() { 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 48bb3588d6c38..26a72c5af5056 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.util +import java.lang.NumberFormatException import java.util.concurrent.TimeUnit import scala.util.Random @@ -62,6 +63,30 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { assert(Utils.timeStringAsUs("1min") === TimeUnit.MINUTES.toMicros(1)) assert(Utils.timeStringAsUs("1h") === TimeUnit.HOURS.toMicros(1)) assert(Utils.timeStringAsUs("1d") === TimeUnit.DAYS.toMicros(1)) + + // Test invalid strings + try { + Utils.timeStringAsMs("This breaks 600s") + assert(false) // We should never reach this + } catch { + case e: NumberFormatException => assert(true) + } + + // Test invalid strings + try { + Utils.timeStringAsMs("600s This breaks") + assert(false) // We should never reach this + } catch { + case e: NumberFormatException => assert(true) + } + + // Test invalid strings + try { + Utils.timeStringAsMs("This 123s breaks") + assert(false) // We should never reach this + } catch { + case e: NumberFormatException => assert(true) + } } test("bytesToString") { diff --git a/docs/configuration.md b/docs/configuration.md index 036a648f25938..a331e377892c6 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -38,8 +38,18 @@ val sc = new SparkContext(conf) Note that we can have more than 1 thread in local mode, and in cases like spark streaming, we may actually require one to prevent any sort of starvation issues. -Properties that specify some time duration shoud be configured with a unit, e.g. `5s`, `10ms`, or -`20us`. If no units are provided, the default unit specified in the documentation is assumed. +Properties that specify some time duration should be configured with a unit of time. +The following format is accepted: + + 200us (microseconds) + 5s (seconds) + 25ms (milliseconds) + 10m or 10min (minutes) + 3h (hours) + 5d (days) + 1y (years) + +If no units are provided, the default unit specified in the documentation is assumed. ## Dynamically Loading Spark Properties In some cases, you may want to avoid hard-coding certain configurations in a `SparkConf`. For @@ -432,7 +442,7 @@ Apart from these, the following properties are also available, and may be useful spark.shuffle.io.retryWait - 5 + 5s (Netty only) Seconds to wait between retries of fetches. The maximum delay caused by retrying is simply maxRetries * retryWait, by default 15 seconds. @@ -735,14 +745,14 @@ Apart from these, the following properties are also available, and may be useful spark.executor.heartbeatInterval - 10000 + 10000ms Interval (milliseconds) between each executor's heartbeats to the driver. Heartbeats let the driver know that the executor is still alive and update it with metrics for in-progress tasks. spark.files.fetchTimeout - 60 + 60s Communication timeout to use when fetching files added through SparkContext.addFile() from the driver, in seconds. @@ -856,7 +866,7 @@ Apart from these, the following properties are also available, and may be useful spark.akka.heartbeat.interval - 1000 + 1000s This is set to a larger value to disable the transport failure detector that comes built in to Akka. It can be enabled again, if you plan to use this feature (Not recommended). A larger @@ -871,7 +881,7 @@ Apart from these, the following properties are also available, and may be useful spark.akka.heartbeat.pauses - 6000 + 6000s This is set to a larger value to disable the transport failure detector that comes built in to Akka. It can be enabled again, if you plan to use this feature (Not recommended). Acceptable heart @@ -889,7 +899,7 @@ Apart from these, the following properties are also available, and may be useful spark.akka.timeout - 100 + 100s Communication timeout between Spark nodes, in seconds. @@ -941,7 +951,7 @@ Apart from these, the following properties are also available, and may be useful spark.network.timeout - 120 + 120s Default timeout for all network interactions, in seconds. This config will be used in place of spark.core.connection.ack.wait.timeout, spark.akka.timeout, @@ -992,7 +1002,7 @@ Apart from these, the following properties are also available, and may be useful spark.locality.wait - 3000 + 3000ms Number of milliseconds to wait to launch a data-local task before giving up and launching it on a less-local node. The same wait will be used to step through multiple locality levels @@ -1027,7 +1037,7 @@ Apart from these, the following properties are also available, and may be useful spark.scheduler.maxRegisteredResourcesWaitingTime - 30000 + 30000ms Maximum amount of time to wait for resources to register before scheduling begins (in milliseconds). @@ -1057,7 +1067,7 @@ Apart from these, the following properties are also available, and may be useful spark.scheduler.revive.interval - 1000 + 1000ms The interval length for the scheduler to revive the worker resource offers to run tasks (in milliseconds). @@ -1073,7 +1083,7 @@ Apart from these, the following properties are also available, and may be useful spark.speculation.interval - 100 + 100ms How often Spark will check for tasks to speculate, in milliseconds. @@ -1130,7 +1140,7 @@ Apart from these, the following properties are also available, and may be useful spark.dynamicAllocation.executorIdleTimeout - 600 + 600s If dynamic allocation is enabled and an executor has been idle for more than this duration (in seconds), the executor will be removed. For more detail, see this @@ -1160,7 +1170,7 @@ Apart from these, the following properties are also available, and may be useful spark.dynamicAllocation.schedulerBacklogTimeout - 5 + 5s If dynamic allocation is enabled and there have been pending tasks backlogged for more than this duration (in seconds), new executors will be requested. For more detail, see this @@ -1218,7 +1228,7 @@ Apart from these, the following properties are also available, and may be useful spark.core.connection.ack.wait.timeout - 60 + 60s Number of seconds for the connection to wait for ack to occur before timing out and giving up. To avoid unwilling timeout caused by long pause like GC, @@ -1227,7 +1237,7 @@ Apart from these, the following properties are also available, and may be useful spark.core.connection.auth.wait.timeout - 30 + 30s Number of seconds for the connection to wait for authentication to occur before timing out and giving up. @@ -1350,7 +1360,7 @@ Apart from these, the following properties are also available, and may be useful Property NameDefaultMeaning spark.streaming.blockInterval - 200 + 200ms Interval (milliseconds) at which data received by Spark Streaming receivers is chunked into blocks of data before storing them in Spark. Minimum recommended - 50 ms. See the diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index d9f3eb2b74b18..f292e3324aaff 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -48,7 +48,7 @@ Most of the configs are the same for Spark on YARN as for other deployment modes spark.yarn.am.waitTime - 100000 + 100000ms In yarn-cluster mode, time in milliseconds for the application master to wait for the SparkContext to be initialized. In yarn-client mode, time for the application master to wait 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 445e58acecedb..18f3eea56b521 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,6 +22,8 @@ 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; @@ -144,16 +146,18 @@ private static long parseTimeString(String str, TimeUnit unit) { try { String suffix = ""; - for (String tail: timeSuffixes.keySet()) { - int charIdx = lower.length() - tail.length() - 1; - if (lower.endsWith(tail) && (charIdx >= 0 && Character.isDigit(lower.charAt(charIdx)))) { - suffix = tail; - break; + long val = -1; + Matcher m = Pattern.compile("([0-9]+)([a-z]+)?").matcher(lower); + if (m.matches()) { + val = Long.parseLong(m.group(1)); + if(m.groupCount() > 1) { + suffix = m.group(2); } + } else { + throw new NumberFormatException("Failed to parse time string."); } - return unit.convert(Long.parseLong(str.substring(0, str.length() - suffix.length())), - timeSuffixes.containsKey(suffix) ? timeSuffixes.get(suffix) : unit); + return unit.convert(val, timeSuffixes.containsKey(suffix) ? 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). " + diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala index 70ceae8398ef7..f4963a78e1d18 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala @@ -79,8 +79,7 @@ private[streaming] class BlockGenerator( private case class Block(id: StreamBlockId, buffer: ArrayBuffer[Any]) private val clock = new SystemClock() - private val blockIntervalMs = - Utils.timeStringAsMs(conf.get("spark.streaming.blockInterval", "200ms")) + private val blockIntervalMs = conf.getTimeAsMs("spark.streaming.blockInterval", "200ms") private val blockIntervalTimer = new RecurringTimer(clock, blockIntervalMs, updateCurrentBuffer, "BlockGenerator") private val blockQueueSize = conf.getInt("spark.streaming.blockQueueSize", 10) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index 12086a6b06e10..58e56638a2dca 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -104,8 +104,8 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { if (processReceivedData) { logInfo("Stopping JobGenerator gracefully") val timeWhenStopStarted = System.currentTimeMillis() - val stopTimeoutMs = Utils.timeStringAsMs(conf.get( - "spark.streaming.gracefulStopTimeout", s"${10 * ssc.graph.batchDuration.milliseconds}ms")) + val stopTimeoutMs = conf.getTimeAsMs( + "spark.streaming.gracefulStopTimeout", s"${10 * ssc.graph.batchDuration.milliseconds}ms") val pollTime = 100 // To prevent graceful stop to get stuck permanently diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index beb8bbfacb27a..20db078d77db0 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -75,7 +75,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) myConf.set("spark.cleaner.ttl", "10s") ssc = new StreamingContext(myConf, batchDuration) - assert(Utils.timeStringAsSec(ssc.conf.get("spark.cleaner.ttl", "-1")) === 10) + assert(ssc.conf.getTimeAsSec("spark.cleaner.ttl", "-1") === 10) } test("from existing SparkContext") { @@ -87,7 +87,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) myConf.set("spark.cleaner.ttl", "10s") ssc = new StreamingContext(myConf, batchDuration) - assert(Utils.timeStringAsSec(ssc.conf.get("spark.cleaner.ttl", "-1")) === 10) + assert(ssc.conf.getTimeAsSec("spark.cleaner.ttl", "-1") === 10) } test("from checkpoint") { @@ -101,9 +101,9 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w Utils.timeStringAsSec(cp.sparkConfPairs.toMap.getOrElse("spark.cleaner.ttl", "-1")) === 10) ssc1.stop() val newCp = Utils.deserialize[Checkpoint](Utils.serialize(cp)) - assert(Utils.timeStringAsSec(newCp.createSparkConf().get("spark.cleaner.ttl", "-1")) === 10) + assert(newCp.createSparkConf().getTimeAsSec("spark.cleaner.ttl", "-1") === 10) ssc = new StreamingContext(null, newCp, null) - assert(Utils.timeStringAsSec(ssc.conf.get("spark.cleaner.ttl", "-1")) === 10) + assert(ssc.conf.getTimeAsSec("spark.cleaner.ttl", "-1") === 10) } test("start and stop state check") { diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 84a0d6745e4ce..c433799dc6ab0 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -295,7 +295,7 @@ private[spark] class ApplicationMaster( // we want to be reasonably responsive without causing too many requests to RM. val schedulerInterval = - Utils.timeStringAsMs(sparkConf.get("spark.yarn.scheduler.heartbeat.interval-ms", "5s")) + sparkConf.getTimeAsMs("spark.yarn.scheduler.heartbeat.interval-ms", "5s") // must be <= expiryInterval / 2. val interval = math.max(0, math.min(expiryInterval / 2, schedulerInterval)) @@ -378,7 +378,8 @@ private[spark] class ApplicationMaster( logWarning( "spark.yarn.applicationMaster.waitTries is deprecated, use spark.yarn.am.waitTime") } - val totalWaitTime = sparkConf.getLong("spark.yarn.am.waitTime", waitTries.getOrElse(100000L)) + val totalWaitTime = sparkConf.getTimeAsMs("spark.yarn.am.waitTime", + s"${waitTries.getOrElse(100000L)}ms") val deadline = System.currentTimeMillis() + totalWaitTime while (sparkContextRef.get() == null && System.currentTimeMillis < deadline && !finished) { @@ -403,8 +404,8 @@ private[spark] class ApplicationMaster( // Spark driver should already be up since it launched us, but we don't want to // wait forever, so wait 100 seconds max to match the cluster mode setting. - val totalWaitTime = sparkConf.getLong("spark.yarn.am.waitTime", 100000L) - val deadline = System.currentTimeMillis + totalWaitTime + val totalWaitTimeMs = sparkConf.getTimeAsMs("spark.yarn.am.waitTime", "100s") + val deadline = System.currentTimeMillis + totalWaitTimeMs while (!driverUp && !finished && System.currentTimeMillis < deadline) { try { From 7d19cdd16fbe9914148331d542a2d636797a83a3 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 8 Apr 2015 14:16:41 -0700 Subject: [PATCH 48/58] Added fix for possible NPE --- .../java/org/apache/spark/network/util/JavaUtils.java | 10 ++++++---- 1 file changed, 6 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 18f3eea56b521..b64b0843279d1 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 @@ -150,14 +150,16 @@ private static long parseTimeString(String str, TimeUnit unit) { Matcher m = Pattern.compile("([0-9]+)([a-z]+)?").matcher(lower); if (m.matches()) { val = Long.parseLong(m.group(1)); - if(m.groupCount() > 1) { + if (m.group(2) != null) { suffix = m.group(2); } - } else { - throw new NumberFormatException("Failed to parse time string."); + else { + throw new NumberFormatException("Failed to parse time string: " + str); + } } - return unit.convert(val, timeSuffixes.containsKey(suffix) ? timeSuffixes.get(suffix) : unit); + return unit.convert(val, (suffix != null) && timeSuffixes.containsKey(suffix) ? + 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). " + From dc7bd0865e84fd6cba0755e0d7cdac873ce7dd3e Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 8 Apr 2015 14:50:35 -0700 Subject: [PATCH 49/58] Fixed error in exception handling --- .../java/org/apache/spark/network/util/JavaUtils.java | 9 +++------ 1 file changed, 3 insertions(+), 6 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 b64b0843279d1..de5fce9ac5dde 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 @@ -150,12 +150,9 @@ private static long parseTimeString(String str, TimeUnit unit) { Matcher m = Pattern.compile("([0-9]+)([a-z]+)?").matcher(lower); if (m.matches()) { val = Long.parseLong(m.group(1)); - if (m.group(2) != null) { - suffix = m.group(2); - } - else { - throw new NumberFormatException("Failed to parse time string: " + str); - } + suffix = m.group(2); + } else { + throw new NumberFormatException("Failed to parse time string: " + str); } return unit.convert(val, (suffix != null) && timeSuffixes.containsKey(suffix) ? From 69fedccd2fcd1916caf5fcf0bb08e9986b4fe2cf Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 8 Apr 2015 15:31:04 -0700 Subject: [PATCH 50/58] Added test for zero --- core/src/test/scala/org/apache/spark/util/UtilsSuite.scala | 3 +++ 1 file changed, 3 insertions(+) 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 26a72c5af5056..938c9a81a642a 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -40,6 +40,9 @@ import org.apache.spark.SparkConf class UtilsSuite extends FunSuite with ResetSystemProperties { test("timeConversion") { + // Test zero + assert(Utils.timeStringAsSec("0") === 0) + assert(Utils.timeStringAsSec("1") === 1) assert(Utils.timeStringAsSec("1s") === 1) assert(Utils.timeStringAsSec("1000ms") === 1) From 8927e66945983081eef0bf9f04f2ff5eac7bc20d Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 8 Apr 2015 18:25:49 -0700 Subject: [PATCH 51/58] Fixed handling of -1 --- core/src/test/scala/org/apache/spark/util/UtilsSuite.scala | 3 +++ .../src/main/java/org/apache/spark/network/util/JavaUtils.java | 2 +- 2 files changed, 4 insertions(+), 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 938c9a81a642a..bbe7f30ad26fa 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -40,6 +40,9 @@ import org.apache.spark.SparkConf class UtilsSuite extends FunSuite with ResetSystemProperties { test("timeConversion") { + // Test -1 + assert(Utils.timeStringAsSec("-1") === -1) + // Test zero assert(Utils.timeStringAsSec("0") === 0) 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 de5fce9ac5dde..786338db3ae9a 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 @@ -147,7 +147,7 @@ private static long parseTimeString(String str, TimeUnit unit) { try { String suffix = ""; long val = -1; - Matcher m = Pattern.compile("([0-9]+)([a-z]+)?").matcher(lower); + Matcher m = Pattern.compile("(-?[0-9]+)([a-z]+)?").matcher(lower); if (m.matches()) { val = Long.parseLong(m.group(1)); suffix = m.group(2); From 642a06db778b40da8364122137c5c671814a46b5 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 8 Apr 2015 22:23:59 -0700 Subject: [PATCH 52/58] Fixed logic for invalid suffixes and addid matching test --- .../scala/org/apache/spark/util/UtilsSuite.scala | 9 +++++++++ .../org/apache/spark/network/util/JavaUtils.java | 13 +++++++++---- 2 files changed, 18 insertions(+), 4 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 bbe7f30ad26fa..31a2dfb77f4c2 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -78,6 +78,15 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { case e: NumberFormatException => assert(true) } + // Test invalid strings + try { + Utils.timeStringAsMs("600ds") + assert(false) // We should never reach this + } catch { + case e: NumberFormatException => assert(true) + } + + // Test invalid strings try { Utils.timeStringAsMs("600s This breaks") 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 786338db3ae9a..69d696c40a060 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 @@ -145,8 +145,8 @@ private static long parseTimeString(String str, TimeUnit unit) { String lower = str.toLowerCase().trim(); try { - String suffix = ""; - long val = -1; + String suffix; + long val; Matcher m = Pattern.compile("(-?[0-9]+)([a-z]+)?").matcher(lower); if (m.matches()) { val = Long.parseLong(m.group(1)); @@ -155,8 +155,13 @@ private static long parseTimeString(String str, TimeUnit unit) { throw new NumberFormatException("Failed to parse time string: " + str); } - return unit.convert(val, (suffix != null) && timeSuffixes.containsKey(suffix) ? - timeSuffixes.get(suffix) : unit); + // Check for invalid suffixes + if(!timeSuffixes.containsKey(suffix) && suffix != null) { + throw new NumberFormatException("Invalid suffix: " + suffix); + } + + // If suffix is valid use that, otherwise none was provided and use the default passed + return unit.convert(val, timeSuffixes.containsKey(suffix) ? 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). " + From 25d3f5228297ed60b9adc43689957b4586636320 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Sat, 11 Apr 2015 07:54:37 -0700 Subject: [PATCH 53/58] Minor nit fixes --- .../main/java/org/apache/spark/network/util/JavaUtils.java | 6 +++--- 1 file changed, 3 insertions(+), 3 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 69d696c40a060..c512671fc7d37 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 @@ -156,12 +156,12 @@ private static long parseTimeString(String str, TimeUnit unit) { } // Check for invalid suffixes - if(!timeSuffixes.containsKey(suffix) && suffix != null) { - throw new NumberFormatException("Invalid suffix: " + suffix); + if (suffix != null && !timeSuffixes.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.convert(val, timeSuffixes.containsKey(suffix) ? timeSuffixes.get(suffix) : 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). " + From bc04e05a1e61dc9eb14949e4f51737ddab7e39f2 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Sun, 12 Apr 2015 20:29:14 -0700 Subject: [PATCH 54/58] Minor fixes and doc updates --- .../scala/org/apache/spark/SparkConf.scala | 8 +-- .../scala/org/apache/spark/util/Utils.scala | 11 +--- .../org/apache/spark/util/UtilsSuite.scala | 12 +---- docs/configuration.md | 51 +++++++++---------- .../apache/spark/network/util/JavaUtils.java | 10 +--- .../spark/streaming/ReceiverSuite.scala | 10 ++-- 6 files changed, 36 insertions(+), 66 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index c6ad8194625ac..1db8d85ad4bc0 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -178,28 +178,28 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { * suffix is provided then seconds are assumed. */ def getTimeAsSec(key: String): Long = { - Utils.timeStringAsSec(getOption(key).getOrElse(throw new NoSuchElementException(key))) + Utils.timeStringAsSec(get(key)) } /** Get a time parameter as seconds, falling back to a default if not set. If no * suffix is provided then seconds are assumed. */ def getTimeAsSec(key: String, defaultValue: String): Long = { - Utils.timeStringAsSec(getOption(key).getOrElse(defaultValue)) + Utils.timeStringAsSec(get(key, defaultValue)) } /** Get a time parameter as milliseconds; throws a NoSuchElementException if it's not set. If no * suffix is provided then milliseconds are assumed. */ def getTimeAsMs(key: String): Long = { - Utils.timeStringAsMs(getOption(key).getOrElse(throw new NoSuchElementException(key))) + Utils.timeStringAsMs(get(key)) } /** Get a time parameter as milliseconds, falling back to a default if not set. If no * suffix is provided then milliseconds are assumed. */ def getTimeAsMs(key: String, defaultValue: String): Long = { - Utils.timeStringAsMs(getOption(key).getOrElse(defaultValue)) + Utils.timeStringAsMs(get(key, defaultValue)) } 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 1b5f696e06175..254621f664868 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -25,8 +25,6 @@ import java.util.{Properties, Locale, Random, UUID} import java.util.concurrent._ import javax.net.ssl.HttpsURLConnection -import org.apache.spark.network.util.JavaUtils - import scala.collection.JavaConversions._ import scala.collection.Map import scala.collection.mutable.ArrayBuffer @@ -48,6 +46,7 @@ import tachyon.client.{TachyonFS, TachyonFile} import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.network.util.JavaUtils import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} /** CallSite represents a place in user code. It can have a short and a long form. */ @@ -1013,14 +1012,6 @@ private[spark] object Utils extends Logging { ) } - /** - * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If - * no suffix is provided, the passed number is assumed to be in us. - */ - def timeStringAsUs(str: String): Long = { - JavaUtils.timeStringAsUs(str) - } - /** * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If * no suffix is provided, the passed number is assumed to be in ms. 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 31a2dfb77f4c2..087ba2cf31407 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -17,15 +17,13 @@ package org.apache.spark.util -import java.lang.NumberFormatException -import java.util.concurrent.TimeUnit - import scala.util.Random import java.io.{File, ByteArrayOutputStream, ByteArrayInputStream, FileOutputStream} import java.net.{BindException, ServerSocket, URI} import java.nio.{ByteBuffer, ByteOrder} import java.text.DecimalFormatSymbols +import java.util.concurrent.TimeUnit import java.util.Locale import com.google.common.base.Charsets.UTF_8 @@ -61,14 +59,6 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { assert(Utils.timeStringAsMs("1min") === TimeUnit.MINUTES.toMillis(1)) assert(Utils.timeStringAsMs("1h") === TimeUnit.HOURS.toMillis(1)) assert(Utils.timeStringAsMs("1d") === TimeUnit.DAYS.toMillis(1)) - - assert(Utils.timeStringAsUs("1") === 1) - assert(Utils.timeStringAsUs("1us") === 1) - assert(Utils.timeStringAsUs("1ms") === TimeUnit.MILLISECONDS.toMicros(1)) - assert(Utils.timeStringAsUs("1s") === TimeUnit.SECONDS.toMicros(1)) - assert(Utils.timeStringAsUs("1min") === TimeUnit.MINUTES.toMicros(1)) - assert(Utils.timeStringAsUs("1h") === TimeUnit.HOURS.toMicros(1)) - assert(Utils.timeStringAsUs("1d") === TimeUnit.DAYS.toMicros(1)) // Test invalid strings try { diff --git a/docs/configuration.md b/docs/configuration.md index a331e377892c6..07ba9e2e20ccb 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -35,13 +35,12 @@ val conf = new SparkConf() val sc = new SparkContext(conf) {% endhighlight %} -Note that we can have more than 1 thread in local mode, and in cases like spark streaming, we may +Note that we can have more than 1 thread in local mode, and in cases like Spark Streaming, we may actually require one to prevent any sort of starvation issues. Properties that specify some time duration should be configured with a unit of time. The following format is accepted: - 200us (microseconds) 5s (seconds) 25ms (milliseconds) 10m or 10min (minutes) @@ -444,8 +443,8 @@ Apart from these, the following properties are also available, and may be useful spark.shuffle.io.retryWait 5s - (Netty only) Seconds to wait between retries of fetches. The maximum delay caused by retrying - is simply maxRetries * retryWait, by default 15 seconds. + (Netty only) How long to wait between retries of fetches. The maximum delay caused by retrying + is simply maxRetries * retryWait, by default 5 seconds. @@ -745,8 +744,8 @@ Apart from these, the following properties are also available, and may be useful spark.executor.heartbeatInterval - 10000ms - Interval (milliseconds) between each executor's heartbeats to the driver. Heartbeats let + 10s + Interval between each executor's heartbeats to the driver. Heartbeats let the driver know that the executor is still alive and update it with metrics for in-progress tasks. @@ -755,7 +754,7 @@ Apart from these, the following properties are also available, and may be useful 60s Communication timeout to use when fetching files added through SparkContext.addFile() from - the driver, in seconds. + the driver. @@ -870,7 +869,7 @@ Apart from these, the following properties are also available, and may be useful This is set to a larger value to disable the transport failure detector that comes built in to Akka. It can be enabled again, if you plan to use this feature (Not recommended). A larger - interval value in seconds reduces network overhead and a smaller value ( ~ 1 s) might be more + interval value reduces network overhead and a smaller value ( ~ 1 s) might be more informative for Akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses` if you need to. A likely positive use case for using failure detector would be: a sensistive failure detector can help evict rogue executors quickly. However this is usually not the case @@ -885,7 +884,7 @@ Apart from these, the following properties are also available, and may be useful This is set to a larger value to disable the transport failure detector that comes built in to Akka. It can be enabled again, if you plan to use this feature (Not recommended). Acceptable heart - beat pause in seconds for Akka. This can be used to control sensitivity to GC pauses. Tune + beat pause for Akka. This can be used to control sensitivity to GC pauses. Tune this along with `spark.akka.heartbeat.interval` if you need to. @@ -901,7 +900,7 @@ Apart from these, the following properties are also available, and may be useful spark.akka.timeout 100s - Communication timeout between Spark nodes, in seconds. + Communication timeout between Spark nodes. @@ -953,8 +952,8 @@ Apart from these, the following properties are also available, and may be useful spark.network.timeout 120s - Default timeout for all network interactions, in seconds. This config will be used in - place of spark.core.connection.ack.wait.timeout, spark.akka.timeout, + Default timeout for all network interactions. This config will be used in place of + spark.core.connection.ack.wait.timeout, spark.akka.timeout, spark.storage.blockManagerSlaveTimeoutMs or spark.shuffle.io.connectionTimeout, if they are not configured. @@ -1002,9 +1001,9 @@ Apart from these, the following properties are also available, and may be useful spark.locality.wait - 3000ms + 3s - Number of milliseconds to wait to launch a data-local task before giving up and launching it + How long to wait to launch a data-local task before giving up and launching it on a less-local node. The same wait will be used to step through multiple locality levels (process-local, node-local, rack-local and then any). It is also possible to customize the waiting time for each level by setting spark.locality.wait.node, etc. @@ -1037,10 +1036,9 @@ Apart from these, the following properties are also available, and may be useful spark.scheduler.maxRegisteredResourcesWaitingTime - 30000ms + 30s - Maximum amount of time to wait for resources to register before scheduling begins - (in milliseconds). + Maximum amount of time to wait for resources to register before scheduling begins. @@ -1067,10 +1065,9 @@ Apart from these, the following properties are also available, and may be useful spark.scheduler.revive.interval - 1000ms + 1s - The interval length for the scheduler to revive the worker resource offers to run tasks - (in milliseconds). + The interval length for the scheduler to revive the worker resource offers to run tasks. @@ -1085,7 +1082,7 @@ Apart from these, the following properties are also available, and may be useful spark.speculation.interval 100ms - How often Spark will check for tasks to speculate, in milliseconds. + How often Spark will check for tasks to speculate. @@ -1142,8 +1139,8 @@ Apart from these, the following properties are also available, and may be useful spark.dynamicAllocation.executorIdleTimeout 600s - If dynamic allocation is enabled and an executor has been idle for more than this duration - (in seconds), the executor will be removed. For more detail, see this + If dynamic allocation is enabled and an executor has been idle for more than this duration, + the executor will be removed. For more detail, see this description. @@ -1173,7 +1170,7 @@ Apart from these, the following properties are also available, and may be useful 5s If dynamic allocation is enabled and there have been pending tasks backlogged for more than - this duration (in seconds), new executors will be requested. For more detail, see this + this duration, new executors will be requested. For more detail, see this description. @@ -1230,7 +1227,7 @@ Apart from these, the following properties are also available, and may be useful spark.core.connection.ack.wait.timeout 60s - Number of seconds for the connection to wait for ack to occur before timing + How long for the connection to wait for ack to occur before timing out and giving up. To avoid unwilling timeout caused by long pause like GC, you can set larger value. @@ -1239,7 +1236,7 @@ Apart from these, the following properties are also available, and may be useful spark.core.connection.auth.wait.timeout 30s - Number of seconds for the connection to wait for authentication to occur before timing + How long for the connection to wait for authentication to occur before timing out and giving up. @@ -1362,7 +1359,7 @@ Apart from these, the following properties are also available, and may be useful spark.streaming.blockInterval 200ms - Interval (milliseconds) at which data received by Spark Streaming receivers is chunked + Interval at which data received by Spark Streaming receivers is chunked into blocks of data before storing them in Spark. Minimum recommended - 50 ms. See the performance tuning section in the Spark Streaming programing guide for more details. 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 c512671fc7d37..b6fbace509a0e 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 @@ -170,15 +170,7 @@ private static long parseTimeString(String str, TimeUnit unit) { throw new NumberFormatException(timeError + "\n" + e.getMessage()); } } - - /** - * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If - * no suffix is provided, the passed number is assumed to be in us. - */ - public static long timeStringAsUs(String str) { - return parseTimeString(str, TimeUnit.MICROSECONDS); - } - + /** * Convert a time parameter such as (50s, 100ms, or 250us) to milliseconds for internal use. If * no suffix is provided, the passed number is assumed to be in ms. diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala index ffd02e79acaeb..8618d9f3f0382 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala @@ -132,7 +132,7 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { test("block generator") { val blockGeneratorListener = new FakeBlockGeneratorListener val blockIntervalMs = 200 - val conf = new SparkConf().set("spark.streaming.blockInterval", s"${blockIntervalMs.toString}ms") + val conf = new SparkConf().set("spark.streaming.blockInterval", s"${blockIntervalMs}ms") val blockGenerator = new BlockGenerator(blockGeneratorListener, 1, conf) val expectedBlocks = 5 val waitTime = expectedBlocks * blockIntervalMs + (blockIntervalMs / 2) @@ -157,15 +157,15 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { test("block generator throttling") { val blockGeneratorListener = new FakeBlockGeneratorListener - val blockInterval = 100 + val blockIntervalMs = 100 val maxRate = 100 - val conf = new SparkConf().set("spark.streaming.blockInterval", s"${blockInterval.toString}ms"). + val conf = new SparkConf().set("spark.streaming.blockInterval", s"${blockIntervalMs}ms"). set("spark.streaming.receiver.maxRate", maxRate.toString) val blockGenerator = new BlockGenerator(blockGeneratorListener, 1, conf) val expectedBlocks = 20 - val waitTime = expectedBlocks * blockInterval + val waitTime = expectedBlocks * blockIntervalMs val expectedMessages = maxRate * waitTime / 1000 - val expectedMessagesPerBlock = maxRate * blockInterval / 1000 + val expectedMessagesPerBlock = maxRate * blockIntervalMs / 1000 val generatedData = new ArrayBuffer[Int] // Generate blocks From 951ca2d150e2d0b046573d65bebcb0e15e158c36 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Mon, 13 Apr 2015 11:24:43 -0700 Subject: [PATCH 55/58] Made the most recent round of changes --- .../spark/ExecutorAllocationManager.scala | 6 +-- .../org/apache/spark/HeartbeatReceiver.scala | 4 +- .../scala/org/apache/spark/SparkConf.scala | 39 +++++++++------- .../spark/network/nio/ConnectionManager.scala | 2 +- .../spark/scheduler/TaskSetManager.scala | 9 +++- .../org/apache/spark/util/AkkaUtils.scala | 8 ++-- .../apache/spark/util/MetadataCleaner.scala | 2 +- .../scala/org/apache/spark/util/Utils.scala | 6 +-- .../org/apache/spark/util/UtilsSuite.scala | 46 +++++++------------ docs/configuration.md | 4 +- .../streaming/StreamingContextSuite.scala | 10 ++-- 11 files changed, 65 insertions(+), 71 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 7977b0e6b5deb..4e7bf51fc0622 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -80,15 +80,15 @@ private[spark] class ExecutorAllocationManager( Integer.MAX_VALUE) // How long there must be backlogged tasks for before an addition is triggered (seconds) - private val schedulerBacklogTimeoutS = conf.getTimeAsSec( + private val schedulerBacklogTimeoutS = conf.getTimeAsSeconds( "spark.dynamicAllocation.schedulerBacklogTimeout", "5s") // Same as above, but used only after `schedulerBacklogTimeoutS` is exceeded - private val sustainedSchedulerBacklogTimeoutS = conf.getTimeAsSec( + private val sustainedSchedulerBacklogTimeoutS = conf.getTimeAsSeconds( "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", s"${schedulerBacklogTimeoutS}s") // How long an executor must be idle for before it is removed (seconds) - private val executorIdleTimeoutS = conf.getTimeAsSec( + private val executorIdleTimeoutS = conf.getTimeAsSeconds( "spark.dynamicAllocation.executorIdleTimeout", "600s") // During testing, the methods to actually kill and add executors are mocked out diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index e19a3707e02be..e3bd16f1cbf24 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -65,14 +65,14 @@ private[spark] class HeartbeatReceiver(sc: SparkContext) private val slaveTimeoutMs = sc.conf.getTimeAsMs("spark.storage.blockManagerSlaveTimeoutMs", "120s") private val executorTimeoutMs = - sc.conf.getTimeAsSec("spark.network.timeout", s"${slaveTimeoutMs}ms") * 1000 + sc.conf.getTimeAsSeconds("spark.network.timeout", s"${slaveTimeoutMs}ms") * 1000 // "spark.network.timeoutInterval" uses "seconds", while // "spark.storage.blockManagerTimeoutIntervalMs" uses "milliseconds" private val timeoutIntervalMs = sc.conf.getTimeAsMs("spark.storage.blockManagerTimeoutIntervalMs", "60s") private val checkTimeoutIntervalMs = - sc.conf.getTimeAsSec("spark.network.timeoutInterval", s"${timeoutIntervalMs}ms") * 1000 + sc.conf.getTimeAsSeconds("spark.network.timeoutInterval", s"${timeoutIntervalMs}ms") * 1000 private var timeoutCheckingTask: ScheduledFuture[_] = null diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 1db8d85ad4bc0..390e631647bd6 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -174,30 +174,37 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { getOption(key).getOrElse(defaultValue) } - /** Get a time parameter as seconds; throws a NoSuchElementException if it's not set. If no - * suffix is provided then seconds are assumed. - */ - def getTimeAsSec(key: String): Long = { - Utils.timeStringAsSec(get(key)) + /** + * Get a time parameter as seconds; throws a NoSuchElementException if it's not set. If no + * suffix is provided then seconds are assumed. + * @throws NoSuchElementException + */ + def getTimeAsSeconds(key: String): Long = { + Utils.timeStringAsSeconds(get(key)) } - /** Get a time parameter as seconds, falling back to a default if not set. If no - * suffix is provided then seconds are assumed. - */ - def getTimeAsSec(key: String, defaultValue: String): Long = { - Utils.timeStringAsSec(get(key, defaultValue)) + /** + * Get a time parameter as seconds, falling back to a default if not set. If no + * suffix is provided then seconds are assumed. + * + */ + def getTimeAsSeconds(key: String, defaultValue: String): Long = { + Utils.timeStringAsSeconds(get(key, defaultValue)) } - /** Get a time parameter as milliseconds; throws a NoSuchElementException if it's not set. If no - * suffix is provided then milliseconds are assumed. - */ + /** + * Get a time parameter as milliseconds; throws a NoSuchElementException if it's not set. If no + * suffix is provided then milliseconds are assumed. + * @throws NoSuchElementException + */ def getTimeAsMs(key: String): Long = { Utils.timeStringAsMs(get(key)) } - /** Get a time parameter as milliseconds, falling back to a default if not set. If no - * suffix is provided then milliseconds are assumed. - */ + /** + * Get a time parameter as milliseconds, falling back to a default if not set. If no + * suffix is provided then milliseconds are assumed. + */ def getTimeAsMs(key: String, defaultValue: String): Long = { Utils.timeStringAsMs(get(key, defaultValue)) } diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala index 9f6be1e2fca55..8e3c30fc3d781 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala @@ -82,7 +82,7 @@ private[nio] class ConnectionManager( new HashedWheelTimer(Utils.namedThreadFactory("AckTimeoutMonitor")) private val ackTimeout = - conf.getTimeAsSec("spark.core.connection.ack.wait.timeout", + conf.getTimeAsSeconds("spark.core.connection.ack.wait.timeout", conf.get("spark.network.timeout", "120s")) // Get the thread counts from the Spark Configuration. diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index ebfb3c8a98bdf..7dc325283d961 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -853,9 +853,14 @@ private[spark] class TaskSetManager( case TaskLocality.PROCESS_LOCAL => "spark.locality.wait.process" case TaskLocality.NODE_LOCAL => "spark.locality.wait.node" case TaskLocality.RACK_LOCAL => "spark.locality.wait.rack" - case _ => "" + case _ => null + } + + if (localityWaitKey != null) { + conf.getTimeAsMs(localityWaitKey, defaultWait) + } else { + 0L } - conf.getTimeAsMs(localityWaitKey, defaultWait) } /** diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 137f897c9e2ce..8e8cc7cc6389e 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -65,7 +65,7 @@ private[spark] object AkkaUtils extends Logging { val akkaThreads = conf.getInt("spark.akka.threads", 4) val akkaBatchSize = conf.getInt("spark.akka.batchSize", 15) - val akkaTimeoutS = conf.getTimeAsSec("spark.akka.timeout", + val akkaTimeoutS = conf.getTimeAsSeconds("spark.akka.timeout", conf.get("spark.network.timeout", "120s")) val akkaFrameSize = maxFrameSizeBytes(conf) val akkaLogLifecycleEvents = conf.getBoolean("spark.akka.logLifecycleEvents", false) @@ -78,10 +78,8 @@ private[spark] object AkkaUtils extends Logging { val logAkkaConfig = if (conf.getBoolean("spark.akka.logAkkaConfig", false)) "on" else "off" - val akkaHeartBeatPausesS = conf.getTimeAsSec("spark.akka.heartbeat.pauses", - "6000s") - val akkaHeartBeatIntervalS = - conf.getTimeAsSec("spark.akka.heartbeat.interval", "1000s") + val akkaHeartBeatPausesS = conf.getTimeAsSeconds("spark.akka.heartbeat.pauses", "6000s") + val akkaHeartBeatIntervalS = conf.getTimeAsSeconds("spark.akka.heartbeat.interval", "1000s") val secretKey = securityManager.getSecretKey() val isAuthOn = securityManager.isAuthenticationEnabled() diff --git a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala index aa8f970d61ba4..2bbfc988a99a8 100644 --- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala @@ -76,7 +76,7 @@ private[spark] object MetadataCleanerType extends Enumeration { // initialization of StreamingContext. It's okay for users trying to configure stuff themselves. private[spark] object MetadataCleaner { def getDelaySeconds(conf: SparkConf): Int = { - conf.getTimeAsSec("spark.cleaner.ttl", "-1").toInt + conf.getTimeAsSeconds("spark.cleaner.ttl", "-1").toInt } def getDelaySeconds( 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 254621f664868..214524cc4f285 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -613,7 +613,7 @@ private[spark] object Utils extends Logging { Utils.setupSecureURLConnection(uc, securityMgr) val timeoutMs = - conf.getTimeAsSec("spark.files.fetchTimeout", "60s").toInt * 1000 + conf.getTimeAsSeconds("spark.files.fetchTimeout", "60s").toInt * 1000 uc.setConnectTimeout(timeoutMs) uc.setReadTimeout(timeoutMs) uc.connect() @@ -1016,7 +1016,7 @@ private[spark] object Utils extends Logging { * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If * no suffix is provided, the passed number is assumed to be in ms. */ - def timeStringAsMs(str: String) : Long = { + def timeStringAsMs(str: String): Long = { JavaUtils.timeStringAsMs(str) } @@ -1024,7 +1024,7 @@ private[spark] object Utils extends Logging { * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If * no suffix is provided, the passed number is assumed to be in seconds. */ - def timeStringAsSec(str: String) : Long = { + def timeStringAsSeconds(str: String): Long = { JavaUtils.timeStringAsSec(str) } 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 087ba2cf31407..dd18d573c93f1 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -39,58 +39,44 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { test("timeConversion") { // Test -1 - assert(Utils.timeStringAsSec("-1") === -1) + assert(Utils.timeStringAsSeconds("-1") === -1) // Test zero - assert(Utils.timeStringAsSec("0") === 0) + assert(Utils.timeStringAsSeconds("0") === 0) - assert(Utils.timeStringAsSec("1") === 1) - assert(Utils.timeStringAsSec("1s") === 1) - assert(Utils.timeStringAsSec("1000ms") === 1) - assert(Utils.timeStringAsSec("1000000us") === 1) - assert(Utils.timeStringAsSec("1min") === TimeUnit.MINUTES.toSeconds(1)) - assert(Utils.timeStringAsSec("1h") === TimeUnit.HOURS.toSeconds(1)) - assert(Utils.timeStringAsSec("1d") === TimeUnit.DAYS.toSeconds(1)) + assert(Utils.timeStringAsSeconds("1") === 1) + assert(Utils.timeStringAsSeconds("1s") === 1) + assert(Utils.timeStringAsSeconds("1000ms") === 1) + assert(Utils.timeStringAsSeconds("1000000us") === 1) + assert(Utils.timeStringAsSeconds("1m") === TimeUnit.MINUTES.toSeconds(1)) + assert(Utils.timeStringAsSeconds("1min") === TimeUnit.MINUTES.toSeconds(1)) + assert(Utils.timeStringAsSeconds("1h") === TimeUnit.HOURS.toSeconds(1)) + assert(Utils.timeStringAsSeconds("1d") === TimeUnit.DAYS.toSeconds(1)) assert(Utils.timeStringAsMs("1") === 1) assert(Utils.timeStringAsMs("1ms") === 1) assert(Utils.timeStringAsMs("1000us") === 1) assert(Utils.timeStringAsMs("1s") === TimeUnit.SECONDS.toMillis(1)) + assert(Utils.timeStringAsMs("1m") === TimeUnit.MINUTES.toMillis(1)) assert(Utils.timeStringAsMs("1min") === TimeUnit.MINUTES.toMillis(1)) assert(Utils.timeStringAsMs("1h") === TimeUnit.HOURS.toMillis(1)) assert(Utils.timeStringAsMs("1d") === TimeUnit.DAYS.toMillis(1)) // Test invalid strings - try { + intercept[NumberFormatException] { Utils.timeStringAsMs("This breaks 600s") - assert(false) // We should never reach this - } catch { - case e: NumberFormatException => assert(true) } - // Test invalid strings - try { - Utils.timeStringAsMs("600ds") - assert(false) // We should never reach this - } catch { - case e: NumberFormatException => assert(true) + intercept[NumberFormatException] { + Utils.timeStringAsMs("This breaks 600ds") } - - // Test invalid strings - try { + intercept[NumberFormatException] { Utils.timeStringAsMs("600s This breaks") - assert(false) // We should never reach this - } catch { - case e: NumberFormatException => assert(true) } - // Test invalid strings - try { + intercept[NumberFormatException] { Utils.timeStringAsMs("This 123s breaks") - assert(false) // We should never reach this - } catch { - case e: NumberFormatException => assert(true) } } diff --git a/docs/configuration.md b/docs/configuration.md index 07ba9e2e20ccb..5150a2a7a7465 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -48,8 +48,6 @@ The following format is accepted: 5d (days) 1y (years) -If no units are provided, the default unit specified in the documentation is assumed. - ## 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 @@ -444,7 +442,7 @@ Apart from these, the following properties are also available, and may be useful 5s (Netty only) How long to wait between retries of fetches. The maximum delay caused by retrying - is simply maxRetries * retryWait, by default 5 seconds. + is 15 seconds by default, calculated as maxRetries * retryWait. diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index 20db078d77db0..6a4829955a592 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -75,7 +75,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) myConf.set("spark.cleaner.ttl", "10s") ssc = new StreamingContext(myConf, batchDuration) - assert(ssc.conf.getTimeAsSec("spark.cleaner.ttl", "-1") === 10) + assert(ssc.conf.getTimeAsSeconds("spark.cleaner.ttl", "-1") === 10) } test("from existing SparkContext") { @@ -87,7 +87,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) myConf.set("spark.cleaner.ttl", "10s") ssc = new StreamingContext(myConf, batchDuration) - assert(ssc.conf.getTimeAsSec("spark.cleaner.ttl", "-1") === 10) + assert(ssc.conf.getTimeAsSeconds("spark.cleaner.ttl", "-1") === 10) } test("from checkpoint") { @@ -98,12 +98,12 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w ssc1.start() val cp = new Checkpoint(ssc1, Time(1000)) assert( - Utils.timeStringAsSec(cp.sparkConfPairs.toMap.getOrElse("spark.cleaner.ttl", "-1")) === 10) + Utils.timeStringAsSeconds(cp.sparkConfPairs.toMap.getOrElse("spark.cleaner.ttl", "-1")) === 10) ssc1.stop() val newCp = Utils.deserialize[Checkpoint](Utils.serialize(cp)) - assert(newCp.createSparkConf().getTimeAsSec("spark.cleaner.ttl", "-1") === 10) + assert(newCp.createSparkConf().getTimeAsSeconds("spark.cleaner.ttl", "-1") === 10) ssc = new StreamingContext(null, newCp, null) - assert(ssc.conf.getTimeAsSec("spark.cleaner.ttl", "-1") === 10) + assert(ssc.conf.getTimeAsSeconds("spark.cleaner.ttl", "-1") === 10) } test("start and stop state check") { From f5fafcd1ad7c1fc383a387b69f5d2db7a1bab799 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Mon, 13 Apr 2015 11:38:42 -0700 Subject: [PATCH 56/58] Doc updates --- docs/configuration.md | 4 +++- docs/running-on-yarn.md | 4 ++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 5150a2a7a7465..d2325f7cdc9c4 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -41,12 +41,14 @@ actually require one to prevent any sort of starvation issues. Properties that specify some time duration should be configured with a unit of time. The following format is accepted: - 5s (seconds) 25ms (milliseconds) + 5s (seconds) 10m or 10min (minutes) 3h (hours) 5d (days) 1y (years) + +If no units are provided, the default unit specified in the documentation is assumed. ## Dynamically Loading Spark Properties In some cases, you may want to avoid hard-coding certain configurations in a `SparkConf`. For diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index f292e3324aaff..7f21801f1eeb5 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -48,9 +48,9 @@ Most of the configs are the same for Spark on YARN as for other deployment modes spark.yarn.am.waitTime - 100000ms + 100s - In yarn-cluster mode, time in milliseconds for the application master to wait for the + In yarn-cluster mode, time for the application master to wait for the SparkContext to be initialized. In yarn-client mode, time for the application master to wait for the driver to connect to it. From de3bff9995f108cdd6183a4fe36b4c5781d861c2 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Mon, 13 Apr 2015 11:57:40 -0700 Subject: [PATCH 57/58] Fixing style errors --- .../org/apache/spark/streaming/StreamingContextSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index 6a4829955a592..924e2a6b0c508 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -98,7 +98,8 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w ssc1.start() val cp = new Checkpoint(ssc1, Time(1000)) assert( - Utils.timeStringAsSeconds(cp.sparkConfPairs.toMap.getOrElse("spark.cleaner.ttl", "-1")) === 10) + Utils.timeStringAsSeconds(cp.sparkConfPairs + .toMap.getOrElse("spark.cleaner.ttl", "-1")) === 10) ssc1.stop() val newCp = Utils.deserialize[Checkpoint](Utils.serialize(cp)) assert(newCp.createSparkConf().getTimeAsSeconds("spark.cleaner.ttl", "-1") === 10) From 4526c817fd68bc24b7534d4fed285fea84b2a2fa Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Mon, 13 Apr 2015 16:10:59 -0700 Subject: [PATCH 58/58] Update configuration.md --- docs/configuration.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index d2325f7cdc9c4..7169ec295ef7f 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -48,8 +48,6 @@ The following format is accepted: 5d (days) 1y (years) -If no units are provided, the default unit specified in the documentation is assumed. - ## 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