From 4180993fdfd5643e5c1a37820266fb6abd626c84 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 3 Sep 2014 17:23:31 +0900 Subject: [PATCH 01/42] Modified SparkContext to retain spark.unique.app.name property in SparkConf --- core/src/main/scala/org/apache/spark/SparkContext.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 9ba21cfcde01..b738f5661d75 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -186,6 +186,8 @@ class SparkContext(config: SparkConf) extends Logging { val master = conf.get("spark.master") val appName = conf.get("spark.app.name") + val uniqueAppName = appName + "-" + System.currentTimeMillis() + conf.set("spark.unique.app.name", uniqueAppName) // Generate the random name for a temp folder in Tachyon // Add a timestamp as the suffix here to make it more safe From 55debab06be171c094ece4ebc5fd9e1bb76b6b00 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 3 Sep 2014 17:25:56 +0900 Subject: [PATCH 02/42] Modified SparkContext and Executor to set spark.executor.id to identifiers --- core/src/main/scala/org/apache/spark/SparkContext.scala | 1 + core/src/main/scala/org/apache/spark/executor/Executor.scala | 1 + 2 files changed, 2 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index b738f5661d75..5c3b9919ecc0 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -202,6 +202,7 @@ class SparkContext(config: SparkConf) extends Logging { private[spark] val listenerBus = new LiveListenerBus // Create the Spark execution environment (cache, map output tracker, etc) + conf.set("spark.executor.id", "driver") private[spark] val env = SparkEnv.create( conf, "", 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 1bb1b4aae91b..0f7e34ecaea6 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -82,6 +82,7 @@ private[spark] class Executor( val executorSource = new ExecutorSource(this, executorId) // Initialize Spark environment (using system properties read above) + conf.set("spark.executor.id", "executor." + executorId) private val env = { if (!isLocal) { val _env = SparkEnv.create(conf, executorId, slaveHostname, 0, From 71609f57ff14f71ef62be74e737416789605e618 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 3 Sep 2014 17:30:12 +0900 Subject: [PATCH 03/42] Modified sourceName of ExecutorSource, DAGSchedulerSource and BlockManagerSource --- .../main/scala/org/apache/spark/executor/ExecutorSource.scala | 2 +- .../scala/org/apache/spark/scheduler/DAGSchedulerSource.scala | 2 +- .../scala/org/apache/spark/storage/BlockManagerSource.scala | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala index 0ed52cfe9df6..dc7e1a8e5468 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala @@ -37,7 +37,7 @@ private[spark] class ExecutorSource(val executor: Executor, executorId: String) val metricRegistry = new MetricRegistry() // TODO: It would be nice to pass the application name here - val sourceName = "executor.%s".format(executorId) + val sourceName = "executor" // Gauge for executor thread pool's actively executing task counts metricRegistry.register(MetricRegistry.name("threadpool", "activeTasks"), new Gauge[Int] { diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala index 5878e733908f..84b2a6174407 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala @@ -25,7 +25,7 @@ import org.apache.spark.metrics.source.Source private[spark] class DAGSchedulerSource(val dagScheduler: DAGScheduler, sc: SparkContext) extends Source { val metricRegistry = new MetricRegistry() - val sourceName = "%s.DAGScheduler".format(sc.appName) + val sourceName = "DAGScheduler" metricRegistry.register(MetricRegistry.name("stage", "failedStages"), new Gauge[Int] { override def getValue: Int = dagScheduler.failedStages.size diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala index 3f14c40ec61c..9bd0d8f39638 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala @@ -25,7 +25,7 @@ import org.apache.spark.metrics.source.Source private[spark] class BlockManagerSource(val blockManager: BlockManager, sc: SparkContext) extends Source { val metricRegistry = new MetricRegistry() - val sourceName = "%s.BlockManager".format(sc.appName) + val sourceName = "BlockManager" metricRegistry.register(MetricRegistry.name("memory", "maxMem_MB"), new Gauge[Long] { override def getValue: Long = { From 868e326f82a86001399842ba4dbfdb51203c0fd3 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 3 Sep 2014 17:33:26 +0900 Subject: [PATCH 04/42] Modified MetricsSystem to set registry name with unique application-id and driver/executor-id --- .../apache/spark/metrics/MetricsSystem.scala | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index 6ef817d0e587..5b496d546b2b 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -95,10 +95,24 @@ private[spark] class MetricsSystem private (val instance: String, sinks.foreach(_.report()) } + def buildRegistryName(source: Source) = { + val appName = conf.get("spark.unique.app.name") + val executorId = conf.get("spark.executor.id") + val registryName = { + if (appName != null && executorId != null) { + MetricRegistry.name(appName, executorId, source.sourceName) + } else { + MetricRegistry.name(source.sourceName) + } + } + registryName + } + def registerSource(source: Source) { sources += source try { - registry.register(source.sourceName, source.metricRegistry) + val regName = buildRegistryName(source) + registry.register(regName, source.metricRegistry) } catch { case e: IllegalArgumentException => logInfo("Metrics already registered", e) } @@ -106,8 +120,9 @@ private[spark] class MetricsSystem private (val instance: String, def removeSource(source: Source) { sources -= source + val regName = buildRegistryName(source) registry.removeMatching(new MetricFilter { - def matches(name: String, metric: Metric): Boolean = name.startsWith(source.sourceName) + def matches(name: String, metric: Metric): Boolean = name.startsWith(regName) }) } From 85ffc0244398b26365bcda751496a35bff3d3080 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 4 Sep 2014 00:12:22 +0900 Subject: [PATCH 05/42] Revert "Modified sourceName of ExecutorSource, DAGSchedulerSource and BlockManagerSource" This reverts commit 71609f57ff14f71ef62be74e737416789605e618. --- .../main/scala/org/apache/spark/executor/ExecutorSource.scala | 2 +- .../scala/org/apache/spark/scheduler/DAGSchedulerSource.scala | 2 +- .../scala/org/apache/spark/storage/BlockManagerSource.scala | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala index dc7e1a8e5468..0ed52cfe9df6 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala @@ -37,7 +37,7 @@ private[spark] class ExecutorSource(val executor: Executor, executorId: String) val metricRegistry = new MetricRegistry() // TODO: It would be nice to pass the application name here - val sourceName = "executor" + val sourceName = "executor.%s".format(executorId) // Gauge for executor thread pool's actively executing task counts metricRegistry.register(MetricRegistry.name("threadpool", "activeTasks"), new Gauge[Int] { diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala index 84b2a6174407..5878e733908f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala @@ -25,7 +25,7 @@ import org.apache.spark.metrics.source.Source private[spark] class DAGSchedulerSource(val dagScheduler: DAGScheduler, sc: SparkContext) extends Source { val metricRegistry = new MetricRegistry() - val sourceName = "DAGScheduler" + val sourceName = "%s.DAGScheduler".format(sc.appName) metricRegistry.register(MetricRegistry.name("stage", "failedStages"), new Gauge[Int] { override def getValue: Int = dagScheduler.failedStages.size diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala index 9bd0d8f39638..3f14c40ec61c 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala @@ -25,7 +25,7 @@ import org.apache.spark.metrics.source.Source private[spark] class BlockManagerSource(val blockManager: BlockManager, sc: SparkContext) extends Source { val metricRegistry = new MetricRegistry() - val sourceName = "BlockManager" + val sourceName = "%s.BlockManager".format(sc.appName) metricRegistry.register(MetricRegistry.name("memory", "maxMem_MB"), new Gauge[Long] { override def getValue: Long = { From 6fc5560846bbd99ebcbc916bbc840cadb8db7849 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 4 Sep 2014 00:17:23 +0900 Subject: [PATCH 06/42] Modified sourceName of ExecutorSource, DAGSchedulerSource and BlockManagerSource --- .../main/scala/org/apache/spark/executor/ExecutorSource.scala | 3 +-- .../scala/org/apache/spark/scheduler/DAGSchedulerSource.scala | 2 +- .../scala/org/apache/spark/storage/BlockManagerSource.scala | 2 +- 3 files changed, 3 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala index d6721586566c..c4d73622c472 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala @@ -37,8 +37,7 @@ private[spark] class ExecutorSource(val executor: Executor, executorId: String) override val metricRegistry = new MetricRegistry() - // TODO: It would be nice to pass the application name here - override val sourceName = "executor.%s".format(executorId) + override val sourceName = "executor" // Gauge for executor thread pool's actively executing task counts metricRegistry.register(MetricRegistry.name("threadpool", "activeTasks"), new Gauge[Int] { diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala index 94944399b134..915d44250608 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala @@ -25,7 +25,7 @@ import org.apache.spark.metrics.source.Source private[spark] class DAGSchedulerSource(val dagScheduler: DAGScheduler, sc: SparkContext) extends Source { override val metricRegistry = new MetricRegistry() - override val sourceName = "%s.DAGScheduler".format(sc.appName) + override val sourceName = "DAGScheduler" metricRegistry.register(MetricRegistry.name("stage", "failedStages"), new Gauge[Int] { override def getValue: Int = dagScheduler.failedStages.size diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala index 49fea6d9e2a7..8d322455d0db 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala @@ -25,7 +25,7 @@ import org.apache.spark.metrics.source.Source private[spark] class BlockManagerSource(val blockManager: BlockManager, sc: SparkContext) extends Source { override val metricRegistry = new MetricRegistry() - override val sourceName = "%s.BlockManager".format(sc.appName) + override val sourceName = "BlockManager" metricRegistry.register(MetricRegistry.name("memory", "maxMem_MB"), new Gauge[Long] { override def getValue: Long = { From 6f7dcd44bb07da4ee0686cf0b1994c97305ecee3 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 4 Sep 2014 01:01:45 +0900 Subject: [PATCH 07/42] Modified constructor of DAGSchedulerSource and BlockManagerSource because the instance of SparkContext is no longer used --- core/src/main/scala/org/apache/spark/SparkContext.scala | 4 ++-- .../scala/org/apache/spark/scheduler/DAGSchedulerSource.scala | 2 +- .../scala/org/apache/spark/storage/BlockManagerSource.scala | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index c761f9e3ca3f..5eaf5be93a37 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -408,8 +408,8 @@ class SparkContext(config: SparkConf) extends Logging { // Post init taskScheduler.postStartHook() - private val dagSchedulerSource = new DAGSchedulerSource(this.dagScheduler, this) - private val blockManagerSource = new BlockManagerSource(SparkEnv.get.blockManager, this) + private val dagSchedulerSource = new DAGSchedulerSource(this.dagScheduler) + private val blockManagerSource = new BlockManagerSource(SparkEnv.get.blockManager) private def initDriverMetrics() { SparkEnv.get.metricsSystem.registerSource(dagSchedulerSource) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala index 915d44250608..12668b6c0988 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala @@ -22,7 +22,7 @@ import com.codahale.metrics.{Gauge,MetricRegistry} import org.apache.spark.SparkContext import org.apache.spark.metrics.source.Source -private[spark] class DAGSchedulerSource(val dagScheduler: DAGScheduler, sc: SparkContext) +private[spark] class DAGSchedulerSource(val dagScheduler: DAGScheduler) extends Source { override val metricRegistry = new MetricRegistry() override val sourceName = "DAGScheduler" diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala index 8d322455d0db..8569c6f3cbbc 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala @@ -22,7 +22,7 @@ import com.codahale.metrics.{Gauge,MetricRegistry} import org.apache.spark.SparkContext import org.apache.spark.metrics.source.Source -private[spark] class BlockManagerSource(val blockManager: BlockManager, sc: SparkContext) +private[spark] class BlockManagerSource(val blockManager: BlockManager) extends Source { override val metricRegistry = new MetricRegistry() override val sourceName = "BlockManager" From 15f88a390fc78181e88e030f47903c0576b664bd Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 4 Sep 2014 03:21:11 +0900 Subject: [PATCH 08/42] Modified MetricsSystem#buildRegistryName because conf.get does not return null when correspondin entry is absent --- .../scala/org/apache/spark/metrics/MetricsSystem.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index 5b496d546b2b..4cc8a3b4c4d5 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -96,11 +96,11 @@ private[spark] class MetricsSystem private (val instance: String, } def buildRegistryName(source: Source) = { - val appName = conf.get("spark.unique.app.name") - val executorId = conf.get("spark.executor.id") + val appNameOpt = conf.getOption("spark.unique.app.name") + val executorIdOpt = conf.getOption("spark.executor.id") val registryName = { - if (appName != null && executorId != null) { - MetricRegistry.name(appName, executorId, source.sourceName) + if (appNameOpt.isDefined && executorIdOpt.isDefined) { + MetricRegistry.name(appNameOpt.get, executorIdOpt.get, source.sourceName) } else { MetricRegistry.name(source.sourceName) } From e4a4593fd52da8a15770b8c3322f39b0f1363dbf Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 4 Sep 2014 15:04:33 +0900 Subject: [PATCH 09/42] tmp --- .../scala/org/apache/spark/SparkContext.scala | 4 +- .../org/apache/spark/executor/Executor.scala | 2 +- .../apache/spark/metrics/MetricsSystem.scala | 4 ++ python/run-tests | 44 +++++++++---------- 4 files changed, 29 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 1f54e2ac6675..431e4fb08385 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -187,7 +187,7 @@ class SparkContext(config: SparkConf) extends Logging { val master = conf.get("spark.master") val appName = conf.get("spark.app.name") val uniqueAppName = appName + "-" + System.currentTimeMillis() - conf.set("spark.unique.app.name", uniqueAppName) +// conf.set("spark.unique.app.name", uniqueAppName) // Generate the random name for a temp folder in Tachyon // Add a timestamp as the suffix here to make it more safe @@ -202,7 +202,7 @@ class SparkContext(config: SparkConf) extends Logging { private[spark] val listenerBus = new LiveListenerBus // Create the Spark execution environment (cache, map output tracker, etc) - conf.set("spark.executor.id", "driver") +// conf.set("spark.executor.id", "driver") private[spark] val env = SparkEnv.create( conf, "", 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 29724810b427..958e818def0f 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -73,7 +73,7 @@ private[spark] class Executor( val executorSource = new ExecutorSource(this, executorId) // Initialize Spark environment (using system properties read above) - conf.set("spark.executor.id", "executor." + executorId) +// conf.set("spark.executor.id", "executor." + executorId) private val env = { if (!isLocal) { val _env = SparkEnv.create(conf, executorId, slaveHostname, 0, diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index 4cc8a3b4c4d5..0402c1bcbd81 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -96,6 +96,8 @@ private[spark] class MetricsSystem private (val instance: String, } def buildRegistryName(source: Source) = { + new Exception().printStackTrace() +/* val appNameOpt = conf.getOption("spark.unique.app.name") val executorIdOpt = conf.getOption("spark.executor.id") val registryName = { @@ -106,6 +108,8 @@ private[spark] class MetricsSystem private (val instance: String, } } registryName + */ + "hoge" } def registerSource(source: Source) { diff --git a/python/run-tests b/python/run-tests index 7b1ee3e1cddb..2c9025ac72f5 100755 --- a/python/run-tests +++ b/python/run-tests @@ -49,37 +49,37 @@ function run_test() { echo "Running PySpark tests. Output is in python/unit-tests.log." # Try to test with Python 2.6, since that's the minimum version that we support: -if [ $(which python2.6) ]; then - export PYSPARK_PYTHON="python2.6" -fi +#if [ $(which python2.6) ]; then +# export PYSPARK_PYTHON="python2.6" +#fi echo "Testing with Python version:" $PYSPARK_PYTHON --version -run_test "pyspark/rdd.py" -run_test "pyspark/context.py" -run_test "pyspark/conf.py" -run_test "pyspark/sql.py" +#run_test "pyspark/rdd.py" +#run_test "pyspark/context.py" +#run_test "pyspark/conf.py" +#run_test "pyspark/sql.py" # These tests are included in the module-level docs, and so must # be handled on a higher level rather than within the python file. export PYSPARK_DOC_TEST=1 -run_test "pyspark/broadcast.py" -run_test "pyspark/accumulators.py" -run_test "pyspark/serializers.py" +#run_test "pyspark/broadcast.py" +#run_test "pyspark/accumulators.py" +#run_test "pyspark/serializers.py" unset PYSPARK_DOC_TEST -run_test "pyspark/shuffle.py" +#run_test "pyspark/shuffle.py" run_test "pyspark/tests.py" -run_test "pyspark/mllib/_common.py" -run_test "pyspark/mllib/classification.py" -run_test "pyspark/mllib/clustering.py" -run_test "pyspark/mllib/linalg.py" -run_test "pyspark/mllib/random.py" -run_test "pyspark/mllib/recommendation.py" -run_test "pyspark/mllib/regression.py" -run_test "pyspark/mllib/stat.py" -run_test "pyspark/mllib/tests.py" -run_test "pyspark/mllib/tree.py" -run_test "pyspark/mllib/util.py" +#run_test "pyspark/mllib/_common.py" +#run_test "pyspark/mllib/classification.py" +#run_test "pyspark/mllib/clustering.py" +#run_test "pyspark/mllib/linalg.py" +#run_test "pyspark/mllib/random.py" +#run_test "pyspark/mllib/recommendation.py" +#run_test "pyspark/mllib/regression.py" +#run_test "pyspark/mllib/stat.py" +#run_test "pyspark/mllib/tests.py" +#run_test "pyspark/mllib/tree.py" +#run_test "pyspark/mllib/util.py" if [[ $FAILED == 0 ]]; then echo -en "\033[32m" # Green From 08e627e12a04de57a9c7eaae9deaa4bbb97688a1 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Mon, 15 Sep 2014 14:04:45 +0900 Subject: [PATCH 10/42] Revert "tmp" This reverts commit e4a4593fd52da8a15770b8c3322f39b0f1363dbf. --- .../scala/org/apache/spark/SparkContext.scala | 4 +- .../org/apache/spark/executor/Executor.scala | 2 +- .../apache/spark/metrics/MetricsSystem.scala | 4 -- python/run-tests | 44 +++++++++---------- 4 files changed, 25 insertions(+), 29 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index c70c5bc6a673..9f34e103efcd 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -187,7 +187,7 @@ class SparkContext(config: SparkConf) extends Logging { val master = conf.get("spark.master") val appName = conf.get("spark.app.name") val uniqueAppName = appName + "-" + System.currentTimeMillis() -// conf.set("spark.unique.app.name", uniqueAppName) + conf.set("spark.unique.app.name", uniqueAppName) // Generate the random name for a temp folder in Tachyon // Add a timestamp as the suffix here to make it more safe @@ -202,7 +202,7 @@ class SparkContext(config: SparkConf) extends Logging { private[spark] val listenerBus = new LiveListenerBus // Create the Spark execution environment (cache, map output tracker, etc) -// conf.set("spark.executor.id", "driver") + conf.set("spark.executor.id", "driver") private[spark] val env = SparkEnv.create( conf, "", 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 96bde705d2e8..d9c6a7eac852 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -73,7 +73,7 @@ private[spark] class Executor( val executorSource = new ExecutorSource(this, executorId) // Initialize Spark environment (using system properties read above) -// conf.set("spark.executor.id", "executor." + executorId) + conf.set("spark.executor.id", "executor." + executorId) private val env = { if (!isLocal) { val _env = SparkEnv.create(conf, executorId, slaveHostname, 0, diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index 0402c1bcbd81..4cc8a3b4c4d5 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -96,8 +96,6 @@ private[spark] class MetricsSystem private (val instance: String, } def buildRegistryName(source: Source) = { - new Exception().printStackTrace() -/* val appNameOpt = conf.getOption("spark.unique.app.name") val executorIdOpt = conf.getOption("spark.executor.id") val registryName = { @@ -108,8 +106,6 @@ private[spark] class MetricsSystem private (val instance: String, } } registryName - */ - "hoge" } def registerSource(source: Source) { diff --git a/python/run-tests b/python/run-tests index 5c19565a464e..a67e5a99fbdc 100755 --- a/python/run-tests +++ b/python/run-tests @@ -53,37 +53,37 @@ echo "Running PySpark tests. Output is in python/unit-tests.log." export PYSPARK_PYTHON="python" # Try to test with Python 2.6, since that's the minimum version that we support: -#if [ $(which python2.6) ]; then -# export PYSPARK_PYTHON="python2.6" -#fi +if [ $(which python2.6) ]; then + export PYSPARK_PYTHON="python2.6" +fi echo "Testing with Python version:" $PYSPARK_PYTHON --version -#run_test "pyspark/rdd.py" -#run_test "pyspark/context.py" -#run_test "pyspark/conf.py" -#run_test "pyspark/sql.py" +run_test "pyspark/rdd.py" +run_test "pyspark/context.py" +run_test "pyspark/conf.py" +run_test "pyspark/sql.py" # These tests are included in the module-level docs, and so must # be handled on a higher level rather than within the python file. export PYSPARK_DOC_TEST=1 -#run_test "pyspark/broadcast.py" -#run_test "pyspark/accumulators.py" -#run_test "pyspark/serializers.py" +run_test "pyspark/broadcast.py" +run_test "pyspark/accumulators.py" +run_test "pyspark/serializers.py" unset PYSPARK_DOC_TEST -#run_test "pyspark/shuffle.py" +run_test "pyspark/shuffle.py" run_test "pyspark/tests.py" -#run_test "pyspark/mllib/_common.py" -#run_test "pyspark/mllib/classification.py" -#run_test "pyspark/mllib/clustering.py" -#run_test "pyspark/mllib/linalg.py" -#run_test "pyspark/mllib/random.py" -#run_test "pyspark/mllib/recommendation.py" -#run_test "pyspark/mllib/regression.py" -#run_test "pyspark/mllib/stat.py" -#run_test "pyspark/mllib/tests.py" -#run_test "pyspark/mllib/tree.py" -#run_test "pyspark/mllib/util.py" +run_test "pyspark/mllib/_common.py" +run_test "pyspark/mllib/classification.py" +run_test "pyspark/mllib/clustering.py" +run_test "pyspark/mllib/linalg.py" +run_test "pyspark/mllib/random.py" +run_test "pyspark/mllib/recommendation.py" +run_test "pyspark/mllib/regression.py" +run_test "pyspark/mllib/stat.py" +run_test "pyspark/mllib/tests.py" +run_test "pyspark/mllib/tree.py" +run_test "pyspark/mllib/util.py" # Try to test with PyPy if [ $(which pypy) ]; then From efcb6e149a274eca9f1780bd042451143fa6af57 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 18 Sep 2014 01:57:21 +0900 Subject: [PATCH 11/42] Modified to add application id to metrics name --- .../scala/org/apache/spark/SparkContext.scala | 10 ++++++++-- .../main/scala/org/apache/spark/SparkEnv.scala | 8 ++++++-- .../executor/CoarseGrainedExecutorBackend.scala | 16 +++++++++------- .../org/apache/spark/metrics/MetricsSystem.scala | 11 ++++++----- .../spark/deploy/yarn/ExecutorRunnable.scala | 3 ++- .../spark/deploy/yarn/ExecutorRunnableUtil.scala | 2 ++ .../apache/spark/deploy/yarn/YarnAllocator.scala | 2 ++ .../spark/deploy/yarn/ExecutorRunnable.scala | 3 ++- .../deploy/yarn/YarnAllocationHandler.scala | 2 +- 9 files changed, 38 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 90894ad717b7..16840eeb2f5c 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -186,8 +186,6 @@ class SparkContext(config: SparkConf) extends Logging { val master = conf.get("spark.master") val appName = conf.get("spark.app.name") - val uniqueAppName = appName + "-" + System.currentTimeMillis() - conf.set("spark.unique.app.name", uniqueAppName) // Generate the random name for a temp folder in Tachyon // Add a timestamp as the suffix here to make it more safe @@ -312,6 +310,14 @@ class SparkContext(config: SparkConf) extends Logging { // constructor taskScheduler.start() + val appId = taskScheduler.applicationId().getOrElse(System.currentTimeMillis().toString) + conf.set("spark.app.id", appId) + + val metricsSystem = env.metricsSystem + metricsSystem.registerSources() + metricsSystem.registerSinks() + metricsSystem.start() + private[spark] val cleaner: Option[ContextCleaner] = { if (conf.getBoolean("spark.cleaner.referenceTracking", true)) { Some(new ContextCleaner(this)) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 009ed6477584..63b12272fece 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -261,9 +261,13 @@ object SparkEnv extends Logging { val metricsSystem = if (isDriver) { MetricsSystem.createMetricsSystem("driver", conf, securityManager) } else { - MetricsSystem.createMetricsSystem("executor", conf, securityManager) + val ms = MetricsSystem.createMetricsSystem("executor", conf, securityManager) + ms.registerSources() + ms.registerSinks() + ms.start() + ms } - metricsSystem.start() + // Set the sparkFiles directory, used when downloading dependencies. In local mode, // this is a temporary directory; in distributed mode, this is the executor's current working diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 13af5b6f5812..d53792892f0e 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -106,7 +106,8 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { executorId: String, hostname: String, cores: Int, - workerUrl: Option[String]) { + workerUrl: Option[String], + appId: String) { SignalLogger.register(log) @@ -122,7 +123,8 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { val driver = fetcher.actorSelection(driverUrl) val timeout = AkkaUtils.askTimeout(executorConf) val fut = Patterns.ask(driver, RetrieveSparkProps, timeout) - val props = Await.result(fut, timeout).asInstanceOf[Seq[(String, String)]] + val props = Await.result(fut, timeout).asInstanceOf[Seq[(String, String)]] ++ + Seq[(String, String)](("spark.app.id", appId)) fetcher.shutdown() // Create a new ActorSystem using driver's Spark properties to run the backend. @@ -144,16 +146,16 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { def main(args: Array[String]) { args.length match { - case x if x < 4 => + case x if x < 5 => System.err.println( // Worker url is used in spark standalone mode to enforce fate-sharing with worker "Usage: CoarseGrainedExecutorBackend " + " []") System.exit(1) - case 4 => - run(args(0), args(1), args(2), args(3).toInt, None) - case x if x > 4 => - run(args(0), args(1), args(2), args(3).toInt, Some(args(4))) + case 5 => + run(args(0), args(1), args(2), args(3).toInt, None, args(4)) + case x if x > 5 => + run(args(0), args(1), args(2), args(3).toInt, Some(args(4)), args(5)) } } } diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index 4cc8a3b4c4d5..1763f7bdd82c 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -80,8 +80,6 @@ private[spark] class MetricsSystem private (val instance: String, def getServletHandlers = metricsServlet.map(_.getHandlers).getOrElse(Array()) metricsConfig.initialize() - registerSources() - registerSinks() def start() { sinks.foreach(_.start) @@ -96,11 +94,14 @@ private[spark] class MetricsSystem private (val instance: String, } def buildRegistryName(source: Source) = { - val appNameOpt = conf.getOption("spark.unique.app.name") + val appNameOpt = conf.getOption("spark.app.name") + val appIdOpt = conf.getOption("spark.app.id") val executorIdOpt = conf.getOption("spark.executor.id") val registryName = { - if (appNameOpt.isDefined && executorIdOpt.isDefined) { - MetricRegistry.name(appNameOpt.get, executorIdOpt.get, source.sourceName) + if (appNameOpt.isDefined && appIdOpt.isDefined && + executorIdOpt.isDefined) { + MetricRegistry.name(appIdOpt.get, appNameOpt.get, + executorIdOpt.get, source.sourceName) } else { MetricRegistry.name(source.sourceName) } diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 10cbeb8b9432..229b7a09f456 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -47,6 +47,7 @@ class ExecutorRunnable( hostname: String, executorMemory: Int, executorCores: Int, + appAttemptId: String, securityMgr: SecurityManager) extends Runnable with ExecutorRunnableUtil with Logging { @@ -83,7 +84,7 @@ class ExecutorRunnable( ctx.setContainerTokens(ByteBuffer.wrap(dob.getData())) val commands = prepareCommand(masterAddress, slaveId, hostname, executorMemory, executorCores, - localResources) + appAttemptId, localResources) logInfo("Setting up executor with commands: " + commands) ctx.setCommands(commands) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala index 312d82a64979..cbdc39ce5d02 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala @@ -44,6 +44,7 @@ trait ExecutorRunnableUtil extends Logging { hostname: String, executorMemory: Int, executorCores: Int, + appId: String, localResources: HashMap[String, LocalResource]): List[String] = { // Extra options for the JVM val javaOpts = ListBuffer[String]() @@ -112,6 +113,7 @@ trait ExecutorRunnableUtil extends Logging { slaveId.toString, hostname.toString, executorCores.toString, + appId, "1>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout", "2>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 299e38a5eb9c..a100c41a2412 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -56,6 +56,7 @@ object AllocationType extends Enumeration { private[yarn] abstract class YarnAllocator( conf: Configuration, sparkConf: SparkConf, + appAttemptId: ApplicationAttemptId, args: ApplicationMasterArguments, preferredNodes: collection.Map[String, collection.Set[SplitInfo]], securityMgr: SecurityManager) @@ -295,6 +296,7 @@ private[yarn] abstract class YarnAllocator( executorHostname, executorMemory, executorCores, + appAttemptId.getApplicationId.toString, securityMgr) launcherPool.execute(executorRunnable) } diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 833be12982e7..0b5a92d87d72 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -47,6 +47,7 @@ class ExecutorRunnable( hostname: String, executorMemory: Int, executorCores: Int, + appId: String, securityMgr: SecurityManager) extends Runnable with ExecutorRunnableUtil with Logging { @@ -80,7 +81,7 @@ class ExecutorRunnable( ctx.setTokens(ByteBuffer.wrap(dob.getData())) val commands = prepareCommand(masterAddress, slaveId, hostname, executorMemory, executorCores, - localResources) + appId, localResources) logInfo(s"Setting up executor with environment: $env") logInfo("Setting up executor with commands: " + commands) diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index e44a8db41b97..2bbf5d7db866 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -41,7 +41,7 @@ private[yarn] class YarnAllocationHandler( args: ApplicationMasterArguments, preferredNodes: collection.Map[String, collection.Set[SplitInfo]], securityMgr: SecurityManager) - extends YarnAllocator(conf, sparkConf, args, preferredNodes, securityMgr) { + extends YarnAllocator(conf, sparkConf, appAttemptId, args, preferredNodes, securityMgr) { override protected def releaseContainer(container: Container) = { amClient.releaseAssignedContainer(container.getId()) From 4776f9e40fcd2bd61025031f0b3c680c29cd81a1 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 18 Sep 2014 04:12:04 +0900 Subject: [PATCH 12/42] Modified MetricsSystemSuite.scala --- .../scala/org/apache/spark/metrics/MetricsSystemSuite.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala index 96a5a1231813..b2de9abf731e 100644 --- a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala @@ -34,6 +34,8 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { test("MetricsSystem with default config") { val metricsSystem = MetricsSystem.createMetricsSystem("default", conf, securityMgr) + metricsSystem.registerSources() + metricsSystem.registerSinks() val sources = metricsSystem.sources val sinks = metricsSystem.sinks @@ -44,6 +46,8 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { test("MetricsSystem with sources add") { val metricsSystem = MetricsSystem.createMetricsSystem("test", conf, securityMgr) + metricsSystem.registerSources() + metricsSystem.registerSinks() val sources = metricsSystem.sources val sinks = metricsSystem.sinks From eea6e19a8a565129cc54a9a9b1e56efd29012ce2 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Mon, 22 Sep 2014 13:07:52 +0900 Subject: [PATCH 13/42] Modified CoarseGrainedMesosSchedulerBackend and MesosSchedulerBackend so that we can get Application ID Modified MesosExecutorBackend to be able to get Application ID --- .../org/apache/spark/executor/MesosExecutorBackend.scala | 3 ++- .../cluster/mesos/CoarseMesosSchedulerBackend.scala | 5 +++++ .../scheduler/cluster/mesos/MesosSchedulerBackend.scala | 7 ++++++- 3 files changed, 13 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala index a42c8b43bbf7..bca0b152268a 100644 --- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -52,7 +52,8 @@ private[spark] class MesosExecutorBackend slaveInfo: SlaveInfo) { logInfo("Registered with Mesos as executor ID " + executorInfo.getExecutorId.getValue) this.driver = driver - val properties = Utils.deserialize[Array[(String, String)]](executorInfo.getData.toByteArray) + val properties = Utils.deserialize[Array[(String, String)]](executorInfo.getData.toByteArray) ++ + Seq[(String, String)](("spark.app.id", frameworkInfo.getId.getValue)) executor = new Executor( executorInfo.getExecutorId.getValue, slaveInfo.getHostname, diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 64568409dbaf..b03edc61827e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -76,6 +76,8 @@ private[spark] class CoarseMesosSchedulerBackend( var nextMesosTaskId = 0 + @volatile var appId: FrameworkID = _ + def newMesosTaskId(): Int = { val id = nextMesosTaskId nextMesosTaskId += 1 @@ -167,6 +169,7 @@ private[spark] class CoarseMesosSchedulerBackend( override def offerRescinded(d: SchedulerDriver, o: OfferID) {} override def registered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) { + appId = frameworkId logInfo("Registered as framework ID " + frameworkId.getValue) registeredLock.synchronized { isRegistered = true @@ -310,4 +313,6 @@ private[spark] class CoarseMesosSchedulerBackend( slaveLost(d, s) } + override def applicationId: Option[String] = Option(appId).map(_.getValue) + } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index a9ef126f5de0..253f4bdbe8cf 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -30,7 +30,7 @@ import org.apache.mesos._ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} import org.apache.spark.{Logging, SparkContext, SparkException, TaskState} -import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason, SchedulerBackend, SlaveLost, TaskDescription, TaskSchedulerImpl, WorkerOffer} +import org.apache.spark.scheduler._ import org.apache.spark.util.Utils /** @@ -62,6 +62,8 @@ private[spark] class MesosSchedulerBackend( var classLoader: ClassLoader = null + @volatile var appId: FrameworkID = _ + override def start() { synchronized { classLoader = Thread.currentThread.getContextClassLoader @@ -168,6 +170,7 @@ private[spark] class MesosSchedulerBackend( override def registered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) { val oldClassLoader = setClassLoader() try { + appId = frameworkId logInfo("Registered as framework ID " + frameworkId.getValue) registeredLock.synchronized { isRegistered = true @@ -350,4 +353,6 @@ private[spark] class MesosSchedulerBackend( // TODO: query Mesos for number of cores override def defaultParallelism() = sc.conf.getInt("spark.default.parallelism", 8) + override def applicationId(): Option[String] = Option(appId).map(_.getValue) + } From 36d2f7a6983d7bb55ea57b513263b661f05ad6d1 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Mon, 22 Sep 2014 14:09:45 +0900 Subject: [PATCH 14/42] Added warning message for the situation we cannot get application id for the prefix for the name of metrics --- core/src/main/scala/org/apache/spark/SparkContext.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 16840eeb2f5c..5ef8be5cfb15 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -310,7 +310,11 @@ class SparkContext(config: SparkConf) extends Logging { // constructor taskScheduler.start() - val appId = taskScheduler.applicationId().getOrElse(System.currentTimeMillis().toString) + val appId = taskScheduler.applicationId().getOrElse({ + logWarning("Failed to get unique Application ID. " + + "Instead of Application ID, UNIX time is used for the prefix of the name of metrics.") + System.currentTimeMillis().toString + }) conf.set("spark.app.id", appId) val metricsSystem = env.metricsSystem From b311806d2039c6d9a4e3dd332c866e3f4d0ca893 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 25 Sep 2014 15:23:04 +0900 Subject: [PATCH 15/42] Swapped last 2 arguments passed to CoarseGrainedExecutorBackend Refactored MetricsSystem#buildRegistryName --- .../CoarseGrainedExecutorBackend.scala | 10 +++++----- .../apache/spark/metrics/MetricsSystem.scala | 20 ++++++++----------- 2 files changed, 13 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index d53792892f0e..06061edfc084 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -106,8 +106,8 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { executorId: String, hostname: String, cores: Int, - workerUrl: Option[String], - appId: String) { + appId: String, + workerUrl: Option[String]) { SignalLogger.register(log) @@ -150,12 +150,12 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { System.err.println( // Worker url is used in spark standalone mode to enforce fate-sharing with worker "Usage: CoarseGrainedExecutorBackend " + - " []") + " [] ") System.exit(1) case 5 => - run(args(0), args(1), args(2), args(3).toInt, None, args(4)) + run(args(0), args(1), args(2), args(3).toInt, args(4), None) case x if x > 5 => - run(args(0), args(1), args(2), args(3).toInt, Some(args(4)), args(5)) + run(args(0), args(1), args(2), args(3).toInt, args(4), Some(args(5))) } } } diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index 1763f7bdd82c..eb5d451ac745 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -94,19 +94,15 @@ private[spark] class MetricsSystem private (val instance: String, } def buildRegistryName(source: Source) = { - val appNameOpt = conf.getOption("spark.app.name") - val appIdOpt = conf.getOption("spark.app.id") - val executorIdOpt = conf.getOption("spark.executor.id") - val registryName = { - if (appNameOpt.isDefined && appIdOpt.isDefined && - executorIdOpt.isDefined) { - MetricRegistry.name(appIdOpt.get, appNameOpt.get, - executorIdOpt.get, source.sourceName) - } else { - MetricRegistry.name(source.sourceName) - } + for { + appName <- conf.getOption("spark.app.name") + appId <- conf.getOption("spark.app.id") + executorId <- conf.getOption("spark.executor.id") + } yield { + MetricRegistry.name(appId, appName, executorId, source.sourceName) } - registryName + }.getOrElse { + MetricRegistry.name(source.sourceName) } def registerSource(source: Source) { From 424fea44023e3ffbe1fb197892b6f07b118cc8c7 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 25 Sep 2014 17:52:27 +0900 Subject: [PATCH 16/42] Modified the subclasses of TaskScheduler and SchedulerBackend so that they can return non-optional Unique Application ID --- .../main/scala/org/apache/spark/SparkContext.scala | 8 ++------ .../org/apache/spark/scheduler/TaskScheduler.scala | 2 +- .../apache/spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../cluster/SparkDeploySchedulerBackend.scala | 6 +++++- .../cluster/mesos/CoarseMesosSchedulerBackend.scala | 6 +++++- .../cluster/mesos/MesosSchedulerBackend.scala | 6 +++++- .../apache/spark/scheduler/local/LocalBackend.scala | 2 ++ .../cluster/YarnClientSchedulerBackend.scala | 6 +++++- .../cluster/YarnClusterSchedulerBackend.scala | 11 ++++++++++- 9 files changed, 36 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index f163a15bbd8a..769852766808 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -310,11 +310,7 @@ class SparkContext(config: SparkConf) extends Logging { // constructor taskScheduler.start() - val appId = taskScheduler.applicationId().getOrElse({ - logWarning("Failed to get unique Application ID. " + - "Instead of Application ID, UNIX time is used for the prefix of the name of metrics.") - System.currentTimeMillis().toString - }) + private val appId = taskScheduler.applicationId() conf.set("spark.app.id", appId) val metricsSystem = env.metricsSystem @@ -1291,7 +1287,7 @@ class SparkContext(config: SparkConf) extends Logging { private def postApplicationStart() { // Note: this code assumes that the task scheduler has been initialized and has contacted // the cluster manager to get an application ID (in case the cluster manager provides one). - listenerBus.post(SparkListenerApplicationStart(appName, taskScheduler.applicationId(), + listenerBus.post(SparkListenerApplicationStart(appName, Some(taskScheduler.applicationId()), startTime, sparkUser)) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 1c1ce666eab0..5006c210a118 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -70,6 +70,6 @@ private[spark] trait TaskScheduler { * * @return The application ID, or None if the backend does not provide an ID. */ - def applicationId(): Option[String] = None + def applicationId(): String = System.currentTimeMillis.toString } 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 633e892554c5..4dc550413c13 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -492,7 +492,7 @@ private[spark] class TaskSchedulerImpl( } } - override def applicationId(): Option[String] = backend.applicationId() + override def applicationId(): String = backend.applicationId() } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 5c5ecc8434d7..dbebbe3313c6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -129,7 +129,11 @@ private[spark] class SparkDeploySchedulerBackend( totalCoreCount.get() >= totalExpectedCores * minRegisteredRatio } - override def applicationId(): Option[String] = Option(appId) + override def applicationId(): String = + Option(appId).getOrElse { + logWarning("Application ID is not initialized yet.") + super.applicationId + } private def waitForRegistration() = { registrationLock.synchronized { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index b03edc61827e..1e9deba89b2c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -313,6 +313,10 @@ private[spark] class CoarseMesosSchedulerBackend( slaveLost(d, s) } - override def applicationId: Option[String] = Option(appId).map(_.getValue) + override def applicationId: String = + Option(appId).map(_.getValue).getOrElse { + logWarning("Application ID is not initialized yet.") + super.applicationId + } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index 253f4bdbe8cf..80b873e18ba9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -353,6 +353,10 @@ private[spark] class MesosSchedulerBackend( // TODO: query Mesos for number of cores override def defaultParallelism() = sc.conf.getInt("spark.default.parallelism", 8) - override def applicationId(): Option[String] = Option(appId).map(_.getValue) + override def applicationId(): String = + Option(appId).map(_.getValue).getOrElse { + logWarning("Application ID is not initialized yet.") + super.applicationId + } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index 9ea25c2bc709..7c7aa02af281 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -115,4 +115,6 @@ private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: localActor ! StatusUpdate(taskId, state, serializedData) } + override def applicationId: String = "local-" + System.currentTimeMillis + } diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 200a30899290..d2b65a7ec6a2 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -155,6 +155,10 @@ private[spark] class YarnClientSchedulerBackend( totalRegisteredExecutors.get() >= totalExpectedExecutors * minRegisteredRatio } - override def applicationId(): Option[String] = Option(appId).map(_.toString()) + override def applicationId(): String = + Option(appId).map(_.toString()).getOrElse { + logWarning("Application ID is not initialized yet.") + super.applicationId + } } diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala index 39436d099966..6f9c96c1a7b3 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala @@ -48,6 +48,15 @@ private[spark] class YarnClusterSchedulerBackend( totalRegisteredExecutors.get() >= totalExpectedExecutors * minRegisteredRatio } - override def applicationId(): Option[String] = sc.getConf.getOption("spark.yarn.app.id") + override def applicationId(): String = + /** + * In YARN Cluster mode, spark.yarn.app.id is expect to be set + * before user application is launched. + * So, if spark.yarn.app.id is not set, it is something wrong. + */ + sc.getConf.getOption("spark.yarn.app.id").getOrElse { + logError("Application ID is not set.") + super.applicationId + } } From 203634e53ed8f5281c0fb3785bc98595123d4281 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 25 Sep 2014 18:04:32 +0900 Subject: [PATCH 17/42] Modified comment in SchedulerBackend#applicationId and TaskScheduler#applicationId --- .../scala/org/apache/spark/scheduler/SchedulerBackend.scala | 6 ++++-- .../scala/org/apache/spark/scheduler/TaskScheduler.scala | 4 +++- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala index a0be8307eff2..94ec34ba29a7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -34,9 +34,11 @@ private[spark] trait SchedulerBackend { /** * The application ID associated with the job, if any. + * It is expected that the subclasses of TaskScheduler or SchedulerBackend + * override this method and return an unique application ID. * - * @return The application ID, or None if the backend does not provide an ID. + * @return The application ID, if the backend does not provide an ID. */ - def applicationId(): Option[String] = None + def applicationId(): String = System.currentTimeMillis.toString } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 5006c210a118..aab6e0ec4402 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -67,8 +67,10 @@ private[spark] trait TaskScheduler { /** * The application ID associated with the job, if any. + * It is expected that the subclasses of TaskScheduler or SchedulerBackend + * override this method and return an unique application ID. * - * @return The application ID, or None if the backend does not provide an ID. + * @return The application ID, if the backend does not provide an ID. */ def applicationId(): String = System.currentTimeMillis.toString From 28d4d939a0f5dda4520f08eaa7f5020c7121af2c Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 25 Sep 2014 18:08:22 +0900 Subject: [PATCH 18/42] Modified SparkContext and EventLoggingListener so that the directory for EventLogs is named same for Application ID --- .../scala/org/apache/spark/SparkContext.scala | 26 +++++++++---------- .../scheduler/EventLoggingListener.scala | 8 +++--- 2 files changed, 17 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 769852766808..71718ecf862a 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -233,19 +233,6 @@ class SparkContext(config: SparkConf) extends Logging { /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ val hadoopConfiguration = SparkHadoopUtil.get.newConfiguration(conf) - // Optionally log Spark events - private[spark] val eventLogger: Option[EventLoggingListener] = { - if (conf.getBoolean("spark.eventLog.enabled", false)) { - val logger = new EventLoggingListener(appName, conf, hadoopConfiguration) - logger.start() - listenerBus.addListener(logger) - Some(logger) - } else None - } - - // At this point, all relevant SparkListeners have been registered, so begin releasing events - listenerBus.start() - val startTime = System.currentTimeMillis() // Add each JAR given through the constructor @@ -318,6 +305,19 @@ class SparkContext(config: SparkConf) extends Logging { metricsSystem.registerSinks() metricsSystem.start() + // Optionally log Spark events + private[spark] val eventLogger: Option[EventLoggingListener] = { + if (conf.getBoolean("spark.eventLog.enabled", false)) { + val logger = new EventLoggingListener(appId, conf, hadoopConfiguration) + logger.start() + listenerBus.addListener(logger) + Some(logger) + } else None + } + + // At this point, all relevant SparkListeners have been registered, so begin releasing events + listenerBus.start() + private[spark] val cleaner: Option[ContextCleaner] = { if (conf.getBoolean("spark.cleaner.referenceTracking", true)) { Some(new ContextCleaner(this)) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 64b32ae0edaa..caa2631fda46 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -43,22 +43,22 @@ import org.apache.spark.util.{FileLogger, JsonProtocol, Utils} * spark.eventLog.buffer.kb - Buffer size to use when writing to output streams */ private[spark] class EventLoggingListener( - appName: String, + appId: String, sparkConf: SparkConf, hadoopConf: Configuration) extends SparkListener with Logging { import EventLoggingListener._ - def this(appName: String, sparkConf: SparkConf) = - this(appName, sparkConf, SparkHadoopUtil.get.newConfiguration(sparkConf)) + def this(appId: String, sparkConf: SparkConf) = + this(appId, sparkConf, SparkHadoopUtil.get.newConfiguration(sparkConf)) private val shouldCompress = sparkConf.getBoolean("spark.eventLog.compress", false) private val shouldOverwrite = sparkConf.getBoolean("spark.eventLog.overwrite", false) private val testing = sparkConf.getBoolean("spark.eventLog.testing", false) private val outputBufferSize = sparkConf.getInt("spark.eventLog.buffer.kb", 100) * 1024 private val logBaseDir = sparkConf.get("spark.eventLog.dir", DEFAULT_LOG_DIR).stripSuffix("/") - private val name = appName.replaceAll("[ :/]", "-").replaceAll("[${}'\"]", "_") + private val name = appId.replaceAll("[ :/]", "-").replaceAll("[${}'\"]", "_") .toLowerCase + "-" + System.currentTimeMillis val logDir = Utils.resolveURI(logBaseDir) + "/" + name.stripSuffix("/") From bcf25bff564a91e2b0e1eca45b2546f5325b4320 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 25 Sep 2014 18:17:38 +0900 Subject: [PATCH 19/42] Modified directory name for EventLogs --- .../org/apache/spark/scheduler/EventLoggingListener.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index caa2631fda46..b4f6ef30eb9a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -58,8 +58,7 @@ private[spark] class EventLoggingListener( private val testing = sparkConf.getBoolean("spark.eventLog.testing", false) private val outputBufferSize = sparkConf.getInt("spark.eventLog.buffer.kb", 100) * 1024 private val logBaseDir = sparkConf.get("spark.eventLog.dir", DEFAULT_LOG_DIR).stripSuffix("/") - private val name = appId.replaceAll("[ :/]", "-").replaceAll("[${}'\"]", "_") - .toLowerCase + "-" + System.currentTimeMillis + private val name = appId.replaceAll("[ :/]", "-").replaceAll("[${}'\"]", "_").toLowerCase val logDir = Utils.resolveURI(logBaseDir) + "/" + name.stripSuffix("/") protected val logger = new FileLogger(logDir, sparkConf, hadoopConf, outputBufferSize, From 0f890e6188da8c9ff1171ba81f1906c6eecaf4b3 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 26 Sep 2014 17:09:31 +0900 Subject: [PATCH 20/42] Modified SparkDeploySchedulerBackend and Master to pass baseLogDir instead f eventLogDir Introduced ApplicationId --- .../scala/org/apache/spark/SparkContext.scala | 32 +++++++++++++------ .../apache/spark/deploy/DeployMessage.scala | 11 ++++--- .../spark/deploy/ExecutorDescription.scala | 4 ++- .../apache/spark/deploy/JsonProtocol.scala | 4 +-- .../spark/deploy/client/AppClient.scala | 4 +-- .../deploy/client/AppClientListener.scala | 4 ++- .../spark/deploy/client/TestClient.scala | 4 +-- .../spark/deploy/master/ApplicationInfo.scala | 3 +- .../apache/spark/deploy/master/Master.scala | 21 ++++++------ .../spark/deploy/worker/ExecutorRunner.scala | 10 +++--- .../scheduler/EventLoggingListener.scala | 22 +++++++++---- .../spark/scheduler/SchedulerBackend.scala | 4 ++- .../spark/scheduler/TaskScheduler.scala | 3 +- .../spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../cluster/SparkDeploySchedulerBackend.scala | 14 ++++---- .../mesos/CoarseMesosSchedulerBackend.scala | 6 ++-- .../cluster/mesos/MesosSchedulerBackend.scala | 6 ++-- .../spark/scheduler/local/LocalBackend.scala | 5 +-- .../spark/deploy/JsonProtocolSuite.scala | 6 ++-- .../deploy/worker/ExecutorRunnerTest.scala | 4 +-- .../scheduler/EventLoggingListenerSuite.scala | 8 ++--- .../spark/scheduler/ReplayListenerSuite.scala | 5 +-- .../cluster/YarnClientSchedulerBackend.scala | 10 +++--- .../cluster/YarnClusterSchedulerBackend.scala | 14 ++++---- 24 files changed, 121 insertions(+), 85 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 71718ecf862a..98469a47fcdb 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -187,6 +187,15 @@ class SparkContext(config: SparkConf) extends Logging { val master = conf.get("spark.master") val appName = conf.get("spark.app.name") + val isEventLogEnabled = conf.getBoolean("spark.eventLog.enabled", false) + val eventLogDir = { + if (isEventLogEnabled) { + Some(conf.get("spark.eventLog.dir", EventLoggingListener.DEFAULT_LOG_DIR).stripSuffix("/")) + } else { + None + } + } + // Generate the random name for a temp folder in Tachyon // Add a timestamp as the suffix here to make it more safe val tachyonFolderName = "spark-" + randomUUID.toString() @@ -282,7 +291,7 @@ class SparkContext(config: SparkConf) extends Logging { executorEnvs("SPARK_USER") = sparkUser // Create and start the scheduler - private[spark] var taskScheduler = SparkContext.createTaskScheduler(this, master) + private[spark] var taskScheduler = SparkContext.createTaskScheduler(this, master, eventLogDir) private val heartbeatReceiver = env.actorSystem.actorOf( Props(new HeartbeatReceiver(taskScheduler)), "HeartbeatReceiver") @volatile private[spark] var dagScheduler: DAGScheduler = _ @@ -298,7 +307,7 @@ class SparkContext(config: SparkConf) extends Logging { taskScheduler.start() private val appId = taskScheduler.applicationId() - conf.set("spark.app.id", appId) + conf.set("spark.app.id", appId.toString) val metricsSystem = env.metricsSystem metricsSystem.registerSources() @@ -307,8 +316,8 @@ class SparkContext(config: SparkConf) extends Logging { // Optionally log Spark events private[spark] val eventLogger: Option[EventLoggingListener] = { - if (conf.getBoolean("spark.eventLog.enabled", false)) { - val logger = new EventLoggingListener(appId, conf, hadoopConfiguration) + if (isEventLogEnabled) { + val logger = new EventLoggingListener(appId, eventLogDir.get, conf, hadoopConfiguration) logger.start() listenerBus.addListener(logger) Some(logger) @@ -343,6 +352,8 @@ class SparkContext(config: SparkConf) extends Logging { localProperties.set(props) } + def getApplicationId = appId + @deprecated("Properties no longer need to be explicitly initialized.", "1.0.0") def initLocalProperties() { localProperties.set(new Properties()) @@ -1287,8 +1298,8 @@ class SparkContext(config: SparkConf) extends Logging { private def postApplicationStart() { // Note: this code assumes that the task scheduler has been initialized and has contacted // the cluster manager to get an application ID (in case the cluster manager provides one). - listenerBus.post(SparkListenerApplicationStart(appName, Some(taskScheduler.applicationId()), - startTime, sparkUser)) + listenerBus.post(SparkListenerApplicationStart(appName, + Some(taskScheduler.applicationId().toString), startTime, sparkUser)) } /** Post the application end event */ @@ -1482,7 +1493,10 @@ object SparkContext extends Logging { } /** Creates a task scheduler based on a given master URL. Extracted for testing. */ - private def createTaskScheduler(sc: SparkContext, master: String): TaskScheduler = { + private def createTaskScheduler( + sc: SparkContext, + master: String, + eventLogDir: Option[String]): TaskScheduler = { // Regular expression used for local[N] and local[*] master formats val LOCAL_N_REGEX = """local\[([0-9]+|\*)\]""".r // Regular expression for local[N, maxRetries], used in tests with failing tasks @@ -1528,7 +1542,7 @@ object SparkContext extends Logging { case SPARK_REGEX(sparkUrl) => val scheduler = new TaskSchedulerImpl(sc) val masterUrls = sparkUrl.split(",").map("spark://" + _) - val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls) + val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls, eventLogDir) scheduler.initialize(backend) scheduler @@ -1545,7 +1559,7 @@ object SparkContext extends Logging { val localCluster = new LocalSparkCluster( numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt) val masterUrls = localCluster.start() - val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls) + val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls, eventLogDir) scheduler.initialize(backend) backend.shutdownCallback = (backend: SparkDeploySchedulerBackend) => { localCluster.stop() diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index a7368f9f3dfb..99f7da5edc46 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -19,6 +19,7 @@ package org.apache.spark.deploy import scala.collection.immutable.List +import org.apache.spark.ApplicationId import org.apache.spark.deploy.ExecutorState.ExecutorState import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, WorkerInfo} import org.apache.spark.deploy.master.DriverState.DriverState @@ -47,7 +48,7 @@ private[deploy] object DeployMessages { } case class ExecutorStateChanged( - appId: String, + appId: ApplicationId, execId: Int, state: ExecutorState, message: Option[String], @@ -71,11 +72,11 @@ private[deploy] object DeployMessages { case class RegisterWorkerFailed(message: String) extends DeployMessage - case class KillExecutor(masterUrl: String, appId: String, execId: Int) extends DeployMessage + case class KillExecutor(masterUrl: String, appId: ApplicationId, execId: Int) extends DeployMessage case class LaunchExecutor( masterUrl: String, - appId: String, + appId: ApplicationId, execId: Int, appDesc: ApplicationDescription, cores: Int, @@ -95,11 +96,11 @@ private[deploy] object DeployMessages { case class RegisterApplication(appDescription: ApplicationDescription) extends DeployMessage - case class MasterChangeAcknowledged(appId: String) + case class MasterChangeAcknowledged(appId: ApplicationId) // Master to AppClient - case class RegisteredApplication(appId: String, masterUrl: String) extends DeployMessage + case class RegisteredApplication(appId: ApplicationId, masterUrl: String) extends DeployMessage // TODO(matei): replace hostPort with host case class ExecutorAdded(id: Int, workerId: String, hostPort: String, cores: Int, memory: Int) { diff --git a/core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala index 2abf0b69dddb..47bdbbff4a68 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala @@ -17,13 +17,15 @@ package org.apache.spark.deploy +import org.apache.spark.ApplicationId + /** * Used to send state on-the-wire about Executors from Worker to Master. * This state is sufficient for the Master to reconstruct its internal data structures during * failover. */ private[spark] class ExecutorDescription( - val appId: String, + val appId: ApplicationId, val execId: Int, val cores: Int, val state: ExecutorState.Value) diff --git a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala index 696f32a6f573..7f127cf76bf0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala @@ -41,7 +41,7 @@ private[spark] object JsonProtocol { def writeApplicationInfo(obj: ApplicationInfo) = { ("starttime" -> obj.startTime) ~ - ("id" -> obj.id) ~ + ("id" -> obj.id.toString) ~ ("name" -> obj.desc.name) ~ ("cores" -> obj.desc.maxCores) ~ ("user" -> obj.desc.user) ~ @@ -62,7 +62,7 @@ private[spark] object JsonProtocol { def writeExecutorRunner(obj: ExecutorRunner) = { ("id" -> obj.execId) ~ ("memory" -> obj.memory) ~ - ("appid" -> obj.appId) ~ + ("appid" -> obj.appId.toString) ~ ("appdesc" -> writeApplicationDescription(obj.appDesc)) } diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala index 32790053a6be..1ec0c1298202 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala @@ -26,7 +26,7 @@ import akka.actor._ import akka.pattern.ask import akka.remote.{AssociationErrorEvent, DisassociatedEvent, RemotingLifecycleEvent} -import org.apache.spark.{Logging, SparkConf, SparkException} +import org.apache.spark.{ApplicationId, Logging, SparkConf, SparkException} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.Master @@ -52,7 +52,7 @@ private[spark] class AppClient( var masterAddress: Address = null var actor: ActorRef = null - var appId: String = null + var appId: ApplicationId = null var registered = false var activeMasterUrl: String = null diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala index e584952a9ad8..f33241df7b8d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala @@ -17,6 +17,8 @@ package org.apache.spark.deploy.client +import org.apache.spark.ApplicationId + /** * Callbacks invoked by deploy client when various events happen. There are currently four events: * connecting to the cluster, disconnecting, being given an executor, and having an executor @@ -25,7 +27,7 @@ package org.apache.spark.deploy.client * Users of this API should *not* block inside the callback methods. */ private[spark] trait AppClientListener { - def connected(appId: String): Unit + def connected(appId: ApplicationId): Unit /** Disconnection may be a temporary state, as we fail over to a new Master. */ def disconnected(): Unit diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala index 88a0862b96af..f9ce7f5585c9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala @@ -17,14 +17,14 @@ package org.apache.spark.deploy.client -import org.apache.spark.{SecurityManager, SparkConf, Logging} +import org.apache.spark.{ApplicationId, SecurityManager, SparkConf, Logging} import org.apache.spark.deploy.{ApplicationDescription, Command} import org.apache.spark.util.{AkkaUtils, Utils} private[spark] object TestClient { class TestListener extends AppClientListener with Logging { - def connected(id: String) { + def connected(id: ApplicationId) { logInfo("Connected to master, got app ID " + id) } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index c3ca43f8d073..5d232489376b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -24,11 +24,12 @@ import scala.collection.mutable.ArrayBuffer import akka.actor.ActorRef +import org.apache.spark.ApplicationId import org.apache.spark.deploy.ApplicationDescription private[spark] class ApplicationInfo( val startTime: Long, - val id: String, + val id: ApplicationId, val desc: ApplicationDescription, val submitDate: Date, val driver: ActorRef, diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 432b552c58cd..cc119cd047f1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -32,9 +32,8 @@ import akka.pattern.ask import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} import akka.serialization.SerializationExtension -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} -import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, ExecutorState, - SparkHadoopUtil} +import org.apache.spark.{ApplicationId, Logging, SecurityManager, SparkConf, SparkException} +import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, ExecutorState, SparkHadoopUtil} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.deploy.master.DriverState.DriverState @@ -69,13 +68,13 @@ private[spark] class Master( val addressToWorker = new HashMap[Address, WorkerInfo] val apps = new HashSet[ApplicationInfo] - val idToApp = new HashMap[String, ApplicationInfo] + val idToApp = new HashMap[ApplicationId, ApplicationInfo] val actorToApp = new HashMap[ActorRef, ApplicationInfo] val addressToApp = new HashMap[Address, ApplicationInfo] val waitingApps = new ArrayBuffer[ApplicationInfo] val completedApps = new ArrayBuffer[ApplicationInfo] var nextAppNumber = 0 - val appIdToUI = new HashMap[String, SparkUI] + val appIdToUI = new HashMap[ApplicationId, SparkUI] val drivers = new HashSet[DriverInfo] val completedDrivers = new ArrayBuffer[DriverInfo] @@ -693,16 +692,18 @@ private[spark] class Master( app.desc.appUiUrl = notFoundBasePath return false } - val fileSystem = Utils.getHadoopFileSystem(eventLogDir, + + val appEventLogDir = EventLoggingListener.getLogDirName(eventLogDir, app.id) + val fileSystem = Utils.getHadoopFileSystem(appEventLogDir, SparkHadoopUtil.get.newConfiguration(conf)) - val eventLogInfo = EventLoggingListener.parseLoggingInfo(eventLogDir, fileSystem) + val eventLogInfo = EventLoggingListener.parseLoggingInfo(appEventLogDir, fileSystem) val eventLogPaths = eventLogInfo.logPaths val compressionCodec = eventLogInfo.compressionCodec if (eventLogPaths.isEmpty) { // Event logging is enabled for this application, but no event logs are found val title = s"Application history not found (${app.id})" - var msg = s"No event logs found for application $appName in $eventLogDir." + var msg = s"No event logs found for application $appName in $appEventLogDir." logWarning(msg) msg += " Did you specify the correct logging directory?" msg = URLEncoder.encode(msg, "UTF-8") @@ -734,10 +735,10 @@ private[spark] class Master( } /** Generate a new app ID given a app's submission date */ - def newApplicationId(submitDate: Date): String = { + def newApplicationId(submitDate: Date) = { val appId = "app-%s-%04d".format(createDateFormat.format(submitDate), nextAppNumber) nextAppNumber += 1 - appId + new ApplicationId(appId) } /** Check for, and remove, any timed-out workers */ diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 00a43673e5cd..2182d7214656 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -23,7 +23,7 @@ import akka.actor.ActorRef import com.google.common.base.Charsets import com.google.common.io.Files -import org.apache.spark.{SparkConf, Logging} +import org.apache.spark.{ApplicationId, SparkConf, Logging} import org.apache.spark.deploy.{ApplicationDescription, Command, ExecutorState} import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged import org.apache.spark.util.logging.FileAppender @@ -33,7 +33,7 @@ import org.apache.spark.util.logging.FileAppender * This is currently only used in standalone mode. */ private[spark] class ExecutorRunner( - val appId: String, + val appId: ApplicationId, val execId: Int, val appDesc: ApplicationDescription, val cores: Int, @@ -48,7 +48,7 @@ private[spark] class ExecutorRunner( var state: ExecutorState.Value) extends Logging { - val fullId = appId + "/" + execId + val fullId = s"${appId}/${execId}" var workerThread: Thread = null var process: Process = null var stdoutAppender: FileAppender = null @@ -117,7 +117,7 @@ private[spark] class ExecutorRunner( def getCommandSeq = { val command = Command( appDesc.command.mainClass, - appDesc.command.arguments.map(substituteVariables) ++ Seq(appId), + appDesc.command.arguments.map(substituteVariables) ++ Seq(appId.toString), appDesc.command.environment, appDesc.command.classPathEntries, appDesc.command.libraryPathEntries, @@ -131,7 +131,7 @@ private[spark] class ExecutorRunner( def fetchAndRunExecutor() { try { // Create the executor's working directory - val executorDir = new File(workDir, appId + "/" + execId) + val executorDir = new File(workDir, s"${appId}/${execId}") if (!executorDir.mkdirs()) { throw new IOException("Failed to create directory " + executorDir) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index b4f6ef30eb9a..c7bcada0a7c5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.fs.permission.FsPermission import org.json4s.JsonAST.JValue import org.json4s.jackson.JsonMethods._ -import org.apache.spark.{Logging, SparkConf, SparkContext} +import org.apache.spark.{ApplicationId, Logging, SparkConf, SparkContext} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec import org.apache.spark.SPARK_VERSION @@ -43,23 +43,26 @@ import org.apache.spark.util.{FileLogger, JsonProtocol, Utils} * spark.eventLog.buffer.kb - Buffer size to use when writing to output streams */ private[spark] class EventLoggingListener( - appId: String, + appId: ApplicationId, + logBaseDir: String, sparkConf: SparkConf, hadoopConf: Configuration) extends SparkListener with Logging { import EventLoggingListener._ - def this(appId: String, sparkConf: SparkConf) = - this(appId, sparkConf, SparkHadoopUtil.get.newConfiguration(sparkConf)) + def this(appId: ApplicationId, logBaseDir: String, sparkConf: SparkConf) = + this(appId, logBaseDir, sparkConf, SparkHadoopUtil.get.newConfiguration(sparkConf)) private val shouldCompress = sparkConf.getBoolean("spark.eventLog.compress", false) private val shouldOverwrite = sparkConf.getBoolean("spark.eventLog.overwrite", false) private val testing = sparkConf.getBoolean("spark.eventLog.testing", false) private val outputBufferSize = sparkConf.getInt("spark.eventLog.buffer.kb", 100) * 1024 - private val logBaseDir = sparkConf.get("spark.eventLog.dir", DEFAULT_LOG_DIR).stripSuffix("/") - private val name = appId.replaceAll("[ :/]", "-").replaceAll("[${}'\"]", "_").toLowerCase - val logDir = Utils.resolveURI(logBaseDir) + "/" + name.stripSuffix("/") + val logDir = EventLoggingListener.getLogDirName(logBaseDir, appId) + private val name = { + val splitPath = logDir.split("/") + splitPath(splitPath.length-1) + } protected val logger = new FileLogger(logDir, sparkConf, hadoopConf, outputBufferSize, shouldCompress, shouldOverwrite, Some(LOG_FILE_PERMISSIONS)) @@ -183,6 +186,11 @@ private[spark] object EventLoggingListener extends Logging { } else "" } + def getLogDirName(logBaseDir: String, appId: ApplicationId) = { + val name = appId.toString.replaceAll("[ :/]", "-").replaceAll("[${}'\"]", "_").toLowerCase + Utils.resolveURI(logBaseDir) + "/" + name.stripSuffix("/") + } + /** * Parse the event logging information associated with the logs in the given directory. * diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala index 94ec34ba29a7..71d05088a425 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -17,6 +17,8 @@ package org.apache.spark.scheduler +import org.apache.spark.ApplicationId + /** * A backend interface for scheduling systems that allows plugging in different ones under * TaskSchedulerImpl. We assume a Mesos-like model where the application gets resource offers as @@ -39,6 +41,6 @@ private[spark] trait SchedulerBackend { * * @return The application ID, if the backend does not provide an ID. */ - def applicationId(): String = System.currentTimeMillis.toString + def applicationId() = new ApplicationId(System.currentTimeMillis.toString) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index aab6e0ec4402..27c9a546cce7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -17,6 +17,7 @@ package org.apache.spark.scheduler +import org.apache.spark.ApplicationId import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId @@ -72,6 +73,6 @@ private[spark] trait TaskScheduler { * * @return The application ID, if the backend does not provide an ID. */ - def applicationId(): String = System.currentTimeMillis.toString + def applicationId() = new ApplicationId(System.currentTimeMillis.toString) } 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 4dc550413c13..232db9e5e54b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -492,7 +492,7 @@ private[spark] class TaskSchedulerImpl( } } - override def applicationId(): String = backend.applicationId() + override def applicationId() = backend.applicationId() } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index dbebbe3313c6..fdde85654e47 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler.cluster -import org.apache.spark.{Logging, SparkConf, SparkContext, SparkEnv} +import org.apache.spark.{ApplicationId, Logging, SparkConf, SparkContext, SparkEnv} import org.apache.spark.deploy.{ApplicationDescription, Command} import org.apache.spark.deploy.client.{AppClient, AppClientListener} import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason, SlaveLost, TaskSchedulerImpl} @@ -26,7 +26,8 @@ import org.apache.spark.util.Utils private[spark] class SparkDeploySchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext, - masters: Array[String]) + masters: Array[String], + baseLogDir: Option[String]) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) with AppClientListener with Logging { @@ -34,7 +35,7 @@ private[spark] class SparkDeploySchedulerBackend( var client: AppClient = null var stopping = false var shutdownCallback : (SparkDeploySchedulerBackend) => Unit = _ - @volatile var appId: String = _ + @volatile var appId: ApplicationId = _ val registrationLock = new Object() var registrationDone = false @@ -68,9 +69,8 @@ private[spark] class SparkDeploySchedulerBackend( val command = Command("org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs, classPathEntries, libraryPathEntries, javaOpts) val appUIAddress = sc.ui.map(_.appUIAddress).getOrElse("") - val eventLogDir = sc.eventLogger.map(_.logDir) val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, - appUIAddress, eventLogDir) + appUIAddress, baseLogDir) client = new AppClient(sc.env.actorSystem, masters, appDesc, this, conf) client.start() @@ -87,7 +87,7 @@ private[spark] class SparkDeploySchedulerBackend( } } - override def connected(appId: String) { + override def connected(appId: ApplicationId) { logInfo("Connected to Spark cluster with app ID " + appId) this.appId = appId notifyContext() @@ -129,7 +129,7 @@ private[spark] class SparkDeploySchedulerBackend( totalCoreCount.get() >= totalExpectedCores * minRegisteredRatio } - override def applicationId(): String = + override def applicationId() = Option(appId).getOrElse { logWarning("Application ID is not initialized yet.") super.applicationId diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 1e9deba89b2c..5bf99bdf9385 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -28,7 +28,7 @@ import org.apache.mesos.{Scheduler => MScheduler} import org.apache.mesos._ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} -import org.apache.spark.{Logging, SparkContext, SparkEnv, SparkException} +import org.apache.spark.{ApplicationId, Logging, SparkContext, SparkEnv, SparkException} import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend @@ -313,8 +313,8 @@ private[spark] class CoarseMesosSchedulerBackend( slaveLost(d, s) } - override def applicationId: String = - Option(appId).map(_.getValue).getOrElse { + override def applicationId = + Option(appId).map(mAppId => new ApplicationId(mAppId.getValue)).getOrElse { logWarning("Application ID is not initialized yet.") super.applicationId } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index 80b873e18ba9..f96cabe6a6d2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -29,7 +29,7 @@ import org.apache.mesos.{Scheduler => MScheduler} import org.apache.mesos._ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} -import org.apache.spark.{Logging, SparkContext, SparkException, TaskState} +import org.apache.spark.{ApplicationId, Logging, SparkContext, SparkException, TaskState} import org.apache.spark.scheduler._ import org.apache.spark.util.Utils @@ -353,8 +353,8 @@ private[spark] class MesosSchedulerBackend( // TODO: query Mesos for number of cores override def defaultParallelism() = sc.conf.getInt("spark.default.parallelism", 8) - override def applicationId(): String = - Option(appId).map(_.getValue).getOrElse { + override def applicationId() = + Option(appId).map(mAppId => new ApplicationId(mAppId.getValue)).getOrElse { logWarning("Application ID is not initialized yet.") super.applicationId } diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index 7c7aa02af281..5c1e8a1bbb80 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -21,7 +21,7 @@ import java.nio.ByteBuffer import akka.actor.{Actor, ActorRef, Props} -import org.apache.spark.{Logging, SparkEnv, TaskState} +import org.apache.spark.{ApplicationId, Logging, SparkEnv, TaskState} import org.apache.spark.TaskState.TaskState import org.apache.spark.executor.{Executor, ExecutorBackend} import org.apache.spark.scheduler.{SchedulerBackend, TaskSchedulerImpl, WorkerOffer} @@ -115,6 +115,7 @@ private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: localActor ! StatusUpdate(taskId, state, serializedData) } - override def applicationId: String = "local-" + System.currentTimeMillis + override def applicationId = + new ApplicationId("local-" + System.currentTimeMillis) } diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index 3f1cd0752e76..ebf659dd03ee 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -28,7 +28,7 @@ import org.scalatest.FunSuite import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse} import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, RecoveryState, WorkerInfo} import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner} -import org.apache.spark.SparkConf +import org.apache.spark.{ApplicationId, SparkConf} class JsonProtocolSuite extends FunSuite { @@ -94,7 +94,7 @@ class JsonProtocolSuite extends FunSuite { def createAppInfo() : ApplicationInfo = { val appInfo = new ApplicationInfo(JsonConstants.appInfoStartTime, - "id", createAppDesc(), JsonConstants.submitDate, null, Int.MaxValue) + new ApplicationId("id"), createAppDesc(), JsonConstants.submitDate, null, Int.MaxValue) appInfo.endTime = JsonConstants.currTimeInMillis appInfo } @@ -117,7 +117,7 @@ class JsonProtocolSuite extends FunSuite { } def createExecutorRunner(): ExecutorRunner = { - new ExecutorRunner("appId", 123, createAppDesc(), 4, 1234, null, "workerId", "host", + new ExecutorRunner(new ApplicationId("appId"), 123, createAppDesc(), 4, 1234, null, "workerId", "host", new File("sparkHome"), new File("workDir"), "akka://worker", new SparkConf, ExecutorState.RUNNING) } diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index 39ab53cf0b5b..4b0a6a6e1851 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -22,7 +22,7 @@ import java.io.File import org.scalatest.FunSuite import org.apache.spark.deploy.{ApplicationDescription, Command, ExecutorState} -import org.apache.spark.SparkConf +import org.apache.spark.{ApplicationId, SparkConf} class ExecutorRunnerTest extends FunSuite { test("command includes appId") { @@ -30,7 +30,7 @@ class ExecutorRunnerTest extends FunSuite { val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!")) val appDesc = new ApplicationDescription("app name", Some(8), 500, Command("foo", Seq(), Map(), Seq(), Seq(), Seq()), "appUiUrl") - val appId = "12345-worker321-9876" + val appId = new ApplicationId("12345-worker321-9876") val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", f(sparkHome), f("ooga"), "blah", new SparkConf, ExecutorState.RUNNING) diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index e5315bc93e21..d26d88b3e182 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.fs.{FileStatus, Path} import org.json4s.jackson.JsonMethods._ import org.scalatest.{BeforeAndAfter, FunSuite} -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.{ApplicationId, SparkConf, SparkContext} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec import org.apache.spark.SPARK_VERSION @@ -169,7 +169,7 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { // Verify logging directory exists val conf = getLoggingConf(logDirPath, compressionCodec) - val eventLogger = new EventLoggingListener("test", conf) + val eventLogger = new EventLoggingListener(new ApplicationId("test"), "test", conf) eventLogger.start() val logPath = new Path(eventLogger.logDir) assert(fileSystem.exists(logPath)) @@ -209,7 +209,7 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { // Verify that all information is correctly parsed before stop() val conf = getLoggingConf(logDirPath, compressionCodec) - val eventLogger = new EventLoggingListener("test", conf) + val eventLogger = new EventLoggingListener(new ApplicationId("test"), "testdir", conf) eventLogger.start() var eventLoggingInfo = EventLoggingListener.parseLoggingInfo(eventLogger.logDir, fileSystem) assertInfoCorrect(eventLoggingInfo, loggerStopped = false) @@ -228,7 +228,7 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { */ private def testEventLogging(compressionCodec: Option[String] = None) { val conf = getLoggingConf(logDirPath, compressionCodec) - val eventLogger = new EventLoggingListener("test", conf) + val eventLogger = new EventLoggingListener(new ApplicationId("test"), "testdir", conf) val listenerBus = new LiveListenerBus val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", None, 125L, "Mickey") diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index 7ab351d1b4d2..cef8ff8702d9 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -24,7 +24,7 @@ import org.json4s.jackson.JsonMethods._ import org.scalatest.{BeforeAndAfter, FunSuite} import org.apache.spark.SparkContext._ -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.{ApplicationId, SparkConf, SparkContext} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec import org.apache.spark.util.{JsonProtocol, Utils} @@ -155,7 +155,8 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { * This child listener inherits only the event buffering functionality, but does not actually * log the events. */ - private class EventMonster(conf: SparkConf) extends EventLoggingListener("test", conf) { + private class EventMonster(conf: SparkConf) + extends EventLoggingListener(new ApplicationId("test"), "testdir", conf) { logger.close() } diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index d2b65a7ec6a2..ce424e6018e0 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -17,8 +17,8 @@ package org.apache.spark.scheduler.cluster -import org.apache.hadoop.yarn.api.records.{ApplicationId, YarnApplicationState} -import org.apache.spark.{SparkException, Logging, SparkContext} +import org.apache.hadoop.yarn.api.records.{ApplicationId => YApplicationId, YarnApplicationState} +import org.apache.spark.{ApplicationId, SparkException, Logging, SparkContext} import org.apache.spark.deploy.yarn.{Client, ClientArguments} import org.apache.spark.scheduler.TaskSchedulerImpl @@ -35,7 +35,7 @@ private[spark] class YarnClientSchedulerBackend( } private var client: Client = null - private var appId: ApplicationId = null + private var appId: YApplicationId = null private var stopping: Boolean = false private var totalExpectedExecutors = 0 @@ -155,8 +155,8 @@ private[spark] class YarnClientSchedulerBackend( totalRegisteredExecutors.get() >= totalExpectedExecutors * minRegisteredRatio } - override def applicationId(): String = - Option(appId).map(_.toString()).getOrElse { + override def applicationId() = + Option(appId).map(yAppId => new ApplicationId(yAppId.toString)).getOrElse { logWarning("Application ID is not initialized yet.") super.applicationId } diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala index 6f9c96c1a7b3..f59ff2d8a039 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler.cluster -import org.apache.spark.SparkContext +import org.apache.spark.{ApplicationId, SparkContext} import org.apache.spark.deploy.yarn.ApplicationMasterArguments import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.util.IntParam @@ -48,15 +48,17 @@ private[spark] class YarnClusterSchedulerBackend( totalRegisteredExecutors.get() >= totalExpectedExecutors * minRegisteredRatio } - override def applicationId(): String = + override def applicationId() = /** * In YARN Cluster mode, spark.yarn.app.id is expect to be set * before user application is launched. * So, if spark.yarn.app.id is not set, it is something wrong. */ - sc.getConf.getOption("spark.yarn.app.id").getOrElse { - logError("Application ID is not set.") - super.applicationId - } + sc.getConf.getOption("spark.yarn.app.id") + .map(strAppId => new ApplicationId(strAppId)) + .getOrElse { + logError("Application ID is not set.") + super.applicationId + } } From 0a2fc14a2c8cf639361e94a5b31414a62ada6e36 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 26 Sep 2014 17:34:26 +0900 Subject: [PATCH 21/42] Modified style --- .../main/scala/org/apache/spark/deploy/DeployMessage.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index 99f7da5edc46..d3e4c610c03d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -72,7 +72,11 @@ private[deploy] object DeployMessages { case class RegisterWorkerFailed(message: String) extends DeployMessage - case class KillExecutor(masterUrl: String, appId: ApplicationId, execId: Int) extends DeployMessage + case class KillExecutor( + masterUrl: String, + appId: ApplicationId, + execId: Int) + extends DeployMessage case class LaunchExecutor( masterUrl: String, From 0325caf6f7c5f42b9e8ffa279e558806119a9035 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 26 Sep 2014 17:44:27 +0900 Subject: [PATCH 22/42] Added ApplicationId.scala --- .../org/apache/spark/ApplicationId.scala | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) create mode 100644 core/src/main/scala/org/apache/spark/ApplicationId.scala diff --git a/core/src/main/scala/org/apache/spark/ApplicationId.scala b/core/src/main/scala/org/apache/spark/ApplicationId.scala new file mode 100644 index 000000000000..5480ba50b916 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ApplicationId.scala @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +/** + * This class represents unique application id for identifying each application + */ +private[spark] case class ApplicationId(appId: String) { + override def toString = appId +} From 9ff4851b263bbb80ee8130347fc9e06d214df96c Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Mon, 29 Sep 2014 14:05:34 +0900 Subject: [PATCH 23/42] Modified MimaExcludes.scala to ignore createTaskScheduler method in SparkContext Converted ApplicationId.scala into ApplicationId.java so that Java API and Py4J can use the class easily --- .../org/apache/spark/ApplicationId.java} | 32 +++++++++++++++++-- project/MimaExcludes.scala | 7 +++- 2 files changed, 35 insertions(+), 4 deletions(-) rename core/src/main/{scala/org/apache/spark/ApplicationId.scala => java/org/apache/spark/ApplicationId.java} (63%) diff --git a/core/src/main/scala/org/apache/spark/ApplicationId.scala b/core/src/main/java/org/apache/spark/ApplicationId.java similarity index 63% rename from core/src/main/scala/org/apache/spark/ApplicationId.scala rename to core/src/main/java/org/apache/spark/ApplicationId.java index 5480ba50b916..14cfae2bfe93 100644 --- a/core/src/main/scala/org/apache/spark/ApplicationId.scala +++ b/core/src/main/java/org/apache/spark/ApplicationId.java @@ -15,11 +15,37 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark; /** * This class represents unique application id for identifying each application */ -private[spark] case class ApplicationId(appId: String) { - override def toString = appId +public class ApplicationId { + + private String appId; + + public ApplicationId(String appId) { + this.appId = appId; + } + + @Override + public boolean equals(Object other) { + if (!(other instanceof ApplicationId)) { + return false; + } else if (other == this) { + return true; + } else { + return appId.equals(other); + } + } + + @Override + public int hashCode() { + return appId.hashCode(); + } + + @Override + public String toString() { + return appId; + } } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 3280e662fa0b..3257d53e781f 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -39,7 +39,12 @@ object MimaExcludes { MimaBuild.excludeSparkPackage("graphx") ) ++ MimaBuild.excludeSparkClass("mllib.linalg.Matrix") ++ - MimaBuild.excludeSparkClass("mllib.linalg.Vector") + MimaBuild.excludeSparkClass("mllib.linalg.Vector") ++ + Seq( + // Ignore SparkContext.createTaskScheduler because it's a private method. + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.SparkContext.org$apache$spark$SparkContext$$createTaskScheduler") + ) case v if v.startsWith("1.1") => Seq( From dfc83fd4ca52af4dd498d888ed8fdd3ba6ec67df Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Mon, 29 Sep 2014 15:26:11 +0900 Subject: [PATCH 24/42] Modified ApplicationId to implement Serializable --- core/src/main/java/org/apache/spark/ApplicationId.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/spark/ApplicationId.java b/core/src/main/java/org/apache/spark/ApplicationId.java index 14cfae2bfe93..a3a6da21d599 100644 --- a/core/src/main/java/org/apache/spark/ApplicationId.java +++ b/core/src/main/java/org/apache/spark/ApplicationId.java @@ -17,10 +17,12 @@ package org.apache.spark; +import java.io.Serializable; + /** * This class represents unique application id for identifying each application */ -public class ApplicationId { +public class ApplicationId implements Serializable { private String appId; From d009c55abcc67edb4d3c6d2e286d81c219faf3b1 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Mon, 29 Sep 2014 16:58:06 +0900 Subject: [PATCH 25/42] Modified ApplicationId#equals to compare appIds --- core/src/main/java/org/apache/spark/ApplicationId.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/spark/ApplicationId.java b/core/src/main/java/org/apache/spark/ApplicationId.java index a3a6da21d599..850163044ec7 100644 --- a/core/src/main/java/org/apache/spark/ApplicationId.java +++ b/core/src/main/java/org/apache/spark/ApplicationId.java @@ -37,7 +37,7 @@ public boolean equals(Object other) { } else if (other == this) { return true; } else { - return appId.equals(other); + return appId.equals(((ApplicationId)other).appId); } } From 3011efcbe3d32a8d0c7fc29f643f3afb57692758 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Mon, 29 Sep 2014 18:51:07 +0900 Subject: [PATCH 26/42] Added ApplicationIdSuite.scala --- .../java/org/apache/spark/ApplicationId.java | 6 +- .../org/apache/spark/ApplicationIdSuite.java | 88 +++++++++++++++++++ 2 files changed, 92 insertions(+), 2 deletions(-) create mode 100644 core/src/test/java/org/apache/spark/ApplicationIdSuite.java diff --git a/core/src/main/java/org/apache/spark/ApplicationId.java b/core/src/main/java/org/apache/spark/ApplicationId.java index 850163044ec7..1da4b6718023 100644 --- a/core/src/main/java/org/apache/spark/ApplicationId.java +++ b/core/src/main/java/org/apache/spark/ApplicationId.java @@ -36,9 +36,11 @@ public boolean equals(Object other) { return false; } else if (other == this) { return true; - } else { + }/* else if (appId != null) { return appId.equals(((ApplicationId)other).appId); - } + } else { + return false; + }*/ else return appId.equals(((ApplicationId)other).appId); } @Override diff --git a/core/src/test/java/org/apache/spark/ApplicationIdSuite.java b/core/src/test/java/org/apache/spark/ApplicationIdSuite.java new file mode 100644 index 000000000000..b828adbc8acd --- /dev/null +++ b/core/src/test/java/org/apache/spark/ApplicationIdSuite.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark; + +import static org.hamcrest.CoreMatchers.*; +import static org.junit.Assert.assertThat; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.ApplicationId; + +public class ApplicationIdSuite { + + private ApplicationId appId1_1, appId1_2, appId1_3, appId2, appIdNull_1, appIdNull_2; + + @Before + public void setUp() { + appId1_1 = new ApplicationId("appId1"); + appId1_2 = new ApplicationId("appId1"); + appId1_3 = new ApplicationId(new String("appId1")); + appId2 = new ApplicationId("appId2"); + appIdNull_1 = new ApplicationId(null); + appIdNull_2 = new ApplicationId(null); + } + + @Test + public void testEquality() { + // When ID strings are same object, ApplicationIds are equivalent + assertThat(appId1_1.toString(), is(sameInstance(appId1_2.toString()))); + assertThat(appId1_1.equals(appId1_2), is(true)); + + // When ID strings are not same object but equivalent, ApplicationIds are equivalent + assertThat(appId1_1.toString(), is(not(sameInstance(appId1_3.toString())))); + assertThat(appId1_1.toString(), is(appId1_3.toString())); + assertThat(appId1_1.equals(appId1_3), is(true)); + + // When ID strings are not equivalent, ApplicationIds are not equivalent + assertThat(appId1_1.toString(), is(not(appId2.toString()))); + assertThat(appId1_1.equals(appId2), is(false)); + + // When one of ApplicationIds has null ID, ApplicationIds are not equivalent + assertThat(appIdNull_1.toString(), is(nullValue())); + assertThat(appIdNull_2.toString(), is(nullValue())); + assertThat(appId1_1.equals(appIdNull_1), is(false)); + assertThat(appIdNull_1.equals(appIdNull_2), is(false)); + } + + @Test + public void testEqualsMethodSpecification() { + // Test reflexivity rule + assertThat(appId1_1.equals(appId1_1), is(true)); + + // Test transitivity rule + assertThat(appId1_1.equals(appId1_2), is(true)); + assertThat(appId1_2.equals(appId1_3), is(true)); + assertThat(appId1_1.equals(appId1_3), is(true)); + + // Test symmetric rule + assertThat(appId1_2.equals(appId1_1), is(true)); + assertThat(appId1_3.equals(appId1_2), is(true)); + assertThat(appId1_3.equals(appId1_1), is(true)); + + // Test consistency rule + assertThat(appId1_1.equals(appId1_2), is(appId1_1.equals(appId1_2))); + assertThat(appId1_3.equals(appId2), is(appId1_3.equals(appId2))); + + // Test comparision with null + assertThat(appId1_1.equals(null), is(false)); + } + +} \ No newline at end of file From 9aadb0b15bd6bff3602035e60b981ca3d9d993b0 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 30 Sep 2014 02:03:54 +0900 Subject: [PATCH 27/42] Modified NetworkReceiverSuite to ensure "executor.start()" is finished in test "network receiver life cycle" --- .../main/java/org/apache/spark/ApplicationId.java | 4 ++-- .../spark/streaming/NetworkReceiverSuite.scala | 14 ++++++++++---- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/org/apache/spark/ApplicationId.java b/core/src/main/java/org/apache/spark/ApplicationId.java index 1da4b6718023..8ef9a09235ff 100644 --- a/core/src/main/java/org/apache/spark/ApplicationId.java +++ b/core/src/main/java/org/apache/spark/ApplicationId.java @@ -36,11 +36,11 @@ public boolean equals(Object other) { return false; } else if (other == this) { return true; - }/* else if (appId != null) { + } else if (appId != null) { return appId.equals(((ApplicationId)other).appId); } else { return false; - }*/ else return appId.equals(((ApplicationId)other).appId); + } } @Override diff --git a/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala index 99c8d13231aa..eb6e88cf5520 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.streaming import java.nio.ByteBuffer +import java.util.concurrent.Semaphore import scala.collection.mutable.ArrayBuffer @@ -36,6 +37,7 @@ class NetworkReceiverSuite extends FunSuite with Timeouts { val receiver = new FakeReceiver val executor = new FakeReceiverSupervisor(receiver) + val executorStarted = new Semaphore(0) assert(executor.isAllEmpty) @@ -43,6 +45,7 @@ class NetworkReceiverSuite extends FunSuite with Timeouts { val executingThread = new Thread() { override def run() { executor.start() + executorStarted.release(1) executor.awaitTermination() } } @@ -57,6 +60,9 @@ class NetworkReceiverSuite extends FunSuite with Timeouts { } } + // Ensure executor is started + executorStarted.acquire() + // Verify that receiver was started assert(receiver.onStartCalled) assert(executor.isReceiverStarted) @@ -186,10 +192,10 @@ class NetworkReceiverSuite extends FunSuite with Timeouts { * An implementation of NetworkReceiver that is used for testing a receiver's life cycle. */ class FakeReceiver extends Receiver[Int](StorageLevel.MEMORY_ONLY) { - var otherThread: Thread = null - var receiving = false - var onStartCalled = false - var onStopCalled = false + @volatile var otherThread: Thread = null + @volatile var receiving = false + @volatile var onStartCalled = false + @volatile var onStopCalled = false def onStart() { otherThread = new Thread() { From 2cdd009158c38040613bbe27c4a62d2c372a40ed Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 30 Sep 2014 08:30:05 +0900 Subject: [PATCH 28/42] Modified defailt implementation of applicationId --- .../main/java/org/apache/spark/ApplicationId.java | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/core/src/main/java/org/apache/spark/ApplicationId.java b/core/src/main/java/org/apache/spark/ApplicationId.java index 8ef9a09235ff..dc394fe3081d 100644 --- a/core/src/main/java/org/apache/spark/ApplicationId.java +++ b/core/src/main/java/org/apache/spark/ApplicationId.java @@ -52,4 +52,18 @@ public int hashCode() { public String toString() { return appId; } + + + private static ApplicationId defaultAppId = + new ApplicationId(String.valueOf(System.currentTimeMillis())); + + /** + * Default implementation of Application Id + * + * @return Default implementation of Application id based on System.currentTimeMillis + */ + public static ApplicationId getApplicationId() { + return defaultAppId; + } + } From f6af132cbfba32d2e7434f6b027f1cd6188ea6f2 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 30 Sep 2014 14:13:22 +0900 Subject: [PATCH 29/42] Modified SchedulerBackend and TaskScheduler to return System.currentTimeMillis as an unique Application Id --- .../main/java/org/apache/spark/ApplicationId.java | 13 ------------- .../apache/spark/scheduler/SchedulerBackend.scala | 4 +++- .../org/apache/spark/scheduler/TaskScheduler.scala | 4 +++- .../apache/spark/scheduler/local/LocalBackend.scala | 5 +++-- 4 files changed, 9 insertions(+), 17 deletions(-) diff --git a/core/src/main/java/org/apache/spark/ApplicationId.java b/core/src/main/java/org/apache/spark/ApplicationId.java index dc394fe3081d..a7a2844fa811 100644 --- a/core/src/main/java/org/apache/spark/ApplicationId.java +++ b/core/src/main/java/org/apache/spark/ApplicationId.java @@ -53,17 +53,4 @@ public String toString() { return appId; } - - private static ApplicationId defaultAppId = - new ApplicationId(String.valueOf(System.currentTimeMillis())); - - /** - * Default implementation of Application Id - * - * @return Default implementation of Application id based on System.currentTimeMillis - */ - public static ApplicationId getApplicationId() { - return defaultAppId; - } - } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala index 71d05088a425..3df64a1b9c19 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -25,6 +25,8 @@ import org.apache.spark.ApplicationId * machines become available and can launch tasks on them. */ private[spark] trait SchedulerBackend { + private val appId = new ApplicationId(System.currentTimeMillis.toString) + def start(): Unit def stop(): Unit def reviveOffers(): Unit @@ -41,6 +43,6 @@ private[spark] trait SchedulerBackend { * * @return The application ID, if the backend does not provide an ID. */ - def applicationId() = new ApplicationId(System.currentTimeMillis.toString) + def applicationId() = appId } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 27c9a546cce7..7be1312cdb14 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -32,6 +32,8 @@ import org.apache.spark.storage.BlockManagerId */ private[spark] trait TaskScheduler { + private val appId = new ApplicationId(System.currentTimeMillis.toString) + def rootPool: Pool def schedulingMode: SchedulingMode @@ -73,6 +75,6 @@ private[spark] trait TaskScheduler { * * @return The application ID, if the backend does not provide an ID. */ - def applicationId() = new ApplicationId(System.currentTimeMillis.toString) + def applicationId() = appId } diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index 5c1e8a1bbb80..464ad5a9d0d8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -88,6 +88,7 @@ private[spark] class LocalActor( private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: Int) extends SchedulerBackend with ExecutorBackend { + private val appId = new ApplicationId("local-" + System.currentTimeMillis) var localActor: ActorRef = null override def start() { @@ -115,7 +116,7 @@ private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: localActor ! StatusUpdate(taskId, state, serializedData) } - override def applicationId = - new ApplicationId("local-" + System.currentTimeMillis) + override def applicationId = appId + } From 0413b90c50fa4c6fc2057bb1d95ddb3785168545 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 2 Oct 2014 10:12:57 +0900 Subject: [PATCH 30/42] Deleted ApplicationId.java and ApplicationIdSuite.java --- .../java/org/apache/spark/ApplicationId.java | 56 ------------ .../org/apache/spark/ApplicationIdSuite.java | 88 ------------------- 2 files changed, 144 deletions(-) delete mode 100644 core/src/main/java/org/apache/spark/ApplicationId.java delete mode 100644 core/src/test/java/org/apache/spark/ApplicationIdSuite.java diff --git a/core/src/main/java/org/apache/spark/ApplicationId.java b/core/src/main/java/org/apache/spark/ApplicationId.java deleted file mode 100644 index a7a2844fa811..000000000000 --- a/core/src/main/java/org/apache/spark/ApplicationId.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark; - -import java.io.Serializable; - -/** - * This class represents unique application id for identifying each application - */ -public class ApplicationId implements Serializable { - - private String appId; - - public ApplicationId(String appId) { - this.appId = appId; - } - - @Override - public boolean equals(Object other) { - if (!(other instanceof ApplicationId)) { - return false; - } else if (other == this) { - return true; - } else if (appId != null) { - return appId.equals(((ApplicationId)other).appId); - } else { - return false; - } - } - - @Override - public int hashCode() { - return appId.hashCode(); - } - - @Override - public String toString() { - return appId; - } - -} diff --git a/core/src/test/java/org/apache/spark/ApplicationIdSuite.java b/core/src/test/java/org/apache/spark/ApplicationIdSuite.java deleted file mode 100644 index b828adbc8acd..000000000000 --- a/core/src/test/java/org/apache/spark/ApplicationIdSuite.java +++ /dev/null @@ -1,88 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark; - -import static org.hamcrest.CoreMatchers.*; -import static org.junit.Assert.assertThat; - -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -import org.apache.spark.ApplicationId; - -public class ApplicationIdSuite { - - private ApplicationId appId1_1, appId1_2, appId1_3, appId2, appIdNull_1, appIdNull_2; - - @Before - public void setUp() { - appId1_1 = new ApplicationId("appId1"); - appId1_2 = new ApplicationId("appId1"); - appId1_3 = new ApplicationId(new String("appId1")); - appId2 = new ApplicationId("appId2"); - appIdNull_1 = new ApplicationId(null); - appIdNull_2 = new ApplicationId(null); - } - - @Test - public void testEquality() { - // When ID strings are same object, ApplicationIds are equivalent - assertThat(appId1_1.toString(), is(sameInstance(appId1_2.toString()))); - assertThat(appId1_1.equals(appId1_2), is(true)); - - // When ID strings are not same object but equivalent, ApplicationIds are equivalent - assertThat(appId1_1.toString(), is(not(sameInstance(appId1_3.toString())))); - assertThat(appId1_1.toString(), is(appId1_3.toString())); - assertThat(appId1_1.equals(appId1_3), is(true)); - - // When ID strings are not equivalent, ApplicationIds are not equivalent - assertThat(appId1_1.toString(), is(not(appId2.toString()))); - assertThat(appId1_1.equals(appId2), is(false)); - - // When one of ApplicationIds has null ID, ApplicationIds are not equivalent - assertThat(appIdNull_1.toString(), is(nullValue())); - assertThat(appIdNull_2.toString(), is(nullValue())); - assertThat(appId1_1.equals(appIdNull_1), is(false)); - assertThat(appIdNull_1.equals(appIdNull_2), is(false)); - } - - @Test - public void testEqualsMethodSpecification() { - // Test reflexivity rule - assertThat(appId1_1.equals(appId1_1), is(true)); - - // Test transitivity rule - assertThat(appId1_1.equals(appId1_2), is(true)); - assertThat(appId1_2.equals(appId1_3), is(true)); - assertThat(appId1_1.equals(appId1_3), is(true)); - - // Test symmetric rule - assertThat(appId1_2.equals(appId1_1), is(true)); - assertThat(appId1_3.equals(appId1_2), is(true)); - assertThat(appId1_3.equals(appId1_1), is(true)); - - // Test consistency rule - assertThat(appId1_1.equals(appId1_2), is(appId1_1.equals(appId1_2))); - assertThat(appId1_3.equals(appId2), is(appId1_3.equals(appId2))); - - // Test comparision with null - assertThat(appId1_1.equals(null), is(false)); - } - -} \ No newline at end of file From 6434b06bdb1356dda00b7cd986e5b78caeeb2225 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 2 Oct 2014 11:03:37 +0900 Subject: [PATCH 31/42] Reverted changes related to ApplicationId --- .../scala/org/apache/spark/SparkContext.scala | 6 +++--- .../org/apache/spark/deploy/DeployMessage.scala | 15 +++++---------- .../apache/spark/deploy/ExecutorDescription.scala | 4 +--- .../org/apache/spark/deploy/JsonProtocol.scala | 4 ++-- .../apache/spark/deploy/client/AppClient.scala | 4 ++-- .../spark/deploy/client/AppClientListener.scala | 4 +--- .../apache/spark/deploy/client/TestClient.scala | 4 ++-- .../spark/deploy/master/ApplicationInfo.scala | 3 +-- .../org/apache/spark/deploy/master/Master.scala | 8 ++++---- .../spark/deploy/worker/ExecutorRunner.scala | 10 +++++----- .../spark/scheduler/EventLoggingListener.scala | 8 ++++---- .../apache/spark/scheduler/SchedulerBackend.scala | 4 +--- .../apache/spark/scheduler/TaskScheduler.scala | 3 +-- .../cluster/SparkDeploySchedulerBackend.scala | 6 +++--- .../mesos/CoarseMesosSchedulerBackend.scala | 4 ++-- .../cluster/mesos/MesosSchedulerBackend.scala | 4 ++-- .../spark/scheduler/local/LocalBackend.scala | 5 ++--- .../apache/spark/deploy/JsonProtocolSuite.scala | 6 +++--- .../spark/deploy/worker/ExecutorRunnerTest.scala | 7 +++---- .../scheduler/EventLoggingListenerSuite.scala | 4 ++-- .../spark/scheduler/ReplayListenerSuite.scala | 4 ++-- .../cluster/YarnClientSchedulerBackend.scala | 8 ++++---- .../cluster/YarnClusterSchedulerBackend.scala | 12 +++++------- 23 files changed, 60 insertions(+), 77 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 98469a47fcdb..b20766172d9c 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -307,7 +307,7 @@ class SparkContext(config: SparkConf) extends Logging { taskScheduler.start() private val appId = taskScheduler.applicationId() - conf.set("spark.app.id", appId.toString) + conf.set("spark.app.id", appId) val metricsSystem = env.metricsSystem metricsSystem.registerSources() @@ -1298,8 +1298,8 @@ class SparkContext(config: SparkConf) extends Logging { private def postApplicationStart() { // Note: this code assumes that the task scheduler has been initialized and has contacted // the cluster manager to get an application ID (in case the cluster manager provides one). - listenerBus.post(SparkListenerApplicationStart(appName, - Some(taskScheduler.applicationId().toString), startTime, sparkUser)) + listenerBus.post(SparkListenerApplicationStart(appName, Some(taskScheduler.applicationId()), + startTime, sparkUser)) } /** Post the application end event */ diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index d3e4c610c03d..a7368f9f3dfb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -19,7 +19,6 @@ package org.apache.spark.deploy import scala.collection.immutable.List -import org.apache.spark.ApplicationId import org.apache.spark.deploy.ExecutorState.ExecutorState import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, WorkerInfo} import org.apache.spark.deploy.master.DriverState.DriverState @@ -48,7 +47,7 @@ private[deploy] object DeployMessages { } case class ExecutorStateChanged( - appId: ApplicationId, + appId: String, execId: Int, state: ExecutorState, message: Option[String], @@ -72,15 +71,11 @@ private[deploy] object DeployMessages { case class RegisterWorkerFailed(message: String) extends DeployMessage - case class KillExecutor( - masterUrl: String, - appId: ApplicationId, - execId: Int) - extends DeployMessage + case class KillExecutor(masterUrl: String, appId: String, execId: Int) extends DeployMessage case class LaunchExecutor( masterUrl: String, - appId: ApplicationId, + appId: String, execId: Int, appDesc: ApplicationDescription, cores: Int, @@ -100,11 +95,11 @@ private[deploy] object DeployMessages { case class RegisterApplication(appDescription: ApplicationDescription) extends DeployMessage - case class MasterChangeAcknowledged(appId: ApplicationId) + case class MasterChangeAcknowledged(appId: String) // Master to AppClient - case class RegisteredApplication(appId: ApplicationId, masterUrl: String) extends DeployMessage + case class RegisteredApplication(appId: String, masterUrl: String) extends DeployMessage // TODO(matei): replace hostPort with host case class ExecutorAdded(id: Int, workerId: String, hostPort: String, cores: Int, memory: Int) { diff --git a/core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala index 47bdbbff4a68..2abf0b69dddb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala @@ -17,15 +17,13 @@ package org.apache.spark.deploy -import org.apache.spark.ApplicationId - /** * Used to send state on-the-wire about Executors from Worker to Master. * This state is sufficient for the Master to reconstruct its internal data structures during * failover. */ private[spark] class ExecutorDescription( - val appId: ApplicationId, + val appId: String, val execId: Int, val cores: Int, val state: ExecutorState.Value) diff --git a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala index 7f127cf76bf0..696f32a6f573 100644 --- a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala @@ -41,7 +41,7 @@ private[spark] object JsonProtocol { def writeApplicationInfo(obj: ApplicationInfo) = { ("starttime" -> obj.startTime) ~ - ("id" -> obj.id.toString) ~ + ("id" -> obj.id) ~ ("name" -> obj.desc.name) ~ ("cores" -> obj.desc.maxCores) ~ ("user" -> obj.desc.user) ~ @@ -62,7 +62,7 @@ private[spark] object JsonProtocol { def writeExecutorRunner(obj: ExecutorRunner) = { ("id" -> obj.execId) ~ ("memory" -> obj.memory) ~ - ("appid" -> obj.appId.toString) ~ + ("appid" -> obj.appId) ~ ("appdesc" -> writeApplicationDescription(obj.appDesc)) } diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala index 1ec0c1298202..32790053a6be 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala @@ -26,7 +26,7 @@ import akka.actor._ import akka.pattern.ask import akka.remote.{AssociationErrorEvent, DisassociatedEvent, RemotingLifecycleEvent} -import org.apache.spark.{ApplicationId, Logging, SparkConf, SparkException} +import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.Master @@ -52,7 +52,7 @@ private[spark] class AppClient( var masterAddress: Address = null var actor: ActorRef = null - var appId: ApplicationId = null + var appId: String = null var registered = false var activeMasterUrl: String = null diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala index f33241df7b8d..e584952a9ad8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala @@ -17,8 +17,6 @@ package org.apache.spark.deploy.client -import org.apache.spark.ApplicationId - /** * Callbacks invoked by deploy client when various events happen. There are currently four events: * connecting to the cluster, disconnecting, being given an executor, and having an executor @@ -27,7 +25,7 @@ import org.apache.spark.ApplicationId * Users of this API should *not* block inside the callback methods. */ private[spark] trait AppClientListener { - def connected(appId: ApplicationId): Unit + def connected(appId: String): Unit /** Disconnection may be a temporary state, as we fail over to a new Master. */ def disconnected(): Unit diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala index f9ce7f5585c9..88a0862b96af 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala @@ -17,14 +17,14 @@ package org.apache.spark.deploy.client -import org.apache.spark.{ApplicationId, SecurityManager, SparkConf, Logging} +import org.apache.spark.{SecurityManager, SparkConf, Logging} import org.apache.spark.deploy.{ApplicationDescription, Command} import org.apache.spark.util.{AkkaUtils, Utils} private[spark] object TestClient { class TestListener extends AppClientListener with Logging { - def connected(id: ApplicationId) { + def connected(id: String) { logInfo("Connected to master, got app ID " + id) } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index 5d232489376b..c3ca43f8d073 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -24,12 +24,11 @@ import scala.collection.mutable.ArrayBuffer import akka.actor.ActorRef -import org.apache.spark.ApplicationId import org.apache.spark.deploy.ApplicationDescription private[spark] class ApplicationInfo( val startTime: Long, - val id: ApplicationId, + val id: String, val desc: ApplicationDescription, val submitDate: Date, val driver: ActorRef, diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index cc119cd047f1..72974a16015e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -32,7 +32,7 @@ import akka.pattern.ask import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} import akka.serialization.SerializationExtension -import org.apache.spark.{ApplicationId, Logging, SecurityManager, SparkConf, SparkException} +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, ExecutorState, SparkHadoopUtil} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.history.HistoryServer @@ -68,13 +68,13 @@ private[spark] class Master( val addressToWorker = new HashMap[Address, WorkerInfo] val apps = new HashSet[ApplicationInfo] - val idToApp = new HashMap[ApplicationId, ApplicationInfo] + val idToApp = new HashMap[String, ApplicationInfo] val actorToApp = new HashMap[ActorRef, ApplicationInfo] val addressToApp = new HashMap[Address, ApplicationInfo] val waitingApps = new ArrayBuffer[ApplicationInfo] val completedApps = new ArrayBuffer[ApplicationInfo] var nextAppNumber = 0 - val appIdToUI = new HashMap[ApplicationId, SparkUI] + val appIdToUI = new HashMap[String, SparkUI] val drivers = new HashSet[DriverInfo] val completedDrivers = new ArrayBuffer[DriverInfo] @@ -738,7 +738,7 @@ private[spark] class Master( def newApplicationId(submitDate: Date) = { val appId = "app-%s-%04d".format(createDateFormat.format(submitDate), nextAppNumber) nextAppNumber += 1 - new ApplicationId(appId) + appId } /** Check for, and remove, any timed-out workers */ diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 2182d7214656..00a43673e5cd 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -23,7 +23,7 @@ import akka.actor.ActorRef import com.google.common.base.Charsets import com.google.common.io.Files -import org.apache.spark.{ApplicationId, SparkConf, Logging} +import org.apache.spark.{SparkConf, Logging} import org.apache.spark.deploy.{ApplicationDescription, Command, ExecutorState} import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged import org.apache.spark.util.logging.FileAppender @@ -33,7 +33,7 @@ import org.apache.spark.util.logging.FileAppender * This is currently only used in standalone mode. */ private[spark] class ExecutorRunner( - val appId: ApplicationId, + val appId: String, val execId: Int, val appDesc: ApplicationDescription, val cores: Int, @@ -48,7 +48,7 @@ private[spark] class ExecutorRunner( var state: ExecutorState.Value) extends Logging { - val fullId = s"${appId}/${execId}" + val fullId = appId + "/" + execId var workerThread: Thread = null var process: Process = null var stdoutAppender: FileAppender = null @@ -117,7 +117,7 @@ private[spark] class ExecutorRunner( def getCommandSeq = { val command = Command( appDesc.command.mainClass, - appDesc.command.arguments.map(substituteVariables) ++ Seq(appId.toString), + appDesc.command.arguments.map(substituteVariables) ++ Seq(appId), appDesc.command.environment, appDesc.command.classPathEntries, appDesc.command.libraryPathEntries, @@ -131,7 +131,7 @@ private[spark] class ExecutorRunner( def fetchAndRunExecutor() { try { // Create the executor's working directory - val executorDir = new File(workDir, s"${appId}/${execId}") + val executorDir = new File(workDir, appId + "/" + execId) if (!executorDir.mkdirs()) { throw new IOException("Failed to create directory " + executorDir) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index c7bcada0a7c5..e0945099ddb0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.fs.permission.FsPermission import org.json4s.JsonAST.JValue import org.json4s.jackson.JsonMethods._ -import org.apache.spark.{ApplicationId, Logging, SparkConf, SparkContext} +import org.apache.spark.{Logging, SparkConf, SparkContext} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec import org.apache.spark.SPARK_VERSION @@ -43,7 +43,7 @@ import org.apache.spark.util.{FileLogger, JsonProtocol, Utils} * spark.eventLog.buffer.kb - Buffer size to use when writing to output streams */ private[spark] class EventLoggingListener( - appId: ApplicationId, + appId: String, logBaseDir: String, sparkConf: SparkConf, hadoopConf: Configuration) @@ -51,7 +51,7 @@ private[spark] class EventLoggingListener( import EventLoggingListener._ - def this(appId: ApplicationId, logBaseDir: String, sparkConf: SparkConf) = + def this(appId: String, logBaseDir: String, sparkConf: SparkConf) = this(appId, logBaseDir, sparkConf, SparkHadoopUtil.get.newConfiguration(sparkConf)) private val shouldCompress = sparkConf.getBoolean("spark.eventLog.compress", false) @@ -186,7 +186,7 @@ private[spark] object EventLoggingListener extends Logging { } else "" } - def getLogDirName(logBaseDir: String, appId: ApplicationId) = { + def getLogDirName(logBaseDir: String, appId: String) = { val name = appId.toString.replaceAll("[ :/]", "-").replaceAll("[${}'\"]", "_").toLowerCase Utils.resolveURI(logBaseDir) + "/" + name.stripSuffix("/") } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala index 3df64a1b9c19..7807c2271fc6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -17,15 +17,13 @@ package org.apache.spark.scheduler -import org.apache.spark.ApplicationId - /** * A backend interface for scheduling systems that allows plugging in different ones under * TaskSchedulerImpl. We assume a Mesos-like model where the application gets resource offers as * machines become available and can launch tasks on them. */ private[spark] trait SchedulerBackend { - private val appId = new ApplicationId(System.currentTimeMillis.toString) + private val appId = "spark-application-" + System.currentTimeMillis def start(): Unit def stop(): Unit diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 7be1312cdb14..12c834c4d388 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -17,7 +17,6 @@ package org.apache.spark.scheduler -import org.apache.spark.ApplicationId import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId @@ -32,7 +31,7 @@ import org.apache.spark.storage.BlockManagerId */ private[spark] trait TaskScheduler { - private val appId = new ApplicationId(System.currentTimeMillis.toString) + private val appId = "spark-application-" + System.currentTimeMillis def rootPool: Pool diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index fdde85654e47..412f24aa5404 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler.cluster -import org.apache.spark.{ApplicationId, Logging, SparkConf, SparkContext, SparkEnv} +import org.apache.spark.{Logging, SparkConf, SparkContext, SparkEnv} import org.apache.spark.deploy.{ApplicationDescription, Command} import org.apache.spark.deploy.client.{AppClient, AppClientListener} import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason, SlaveLost, TaskSchedulerImpl} @@ -35,7 +35,7 @@ private[spark] class SparkDeploySchedulerBackend( var client: AppClient = null var stopping = false var shutdownCallback : (SparkDeploySchedulerBackend) => Unit = _ - @volatile var appId: ApplicationId = _ + @volatile var appId: String = _ val registrationLock = new Object() var registrationDone = false @@ -87,7 +87,7 @@ private[spark] class SparkDeploySchedulerBackend( } } - override def connected(appId: ApplicationId) { + override def connected(appId: String) { logInfo("Connected to Spark cluster with app ID " + appId) this.appId = appId notifyContext() diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 5bf99bdf9385..23f17c2ce50b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -28,7 +28,7 @@ import org.apache.mesos.{Scheduler => MScheduler} import org.apache.mesos._ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} -import org.apache.spark.{ApplicationId, Logging, SparkContext, SparkEnv, SparkException} +import org.apache.spark.{Logging, SparkContext, SparkEnv, SparkException} import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend @@ -314,7 +314,7 @@ private[spark] class CoarseMesosSchedulerBackend( } override def applicationId = - Option(appId).map(mAppId => new ApplicationId(mAppId.getValue)).getOrElse { + Option(appId).map(_.getValue).getOrElse { logWarning("Application ID is not initialized yet.") super.applicationId } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index f96cabe6a6d2..d1352d2b254a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -29,7 +29,7 @@ import org.apache.mesos.{Scheduler => MScheduler} import org.apache.mesos._ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} -import org.apache.spark.{ApplicationId, Logging, SparkContext, SparkException, TaskState} +import org.apache.spark.{Logging, SparkContext, SparkException, TaskState} import org.apache.spark.scheduler._ import org.apache.spark.util.Utils @@ -354,7 +354,7 @@ private[spark] class MesosSchedulerBackend( override def defaultParallelism() = sc.conf.getInt("spark.default.parallelism", 8) override def applicationId() = - Option(appId).map(mAppId => new ApplicationId(mAppId.getValue)).getOrElse { + Option(appId).map(_.getValue).getOrElse { logWarning("Application ID is not initialized yet.") super.applicationId } diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index 464ad5a9d0d8..70c337b3522e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -21,7 +21,7 @@ import java.nio.ByteBuffer import akka.actor.{Actor, ActorRef, Props} -import org.apache.spark.{ApplicationId, Logging, SparkEnv, TaskState} +import org.apache.spark.{Logging, SparkEnv, TaskState} import org.apache.spark.TaskState.TaskState import org.apache.spark.executor.{Executor, ExecutorBackend} import org.apache.spark.scheduler.{SchedulerBackend, TaskSchedulerImpl, WorkerOffer} @@ -88,7 +88,7 @@ private[spark] class LocalActor( private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: Int) extends SchedulerBackend with ExecutorBackend { - private val appId = new ApplicationId("local-" + System.currentTimeMillis) + private val appId = "local-" + System.currentTimeMillis var localActor: ActorRef = null override def start() { @@ -118,5 +118,4 @@ private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: override def applicationId = appId - } diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index ebf659dd03ee..3f1cd0752e76 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -28,7 +28,7 @@ import org.scalatest.FunSuite import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse} import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, RecoveryState, WorkerInfo} import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner} -import org.apache.spark.{ApplicationId, SparkConf} +import org.apache.spark.SparkConf class JsonProtocolSuite extends FunSuite { @@ -94,7 +94,7 @@ class JsonProtocolSuite extends FunSuite { def createAppInfo() : ApplicationInfo = { val appInfo = new ApplicationInfo(JsonConstants.appInfoStartTime, - new ApplicationId("id"), createAppDesc(), JsonConstants.submitDate, null, Int.MaxValue) + "id", createAppDesc(), JsonConstants.submitDate, null, Int.MaxValue) appInfo.endTime = JsonConstants.currTimeInMillis appInfo } @@ -117,7 +117,7 @@ class JsonProtocolSuite extends FunSuite { } def createExecutorRunner(): ExecutorRunner = { - new ExecutorRunner(new ApplicationId("appId"), 123, createAppDesc(), 4, 1234, null, "workerId", "host", + new ExecutorRunner("appId", 123, createAppDesc(), 4, 1234, null, "workerId", "host", new File("sparkHome"), new File("workDir"), "akka://worker", new SparkConf, ExecutorState.RUNNING) } diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index b7fc5ad8dec0..39ab53cf0b5b 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -22,7 +22,7 @@ import java.io.File import org.scalatest.FunSuite import org.apache.spark.deploy.{ApplicationDescription, Command, ExecutorState} -import org.apache.spark.{ApplicationId, SparkConf} +import org.apache.spark.SparkConf class ExecutorRunnerTest extends FunSuite { test("command includes appId") { @@ -30,11 +30,10 @@ class ExecutorRunnerTest extends FunSuite { val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!")) val appDesc = new ApplicationDescription("app name", Some(8), 500, Command("foo", Seq(), Map(), Seq(), Seq(), Seq()), "appUiUrl") - val appIdStr = "12345-worker321-9876" - val appId = new ApplicationId(appIdStr) + val appId = "12345-worker321-9876" val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", f(sparkHome), f("ooga"), "blah", new SparkConf, ExecutorState.RUNNING) - assert(er.getCommandSeq.last === appIdStr) + assert(er.getCommandSeq.last === appId) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index 84d14b4e4234..3efa85431876 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.fs.{FileStatus, Path} import org.json4s.jackson.JsonMethods._ import org.scalatest.{BeforeAndAfter, FunSuite} -import org.apache.spark.{ApplicationId, SparkConf, SparkContext} +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec import org.apache.spark.SPARK_VERSION @@ -415,5 +415,5 @@ object EventLoggingListenerSuite { conf } - def getUniqueApplicationId = new ApplicationId("test-" + System.currentTimeMillis()) + def getUniqueApplicationId = "test-" + System.currentTimeMillis } diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index cef8ff8702d9..48114feee623 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -24,7 +24,7 @@ import org.json4s.jackson.JsonMethods._ import org.scalatest.{BeforeAndAfter, FunSuite} import org.apache.spark.SparkContext._ -import org.apache.spark.{ApplicationId, SparkConf, SparkContext} +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec import org.apache.spark.util.{JsonProtocol, Utils} @@ -156,7 +156,7 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { * log the events. */ private class EventMonster(conf: SparkConf) - extends EventLoggingListener(new ApplicationId("test"), "testdir", conf) { + extends EventLoggingListener("test", "testdir", conf) { logger.close() } diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index ce424e6018e0..5dd3ef437697 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -17,8 +17,8 @@ package org.apache.spark.scheduler.cluster -import org.apache.hadoop.yarn.api.records.{ApplicationId => YApplicationId, YarnApplicationState} -import org.apache.spark.{ApplicationId, SparkException, Logging, SparkContext} +import org.apache.hadoop.yarn.api.records.{ApplicationId, YarnApplicationState} +import org.apache.spark.{SparkException, Logging, SparkContext} import org.apache.spark.deploy.yarn.{Client, ClientArguments} import org.apache.spark.scheduler.TaskSchedulerImpl @@ -35,7 +35,7 @@ private[spark] class YarnClientSchedulerBackend( } private var client: Client = null - private var appId: YApplicationId = null + private var appId: ApplicationId = null private var stopping: Boolean = false private var totalExpectedExecutors = 0 @@ -156,7 +156,7 @@ private[spark] class YarnClientSchedulerBackend( } override def applicationId() = - Option(appId).map(yAppId => new ApplicationId(yAppId.toString)).getOrElse { + Option(appId).map(_.toString).getOrElse { logWarning("Application ID is not initialized yet.") super.applicationId } diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala index f59ff2d8a039..25bfabdd97c4 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler.cluster -import org.apache.spark.{ApplicationId, SparkContext} +import org.apache.spark.SparkContext import org.apache.spark.deploy.yarn.ApplicationMasterArguments import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.util.IntParam @@ -54,11 +54,9 @@ private[spark] class YarnClusterSchedulerBackend( * before user application is launched. * So, if spark.yarn.app.id is not set, it is something wrong. */ - sc.getConf.getOption("spark.yarn.app.id") - .map(strAppId => new ApplicationId(strAppId)) - .getOrElse { - logError("Application ID is not set.") - super.applicationId - } + sc.getConf.getOption("spark.yarn.app.id").map(_.toString).getOrElse { + logError("Application ID is not set.") + super.applicationId + } } From 16a9f01bfa0f0915f860febb0435af5ed29b91de Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 2 Oct 2014 15:30:11 +0900 Subject: [PATCH 32/42] Added data types to be returned to some methods Modified the signature of SparkContext#createTaskScheduler --- .../main/scala/org/apache/spark/SparkContext.scala | 13 +++++-------- .../org/apache/spark/deploy/master/Master.scala | 2 +- .../spark/scheduler/EventLoggingListener.scala | 7 +------ .../apache/spark/scheduler/SchedulerBackend.scala | 2 +- .../cluster/SparkDeploySchedulerBackend.scala | 2 +- .../cluster/mesos/CoarseMesosSchedulerBackend.scala | 10 +++++----- .../cluster/mesos/MesosSchedulerBackend.scala | 10 +++++----- .../apache/spark/scheduler/local/LocalBackend.scala | 2 +- project/MimaExcludes.scala | 5 ----- .../cluster/YarnClientSchedulerBackend.scala | 2 +- .../cluster/YarnClusterSchedulerBackend.scala | 2 +- 11 files changed, 22 insertions(+), 35 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index b20766172d9c..badb250fa265 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -188,7 +188,7 @@ class SparkContext(config: SparkConf) extends Logging { val appName = conf.get("spark.app.name") val isEventLogEnabled = conf.getBoolean("spark.eventLog.enabled", false) - val eventLogDir = { + val eventLogDir: Option[String] = { if (isEventLogEnabled) { Some(conf.get("spark.eventLog.dir", EventLoggingListener.DEFAULT_LOG_DIR).stripSuffix("/")) } else { @@ -291,7 +291,7 @@ class SparkContext(config: SparkConf) extends Logging { executorEnvs("SPARK_USER") = sparkUser // Create and start the scheduler - private[spark] var taskScheduler = SparkContext.createTaskScheduler(this, master, eventLogDir) + private[spark] var taskScheduler = SparkContext.createTaskScheduler(this, master) private val heartbeatReceiver = env.actorSystem.actorOf( Props(new HeartbeatReceiver(taskScheduler)), "HeartbeatReceiver") @volatile private[spark] var dagScheduler: DAGScheduler = _ @@ -1493,10 +1493,7 @@ object SparkContext extends Logging { } /** Creates a task scheduler based on a given master URL. Extracted for testing. */ - private def createTaskScheduler( - sc: SparkContext, - master: String, - eventLogDir: Option[String]): TaskScheduler = { + private def createTaskScheduler(sc: SparkContext, master: String): TaskScheduler = { // Regular expression used for local[N] and local[*] master formats val LOCAL_N_REGEX = """local\[([0-9]+|\*)\]""".r // Regular expression for local[N, maxRetries], used in tests with failing tasks @@ -1542,7 +1539,7 @@ object SparkContext extends Logging { case SPARK_REGEX(sparkUrl) => val scheduler = new TaskSchedulerImpl(sc) val masterUrls = sparkUrl.split(",").map("spark://" + _) - val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls, eventLogDir) + val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls, sc.eventLogDir) scheduler.initialize(backend) scheduler @@ -1559,7 +1556,7 @@ object SparkContext extends Logging { val localCluster = new LocalSparkCluster( numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt) val masterUrls = localCluster.start() - val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls, eventLogDir) + val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls, sc.eventLogDir) scheduler.initialize(backend) backend.shutdownCallback = (backend: SparkDeploySchedulerBackend) => { localCluster.stop() diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 72974a16015e..9b4a241a7a44 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -735,7 +735,7 @@ private[spark] class Master( } /** Generate a new app ID given a app's submission date */ - def newApplicationId(submitDate: Date) = { + def newApplicationId(submitDate: Date): String = { val appId = "app-%s-%04d".format(createDateFormat.format(submitDate), nextAppNumber) nextAppNumber += 1 appId diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index e0945099ddb0..bff11482f533 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -59,11 +59,6 @@ private[spark] class EventLoggingListener( private val testing = sparkConf.getBoolean("spark.eventLog.testing", false) private val outputBufferSize = sparkConf.getInt("spark.eventLog.buffer.kb", 100) * 1024 val logDir = EventLoggingListener.getLogDirName(logBaseDir, appId) - private val name = { - val splitPath = logDir.split("/") - splitPath(splitPath.length-1) - } - protected val logger = new FileLogger(logDir, sparkConf, hadoopConf, outputBufferSize, shouldCompress, shouldOverwrite, Some(LOG_FILE_PERMISSIONS)) @@ -74,7 +69,7 @@ private[spark] class EventLoggingListener( * Return only the unique application directory without the base directory. */ def getApplicationLogDir(): String = { - name + logDir.split("/").last } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala index 7807c2271fc6..bb0ba9ee092c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -41,6 +41,6 @@ private[spark] trait SchedulerBackend { * * @return The application ID, if the backend does not provide an ID. */ - def applicationId() = appId + def applicationId(): String = appId } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 412f24aa5404..1eb9425d0bd4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -129,7 +129,7 @@ private[spark] class SparkDeploySchedulerBackend( totalCoreCount.get() >= totalExpectedCores * minRegisteredRatio } - override def applicationId() = + override def applicationId(): String = Option(appId).getOrElse { logWarning("Application ID is not initialized yet.") super.applicationId diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 23f17c2ce50b..e823e43c3897 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -76,7 +76,7 @@ private[spark] class CoarseMesosSchedulerBackend( var nextMesosTaskId = 0 - @volatile var appId: FrameworkID = _ + @volatile var appId: String = _ def newMesosTaskId(): Int = { val id = nextMesosTaskId @@ -169,8 +169,8 @@ private[spark] class CoarseMesosSchedulerBackend( override def offerRescinded(d: SchedulerDriver, o: OfferID) {} override def registered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) { - appId = frameworkId - logInfo("Registered as framework ID " + frameworkId.getValue) + appId = frameworkId.getValue + logInfo("Registered as framework ID " + appId) registeredLock.synchronized { isRegistered = true registeredLock.notifyAll() @@ -313,8 +313,8 @@ private[spark] class CoarseMesosSchedulerBackend( slaveLost(d, s) } - override def applicationId = - Option(appId).map(_.getValue).getOrElse { + override def applicationId(): String = + Option(appId).getOrElse { logWarning("Application ID is not initialized yet.") super.applicationId } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index d1352d2b254a..d6321bcc98a8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -62,7 +62,7 @@ private[spark] class MesosSchedulerBackend( var classLoader: ClassLoader = null - @volatile var appId: FrameworkID = _ + @volatile var appId: String = _ override def start() { synchronized { @@ -170,8 +170,8 @@ private[spark] class MesosSchedulerBackend( override def registered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) { val oldClassLoader = setClassLoader() try { - appId = frameworkId - logInfo("Registered as framework ID " + frameworkId.getValue) + appId = frameworkId.getValue + logInfo("Registered as framework ID " + appId) registeredLock.synchronized { isRegistered = true registeredLock.notifyAll() @@ -353,8 +353,8 @@ private[spark] class MesosSchedulerBackend( // TODO: query Mesos for number of cores override def defaultParallelism() = sc.conf.getInt("spark.default.parallelism", 8) - override def applicationId() = - Option(appId).map(_.getValue).getOrElse { + override def applicationId(): String = + Option(appId).getOrElse { logWarning("Application ID is not initialized yet.") super.applicationId } diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index 70c337b3522e..58b78f041cd8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -116,6 +116,6 @@ private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: localActor ! StatusUpdate(taskId, state, serializedData) } - override def applicationId = appId + override def applicationId(): String = appId } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 4298e40b4487..4076ebc6fc8d 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -40,11 +40,6 @@ object MimaExcludes { ) ++ MimaBuild.excludeSparkClass("mllib.linalg.Matrix") ++ MimaBuild.excludeSparkClass("mllib.linalg.Vector") ++ - Seq( - // Ignore SparkContext.createTaskScheduler because it's a private method. - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.SparkContext.org$apache$spark$SparkContext$$createTaskScheduler") - ) ++ Seq( // Added normL1 and normL2 to trait MultivariateStatisticalSummary ProblemFilters.exclude[MissingMethodProblem]( diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 5dd3ef437697..6bb4b82316ad 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -155,7 +155,7 @@ private[spark] class YarnClientSchedulerBackend( totalRegisteredExecutors.get() >= totalExpectedExecutors * minRegisteredRatio } - override def applicationId() = + override def applicationId(): String = Option(appId).map(_.toString).getOrElse { logWarning("Application ID is not initialized yet.") super.applicationId diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala index 25bfabdd97c4..3d042907ee42 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala @@ -48,7 +48,7 @@ private[spark] class YarnClusterSchedulerBackend( totalRegisteredExecutors.get() >= totalExpectedExecutors * minRegisteredRatio } - override def applicationId() = + override def applicationId(): String = /** * In YARN Cluster mode, spark.yarn.app.id is expect to be set * before user application is launched. From 5cca0d2a794ef639b6ff0332f494adc8745634e8 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 2 Oct 2014 17:16:31 +0900 Subject: [PATCH 33/42] Added Javadoc comment to SparkContext#getApplicationId --- core/src/main/scala/org/apache/spark/SparkContext.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index badb250fa265..ee1a2c5fa224 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -352,7 +352,12 @@ class SparkContext(config: SparkConf) extends Logging { localProperties.set(props) } - def getApplicationId = appId + /** + * Get an unique identifier which identifies each application. + * + * @return Application ID which is an unique identifier assigned to each application. + */ + def getApplicationId(): String = appId @deprecated("Properties no longer need to be explicitly initialized.", "1.0.0") def initLocalProperties() { From 69c46a6c94f90b701b1495b2f6308f2142213ad2 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 2 Oct 2014 17:17:47 +0900 Subject: [PATCH 34/42] Added warning logging logic to MetricsSystem#buildRegistryName Added Javadoc comment to MetricsSystem#buildRegistryName --- .../apache/spark/metrics/MetricsSystem.scala | 41 ++++++++++++++----- 1 file changed, 31 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index 23da502e687b..59b3c9caeaf4 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -96,17 +96,38 @@ private[spark] class MetricsSystem private ( sinks.foreach(_.report()) } - def buildRegistryName(source: Source) = { - for { - appName <- conf.getOption("spark.app.name") - appId <- conf.getOption("spark.app.id") - executorId <- conf.getOption("spark.executor.id") - } yield { - MetricRegistry.name(appId, appName, executorId, source.sourceName) + /** + * Build a unique name for each combination of application, executor/driver and metric source. + * The name is built like .. + * + * @param source Metric source to be named by this method. + * @return An unique metric name for each combination of application, executor/driver and metric source. + */ + def buildRegistryName(source: Source): String = + instance match { + /** + * Only Driver and Executor are set spark.app.id and spark.executor.id. + * For instance, Master and Worker are not related to a specific application. + */ + case "driver" | "executor" => + val appIdOpt = conf.getOption("spark.app.id") + val executorIdOpt = conf.getOption("spark.executor.id") + (for { + appId <- appIdOpt + executorId <- executorIdOpt + } yield { + MetricRegistry.name(appId, executorId, source.sourceName) + }).getOrElse { + if (appIdOpt == None) { + logWarning("spark.app.id is not set.") + } + if (executorIdOpt == None) { + logWarning("spark.executor.id is not set") + } + MetricRegistry.name(source.sourceName) + } + case _ => MetricRegistry.name(source.sourceName) } - }.getOrElse { - MetricRegistry.name(source.sourceName) - } def registerSource(source: Source) { sources += source From ff45c893fcd3e3a9217091c52bdd2b82c3f67187 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 2 Oct 2014 17:18:10 +0900 Subject: [PATCH 35/42] Added some test cases to MetricsSystemSuite --- .../spark/metrics/MetricsSystemSuite.scala | 126 +++++++++++++++++- 1 file changed, 124 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala index ea8111a6a72f..1b014856f2f5 100644 --- a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala @@ -17,14 +17,15 @@ package org.apache.spark.metrics -import org.apache.spark.metrics.source.Source import org.scalatest.{BeforeAndAfter, FunSuite, PrivateMethodTester} import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.master.MasterSource +import org.apache.spark.metrics.source.Source -import scala.collection.mutable.ArrayBuffer +import com.codahale.metrics.MetricRegistry +import scala.collection.mutable.ArrayBuffer class MetricsSystemSuite extends FunSuite with BeforeAndAfter with PrivateMethodTester{ var filePath: String = _ @@ -64,4 +65,125 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter with PrivateMethod metricsSystem.registerSource(source) assert(metricsSystem.invokePrivate(sources()).length === 1) } + + test("MetricsSystem with Driver instance") { + val source = new Source { + override val sourceName = "dummySource" + override val metricRegistry = new MetricRegistry() + } + + val appId = "testId" + val executorId = "driver" + conf.set("spark.app.id", appId) + conf.set("spark.executor.id", executorId) + + val instanceName = "driver" + val driverMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) + + val metricName = driverMetricsSystem.buildRegistryName(source) + assert(metricName == s"$appId.$executorId.${source.sourceName}") + } + + test("MetricsSystem with Driver instance and spark.app.id is not set") { + val source = new Source { + override val sourceName = "dummySource" + override val metricRegistry = new MetricRegistry() + } + + val executorId = "driver" + conf.set("spark.executor.id", executorId) + + val instanceName = "driver" + val driverMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) + + val metricName = driverMetricsSystem.buildRegistryName(source) + assert(metricName == source.sourceName) + } + + test("MetricsSystem with Driver instance and spark.executor.id is not set") { + val source = new Source { + override val sourceName = "dummySource" + override val metricRegistry = new MetricRegistry() + } + + val appId = "testId" + conf.set("spark.app.id", appId) + + val instanceName = "driver" + val driverMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) + + val metricName = driverMetricsSystem.buildRegistryName(source) + assert(metricName == source.sourceName) + } + + test("MetricsSystem with Executor instance") { + val source = new Source { + override val sourceName = "dummySource" + override val metricRegistry = new MetricRegistry() + } + + val appId = "testId" + val executorId = "executor.1" + conf.set("spark.app.id", appId) + conf.set("spark.executor.id", executorId) + + val instanceName = "executor" + val driverMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) + + val metricName = driverMetricsSystem.buildRegistryName(source) + assert(metricName == s"$appId.$executorId.${source.sourceName}") + } + + test("MetricsSystem with Executor instance and spark.app.id is not set") { + val source = new Source { + override val sourceName = "dummySource" + override val metricRegistry = new MetricRegistry() + } + + val executorId = "executor.1" + conf.set("spark.executor.id", executorId) + + val instanceName = "executor" + val driverMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) + + val metricName = driverMetricsSystem.buildRegistryName(source) + assert(metricName == source.sourceName) + } + + test("MetricsSystem with Executor instance and spark.executor.id is not set") { + val source = new Source { + override val sourceName = "dummySource" + override val metricRegistry = new MetricRegistry() + } + + val appId = "testId" + conf.set("spark.app.id", appId) + + val instanceName = "executor" + val driverMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) + + val metricName = driverMetricsSystem.buildRegistryName(source) + assert(metricName == source.sourceName) + } + + test("MetricsSystem with instance which is neither Driver nor Executor") { + val source = new Source { + override val sourceName = "dummySource" + override val metricRegistry = new MetricRegistry() + } + + val appId = "testId" + val executorId = "dummyExecutorId" + conf.set("spark.app.id", appId) + conf.set("spark.executor.id", executorId) + + val instanceName = "testInstance" + val driverMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) + + val metricName = driverMetricsSystem.buildRegistryName(source) + + // Even if spark.app.id and spark.executor.id are set, they are not used for the metric name. + assert(metricName != s"$appId.$executorId.${source.sourceName}") + assert(metricName == source.sourceName) + } } From 389090d33190acaaa72499d81cacce732ec34cf9 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 2 Oct 2014 17:33:15 +0900 Subject: [PATCH 36/42] Replaced taskScheduler.applicationId() with getApplicationId in SparkContext#postApplicationStart --- core/src/main/scala/org/apache/spark/SparkContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index ee1a2c5fa224..1ce44ccb078b 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1303,7 +1303,7 @@ class SparkContext(config: SparkConf) extends Logging { private def postApplicationStart() { // Note: this code assumes that the task scheduler has been initialized and has contacted // the cluster manager to get an application ID (in case the cluster manager provides one). - listenerBus.post(SparkListenerApplicationStart(appName, Some(taskScheduler.applicationId()), + listenerBus.post(SparkListenerApplicationStart(appName, Some(getApplicationId()), startTime, sparkUser)) } From f9b6fb36a6e0b385a1fe5a769295c133e519f8ce Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 2 Oct 2014 17:55:16 +0900 Subject: [PATCH 37/42] Modified style. --- .../main/scala/org/apache/spark/metrics/MetricsSystem.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index 59b3c9caeaf4..aa5b10ce3c4f 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -101,7 +101,8 @@ private[spark] class MetricsSystem private ( * The name is built like .. * * @param source Metric source to be named by this method. - * @return An unique metric name for each combination of application, executor/driver and metric source. + * @return An unique metric name for each combination of + * application, executor/driver and metric source. */ def buildRegistryName(source: Source): String = instance match { From 59cc2cd1bbc4be1a7773cec0f7b11b6def6ca442 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 2 Oct 2014 19:52:47 +0900 Subject: [PATCH 38/42] Modified SparkContextSchedulerCreationSuite --- .../org/apache/spark/SparkContextSchedulerCreationSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index 7e874c7897c0..495a0d48633a 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -32,7 +32,7 @@ class SparkContextSchedulerCreationSuite // real schedulers, so we don't want to create a full SparkContext with the desired scheduler. val sc = new SparkContext("local", "test") val createTaskSchedulerMethod = PrivateMethod[TaskScheduler]('createTaskScheduler) - val sched = SparkContext invokePrivate createTaskSchedulerMethod(sc, master, Some("testdir")) + val sched = SparkContext invokePrivate createTaskSchedulerMethod(sc, master) sched.asInstanceOf[TaskSchedulerImpl] } From 10be6547a443c8b8c30a47de122afdac9dbce930 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 3 Oct 2014 11:58:25 +0900 Subject: [PATCH 39/42] Fixed style. --- .../scala/org/apache/spark/SparkContext.scala | 4 +--- .../scala/org/apache/spark/SparkEnv.scala | 6 +++++- .../apache/spark/deploy/master/Master.scala | 2 +- .../apache/spark/metrics/MetricsSystem.scala | 5 +++-- .../scheduler/EventLoggingListener.scala | 21 ++++++++++--------- .../spark/scheduler/SchedulerBackend.scala | 6 ++---- .../spark/scheduler/TaskScheduler.scala | 8 +++---- .../spark/scheduler/TaskSchedulerImpl.scala | 2 +- 8 files changed, 27 insertions(+), 27 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 1ce44ccb078b..f0bbc5f2549a 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -353,9 +353,7 @@ class SparkContext(config: SparkConf) extends Logging { } /** - * Get an unique identifier which identifies each application. - * - * @return Application ID which is an unique identifier assigned to each application. + * Return a unique identifier for this application. */ def getApplicationId(): String = appId diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 63b12272fece..0b0ae12ae6fc 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -259,6 +259,11 @@ object SparkEnv extends Logging { } val metricsSystem = if (isDriver) { + /** + * Don't initialize/start metrics system right now for Driver. + * We need to wait for the task scheduler to give us an app ID. + * Then we can initialize/start the metrics system. + */ MetricsSystem.createMetricsSystem("driver", conf, securityManager) } else { val ms = MetricsSystem.createMetricsSystem("executor", conf, securityManager) @@ -268,7 +273,6 @@ object SparkEnv extends Logging { ms } - // Set the sparkFiles directory, used when downloading dependencies. In local mode, // this is a temporary directory; in distributed mode, this is the executor's current working // directory. diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 9b4a241a7a44..8872c2178181 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -693,7 +693,7 @@ private[spark] class Master( return false } - val appEventLogDir = EventLoggingListener.getLogDirName(eventLogDir, app.id) + val appEventLogDir = EventLoggingListener.getLogDirPath(eventLogDir, app.id) val fileSystem = Utils.getHadoopFileSystem(appEventLogDir, SparkHadoopUtil.get.newConfiguration(conf)) val eventLogInfo = EventLoggingListener.parseLoggingInfo(appEventLogDir, fileSystem) diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index aa5b10ce3c4f..270d245bdb66 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -97,8 +97,9 @@ private[spark] class MetricsSystem private ( } /** - * Build a unique name for each combination of application, executor/driver and metric source. - * The name is built like .. + * Build a name that uniquely identifies each metric source. + * The name is structured as follows: ... + * If either ID is not available, this defaults to just using . * * @param source Metric source to be named by this method. * @return An unique metric name for each combination of diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index bff11482f533..100c9ba9b780 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -58,20 +58,14 @@ private[spark] class EventLoggingListener( private val shouldOverwrite = sparkConf.getBoolean("spark.eventLog.overwrite", false) private val testing = sparkConf.getBoolean("spark.eventLog.testing", false) private val outputBufferSize = sparkConf.getInt("spark.eventLog.buffer.kb", 100) * 1024 - val logDir = EventLoggingListener.getLogDirName(logBaseDir, appId) + val logDir = EventLoggingListener.getLogDirPath(logBaseDir, appId) + val logDirName: String = logDir.split("/").last protected val logger = new FileLogger(logDir, sparkConf, hadoopConf, outputBufferSize, shouldCompress, shouldOverwrite, Some(LOG_FILE_PERMISSIONS)) // For testing. Keep track of all JSON serialized events that have been logged. private[scheduler] val loggedEvents = new ArrayBuffer[JValue] - /** - * Return only the unique application directory without the base directory. - */ - def getApplicationLogDir(): String = { - logDir.split("/").last - } - /** * Begin logging events. * If compression is used, log a file that indicates which compression library is used. @@ -181,8 +175,15 @@ private[spark] object EventLoggingListener extends Logging { } else "" } - def getLogDirName(logBaseDir: String, appId: String) = { - val name = appId.toString.replaceAll("[ :/]", "-").replaceAll("[${}'\"]", "_").toLowerCase + /** + * Return a file-system-safe path to the log directory for the given application. + * + * @param logBaseDir A base directory for the path to the log directory for given application. + * @param appId A unique app ID. + * @return A path which consists of file-system-safe characters. + */ + def getLogDirPath(logBaseDir: String, appId: String): String = { + val name = appId.replaceAll("[ :/]", "-").replaceAll("[${}'\"]", "_").toLowerCase Utils.resolveURI(logBaseDir) + "/" + name.stripSuffix("/") } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala index bb0ba9ee092c..992c477493d8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -35,11 +35,9 @@ private[spark] trait SchedulerBackend { def isReady(): Boolean = true /** - * The application ID associated with the job, if any. - * It is expected that the subclasses of TaskScheduler or SchedulerBackend - * override this method and return an unique application ID. + * Get an application ID associated with the job. * - * @return The application ID, if the backend does not provide an ID. + * @return An application ID */ def applicationId(): String = appId diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 12c834c4d388..a129a434c9a1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -68,12 +68,10 @@ private[spark] trait TaskScheduler { blockManagerId: BlockManagerId): Boolean /** - * The application ID associated with the job, if any. - * It is expected that the subclasses of TaskScheduler or SchedulerBackend - * override this method and return an unique application ID. + * Get an application ID associated with the job. * - * @return The application ID, if the backend does not provide an ID. + * @return An application ID */ - def applicationId() = appId + def applicationId(): String = appId } 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 232db9e5e54b..8ad5fb6b9e6e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -492,7 +492,7 @@ private[spark] class TaskSchedulerImpl( } } - override def applicationId() = backend.applicationId() + override def applicationId():String = backend.applicationId() } From 67fa5eb93dbb58576b7cf1c5555b7b8a4c6e5fd6 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 3 Oct 2014 13:08:01 +0900 Subject: [PATCH 40/42] Unified MetricsSystem#registerSources and registerSinks in start Fixed signature of the constructor of SparkDeploySchedulerBackend --- .../src/main/scala/org/apache/spark/SparkContext.scala | 9 +++++---- core/src/main/scala/org/apache/spark/SparkEnv.scala | 10 +++------- .../scala/org/apache/spark/metrics/MetricsSystem.scala | 4 +++- .../cluster/SparkDeploySchedulerBackend.scala | 5 ++--- .../org/apache/spark/metrics/MetricsSystemSuite.scala | 6 ++---- .../cluster/YarnClusterSchedulerBackend.scala | 8 +++----- 6 files changed, 18 insertions(+), 24 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index f0bbc5f2549a..9d75cb02eac2 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -310,8 +310,9 @@ class SparkContext(config: SparkConf) extends Logging { conf.set("spark.app.id", appId) val metricsSystem = env.metricsSystem - metricsSystem.registerSources() - metricsSystem.registerSinks() + + // The metrics system for Driver need to be set spark.app.id to app ID. + // So it should start after we get app ID from the task scheduler and set spark.app.id. metricsSystem.start() // Optionally log Spark events @@ -1542,7 +1543,7 @@ object SparkContext extends Logging { case SPARK_REGEX(sparkUrl) => val scheduler = new TaskSchedulerImpl(sc) val masterUrls = sparkUrl.split(",").map("spark://" + _) - val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls, sc.eventLogDir) + val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls) scheduler.initialize(backend) scheduler @@ -1559,7 +1560,7 @@ object SparkContext extends Logging { val localCluster = new LocalSparkCluster( numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt) val masterUrls = localCluster.start() - val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls, sc.eventLogDir) + val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls) scheduler.initialize(backend) backend.shutdownCallback = (backend: SparkDeploySchedulerBackend) => { localCluster.stop() diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 0b0ae12ae6fc..ab60fa0c137c 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -259,16 +259,12 @@ object SparkEnv extends Logging { } val metricsSystem = if (isDriver) { - /** - * Don't initialize/start metrics system right now for Driver. - * We need to wait for the task scheduler to give us an app ID. - * Then we can initialize/start the metrics system. - */ + // Don't start metrics system right now for Driver. + // We need to wait for the task scheduler to give us an app ID. + // Then we can start the metrics system. MetricsSystem.createMetricsSystem("driver", conf, securityManager) } else { val ms = MetricsSystem.createMetricsSystem("executor", conf, securityManager) - ms.registerSources() - ms.registerSinks() ms.start() ms } diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index 270d245bdb66..f3d4ed4ee0cc 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -85,6 +85,8 @@ private[spark] class MetricsSystem private ( metricsConfig.initialize() def start() { + registerSources() + registerSinks() sinks.foreach(_.start) } @@ -160,7 +162,7 @@ private[spark] class MetricsSystem private ( val source = Class.forName(classPath).newInstance() registerSource(source.asInstanceOf[Source]) } catch { - case e: Exception => logError("Source class " + classPath + " cannot be instantialized", e) + case e: Exception => logError("Source class " + classPath + " cannot be instantiated", e) } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 1eb9425d0bd4..ed209d195ec9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -26,8 +26,7 @@ import org.apache.spark.util.Utils private[spark] class SparkDeploySchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext, - masters: Array[String], - baseLogDir: Option[String]) + masters: Array[String]) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) with AppClientListener with Logging { @@ -70,7 +69,7 @@ private[spark] class SparkDeploySchedulerBackend( args, sc.executorEnvs, classPathEntries, libraryPathEntries, javaOpts) val appUIAddress = sc.ui.map(_.appUIAddress).getOrElse("") val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, - appUIAddress, baseLogDir) + appUIAddress, sc.eventLogDir) client = new AppClient(sc.env.actorSystem, masters, appDesc, this, conf) client.start() diff --git a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala index 1b014856f2f5..2fc59a841322 100644 --- a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala @@ -40,8 +40,7 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter with PrivateMethod test("MetricsSystem with default config") { val metricsSystem = MetricsSystem.createMetricsSystem("default", conf, securityMgr) - metricsSystem.registerSources() - metricsSystem.registerSinks() + metricsSystem.start() val sources = PrivateMethod[ArrayBuffer[Source]]('sources) val sinks = PrivateMethod[ArrayBuffer[Source]]('sinks) @@ -52,8 +51,7 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter with PrivateMethod test("MetricsSystem with sources add") { val metricsSystem = MetricsSystem.createMetricsSystem("test", conf, securityMgr) - metricsSystem.registerSources() - metricsSystem.registerSinks() + metricsSystem.start() val sources = PrivateMethod[ArrayBuffer[Source]]('sources) val sinks = PrivateMethod[ArrayBuffer[Source]]('sinks) diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala index 3d042907ee42..e4551babed42 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala @@ -49,11 +49,9 @@ private[spark] class YarnClusterSchedulerBackend( } override def applicationId(): String = - /** - * In YARN Cluster mode, spark.yarn.app.id is expect to be set - * before user application is launched. - * So, if spark.yarn.app.id is not set, it is something wrong. - */ + // In YARN Cluster mode, spark.yarn.app.id is expect to be set + // before user application is launched. + // So, if spark.yarn.app.id is not set, it is something wrong. sc.getConf.getOption("spark.yarn.app.id").map(_.toString).getOrElse { logError("Application ID is not set.") super.applicationId From 817e4f03ad85e242855f805d795aca50e0599339 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 3 Oct 2014 13:30:41 +0900 Subject: [PATCH 41/42] Simplified MetricsSystem#buildRegistryName --- .../apache/spark/metrics/MetricsSystem.scala | 43 ++++++++----------- 1 file changed, 18 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index f3d4ed4ee0cc..5dd67b0cbf68 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -107,31 +107,24 @@ private[spark] class MetricsSystem private ( * @return An unique metric name for each combination of * application, executor/driver and metric source. */ - def buildRegistryName(source: Source): String = - instance match { - /** - * Only Driver and Executor are set spark.app.id and spark.executor.id. - * For instance, Master and Worker are not related to a specific application. - */ - case "driver" | "executor" => - val appIdOpt = conf.getOption("spark.app.id") - val executorIdOpt = conf.getOption("spark.executor.id") - (for { - appId <- appIdOpt - executorId <- executorIdOpt - } yield { - MetricRegistry.name(appId, executorId, source.sourceName) - }).getOrElse { - if (appIdOpt == None) { - logWarning("spark.app.id is not set.") - } - if (executorIdOpt == None) { - logWarning("spark.executor.id is not set") - } - MetricRegistry.name(source.sourceName) - } - case _ => MetricRegistry.name(source.sourceName) - } + def buildRegistryName(source: Source): String = { + val appId = conf.getOption("spark.app.id") + val executorId = conf.getOption("spark.executor.id") + val defaultName = MetricRegistry.name(source.sourceName) + + if (instance == "driver" || instance == "executor") { + if (appId.isDefined && executorId.isDefined) { + MetricRegistry.name(appId.get, executorId.get, source.sourceName) + } else { + // Only Driver and Executor are set spark.app.id and spark.executor.id. + // For instance, Master and Worker are not related to a specific application. + val warningMsg = s"Using default name $defaultName for source because %s is not set." + if (appId.isEmpty) { logWarning(warningMsg.format("spark.app.id")) } + if (executorId.isEmpty) { logWarning(warningMsg.format("spark.executor.id")) } + defaultName + } + } else { defaultName } + } def registerSource(source: Source) { sources += source From 39169e4b948d7837a4401a11a236e4cd87336582 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Sat, 4 Oct 2014 04:28:53 +0900 Subject: [PATCH 42/42] Fixed style Replaced getApplicationId with applicationId in SparkContext Replaced == with === in MetricsSystemSuite --- bin/compute-classpath.cmd | 8 +- bin/compute-classpath.sh | 8 +- .../scala/org/apache/spark/SparkContext.scala | 18 +- .../scala/org/apache/spark/SparkEnv.scala | 6 +- .../spark/deploy/SparkSubmitArguments.scala | 42 +-- .../apache/spark/deploy/master/Master.scala | 3 +- .../spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../spark/deploy/SparkSubmitSuite.scala | 34 +- .../spark/metrics/MetricsSystemSuite.scala | 14 +- docs/configuration.md | 7 + .../spark/mllib/tree/DecisionTree.scala | 140 ++++++--- .../spark/mllib/tree/RandomForest.scala | 5 +- .../mllib/tree/impl/DTStatsAggregator.scala | 292 +++++------------- .../tree/model/InformationGainStats.scala | 11 + .../spark/mllib/tree/RandomForestSuite.scala | 1 + sbin/spark-config.sh | 2 +- .../org/apache/spark/sql/hive/hiveUdfs.scala | 46 ++- .../sql/hive/execution/HiveUdfSuite.scala | 4 + .../cluster/YarnClusterSchedulerBackend.scala | 2 +- .../spark/deploy/yarn/YarnClusterSuite.scala | 35 ++- 20 files changed, 352 insertions(+), 328 deletions(-) diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd index 5ad52452a5c9..9b9e40321ea9 100644 --- a/bin/compute-classpath.cmd +++ b/bin/compute-classpath.cmd @@ -36,7 +36,13 @@ rem Load environment variables from conf\spark-env.cmd, if it exists if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" rem Build up classpath -set CLASSPATH=%SPARK_CLASSPATH%;%SPARK_SUBMIT_CLASSPATH%;%FWDIR%conf +set CLASSPATH=%SPARK_CLASSPATH%;%SPARK_SUBMIT_CLASSPATH% + +if "x%SPARK_CONF_DIR%"!="x" ( + set CLASSPATH=%CLASSPATH%;%SPARK_CONF_DIR% +) else ( + set CLASSPATH=%CLASSPATH%;%FWDIR%conf +) if exist "%FWDIR%RELEASE" ( for %%d in ("%FWDIR%lib\spark-assembly*.jar") do ( diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 0f63e36d8aec..905bbaf99b37 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -27,8 +27,14 @@ FWDIR="$(cd "`dirname "$0"`"/..; pwd)" . "$FWDIR"/bin/load-spark-env.sh +CLASSPATH="$SPARK_CLASSPATH:$SPARK_SUBMIT_CLASSPATH" + # Build up classpath -CLASSPATH="$SPARK_CLASSPATH:$SPARK_SUBMIT_CLASSPATH:$FWDIR/conf" +if [ -n "$SPARK_CONF_DIR" ]; then + CLASSPATH="$CLASSPATH:$SPARK_CONF_DIR" +else + CLASSPATH="$CLASSPATH:$FWDIR/conf" +fi ASSEMBLY_DIR="$FWDIR/assembly/target/scala-$SCALA_VERSION" diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 9d75cb02eac2..97109b9f41b6 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -187,8 +187,8 @@ class SparkContext(config: SparkConf) extends Logging { val master = conf.get("spark.master") val appName = conf.get("spark.app.name") - val isEventLogEnabled = conf.getBoolean("spark.eventLog.enabled", false) - val eventLogDir: Option[String] = { + private[spark] val isEventLogEnabled = conf.getBoolean("spark.eventLog.enabled", false) + private[spark] val eventLogDir: Option[String] = { if (isEventLogEnabled) { Some(conf.get("spark.eventLog.dir", EventLoggingListener.DEFAULT_LOG_DIR).stripSuffix("/")) } else { @@ -306,8 +306,8 @@ class SparkContext(config: SparkConf) extends Logging { // constructor taskScheduler.start() - private val appId = taskScheduler.applicationId() - conf.set("spark.app.id", appId) + val applicationId: String = taskScheduler.applicationId() + conf.set("spark.app.id", applicationId) val metricsSystem = env.metricsSystem @@ -318,7 +318,8 @@ class SparkContext(config: SparkConf) extends Logging { // Optionally log Spark events private[spark] val eventLogger: Option[EventLoggingListener] = { if (isEventLogEnabled) { - val logger = new EventLoggingListener(appId, eventLogDir.get, conf, hadoopConfiguration) + val logger = + new EventLoggingListener(applicationId, eventLogDir.get, conf, hadoopConfiguration) logger.start() listenerBus.addListener(logger) Some(logger) @@ -353,11 +354,6 @@ class SparkContext(config: SparkConf) extends Logging { localProperties.set(props) } - /** - * Return a unique identifier for this application. - */ - def getApplicationId(): String = appId - @deprecated("Properties no longer need to be explicitly initialized.", "1.0.0") def initLocalProperties() { localProperties.set(new Properties()) @@ -1302,7 +1298,7 @@ class SparkContext(config: SparkConf) extends Logging { private def postApplicationStart() { // Note: this code assumes that the task scheduler has been initialized and has contacted // the cluster manager to get an application ID (in case the cluster manager provides one). - listenerBus.post(SparkListenerApplicationStart(appName, Some(getApplicationId()), + listenerBus.post(SparkListenerApplicationStart(appName, Some(applicationId), startTime, sparkUser)) } diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index ab60fa0c137c..72cac42cd2b2 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -259,9 +259,9 @@ object SparkEnv extends Logging { } val metricsSystem = if (isDriver) { - // Don't start metrics system right now for Driver. - // We need to wait for the task scheduler to give us an app ID. - // Then we can start the metrics system. + // Don't start metrics system right now for Driver. + // We need to wait for the task scheduler to give us an app ID. + // Then we can start the metrics system. MetricsSystem.createMetricsSystem("driver", conf, securityManager) } else { val ms = MetricsSystem.createMetricsSystem("executor", conf, securityManager) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 2b72c61cc817..57b251ff4771 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -29,8 +29,9 @@ import org.apache.spark.util.Utils /** * Parses and encapsulates arguments from the spark-submit script. + * The env argument is used for testing. */ -private[spark] class SparkSubmitArguments(args: Seq[String]) { +private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, String] = sys.env) { var master: String = null var deployMode: String = null var executorMemory: String = null @@ -90,20 +91,12 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { private def mergeSparkProperties(): Unit = { // Use common defaults file, if not specified by user if (propertiesFile == null) { - sys.env.get("SPARK_CONF_DIR").foreach { sparkConfDir => - val sep = File.separator - val defaultPath = s"${sparkConfDir}${sep}spark-defaults.conf" - val file = new File(defaultPath) - if (file.exists()) { - propertiesFile = file.getAbsolutePath - } - } - } + val sep = File.separator + val sparkHomeConfig = env.get("SPARK_HOME").map(sparkHome => s"${sparkHome}${sep}conf") + val confDir = env.get("SPARK_CONF_DIR").orElse(sparkHomeConfig) - if (propertiesFile == null) { - sys.env.get("SPARK_HOME").foreach { sparkHome => - val sep = File.separator - val defaultPath = s"${sparkHome}${sep}conf${sep}spark-defaults.conf" + confDir.foreach { sparkConfDir => + val defaultPath = s"${sparkConfDir}${sep}spark-defaults.conf" val file = new File(defaultPath) if (file.exists()) { propertiesFile = file.getAbsolutePath @@ -117,19 +110,18 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { // Use properties file as fallback for values which have a direct analog to // arguments in this script. - master = Option(master).getOrElse(properties.get("spark.master").orNull) - executorMemory = Option(executorMemory) - .getOrElse(properties.get("spark.executor.memory").orNull) - executorCores = Option(executorCores) - .getOrElse(properties.get("spark.executor.cores").orNull) + master = Option(master).orElse(properties.get("spark.master")).orNull + executorMemory = Option(executorMemory).orElse(properties.get("spark.executor.memory")).orNull + executorCores = Option(executorCores).orElse(properties.get("spark.executor.cores")).orNull totalExecutorCores = Option(totalExecutorCores) - .getOrElse(properties.get("spark.cores.max").orNull) - name = Option(name).getOrElse(properties.get("spark.app.name").orNull) - jars = Option(jars).getOrElse(properties.get("spark.jars").orNull) + .orElse(properties.get("spark.cores.max")) + .orNull + name = Option(name).orElse(properties.get("spark.app.name")).orNull + jars = Option(jars).orElse(properties.get("spark.jars")).orNull // This supports env vars in older versions of Spark - master = Option(master).getOrElse(System.getenv("MASTER")) - deployMode = Option(deployMode).getOrElse(System.getenv("DEPLOY_MODE")) + master = Option(master).orElse(env.get("MASTER")).orNull + deployMode = Option(deployMode).orElse(env.get("DEPLOY_MODE")).orNull // Try to set main class from JAR if no --class argument is given if (mainClass == null && !isPython && primaryResource != null) { @@ -182,7 +174,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { } if (master.startsWith("yarn")) { - val hasHadoopEnv = sys.env.contains("HADOOP_CONF_DIR") || sys.env.contains("YARN_CONF_DIR") + val hasHadoopEnv = env.contains("HADOOP_CONF_DIR") || env.contains("YARN_CONF_DIR") if (!hasHadoopEnv && !Utils.isTesting) { throw new Exception(s"When running with master '$master' " + "either HADOOP_CONF_DIR or YARN_CONF_DIR must be set in the environment.") diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 8872c2178181..f98b531316a3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -33,7 +33,8 @@ import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} import akka.serialization.SerializationExtension import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} -import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, ExecutorState, SparkHadoopUtil} +import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, + ExecutorState, SparkHadoopUtil} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.deploy.master.DriverState.DriverState 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 8ad5fb6b9e6e..4dc550413c13 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -492,7 +492,7 @@ private[spark] class TaskSchedulerImpl( } } - override def applicationId():String = backend.applicationId() + override def applicationId(): String = backend.applicationId() } diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 0c324d8bdf6a..4cba90e8f2af 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy -import java.io.{File, OutputStream, PrintStream} +import java.io._ import scala.collection.mutable.ArrayBuffer @@ -26,6 +26,7 @@ import org.apache.spark.deploy.SparkSubmit._ import org.apache.spark.util.Utils import org.scalatest.FunSuite import org.scalatest.Matchers +import com.google.common.io.Files class SparkSubmitSuite extends FunSuite with Matchers { def beforeAll() { @@ -306,6 +307,21 @@ class SparkSubmitSuite extends FunSuite with Matchers { runSparkSubmit(args) } + test("SPARK_CONF_DIR overrides spark-defaults.conf") { + forConfDir(Map("spark.executor.memory" -> "2.3g")) { path => + val unusedJar = TestUtils.createJarWithClasses(Seq.empty) + val args = Seq( + "--class", SimpleApplicationTest.getClass.getName.stripSuffix("$"), + "--name", "testApp", + "--master", "local", + unusedJar.toString) + val appArgs = new SparkSubmitArguments(args, Map("SPARK_CONF_DIR" -> path)) + assert(appArgs.propertiesFile != null) + assert(appArgs.propertiesFile.startsWith(path)) + appArgs.executorMemory should be ("2.3g") + } + } + // NOTE: This is an expensive operation in terms of time (10 seconds+). Use sparingly. def runSparkSubmit(args: Seq[String]): String = { val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!")) @@ -314,6 +330,22 @@ class SparkSubmitSuite extends FunSuite with Matchers { new File(sparkHome), Map("SPARK_TESTING" -> "1", "SPARK_HOME" -> sparkHome)) } + + def forConfDir(defaults: Map[String, String]) (f: String => Unit) = { + val tmpDir = Files.createTempDir() + + val defaultsConf = new File(tmpDir.getAbsolutePath, "spark-defaults.conf") + val writer = new OutputStreamWriter(new FileOutputStream(defaultsConf)) + for ((key, value) <- defaults) writer.write(s"$key $value\n") + + writer.close() + + try { + f(tmpDir.getAbsolutePath) + } finally { + Utils.deleteRecursively(tmpDir) + } + } } object JarCreationTest { diff --git a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala index 2fc59a841322..3925f0ccbdbf 100644 --- a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala @@ -79,7 +79,7 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter with PrivateMethod val driverMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) val metricName = driverMetricsSystem.buildRegistryName(source) - assert(metricName == s"$appId.$executorId.${source.sourceName}") + assert(metricName === s"$appId.$executorId.${source.sourceName}") } test("MetricsSystem with Driver instance and spark.app.id is not set") { @@ -95,7 +95,7 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter with PrivateMethod val driverMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) val metricName = driverMetricsSystem.buildRegistryName(source) - assert(metricName == source.sourceName) + assert(metricName === source.sourceName) } test("MetricsSystem with Driver instance and spark.executor.id is not set") { @@ -111,7 +111,7 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter with PrivateMethod val driverMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) val metricName = driverMetricsSystem.buildRegistryName(source) - assert(metricName == source.sourceName) + assert(metricName === source.sourceName) } test("MetricsSystem with Executor instance") { @@ -129,7 +129,7 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter with PrivateMethod val driverMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) val metricName = driverMetricsSystem.buildRegistryName(source) - assert(metricName == s"$appId.$executorId.${source.sourceName}") + assert(metricName === s"$appId.$executorId.${source.sourceName}") } test("MetricsSystem with Executor instance and spark.app.id is not set") { @@ -145,7 +145,7 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter with PrivateMethod val driverMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) val metricName = driverMetricsSystem.buildRegistryName(source) - assert(metricName == source.sourceName) + assert(metricName === source.sourceName) } test("MetricsSystem with Executor instance and spark.executor.id is not set") { @@ -161,7 +161,7 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter with PrivateMethod val driverMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) val metricName = driverMetricsSystem.buildRegistryName(source) - assert(metricName == source.sourceName) + assert(metricName === source.sourceName) } test("MetricsSystem with instance which is neither Driver nor Executor") { @@ -182,6 +182,6 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter with PrivateMethod // Even if spark.app.id and spark.executor.id are set, they are not used for the metric name. assert(metricName != s"$appId.$executorId.${source.sourceName}") - assert(metricName == source.sourceName) + assert(metricName === source.sourceName) } } diff --git a/docs/configuration.md b/docs/configuration.md index 316490f0f43f..a782809a55ec 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1108,3 +1108,10 @@ compute `SPARK_LOCAL_IP` by looking up the IP of a specific network interface. Spark uses [log4j](http://logging.apache.org/log4j/) for logging. You can configure it by adding a `log4j.properties` file in the `conf` directory. One way to start is to copy the existing `log4j.properties.template` located there. + +# Overriding configuration directory + +To specify a different configuration directory other than the default "SPARK_HOME/conf", +you can set SPARK_CONF_DIR. Spark will use the the configuration files (spark-defaults.conf, spark-env.sh, log4j.properties, etc) +from this directory. + diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index b7dc373ebd9c..b311d1002389 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -23,7 +23,6 @@ import scala.collection.mutable import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaRDD import org.apache.spark.Logging -import org.apache.spark.mllib.rdd.RDDFunctions._ import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.RandomForest.NodeIndexInfo import org.apache.spark.mllib.tree.configuration.Strategy @@ -36,6 +35,7 @@ import org.apache.spark.mllib.tree.impurity._ import org.apache.spark.mllib.tree.model._ import org.apache.spark.rdd.RDD import org.apache.spark.util.random.XORShiftRandom +import org.apache.spark.SparkContext._ /** @@ -328,9 +328,8 @@ object DecisionTree extends Serializable with Logging { * for each subset is updated. * * @param agg Array storing aggregate calculation, with a set of sufficient statistics for - * each (node, feature, bin). + * each (feature, bin). * @param treePoint Data point being aggregated. - * @param nodeIndex Node corresponding to treePoint. agg is indexed in [0, numNodes). * @param bins possible bins for all features, indexed (numFeatures)(numBins) * @param unorderedFeatures Set of indices of unordered features. * @param instanceWeight Weight (importance) of instance in dataset. @@ -338,7 +337,6 @@ object DecisionTree extends Serializable with Logging { private def mixedBinSeqOp( agg: DTStatsAggregator, treePoint: TreePoint, - nodeIndex: Int, bins: Array[Array[Bin]], unorderedFeatures: Set[Int], instanceWeight: Double, @@ -350,7 +348,6 @@ object DecisionTree extends Serializable with Logging { // Use all features agg.metadata.numFeatures } - val nodeOffset = agg.getNodeOffset(nodeIndex) // Iterate over features. var featureIndexIdx = 0 while (featureIndexIdx < numFeaturesPerNode) { @@ -363,16 +360,16 @@ object DecisionTree extends Serializable with Logging { // Unordered feature val featureValue = treePoint.binnedFeatures(featureIndex) val (leftNodeFeatureOffset, rightNodeFeatureOffset) = - agg.getLeftRightNodeFeatureOffsets(nodeIndex, featureIndexIdx) + agg.getLeftRightFeatureOffsets(featureIndexIdx) // Update the left or right bin for each split. val numSplits = agg.metadata.numSplits(featureIndex) var splitIndex = 0 while (splitIndex < numSplits) { if (bins(featureIndex)(splitIndex).highSplit.categories.contains(featureValue)) { - agg.nodeFeatureUpdate(leftNodeFeatureOffset, splitIndex, treePoint.label, + agg.featureUpdate(leftNodeFeatureOffset, splitIndex, treePoint.label, instanceWeight) } else { - agg.nodeFeatureUpdate(rightNodeFeatureOffset, splitIndex, treePoint.label, + agg.featureUpdate(rightNodeFeatureOffset, splitIndex, treePoint.label, instanceWeight) } splitIndex += 1 @@ -380,8 +377,7 @@ object DecisionTree extends Serializable with Logging { } else { // Ordered feature val binIndex = treePoint.binnedFeatures(featureIndex) - agg.nodeUpdate(nodeOffset, nodeIndex, featureIndexIdx, binIndex, treePoint.label, - instanceWeight) + agg.update(featureIndexIdx, binIndex, treePoint.label, instanceWeight) } featureIndexIdx += 1 } @@ -393,26 +389,24 @@ object DecisionTree extends Serializable with Logging { * For each feature, the sufficient statistics of one bin are updated. * * @param agg Array storing aggregate calculation, with a set of sufficient statistics for - * each (node, feature, bin). + * each (feature, bin). * @param treePoint Data point being aggregated. - * @param nodeIndex Node corresponding to treePoint. agg is indexed in [0, numNodes). * @param instanceWeight Weight (importance) of instance in dataset. */ private def orderedBinSeqOp( agg: DTStatsAggregator, treePoint: TreePoint, - nodeIndex: Int, instanceWeight: Double, featuresForNode: Option[Array[Int]]): Unit = { val label = treePoint.label - val nodeOffset = agg.getNodeOffset(nodeIndex) + // Iterate over features. if (featuresForNode.nonEmpty) { // Use subsampled features var featureIndexIdx = 0 while (featureIndexIdx < featuresForNode.get.size) { val binIndex = treePoint.binnedFeatures(featuresForNode.get.apply(featureIndexIdx)) - agg.nodeUpdate(nodeOffset, nodeIndex, featureIndexIdx, binIndex, label, instanceWeight) + agg.update(featureIndexIdx, binIndex, label, instanceWeight) featureIndexIdx += 1 } } else { @@ -421,7 +415,7 @@ object DecisionTree extends Serializable with Logging { var featureIndex = 0 while (featureIndex < numFeatures) { val binIndex = treePoint.binnedFeatures(featureIndex) - agg.nodeUpdate(nodeOffset, nodeIndex, featureIndex, binIndex, label, instanceWeight) + agg.update(featureIndex, binIndex, label, instanceWeight) featureIndex += 1 } } @@ -496,8 +490,8 @@ object DecisionTree extends Serializable with Logging { * @return agg */ def binSeqOp( - agg: DTStatsAggregator, - baggedPoint: BaggedPoint[TreePoint]): DTStatsAggregator = { + agg: Array[DTStatsAggregator], + baggedPoint: BaggedPoint[TreePoint]): Array[DTStatsAggregator] = { treeToNodeToIndexInfo.foreach { case (treeIndex, nodeIndexToInfo) => val nodeIndex = predictNodeIndex(topNodes(treeIndex), baggedPoint.datum.binnedFeatures, bins, metadata.unorderedFeatures) @@ -508,9 +502,9 @@ object DecisionTree extends Serializable with Logging { val featuresForNode = nodeInfo.featureSubset val instanceWeight = baggedPoint.subsampleWeights(treeIndex) if (metadata.unorderedFeatures.isEmpty) { - orderedBinSeqOp(agg, baggedPoint.datum, aggNodeIndex, instanceWeight, featuresForNode) + orderedBinSeqOp(agg(aggNodeIndex), baggedPoint.datum, instanceWeight, featuresForNode) } else { - mixedBinSeqOp(agg, baggedPoint.datum, aggNodeIndex, bins, metadata.unorderedFeatures, + mixedBinSeqOp(agg(aggNodeIndex), baggedPoint.datum, bins, metadata.unorderedFeatures, instanceWeight, featuresForNode) } } @@ -518,30 +512,76 @@ object DecisionTree extends Serializable with Logging { agg } - // Calculate bin aggregates. - timer.start("aggregation") - val binAggregates: DTStatsAggregator = { - val initAgg = if (metadata.subsamplingFeatures) { - new DTStatsAggregatorSubsampledFeatures(metadata, treeToNodeToIndexInfo) - } else { - new DTStatsAggregatorFixedFeatures(metadata, numNodes) + /** + * Get node index in group --> features indices map, + * which is a short cut to find feature indices for a node given node index in group + * @param treeToNodeToIndexInfo + * @return + */ + def getNodeToFeatures(treeToNodeToIndexInfo: Map[Int, Map[Int, NodeIndexInfo]]) + : Option[Map[Int, Array[Int]]] = if (!metadata.subsamplingFeatures) { + None + } else { + val mutableNodeToFeatures = new mutable.HashMap[Int, Array[Int]]() + treeToNodeToIndexInfo.values.foreach { nodeIdToNodeInfo => + nodeIdToNodeInfo.values.foreach { nodeIndexInfo => + assert(nodeIndexInfo.featureSubset.isDefined) + mutableNodeToFeatures(nodeIndexInfo.nodeIndexInGroup) = nodeIndexInfo.featureSubset.get + } } - input.treeAggregate(initAgg)(binSeqOp, DTStatsAggregator.binCombOp) + Some(mutableNodeToFeatures.toMap) } - timer.stop("aggregation") // Calculate best splits for all nodes in the group timer.start("chooseSplits") + // In each partition, iterate all instances and compute aggregate stats for each node, + // yield an (nodeIndex, nodeAggregateStats) pair for each node. + // After a `reduceByKey` operation, + // stats of a node will be shuffled to a particular partition and be combined together, + // then best splits for nodes are found there. + // Finally, only best Splits for nodes are collected to driver to construct decision tree. + val nodeToFeatures = getNodeToFeatures(treeToNodeToIndexInfo) + val nodeToFeaturesBc = input.sparkContext.broadcast(nodeToFeatures) + val nodeToBestSplits = + input.mapPartitions { points => + // Construct a nodeStatsAggregators array to hold node aggregate stats, + // each node will have a nodeStatsAggregator + val nodeStatsAggregators = Array.tabulate(numNodes) { nodeIndex => + val featuresForNode = nodeToFeaturesBc.value.flatMap { nodeToFeatures => + Some(nodeToFeatures(nodeIndex)) + } + new DTStatsAggregator(metadata, featuresForNode) + } + + // iterator all instances in current partition and update aggregate stats + points.foreach(binSeqOp(nodeStatsAggregators, _)) + + // transform nodeStatsAggregators array to (nodeIndex, nodeAggregateStats) pairs, + // which can be combined with other partition using `reduceByKey` + nodeStatsAggregators.view.zipWithIndex.map(_.swap).iterator + }.reduceByKey((a, b) => a.merge(b)) + .map { case (nodeIndex, aggStats) => + val featuresForNode = nodeToFeaturesBc.value.flatMap { nodeToFeatures => + Some(nodeToFeatures(nodeIndex)) + } + + // find best split for each node + val (split: Split, stats: InformationGainStats, predict: Predict) = + binsToBestSplit(aggStats, splits, featuresForNode) + (nodeIndex, (split, stats, predict)) + }.collectAsMap() + + timer.stop("chooseSplits") + // Iterate over all nodes in this group. nodesForGroup.foreach { case (treeIndex, nodesForTree) => nodesForTree.foreach { node => val nodeIndex = node.id val nodeInfo = treeToNodeToIndexInfo(treeIndex)(nodeIndex) val aggNodeIndex = nodeInfo.nodeIndexInGroup - val featuresForNode = nodeInfo.featureSubset val (split: Split, stats: InformationGainStats, predict: Predict) = - binsToBestSplit(binAggregates, aggNodeIndex, splits, featuresForNode) + nodeToBestSplits(aggNodeIndex) logDebug("best split = " + split) // Extract info for this node. Create children if not leaf. @@ -565,7 +605,7 @@ object DecisionTree extends Serializable with Logging { } } } - timer.stop("chooseSplits") + } /** @@ -633,36 +673,33 @@ object DecisionTree extends Serializable with Logging { /** * Find the best split for a node. * @param binAggregates Bin statistics. - * @param nodeIndex Index into aggregates for node to split in this group. * @return tuple for best split: (Split, information gain, prediction at node) */ private def binsToBestSplit( binAggregates: DTStatsAggregator, - nodeIndex: Int, splits: Array[Array[Split]], featuresForNode: Option[Array[Int]]): (Split, InformationGainStats, Predict) = { - val metadata: DecisionTreeMetadata = binAggregates.metadata - // calculate predict only once var predict: Option[Predict] = None // For each (feature, split), calculate the gain, and select the best (feature, split). - val (bestSplit, bestSplitStats) = Range(0, metadata.numFeaturesPerNode).map { featureIndexIdx => + val (bestSplit, bestSplitStats) = + Range(0, binAggregates.metadata.numFeaturesPerNode).map { featureIndexIdx => val featureIndex = if (featuresForNode.nonEmpty) { featuresForNode.get.apply(featureIndexIdx) } else { featureIndexIdx } - val numSplits = metadata.numSplits(featureIndex) - if (metadata.isContinuous(featureIndex)) { + val numSplits = binAggregates.metadata.numSplits(featureIndex) + if (binAggregates.metadata.isContinuous(featureIndex)) { // Cumulative sum (scanLeft) of bin statistics. // Afterwards, binAggregates for a bin is the sum of aggregates for // that bin + all preceding bins. - val nodeFeatureOffset = binAggregates.getNodeFeatureOffset(nodeIndex, featureIndexIdx) + val nodeFeatureOffset = binAggregates.getFeatureOffset(featureIndexIdx) var splitIndex = 0 while (splitIndex < numSplits) { - binAggregates.mergeForNodeFeature(nodeFeatureOffset, splitIndex + 1, splitIndex) + binAggregates.mergeForFeature(nodeFeatureOffset, splitIndex + 1, splitIndex) splitIndex += 1 } // Find best split. @@ -672,27 +709,29 @@ object DecisionTree extends Serializable with Logging { val rightChildStats = binAggregates.getImpurityCalculator(nodeFeatureOffset, numSplits) rightChildStats.subtract(leftChildStats) predict = Some(predict.getOrElse(calculatePredict(leftChildStats, rightChildStats))) - val gainStats = calculateGainForSplit(leftChildStats, rightChildStats, metadata) + val gainStats = calculateGainForSplit(leftChildStats, + rightChildStats, binAggregates.metadata) (splitIdx, gainStats) }.maxBy(_._2.gain) (splits(featureIndex)(bestFeatureSplitIndex), bestFeatureGainStats) - } else if (metadata.isUnordered(featureIndex)) { + } else if (binAggregates.metadata.isUnordered(featureIndex)) { // Unordered categorical feature val (leftChildOffset, rightChildOffset) = - binAggregates.getLeftRightNodeFeatureOffsets(nodeIndex, featureIndexIdx) + binAggregates.getLeftRightFeatureOffsets(featureIndexIdx) val (bestFeatureSplitIndex, bestFeatureGainStats) = Range(0, numSplits).map { splitIndex => val leftChildStats = binAggregates.getImpurityCalculator(leftChildOffset, splitIndex) val rightChildStats = binAggregates.getImpurityCalculator(rightChildOffset, splitIndex) predict = Some(predict.getOrElse(calculatePredict(leftChildStats, rightChildStats))) - val gainStats = calculateGainForSplit(leftChildStats, rightChildStats, metadata) + val gainStats = calculateGainForSplit(leftChildStats, + rightChildStats, binAggregates.metadata) (splitIndex, gainStats) }.maxBy(_._2.gain) (splits(featureIndex)(bestFeatureSplitIndex), bestFeatureGainStats) } else { // Ordered categorical feature - val nodeFeatureOffset = binAggregates.getNodeFeatureOffset(nodeIndex, featureIndexIdx) - val numBins = metadata.numBins(featureIndex) + val nodeFeatureOffset = binAggregates.getFeatureOffset(featureIndexIdx) + val numBins = binAggregates.metadata.numBins(featureIndex) /* Each bin is one category (feature value). * The bins are ordered based on centroidForCategories, and this ordering determines which @@ -700,7 +739,7 @@ object DecisionTree extends Serializable with Logging { * * centroidForCategories is a list: (category, centroid) */ - val centroidForCategories = if (metadata.isMulticlass) { + val centroidForCategories = if (binAggregates.metadata.isMulticlass) { // For categorical variables in multiclass classification, // the bins are ordered by the impurity of their corresponding labels. Range(0, numBins).map { case featureValue => @@ -741,7 +780,7 @@ object DecisionTree extends Serializable with Logging { while (splitIndex < numSplits) { val currentCategory = categoriesSortedByCentroid(splitIndex)._1 val nextCategory = categoriesSortedByCentroid(splitIndex + 1)._1 - binAggregates.mergeForNodeFeature(nodeFeatureOffset, nextCategory, currentCategory) + binAggregates.mergeForFeature(nodeFeatureOffset, nextCategory, currentCategory) splitIndex += 1 } // lastCategory = index of bin with total aggregates for this (node, feature) @@ -756,7 +795,8 @@ object DecisionTree extends Serializable with Logging { binAggregates.getImpurityCalculator(nodeFeatureOffset, lastCategory) rightChildStats.subtract(leftChildStats) predict = Some(predict.getOrElse(calculatePredict(leftChildStats, rightChildStats))) - val gainStats = calculateGainForSplit(leftChildStats, rightChildStats, metadata) + val gainStats = calculateGainForSplit(leftChildStats, + rightChildStats, binAggregates.metadata) (splitIndex, gainStats) }.maxBy(_._2.gain) val categoriesForSplit = diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala index 7fa7725e79e4..fa7a26f17c3c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala @@ -171,8 +171,8 @@ private class RandomForest ( // Choose node splits, and enqueue new nodes as needed. timer.start("findBestSplits") - DecisionTree.findBestSplits(baggedInput, - metadata, topNodes, nodesForGroup, treeToNodeToIndexInfo, splits, bins, nodeQueue, timer) + DecisionTree.findBestSplits(baggedInput, metadata, topNodes, nodesForGroup, + treeToNodeToIndexInfo, splits, bins, nodeQueue, timer) timer.stop("findBestSplits") } @@ -382,6 +382,7 @@ object RandomForest extends Serializable with Logging { * @param maxMemoryUsage Bound on size of aggregate statistics. * @return (nodesForGroup, treeToNodeToIndexInfo). * nodesForGroup holds the nodes to split: treeIndex --> nodes in tree. + * * treeToNodeToIndexInfo holds indices selected features for each node: * treeIndex --> (global) node index --> (node index in group, feature indices). * The (global) node index is the index in the tree; the node index in group is the diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala index d49df7a01637..55f422dff0d7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala @@ -17,17 +17,19 @@ package org.apache.spark.mllib.tree.impl -import org.apache.spark.mllib.tree.RandomForest.NodeIndexInfo import org.apache.spark.mllib.tree.impurity._ + + /** - * DecisionTree statistics aggregator. - * This holds a flat array of statistics for a set of (nodes, features, bins) + * DecisionTree statistics aggregator for a node. + * This holds a flat array of statistics for a set of (features, bins) * and helps with indexing. * This class is abstract to support learning with and without feature subsampling. */ -private[tree] abstract class DTStatsAggregator( - val metadata: DecisionTreeMetadata) extends Serializable { +private[tree] class DTStatsAggregator( + val metadata: DecisionTreeMetadata, + featureSubset: Option[Array[Int]]) extends Serializable { /** * [[ImpurityAggregator]] instance specifying the impurity type. @@ -42,7 +44,25 @@ private[tree] abstract class DTStatsAggregator( /** * Number of elements (Double values) used for the sufficient statistics of each bin. */ - val statsSize: Int = impurityAggregator.statsSize + private val statsSize: Int = impurityAggregator.statsSize + + /** + * Number of bins for each feature. This is indexed by the feature index. + */ + private val numBins: Array[Int] = { + if (featureSubset.isDefined) { + featureSubset.get.map(metadata.numBins(_)) + } else { + metadata.numBins + } + } + + /** + * Offset for each feature for calculating indices into the [[allStats]] array. + */ + private val featureOffsets: Array[Int] = { + numBins.scanLeft(0)((total, nBins) => total + statsSize * nBins) + } /** * Indicator for each feature of whether that feature is an unordered feature. @@ -51,107 +71,95 @@ private[tree] abstract class DTStatsAggregator( def isUnordered(featureIndex: Int): Boolean = metadata.isUnordered(featureIndex) /** - * Total number of elements stored in this aggregator. + * Total number of elements stored in this aggregator */ - def allStatsSize: Int + private val allStatsSize: Int = featureOffsets.last /** - * Get flat array of elements stored in this aggregator. + * Flat array of elements. + * Index for start of stats for a (feature, bin) is: + * index = featureOffsets(featureIndex) + binIndex * statsSize + * Note: For unordered features, + * the left child stats have binIndex in [0, numBins(featureIndex) / 2)) + * and the right child stats in [numBins(featureIndex) / 2), numBins(featureIndex)) */ - protected def allStats: Array[Double] + private val allStats: Array[Double] = new Array[Double](allStatsSize) + /** * Get an [[ImpurityCalculator]] for a given (node, feature, bin). - * @param nodeFeatureOffset For ordered features, this is a pre-computed (node, feature) offset - * from [[getNodeFeatureOffset]]. + * @param featureOffset For ordered features, this is a pre-computed (node, feature) offset + * from [[getFeatureOffset]]. * For unordered features, this is a pre-computed * (node, feature, left/right child) offset from - * [[getLeftRightNodeFeatureOffsets]]. + * [[getLeftRightFeatureOffsets]]. */ - def getImpurityCalculator(nodeFeatureOffset: Int, binIndex: Int): ImpurityCalculator = { - impurityAggregator.getCalculator(allStats, nodeFeatureOffset + binIndex * statsSize) + def getImpurityCalculator(featureOffset: Int, binIndex: Int): ImpurityCalculator = { + impurityAggregator.getCalculator(allStats, featureOffset + binIndex * statsSize) } /** - * Update the stats for a given (node, feature, bin) for ordered features, using the given label. + * Update the stats for a given (feature, bin) for ordered features, using the given label. */ - def update( - nodeIndex: Int, - featureIndex: Int, - binIndex: Int, - label: Double, - instanceWeight: Double): Unit = { - val i = getNodeFeatureOffset(nodeIndex, featureIndex) + binIndex * statsSize + def update(featureIndex: Int, binIndex: Int, label: Double, instanceWeight: Double): Unit = { + val i = featureOffsets(featureIndex) + binIndex * statsSize impurityAggregator.update(allStats, i, label, instanceWeight) } - /** - * Pre-compute node offset for use with [[nodeUpdate]]. - */ - def getNodeOffset(nodeIndex: Int): Int - /** * Faster version of [[update]]. - * Update the stats for a given (node, feature, bin) for ordered features, using the given label. - * @param nodeOffset Pre-computed node offset from [[getNodeOffset]]. + * Update the stats for a given (feature, bin), using the given label. + * @param featureOffset For ordered features, this is a pre-computed feature offset + * from [[getFeatureOffset]]. + * For unordered features, this is a pre-computed + * (feature, left/right child) offset from + * [[getLeftRightFeatureOffsets]]. */ - def nodeUpdate( - nodeOffset: Int, - nodeIndex: Int, - featureIndex: Int, + def featureUpdate( + featureOffset: Int, binIndex: Int, label: Double, - instanceWeight: Double): Unit + instanceWeight: Double): Unit = { + impurityAggregator.update(allStats, featureOffset + binIndex * statsSize, + label, instanceWeight) + } /** - * Pre-compute (node, feature) offset for use with [[nodeFeatureUpdate]]. + * Pre-compute feature offset for use with [[featureUpdate]]. * For ordered features only. */ - def getNodeFeatureOffset(nodeIndex: Int, featureIndex: Int): Int + def getFeatureOffset(featureIndex: Int): Int = { + require(!isUnordered(featureIndex), + s"DTStatsAggregator.getFeatureOffset is for ordered features only, but was called" + + s" for unordered feature $featureIndex.") + featureOffsets(featureIndex) + } /** - * Pre-compute (node, feature) offset for use with [[nodeFeatureUpdate]]. + * Pre-compute feature offset for use with [[featureUpdate]]. * For unordered features only. */ - def getLeftRightNodeFeatureOffsets(nodeIndex: Int, featureIndex: Int): (Int, Int) = { + def getLeftRightFeatureOffsets(featureIndex: Int): (Int, Int) = { require(isUnordered(featureIndex), - s"DTStatsAggregator.getLeftRightNodeFeatureOffsets is for unordered features only," + + s"DTStatsAggregator.getLeftRightFeatureOffsets is for unordered features only," + s" but was called for ordered feature $featureIndex.") - val baseOffset = getNodeFeatureOffset(nodeIndex, featureIndex) - (baseOffset, baseOffset + (metadata.numBins(featureIndex) >> 1) * statsSize) - } - - /** - * Faster version of [[update]]. - * Update the stats for a given (node, feature, bin), using the given label. - * @param nodeFeatureOffset For ordered features, this is a pre-computed (node, feature) offset - * from [[getNodeFeatureOffset]]. - * For unordered features, this is a pre-computed - * (node, feature, left/right child) offset from - * [[getLeftRightNodeFeatureOffsets]]. - */ - def nodeFeatureUpdate( - nodeFeatureOffset: Int, - binIndex: Int, - label: Double, - instanceWeight: Double): Unit = { - impurityAggregator.update(allStats, nodeFeatureOffset + binIndex * statsSize, label, - instanceWeight) + val baseOffset = featureOffsets(featureIndex) + (baseOffset, baseOffset + (numBins(featureIndex) >> 1) * statsSize) } /** - * For a given (node, feature), merge the stats for two bins. - * @param nodeFeatureOffset For ordered features, this is a pre-computed (node, feature) offset - * from [[getNodeFeatureOffset]]. + * For a given feature, merge the stats for two bins. + * @param featureOffset For ordered features, this is a pre-computed feature offset + * from [[getFeatureOffset]]. * For unordered features, this is a pre-computed - * (node, feature, left/right child) offset from - * [[getLeftRightNodeFeatureOffsets]]. + * (feature, left/right child) offset from + * [[getLeftRightFeatureOffsets]]. * @param binIndex The other bin is merged into this bin. * @param otherBinIndex This bin is not modified. */ - def mergeForNodeFeature(nodeFeatureOffset: Int, binIndex: Int, otherBinIndex: Int): Unit = { - impurityAggregator.merge(allStats, nodeFeatureOffset + binIndex * statsSize, - nodeFeatureOffset + otherBinIndex * statsSize) + def mergeForFeature(featureOffset: Int, binIndex: Int, otherBinIndex: Int): Unit = { + impurityAggregator.merge(allStats, featureOffset + binIndex * statsSize, + featureOffset + otherBinIndex * statsSize) } /** @@ -161,7 +169,7 @@ private[tree] abstract class DTStatsAggregator( def merge(other: DTStatsAggregator): DTStatsAggregator = { require(allStatsSize == other.allStatsSize, s"DTStatsAggregator.merge requires that both aggregators have the same length stats vectors." - + s" This aggregator is of length $allStatsSize, but the other is ${other.allStatsSize}.") + + s" This aggregator is of length $allStatsSize, but the other is ${other.allStatsSize}.") var i = 0 // TODO: Test BLAS.axpy while (i < allStatsSize) { @@ -171,149 +179,3 @@ private[tree] abstract class DTStatsAggregator( this } } - -/** - * DecisionTree statistics aggregator. - * This holds a flat array of statistics for a set of (nodes, features, bins) - * and helps with indexing. - * - * This instance of [[DTStatsAggregator]] is used when not subsampling features. - * - * @param numNodes Number of nodes to collect statistics for. - */ -private[tree] class DTStatsAggregatorFixedFeatures( - metadata: DecisionTreeMetadata, - numNodes: Int) extends DTStatsAggregator(metadata) { - - /** - * Offset for each feature for calculating indices into the [[allStats]] array. - * Mapping: featureIndex --> offset - */ - private val featureOffsets: Array[Int] = { - metadata.numBins.scanLeft(0)((total, nBins) => total + statsSize * nBins) - } - - /** - * Number of elements for each node, corresponding to stride between nodes in [[allStats]]. - */ - private val nodeStride: Int = featureOffsets.last - - override val allStatsSize: Int = numNodes * nodeStride - - /** - * Flat array of elements. - * Index for start of stats for a (node, feature, bin) is: - * index = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize - * Note: For unordered features, the left child stats precede the right child stats - * in the binIndex order. - */ - override protected val allStats: Array[Double] = new Array[Double](allStatsSize) - - override def getNodeOffset(nodeIndex: Int): Int = nodeIndex * nodeStride - - override def nodeUpdate( - nodeOffset: Int, - nodeIndex: Int, - featureIndex: Int, - binIndex: Int, - label: Double, - instanceWeight: Double): Unit = { - val i = nodeOffset + featureOffsets(featureIndex) + binIndex * statsSize - impurityAggregator.update(allStats, i, label, instanceWeight) - } - - override def getNodeFeatureOffset(nodeIndex: Int, featureIndex: Int): Int = { - nodeIndex * nodeStride + featureOffsets(featureIndex) - } -} - -/** - * DecisionTree statistics aggregator. - * This holds a flat array of statistics for a set of (nodes, features, bins) - * and helps with indexing. - * - * This instance of [[DTStatsAggregator]] is used when subsampling features. - * - * @param treeToNodeToIndexInfo Mapping: treeIndex --> nodeIndex --> nodeIndexInfo, - * where nodeIndexInfo stores the index in the group and the - * feature subsets (if using feature subsets). - */ -private[tree] class DTStatsAggregatorSubsampledFeatures( - metadata: DecisionTreeMetadata, - treeToNodeToIndexInfo: Map[Int, Map[Int, NodeIndexInfo]]) extends DTStatsAggregator(metadata) { - - /** - * For each node, offset for each feature for calculating indices into the [[allStats]] array. - * Mapping: nodeIndex --> featureIndex --> offset - */ - private val featureOffsets: Array[Array[Int]] = { - val numNodes: Int = treeToNodeToIndexInfo.values.map(_.size).sum - val offsets = new Array[Array[Int]](numNodes) - treeToNodeToIndexInfo.foreach { case (treeIndex, nodeToIndexInfo) => - nodeToIndexInfo.foreach { case (globalNodeIndex, nodeInfo) => - offsets(nodeInfo.nodeIndexInGroup) = nodeInfo.featureSubset.get.map(metadata.numBins(_)) - .scanLeft(0)((total, nBins) => total + statsSize * nBins) - } - } - offsets - } - - /** - * For each node, offset for each feature for calculating indices into the [[allStats]] array. - */ - protected val nodeOffsets: Array[Int] = featureOffsets.map(_.last).scanLeft(0)(_ + _) - - override val allStatsSize: Int = nodeOffsets.last - - /** - * Flat array of elements. - * Index for start of stats for a (node, feature, bin) is: - * index = nodeOffsets(nodeIndex) + featureOffsets(featureIndex) + binIndex * statsSize - * Note: For unordered features, the left child stats precede the right child stats - * in the binIndex order. - */ - override protected val allStats: Array[Double] = new Array[Double](allStatsSize) - - override def getNodeOffset(nodeIndex: Int): Int = nodeOffsets(nodeIndex) - - /** - * Faster version of [[update]]. - * Update the stats for a given (node, feature, bin) for ordered features, using the given label. - * @param nodeOffset Pre-computed node offset from [[getNodeOffset]]. - * @param featureIndex Index of feature in featuresForNodes(nodeIndex). - * Note: This is NOT the original feature index. - */ - override def nodeUpdate( - nodeOffset: Int, - nodeIndex: Int, - featureIndex: Int, - binIndex: Int, - label: Double, - instanceWeight: Double): Unit = { - val i = nodeOffset + featureOffsets(nodeIndex)(featureIndex) + binIndex * statsSize - impurityAggregator.update(allStats, i, label, instanceWeight) - } - - /** - * Pre-compute (node, feature) offset for use with [[nodeFeatureUpdate]]. - * For ordered features only. - * @param featureIndex Index of feature in featuresForNodes(nodeIndex). - * Note: This is NOT the original feature index. - */ - override def getNodeFeatureOffset(nodeIndex: Int, featureIndex: Int): Int = { - nodeOffsets(nodeIndex) + featureOffsets(nodeIndex)(featureIndex) - } -} - -private[tree] object DTStatsAggregator extends Serializable { - - /** - * Combines two aggregates (modifying the first) and returns the combination. - */ - def binCombOp( - agg1: DTStatsAggregator, - agg2: DTStatsAggregator): DTStatsAggregator = { - agg1.merge(agg2) - } - -} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala index f3e2619bd8ba..a89e71e11580 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala @@ -38,6 +38,17 @@ class InformationGainStats( "gain = %f, impurity = %f, left impurity = %f, right impurity = %f" .format(gain, impurity, leftImpurity, rightImpurity) } + + override def equals(o: Any) = + o match { + case other: InformationGainStats => { + gain == other.gain && + impurity == other.impurity && + leftImpurity == other.leftImpurity && + rightImpurity == other.rightImpurity + } + case _ => false + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala index 30669fcd1c75..20d372dc1d3c 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala @@ -145,6 +145,7 @@ class RandomForestSuite extends FunSuite with LocalSparkContext { assert(nodesForGroup.size === numTrees, failString) assert(nodesForGroup.values.forall(_.size == 1), failString) // 1 node per tree + if (numFeaturesPerNode == numFeatures) { // featureSubset values should all be None assert(treeToNodeToIndexInfo.values.forall(_.values.forall(_.featureSubset.isEmpty)), diff --git a/sbin/spark-config.sh b/sbin/spark-config.sh index 2718d6cba1c9..1d154e62ed5b 100755 --- a/sbin/spark-config.sh +++ b/sbin/spark-config.sh @@ -33,7 +33,7 @@ this="$config_bin/$script" export SPARK_PREFIX="`dirname "$this"`"/.. export SPARK_HOME="${SPARK_PREFIX}" -export SPARK_CONF_DIR="$SPARK_HOME/conf" +export SPARK_CONF_DIR="${SPARK_CONF_DIR:-"$SPARK_HOME/conf"}" # Add the PySpark classes to the PYTHONPATH: export PYTHONPATH="$SPARK_HOME/python:$PYTHONPATH" export PYTHONPATH="$SPARK_HOME/python/lib/py4j-0.8.2.1-src.zip:$PYTHONPATH" diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 732e4976f684..68f93f247d9b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -22,7 +22,7 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDFUtils.ConversionHelper import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.hive.common.`type`.HiveDecimal -import org.apache.hadoop.hive.ql.exec.UDF +import org.apache.hadoop.hive.ql.exec.{UDF, UDAF} import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry} import org.apache.hadoop.hive.ql.udf.{UDFType => HiveUDFType} import org.apache.hadoop.hive.ql.udf.generic._ @@ -57,7 +57,8 @@ private[hive] abstract class HiveFunctionRegistry } else if ( classOf[AbstractGenericUDAFResolver].isAssignableFrom(functionInfo.getFunctionClass)) { HiveGenericUdaf(functionClassName, children) - + } else if (classOf[UDAF].isAssignableFrom(functionInfo.getFunctionClass)) { + HiveUdaf(functionClassName, children) } else if (classOf[GenericUDTF].isAssignableFrom(functionInfo.getFunctionClass)) { HiveGenericUdtf(functionClassName, Nil, children) } else { @@ -194,6 +195,37 @@ private[hive] case class HiveGenericUdaf( def newInstance() = new HiveUdafFunction(functionClassName, children, this) } +/** It is used as a wrapper for the hive functions which uses UDAF interface */ +private[hive] case class HiveUdaf( + functionClassName: String, + children: Seq[Expression]) extends AggregateExpression + with HiveInspectors + with HiveFunctionFactory { + + type UDFType = UDAF + + @transient + protected lazy val resolver: AbstractGenericUDAFResolver = new GenericUDAFBridge(createFunction()) + + @transient + protected lazy val objectInspector = { + resolver.getEvaluator(children.map(_.dataType.toTypeInfo).toArray) + .init(GenericUDAFEvaluator.Mode.COMPLETE, inspectors.toArray) + } + + @transient + protected lazy val inspectors = children.map(_.dataType).map(toInspector) + + def dataType: DataType = inspectorToDataType(objectInspector) + + def nullable: Boolean = true + + override def toString = s"$nodeName#$functionClassName(${children.mkString(",")})" + + def newInstance() = + new HiveUdafFunction(functionClassName, children, this, true) +} + /** * Converts a Hive Generic User Defined Table Generating Function (UDTF) to a * [[catalyst.expressions.Generator Generator]]. Note that the semantics of Generators do not allow @@ -275,14 +307,20 @@ private[hive] case class HiveGenericUdtf( private[hive] case class HiveUdafFunction( functionClassName: String, exprs: Seq[Expression], - base: AggregateExpression) + base: AggregateExpression, + isUDAFBridgeRequired: Boolean = false) extends AggregateFunction with HiveInspectors with HiveFunctionFactory { def this() = this(null, null, null) - private val resolver = createFunction[AbstractGenericUDAFResolver]() + private val resolver = + if (isUDAFBridgeRequired) { + new GenericUDAFBridge(createFunction[UDAF]()) + } else { + createFunction[AbstractGenericUDAFResolver]() + } private val inspectors = exprs.map(_.dataType).map(toInspector).toArray diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala index cc125d539c3c..e4324e9528f9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala @@ -79,6 +79,10 @@ class HiveUdfSuite extends HiveComparisonTest { sql("SELECT testUdf(pair) FROM hiveUdfTestTable") sql("DROP TEMPORARY FUNCTION IF EXISTS testUdf") } + + test("SPARK-2693 udaf aggregates test") { + assert(sql("SELECT percentile(key,1) FROM src").first === sql("SELECT max(key) FROM src").first) + } } class TestPair(x: Int, y: Int) extends Writable with Serializable { diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala index e4551babed42..3a186cfeb4ee 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala @@ -52,7 +52,7 @@ private[spark] class YarnClusterSchedulerBackend( // In YARN Cluster mode, spark.yarn.app.id is expect to be set // before user application is launched. // So, if spark.yarn.app.id is not set, it is something wrong. - sc.getConf.getOption("spark.yarn.app.id").map(_.toString).getOrElse { + sc.getConf.getOption("spark.yarn.app.id").getOrElse { logError("Application ID is not set.") super.applicationId } diff --git a/yarn/stable/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/stable/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index 4b6635679f05..a826b2a78a8f 100644 --- a/yarn/stable/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/yarn/stable/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.deploy.yarn import java.io.File +import java.util.concurrent.TimeUnit import scala.collection.JavaConversions._ @@ -32,7 +33,7 @@ import org.apache.spark.{Logging, SparkConf, SparkContext} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.Utils -class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers { +class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers with Logging { // log4j configuration for the Yarn containers, so that their output is collected // by Yarn instead of trying to overwrite unit-tests.log. @@ -66,7 +67,33 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers { yarnCluster = new MiniYARNCluster(getClass().getName(), 1, 1, 1) yarnCluster.init(new YarnConfiguration()) yarnCluster.start() - yarnCluster.getConfig().foreach { e => + + // There's a race in MiniYARNCluster in which start() may return before the RM has updated + // its address in the configuration. You can see this in the logs by noticing that when + // MiniYARNCluster prints the address, it still has port "0" assigned, although later the + // test works sometimes: + // + // INFO MiniYARNCluster: MiniYARN ResourceManager address: blah:0 + // + // That log message prints the contents of the RM_ADDRESS config variable. If you check it + // later on, it looks something like this: + // + // INFO YarnClusterSuite: RM address in configuration is blah:42631 + // + // This hack loops for a bit waiting for the port to change, and fails the test if it hasn't + // done so in a timely manner (defined to be 10 seconds). + val config = yarnCluster.getConfig() + val deadline = System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(10) + while (config.get(YarnConfiguration.RM_ADDRESS).split(":")(1) == "0") { + if (System.currentTimeMillis() > deadline) { + throw new IllegalStateException("Timed out waiting for RM to come up.") + } + logDebug("RM address still not set in configuration, waiting...") + TimeUnit.MILLISECONDS.sleep(100) + } + + logInfo(s"RM address in configuration is ${config.get(YarnConfiguration.RM_ADDRESS)}") + config.foreach { e => sys.props += ("spark.hadoop." + e.getKey() -> e.getValue()) } @@ -86,13 +113,13 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers { super.afterAll() } - ignore("run Spark in yarn-client mode") { + test("run Spark in yarn-client mode") { var result = File.createTempFile("result", null, tempDir) YarnClusterDriver.main(Array("yarn-client", result.getAbsolutePath())) checkResult(result) } - ignore("run Spark in yarn-cluster mode") { + test("run Spark in yarn-cluster mode") { val main = YarnClusterDriver.getClass.getName().stripSuffix("$") var result = File.createTempFile("result", null, tempDir)