From 2573add94cf920a88f74d80d8ea94218d812704d Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 24 Dec 2013 18:30:31 +0530 Subject: [PATCH] spark-544, introducing SparkConf and related configuration overhaul. --- .../org/apache/spark/MapOutputTracker.scala | 7 +- .../scala/org/apache/spark/Partitioner.scala | 4 +- .../scala/org/apache/spark/SparkConf.scala | 71 +++++++++ .../scala/org/apache/spark/SparkContext.scala | 140 ++++++++++-------- .../scala/org/apache/spark/SparkEnv.scala | 44 +++--- .../apache/spark/api/python/PythonRDD.scala | 6 +- .../apache/spark/broadcast/Broadcast.scala | 6 +- .../spark/broadcast/BroadcastFactory.scala | 4 +- .../spark/broadcast/HttpBroadcast.scala | 14 +- .../spark/broadcast/TorrentBroadcast.scala | 9 +- .../spark/deploy/ApplicationDescription.scala | 2 +- .../apache/spark/deploy/SparkHadoopUtil.scala | 3 +- .../apache/spark/deploy/client/Client.scala | 13 +- .../spark/deploy/client/TestClient.scala | 7 +- .../apache/spark/deploy/master/Master.scala | 31 ++-- .../spark/deploy/master/MasterArguments.scala | 7 +- .../deploy/master/SparkZooKeeperSession.scala | 7 +- .../master/ZooKeeperLeaderElectionAgent.scala | 9 +- .../master/ZooKeeperPersistenceEngine.scala | 8 +- .../spark/deploy/master/ui/MasterWebUI.scala | 2 +- .../apache/spark/deploy/worker/Worker.scala | 28 ++-- .../spark/deploy/worker/ui/WorkerWebUI.scala | 6 +- .../CoarseGrainedExecutorBackend.scala | 6 +- .../org/apache/spark/executor/Executor.scala | 10 +- .../apache/spark/io/CompressionCodec.scala | 14 +- .../apache/spark/metrics/MetricsSystem.scala | 10 +- .../spark/network/ConnectionManager.scala | 24 +-- .../apache/spark/network/ReceiverTest.scala | 4 +- .../org/apache/spark/network/SenderTest.scala | 4 +- .../spark/network/netty/ShuffleCopier.scala | 8 +- .../org/apache/spark/rdd/CheckpointRDD.scala | 6 +- .../main/scala/org/apache/spark/rdd/RDD.scala | 1 + .../apache/spark/scheduler/JobLogger.scala | 2 +- .../spark/scheduler/SchedulableBuilder.scala | 6 +- .../scheduler/cluster/ClusterScheduler.scala | 11 +- .../cluster/ClusterTaskSetManager.scala | 19 +-- .../CoarseGrainedSchedulerBackend.scala | 12 +- .../cluster/SimrSchedulerBackend.scala | 4 +- .../cluster/SparkDeploySchedulerBackend.scala | 6 +- .../scheduler/cluster/TaskResultGetter.scala | 3 +- .../mesos/CoarseMesosSchedulerBackend.scala | 10 +- .../cluster/mesos/MesosSchedulerBackend.scala | 4 +- .../scheduler/local/LocalScheduler.scala | 5 +- .../spark/serializer/KryoSerializer.scala | 10 +- .../spark/storage/BlockFetcherIterator.scala | 4 +- .../apache/spark/storage/BlockManager.scala | 38 ++--- .../spark/storage/BlockManagerMaster.scala | 11 +- .../storage/BlockManagerMasterActor.scala | 10 +- .../spark/storage/BlockObjectWriter.scala | 5 +- .../spark/storage/DiskBlockManager.scala | 2 +- .../spark/storage/ShuffleBlockManager.scala | 7 +- .../apache/spark/storage/ThreadingTest.scala | 6 +- .../scala/org/apache/spark/ui/SparkUI.scala | 2 +- .../apache/spark/ui/UIWorkloadGenerator.scala | 4 +- .../apache/spark/ui/env/EnvironmentUI.scala | 2 +- .../spark/ui/jobs/JobProgressListener.scala | 2 +- .../org/apache/spark/util/AkkaUtils.scala | 25 ++-- .../apache/spark/util/MetadataCleaner.scala | 12 +- .../org/apache/spark/util/SizeEstimator.scala | 7 +- .../scala/org/apache/spark/util/Utils.scala | 7 +- .../apache/spark/MapOutputTrackerSuite.scala | 16 +- .../spark/metrics/MetricsSystemSuite.scala | 8 +- .../spark/scheduler/DAGSchedulerSuite.scala | 23 +-- .../spark/scheduler/JobLoggerSuite.scala | 2 +- .../cluster/ClusterSchedulerSuite.scala | 2 +- .../cluster/ClusterTaskSetManagerSuite.scala | 4 +- .../cluster/TaskResultGetterSuite.scala | 2 +- .../spark/storage/BlockManagerSuite.scala | 95 ++++++------ .../spark/storage/DiskBlockManagerSuite.scala | 12 +- .../spark/util/SizeEstimatorSuite.scala | 4 +- .../examples/bagel/WikipediaPageRank.scala | 10 +- .../bagel/WikipediaPageRankStandalone.scala | 8 +- .../streaming/examples/ActorWordCount.scala | 3 +- .../spark/mllib/recommendation/ALS.scala | 13 +- .../spark/deploy/yarn/ApplicationMaster.scala | 16 +- .../org/apache/spark/deploy/yarn/Client.scala | 4 +- .../spark/deploy/yarn/ClientArguments.scala | 2 +- .../spark/deploy/yarn/WorkerLauncher.scala | 4 +- .../deploy/yarn/YarnAllocationHandler.scala | 4 +- .../cluster/YarnClientSchedulerBackend.scala | 4 +- project/SparkBuild.scala | 3 +- .../org/apache/spark/repl/SparkILoop.scala | 7 +- .../org/apache/spark/repl/SparkIMain.scala | 7 +- .../apache/spark/streaming/Checkpoint.scala | 3 +- .../apache/spark/streaming/Scheduler.scala | 6 +- .../spark/streaming/StreamingContext.scala | 2 +- .../dstream/NetworkInputDStream.scala | 6 +- .../spark/streaming/CheckpointSuite.scala | 6 +- .../spark/streaming/InputStreamsSuite.scala | 18 +-- .../spark/streaming/TestSuiteBase.scala | 11 +- .../spark/deploy/yarn/ApplicationMaster.scala | 16 +- .../org/apache/spark/deploy/yarn/Client.scala | 6 +- .../spark/deploy/yarn/ClientArguments.scala | 2 +- .../spark/deploy/yarn/WorkerLauncher.scala | 4 +- .../deploy/yarn/YarnAllocationHandler.scala | 2 +- .../cluster/YarnClientSchedulerBackend.scala | 4 +- 96 files changed, 612 insertions(+), 478 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/SparkConf.scala diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index ccffcc356ca04..4520edb10d491 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -50,9 +50,9 @@ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster } } -private[spark] class MapOutputTracker extends Logging { +private[spark] class MapOutputTracker(conf: SparkConf) extends Logging { - private val timeout = AkkaUtils.askTimeout + private val timeout = AkkaUtils.askTimeout(conf) // Set to the MapOutputTrackerActor living on the driver var trackerActor: Either[ActorRef, ActorSelection] = _ @@ -192,7 +192,8 @@ private[spark] class MapOutputTracker extends Logging { } } -private[spark] class MapOutputTrackerMaster extends MapOutputTracker { +private[spark] class MapOutputTrackerMaster(conf: SparkConf) + extends MapOutputTracker(conf) { // Cache a serialized version of the output statuses for each shuffle to send them out faster private var cacheEpoch = epoch diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index bcec41c439afb..04c1eedfeb455 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -32,6 +32,8 @@ abstract class Partitioner extends Serializable { } object Partitioner { + + import SparkContext.{globalConf => conf} /** * Choose a partitioner to use for a cogroup-like operation between a number of RDDs. * @@ -52,7 +54,7 @@ object Partitioner { for (r <- bySize if r.partitioner != None) { return r.partitioner.get } - if (System.getProperty("spark.default.parallelism") != null) { + if (conf.getOrElse("spark.default.parallelism", null) != null) { return new HashPartitioner(rdd.context.defaultParallelism) } else { return new HashPartitioner(bySize.head.partitions.size) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala new file mode 100644 index 0000000000000..9a4eefad2ee52 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -0,0 +1,71 @@ +package org.apache.spark + +import scala.collection.JavaConversions._ +import scala.collection.concurrent.TrieMap + +import com.typesafe.config.ConfigFactory + +private[spark] class SparkConf(loadClasspathRes: Boolean = true) extends Serializable { + @transient lazy val config = ConfigFactory.systemProperties() + .withFallback(ConfigFactory.parseResources("spark.conf")) + // TODO this should actually be synchronized + private val configMap = TrieMap[String, String]() + + if (loadClasspathRes && !config.entrySet().isEmpty) { + for (e <- config.entrySet()) { + configMap += ((e.getKey, e.getValue.unwrapped().toString)) + } + } + + def setMasterUrl(master: String) = { + if (master != null) + configMap += (("spark.master", master)) + this + } + + def setAppName(name: String) = { + if (name != null) + configMap += (("spark.appName", name)) + this + } + + def setJars(jars: Seq[String]) = { + if (!jars.isEmpty) + configMap += (("spark.jars", jars.mkString(","))) + this + } + + def set(k: String, value: String) = { + configMap += ((k, value)) + this + } + + def setSparkHome(home: String) = { + if (home != null) + configMap += (("spark.home", home)) + this + } + + def set(map: Seq[(String, String)]) = { + if (map != null && !map.isEmpty) + configMap ++= map + this + } + + def get(k: String): String = { + configMap(k) + } + + def getAllConfiguration = configMap.clone.entrySet().iterator + + def getOrElse(k: String, defaultValue: String): String = { + configMap.getOrElse(k, defaultValue) + } + + override def clone: SparkConf = { + val conf = new SparkConf(false) + conf.set(configMap.toSeq) + conf + } + +} diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index a0f794edfdfcb..4300b07bdb7ae 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -22,91 +22,99 @@ import java.net.URI import java.util.Properties import java.util.concurrent.atomic.AtomicInteger -import scala.collection.Map +import scala.collection.{Map, immutable} +import scala.collection.JavaConversions._ import scala.collection.generic.Growable -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap + +import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.reflect.{ClassTag, classTag} +import scala.util.Try import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.hadoop.io.ArrayWritable -import org.apache.hadoop.io.BooleanWritable -import org.apache.hadoop.io.BytesWritable -import org.apache.hadoop.io.DoubleWritable -import org.apache.hadoop.io.FloatWritable -import org.apache.hadoop.io.IntWritable -import org.apache.hadoop.io.LongWritable -import org.apache.hadoop.io.NullWritable -import org.apache.hadoop.io.Text -import org.apache.hadoop.io.Writable -import org.apache.hadoop.mapred.FileInputFormat -import org.apache.hadoop.mapred.InputFormat -import org.apache.hadoop.mapred.JobConf -import org.apache.hadoop.mapred.SequenceFileInputFormat -import org.apache.hadoop.mapred.TextInputFormat -import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} -import org.apache.hadoop.mapreduce.{Job => NewHadoopJob} +import org.apache.hadoop.io.{ArrayWritable, BooleanWritable, BytesWritable, DoubleWritable, +FloatWritable, IntWritable, LongWritable, NullWritable, Text, Writable} +import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf, SequenceFileInputFormat, +TextInputFormat} +import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHadoopJob} import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} - import org.apache.mesos.MesosNativeLibrary import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} import org.apache.spark.rdd._ import org.apache.spark.scheduler._ -import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, - SparkDeploySchedulerBackend, ClusterScheduler, SimrSchedulerBackend} -import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} +import org.apache.spark.scheduler.cluster.{ClusterScheduler, CoarseGrainedSchedulerBackend, +SimrSchedulerBackend, SparkDeploySchedulerBackend} +import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, +MesosSchedulerBackend} import org.apache.spark.scheduler.local.LocalScheduler -import org.apache.spark.scheduler.StageInfo import org.apache.spark.storage.{BlockManagerSource, RDDInfo, StorageStatus, StorageUtils} import org.apache.spark.ui.SparkUI -import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerType, - TimeStampedHashMap, Utils} +import org.apache.spark.util._ /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark * cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster. * - * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). - * @param appName A name for your application, to display on the cluster web UI. - * @param sparkHome Location where Spark is installed on cluster nodes. - * @param jars Collection of JARs to send to the cluster. These can be paths on the local file - * system or HDFS, HTTP, HTTPS, or FTP URLs. + * @param conf a Spark Config object describing the context configuration. Any settings in this + * config overrides the default configs as well as system properties. + * * @param environment Environment variables to set on worker nodes. */ class SparkContext( - val master: String, - val appName: String, - val sparkHome: String = null, - val jars: Seq[String] = Nil, + val conf: SparkConf, val environment: Map[String, String] = Map(), // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, etc) // too. This is typically generated from InputFormatInfo.computePreferredLocations .. host, set // of data-local splits on host - val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] = - scala.collection.immutable.Map()) + val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] = immutable.Map()) extends Logging { - // Ensure logging is initialized before we spawn any threads - initLogging() + /** + * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark + * cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster. + * + * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + * @param appName A name for your application, to display on the cluster web UI. + * @param sparkHome Location where Spark is installed on cluster nodes. + * @param jars Collection of JARs to send to the cluster. These can be paths on the local file + * system or HDFS, HTTP, HTTPS, or FTP URLs. + * @param environment Environment variables to set on worker nodes. + */ + def this(master: String, appName: String, sparkHome: String = null, + jars: Seq[String] = Nil, environment: Map[String, String] = Map(), + preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] = + immutable.Map()) = + this(new SparkConf(false).setAppName(appName).setMasterUrl(master) + .setJars(jars).set(environment.toSeq).setSparkHome(sparkHome), + environment, preferredNodeLocationData) // Set Spark driver host and port system properties - if (System.getProperty("spark.driver.host") == null) { - System.setProperty("spark.driver.host", Utils.localHostName()) - } - if (System.getProperty("spark.driver.port") == null) { - System.setProperty("spark.driver.port", "0") - } + Try(conf.get("spark.driver.host")) + .getOrElse(conf.set("spark.driver.host", Utils.localHostName())) + + Try(conf.get("spark.driver.port")) + .getOrElse(conf.set("spark.driver.port", "0")) + + val jars: Seq[String] = if (conf.getOrElse("spark.jars", null) != null) { + conf.get("spark.jars").split(",") + } else null + + val master = conf.get("spark.master") + val appName = conf.get("spark.appName") val isLocal = (master == "local" || master.startsWith("local[")) + // Ensure logging is initialized before we spawn any threads + initLogging() + // Create the Spark execution environment (cache, map output tracker, etc) private[spark] val env = SparkEnv.createFromSystemProperties( "", - System.getProperty("spark.driver.host"), - System.getProperty("spark.driver.port").toInt, + conf.get("spark.driver.host"), + conf.get("spark.driver.port").toInt, + conf, true, isLocal) SparkEnv.set(env) @@ -165,24 +173,24 @@ class SparkContext( /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ val hadoopConfiguration = { val env = SparkEnv.get - val conf = SparkHadoopUtil.get.newConfiguration() + val hadoopConf = SparkHadoopUtil.get.newConfiguration() // Explicitly check for S3 environment variables if (System.getenv("AWS_ACCESS_KEY_ID") != null && System.getenv("AWS_SECRET_ACCESS_KEY") != null) { - conf.set("fs.s3.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID")) - conf.set("fs.s3n.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID")) - conf.set("fs.s3.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY")) - conf.set("fs.s3n.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY")) + hadoopConf.set("fs.s3.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID")) + hadoopConf.set("fs.s3n.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID")) + hadoopConf.set("fs.s3.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY")) + hadoopConf.set("fs.s3n.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY")) } // Copy any "spark.hadoop.foo=bar" system properties into conf as "foo=bar" Utils.getSystemProperties.foreach { case (key, value) => if (key.startsWith("spark.hadoop.")) { - conf.set(key.substring("spark.hadoop.".length), value) + hadoopConf.set(key.substring("spark.hadoop.".length), value) } } - val bufferSize = System.getProperty("spark.buffer.size", "65536") - conf.set("io.file.buffer.size", bufferSize) - conf + val bufferSize = conf.getOrElse("spark.buffer.size", "65536") + hadoopConf.set("io.file.buffer.size", bufferSize) + hadoopConf } private[spark] var checkpointDir: Option[String] = None @@ -695,10 +703,8 @@ class SparkContext( * (in that order of preference). If neither of these is set, return None. */ private[spark] def getSparkHome(): Option[String] = { - if (sparkHome != null) { - Some(sparkHome) - } else if (System.getProperty("spark.home") != null) { - Some(System.getProperty("spark.home")) + if (conf.getOrElse("spark.home", null) != null) { + Some(conf.get("spark.home")) } else if (System.getenv("SPARK_HOME") != null) { Some(System.getenv("SPARK_HOME")) } else { @@ -909,6 +915,14 @@ object SparkContext { private[spark] val SPARK_UNKNOWN_USER = "" + private lazy val conf = new SparkConf() + + private[spark] def globalConf = { + if (SparkEnv.get != null) { + SparkEnv.get.conf + } else conf + } + implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] { def addInPlace(t1: Double, t2: Double): Double = t1 + t2 def zero(initialValue: Double) = 0.0 @@ -1020,7 +1034,7 @@ object SparkContext { /** Get the amount of memory per executor requested through system properties or SPARK_MEM */ private[spark] val executorMemoryRequested = { // TODO: Might need to add some extra memory for the non-heap parts of the JVM - Option(System.getProperty("spark.executor.memory")) + Try(globalConf.get("spark.executor.memory")).toOption .orElse(Option(System.getenv("SPARK_MEM"))) .map(Utils.memoryStringToMb) .getOrElse(512) @@ -1123,7 +1137,7 @@ object SparkContext { case mesosUrl @ MESOS_REGEX(_) => MesosNativeLibrary.load() val scheduler = new ClusterScheduler(sc) - val coarseGrained = System.getProperty("spark.mesos.coarse", "false").toBoolean + val coarseGrained = globalConf.getOrElse("spark.mesos.coarse", "false").toBoolean val url = mesosUrl.stripPrefix("mesos://") // strip scheme from raw Mesos URLs val backend = if (coarseGrained) { new CoarseMesosSchedulerBackend(scheduler, sc, url, appName) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 826f5c2d8c6bd..78e4ae27b2bb5 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -54,7 +54,8 @@ class SparkEnv ( val connectionManager: ConnectionManager, val httpFileServer: HttpFileServer, val sparkFilesDir: String, - val metricsSystem: MetricsSystem) { + val metricsSystem: MetricsSystem, + val conf: SparkConf) { private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]() @@ -114,25 +115,27 @@ object SparkEnv extends Logging { executorId: String, hostname: String, port: Int, + conf: SparkConf, isDriver: Boolean, isLocal: Boolean): SparkEnv = { - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, port) + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, port, + conf = conf) // Bit of a hack: If this is the driver and our port was 0 (meaning bind to any free port), // figure out which port number Akka actually bound to and set spark.driver.port to it. if (isDriver && port == 0) { - System.setProperty("spark.driver.port", boundPort.toString) + conf.set("spark.driver.port", boundPort.toString) } // set only if unset until now. - if (System.getProperty("spark.hostPort", null) == null) { + if (conf.getOrElse("spark.hostPort", null) == null) { if (!isDriver){ // unexpected Utils.logErrorWithStack("Unexpected NOT to have spark.hostPort set") } Utils.checkHost(hostname) - System.setProperty("spark.hostPort", hostname + ":" + boundPort) + conf.set("spark.hostPort", hostname + ":" + boundPort) } val classLoader = Thread.currentThread.getContextClassLoader @@ -140,25 +143,25 @@ object SparkEnv extends Logging { // Create an instance of the class named by the given Java system property, or by // defaultClassName if the property is not set, and return it as a T def instantiateClass[T](propertyName: String, defaultClassName: String): T = { - val name = System.getProperty(propertyName, defaultClassName) + val name = conf.getOrElse(propertyName, defaultClassName) Class.forName(name, true, classLoader).newInstance().asInstanceOf[T] } val serializerManager = new SerializerManager val serializer = serializerManager.setDefault( - System.getProperty("spark.serializer", "org.apache.spark.serializer.JavaSerializer")) + conf.getOrElse("spark.serializer", "org.apache.spark.serializer.JavaSerializer")) val closureSerializer = serializerManager.get( - System.getProperty("spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer")) + conf.getOrElse("spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer")) def registerOrLookup(name: String, newActor: => Actor): Either[ActorRef, ActorSelection] = { if (isDriver) { logInfo("Registering " + name) Left(actorSystem.actorOf(Props(newActor), name = name)) } else { - val driverHost: String = System.getProperty("spark.driver.host", "localhost") - val driverPort: Int = System.getProperty("spark.driver.port", "7077").toInt + val driverHost: String = conf.getOrElse("spark.driver.host", "localhost") + val driverPort: Int = conf.getOrElse("spark.driver.port", "7077").toInt Utils.checkHost(driverHost, "Expected hostname") val url = "akka.tcp://spark@%s:%s/user/%s".format(driverHost, driverPort, name) logInfo("Connecting to " + name + ": " + url) @@ -168,21 +171,21 @@ object SparkEnv extends Logging { val blockManagerMaster = new BlockManagerMaster(registerOrLookup( "BlockManagerMaster", - new BlockManagerMasterActor(isLocal))) - val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, serializer) + new BlockManagerMasterActor(isLocal, conf)), conf) + val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, serializer, conf) val connectionManager = blockManager.connectionManager - val broadcastManager = new BroadcastManager(isDriver) + val broadcastManager = new BroadcastManager(isDriver, conf) val cacheManager = new CacheManager(blockManager) // Have to assign trackerActor after initialization as MapOutputTrackerActor // requires the MapOutputTracker itself val mapOutputTracker = if (isDriver) { - new MapOutputTrackerMaster() + new MapOutputTrackerMaster(conf) } else { - new MapOutputTracker() + new MapOutputTracker(conf) } mapOutputTracker.trackerActor = registerOrLookup( "MapOutputTracker", @@ -193,12 +196,12 @@ object SparkEnv extends Logging { val httpFileServer = new HttpFileServer() httpFileServer.initialize() - System.setProperty("spark.fileserver.uri", httpFileServer.serverUri) + conf.set("spark.fileserver.uri", httpFileServer.serverUri) val metricsSystem = if (isDriver) { - MetricsSystem.createMetricsSystem("driver") + MetricsSystem.createMetricsSystem("driver", conf) } else { - MetricsSystem.createMetricsSystem("executor") + MetricsSystem.createMetricsSystem("executor", conf) } metricsSystem.start() @@ -212,7 +215,7 @@ object SparkEnv extends Logging { } // Warn about deprecated spark.cache.class property - if (System.getProperty("spark.cache.class") != null) { + if (conf.getOrElse("spark.cache.class", null) != null) { logWarning("The spark.cache.class property is no longer being used! Specify storage " + "levels using the RDD.persist() method instead.") } @@ -231,6 +234,7 @@ object SparkEnv extends Logging { connectionManager, httpFileServer, sparkFilesDir, - metricsSystem) + metricsSystem, + conf) } } diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index ca42c769286e6..d6eacfe23ef11 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -41,7 +41,7 @@ private[spark] class PythonRDD[T: ClassTag]( accumulator: Accumulator[JList[Array[Byte]]]) extends RDD[Array[Byte]](parent) { - val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt + val bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt override def getPartitions = parent.partitions @@ -247,10 +247,10 @@ private class BytesToString extends org.apache.spark.api.java.function.Function[ */ private class PythonAccumulatorParam(@transient serverHost: String, serverPort: Int) extends AccumulatorParam[JList[Array[Byte]]] { - + import SparkContext.{globalConf => conf} Utils.checkHost(serverHost, "Expected hostname") - val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt + val bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt override def zero(value: JList[Array[Byte]]): JList[Array[Byte]] = new JArrayList diff --git a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala index 43c18294c552b..be99d229efb53 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala @@ -32,7 +32,7 @@ abstract class Broadcast[T](private[spark] val id: Long) extends Serializable { } private[spark] -class BroadcastManager(val _isDriver: Boolean) extends Logging with Serializable { +class BroadcastManager(val _isDriver: Boolean, conf: SparkConf) extends Logging with Serializable { private var initialized = false private var broadcastFactory: BroadcastFactory = null @@ -43,14 +43,14 @@ class BroadcastManager(val _isDriver: Boolean) extends Logging with Serializable private def initialize() { synchronized { if (!initialized) { - val broadcastFactoryClass = System.getProperty( + val broadcastFactoryClass = conf.getOrElse( "spark.broadcast.factory", "org.apache.spark.broadcast.HttpBroadcastFactory") broadcastFactory = Class.forName(broadcastFactoryClass).newInstance.asInstanceOf[BroadcastFactory] // Initialize appropriate BroadcastFactory and BroadcastObject - broadcastFactory.initialize(isDriver) + broadcastFactory.initialize(isDriver, conf) initialized = true } diff --git a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala index 68bff75b908c7..fb161ce69d40b 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala @@ -17,6 +17,8 @@ package org.apache.spark.broadcast +import org.apache.spark.SparkConf + /** * An interface for all the broadcast implementations in Spark (to allow * multiple broadcast implementations). SparkContext uses a user-specified @@ -24,7 +26,7 @@ package org.apache.spark.broadcast * entire Spark job. */ private[spark] trait BroadcastFactory { - def initialize(isDriver: Boolean): Unit + def initialize(isDriver: Boolean, conf: SparkConf): Unit def newBroadcast[T](value: T, isLocal: Boolean, id: Long): Broadcast[T] def stop(): Unit } diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala index 47db720416447..cecb8c228bb9f 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -24,7 +24,7 @@ import java.util.concurrent.TimeUnit import it.unimi.dsi.fastutil.io.FastBufferedInputStream import it.unimi.dsi.fastutil.io.FastBufferedOutputStream -import org.apache.spark.{HttpServer, Logging, SparkEnv} +import org.apache.spark.{SparkConf, HttpServer, Logging, SparkEnv} import org.apache.spark.io.CompressionCodec import org.apache.spark.storage.{BroadcastBlockId, StorageLevel} import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashSet, Utils} @@ -64,7 +64,7 @@ private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolea } private[spark] class HttpBroadcastFactory extends BroadcastFactory { - def initialize(isDriver: Boolean) { HttpBroadcast.initialize(isDriver) } + def initialize(isDriver: Boolean, conf: SparkConf) { HttpBroadcast.initialize(isDriver, conf) } def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) = new HttpBroadcast[T](value_, isLocal, id) @@ -88,15 +88,16 @@ private object HttpBroadcast extends Logging { private lazy val compressionCodec = CompressionCodec.createCodec() - def initialize(isDriver: Boolean) { + def initialize(isDriver: Boolean, conf: SparkConf) { synchronized { if (!initialized) { - bufferSize = System.getProperty("spark.buffer.size", "65536").toInt - compress = System.getProperty("spark.broadcast.compress", "true").toBoolean + bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt + compress = conf.getOrElse("spark.broadcast.compress", "true").toBoolean if (isDriver) { createServer() + conf.set("spark.httpBroadcast.uri", serverUri) } - serverUri = System.getProperty("spark.httpBroadcast.uri") + serverUri = conf.get("spark.httpBroadcast.uri") initialized = true } } @@ -118,7 +119,6 @@ private object HttpBroadcast extends Logging { server = new HttpServer(broadcastDir) server.start() serverUri = server.uri - System.setProperty("spark.httpBroadcast.uri", serverUri) logInfo("Broadcast server started at " + serverUri) } diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 073a0a50297d0..4a3801dc48691 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -166,8 +166,9 @@ private object TorrentBroadcast extends Logging { private var initialized = false - - def initialize(_isDriver: Boolean) { + private var conf: SparkConf = null + def initialize(_isDriver: Boolean, conf: SparkConf) { + TorrentBroadcast.conf = conf //TODO: we might have to fix it in tests synchronized { if (!initialized) { initialized = true @@ -179,7 +180,7 @@ extends Logging { initialized = false } - val BLOCK_SIZE = System.getProperty("spark.broadcast.blockSize", "4096").toInt * 1024 + lazy val BLOCK_SIZE = conf.getOrElse("spark.broadcast.blockSize", "4096").toInt * 1024 def blockifyObject[T](obj: T): TorrentInfo = { val byteArray = Utils.serialize[T](obj) @@ -238,7 +239,7 @@ private[spark] case class TorrentInfo( private[spark] class TorrentBroadcastFactory extends BroadcastFactory { - def initialize(isDriver: Boolean) { TorrentBroadcast.initialize(isDriver) } + def initialize(isDriver: Boolean, conf: SparkConf) { TorrentBroadcast.initialize(isDriver, conf) } def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) = new TorrentBroadcast[T](value_, isLocal, id) diff --git a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala index 19d393a0dbce7..dda43dc01820a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala @@ -26,7 +26,7 @@ private[spark] class ApplicationDescription( val appUiUrl: String) extends Serializable { - val user = System.getProperty("user.name", "") + val user = System.getProperty("user.name", "") override def toString: String = "ApplicationDescription(" + name + ")" } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index fc1537f7963c4..1c979ac3e0aa5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -67,8 +67,9 @@ class SparkHadoopUtil { } object SparkHadoopUtil { + import SparkContext.{globalConf => conf} private val hadoop = { - val yarnMode = java.lang.Boolean.valueOf(System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE"))) + val yarnMode = java.lang.Boolean.valueOf(conf.getOrElse("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE"))) if (yarnMode) { try { Class.forName("org.apache.spark.deploy.yarn.YarnSparkHadoopUtil").newInstance.asInstanceOf[SparkHadoopUtil] diff --git a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala index 953755e40dbbf..9bbd635ab9bb0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala @@ -19,20 +19,18 @@ package org.apache.spark.deploy.client import java.util.concurrent.TimeoutException -import scala.concurrent.duration._ import scala.concurrent.Await +import scala.concurrent.duration._ import akka.actor._ import akka.pattern.ask -import akka.remote.{RemotingLifecycleEvent, DisassociatedEvent} - -import org.apache.spark.{SparkException, Logging} +import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} +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 import org.apache.spark.util.AkkaUtils - /** * The main class used to talk to a Spark deploy cluster. Takes a master URL, an app description, * and a listener for cluster events, and calls back the listener when various events occur. @@ -43,7 +41,8 @@ private[spark] class Client( actorSystem: ActorSystem, masterUrls: Array[String], appDescription: ApplicationDescription, - listener: ClientListener) + listener: ClientListener, + conf: SparkConf) extends Logging { val REGISTRATION_TIMEOUT = 20.seconds @@ -178,7 +177,7 @@ private[spark] class Client( def stop() { if (actor != null) { try { - val timeout = AkkaUtils.askTimeout + val timeout = AkkaUtils.askTimeout(conf) val future = actor.ask(StopClient)(timeout) Await.result(future, timeout) } catch { 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 5b62d3ba6c0e0..426cf524ae848 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 @@ -18,7 +18,7 @@ package org.apache.spark.deploy.client import org.apache.spark.util.{Utils, AkkaUtils} -import org.apache.spark.{Logging} +import org.apache.spark.{SparkContext, Logging} import org.apache.spark.deploy.{Command, ApplicationDescription} private[spark] object TestClient { @@ -45,11 +45,12 @@ private[spark] object TestClient { def main(args: Array[String]) { val url = args(0) - val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0) + val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0, + conf = SparkContext.globalConf) val desc = new ApplicationDescription( "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), "dummy-spark-home", "ignored") val listener = new TestListener - val client = new Client(actorSystem, Array(url), desc, listener) + val client = new Client(actorSystem, Array(url), desc, listener, SparkContext.globalConf) client.start() actorSystem.awaitTermination() } 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 eebd0794b8585..2c162c4fa23fd 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 @@ -29,7 +29,7 @@ import akka.pattern.ask import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} import akka.serialization.SerializationExtension -import org.apache.spark.{Logging, SparkException} +import org.apache.spark.{SparkContext, Logging, SparkException} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.MasterMessages._ @@ -39,13 +39,13 @@ import org.apache.spark.util.{AkkaUtils, Utils} private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { import context.dispatcher - + val conf = SparkContext.globalConf val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs - val WORKER_TIMEOUT = System.getProperty("spark.worker.timeout", "60").toLong * 1000 - val RETAINED_APPLICATIONS = System.getProperty("spark.deploy.retainedApplications", "200").toInt - val REAPER_ITERATIONS = System.getProperty("spark.dead.worker.persistence", "15").toInt - val RECOVERY_DIR = System.getProperty("spark.deploy.recoveryDirectory", "") - val RECOVERY_MODE = System.getProperty("spark.deploy.recoveryMode", "NONE") + val WORKER_TIMEOUT = conf.getOrElse("spark.worker.timeout", "60").toLong * 1000 + val RETAINED_APPLICATIONS = conf.getOrElse("spark.deploy.retainedApplications", "200").toInt + val REAPER_ITERATIONS = conf.getOrElse("spark.dead.worker.persistence", "15").toInt + val RECOVERY_DIR = conf.getOrElse("spark.deploy.recoveryDirectory", "") + val RECOVERY_MODE = conf.getOrElse("spark.deploy.recoveryMode", "NONE") var nextAppNumber = 0 val workers = new HashSet[WorkerInfo] @@ -63,8 +63,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act Utils.checkHost(host, "Expected hostname") - val masterMetricsSystem = MetricsSystem.createMetricsSystem("master") - val applicationMetricsSystem = MetricsSystem.createMetricsSystem("applications") + val masterMetricsSystem = MetricsSystem.createMetricsSystem("master", conf) + val applicationMetricsSystem = MetricsSystem.createMetricsSystem("applications", conf) val masterSource = new MasterSource(this) val webUi = new MasterWebUI(this, webUiPort) @@ -86,7 +86,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act // As a temporary workaround before better ways of configuring memory, we allow users to set // a flag that will perform round-robin scheduling across the nodes (spreading out each app // among all the nodes) instead of trying to consolidate each app onto a small # of nodes. - val spreadOutApps = System.getProperty("spark.deploy.spreadOut", "true").toBoolean + val spreadOutApps = conf.getOrElse("spark.deploy.spreadOut", "true").toBoolean override def preStart() { logInfo("Starting Spark master at " + masterUrl) @@ -103,7 +103,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act persistenceEngine = RECOVERY_MODE match { case "ZOOKEEPER" => logInfo("Persisting recovery state to ZooKeeper") - new ZooKeeperPersistenceEngine(SerializationExtension(context.system)) + new ZooKeeperPersistenceEngine(SerializationExtension(context.system), conf) case "FILESYSTEM" => logInfo("Persisting recovery state to directory: " + RECOVERY_DIR) new FileSystemPersistenceEngine(RECOVERY_DIR, SerializationExtension(context.system)) @@ -113,7 +113,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act leaderElectionAgent = RECOVERY_MODE match { case "ZOOKEEPER" => - context.actorOf(Props(classOf[ZooKeeperLeaderElectionAgent], self, masterUrl)) + context.actorOf(Props(classOf[ZooKeeperLeaderElectionAgent], self, masterUrl, conf)) case _ => context.actorOf(Props(classOf[MonarchyLeaderAgent], self)) } @@ -507,7 +507,7 @@ private[spark] object Master { val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r def main(argStrings: Array[String]) { - val args = new MasterArguments(argStrings) + val args = new MasterArguments(argStrings, SparkContext.globalConf) val (actorSystem, _, _) = startSystemAndActor(args.host, args.port, args.webUiPort) actorSystem.awaitTermination() } @@ -523,9 +523,10 @@ private[spark] object Master { } def startSystemAndActor(host: String, port: Int, webUiPort: Int): (ActorSystem, Int, Int) = { - val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port) + val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port, + conf = SparkContext.globalConf) val actor = actorSystem.actorOf(Props(classOf[Master], host, boundPort, webUiPort), actorName) - val timeout = AkkaUtils.askTimeout + val timeout = AkkaUtils.askTimeout(SparkContext.globalConf) val respFuture = actor.ask(RequestWebUIPort)(timeout) val resp = Await.result(respFuture, timeout).asInstanceOf[WebUIPortResponse] (actorSystem, boundPort, resp.webUIBoundPort) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala index 9d89b455fb961..7ce83f9c368a4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala @@ -18,11 +18,12 @@ package org.apache.spark.deploy.master import org.apache.spark.util.{Utils, IntParam} +import org.apache.spark.SparkConf /** * Command-line parser for the master. */ -private[spark] class MasterArguments(args: Array[String]) { +private[spark] class MasterArguments(args: Array[String], conf: SparkConf) { var host = Utils.localHostName() var port = 7077 var webUiPort = 8080 @@ -37,8 +38,8 @@ private[spark] class MasterArguments(args: Array[String]) { if (System.getenv("SPARK_MASTER_WEBUI_PORT") != null) { webUiPort = System.getenv("SPARK_MASTER_WEBUI_PORT").toInt } - if (System.getProperty("master.ui.port") != null) { - webUiPort = System.getProperty("master.ui.port").toInt + if (conf.get("master.ui.port") != null) { + webUiPort = conf.get("master.ui.port").toInt } parse(args.toList) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala index 6cc7fd2ff498f..79d95b1a833a5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala @@ -23,7 +23,7 @@ import org.apache.zookeeper._ import org.apache.zookeeper.Watcher.Event.KeeperState import org.apache.zookeeper.data.Stat -import org.apache.spark.Logging +import org.apache.spark.{SparkConf, Logging} /** * Provides a Scala-side interface to the standard ZooKeeper client, with the addition of retry @@ -35,8 +35,9 @@ import org.apache.spark.Logging * Additionally, all commands sent to ZooKeeper will be retried until they either fail too many * times or a semantic exception is thrown (e.g., "node already exists"). */ -private[spark] class SparkZooKeeperSession(zkWatcher: SparkZooKeeperWatcher) extends Logging { - val ZK_URL = System.getProperty("spark.deploy.zookeeper.url", "") +private[spark] class SparkZooKeeperSession(zkWatcher: SparkZooKeeperWatcher, + conf: SparkConf) extends Logging { + val ZK_URL = conf.getOrElse("spark.deploy.zookeeper.url", "") val ZK_ACL = ZooDefs.Ids.OPEN_ACL_UNSAFE val ZK_TIMEOUT_MILLIS = 30000 diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala index 7d535b08def16..df5bb368a2eeb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala @@ -21,16 +21,17 @@ import akka.actor.ActorRef import org.apache.zookeeper._ import org.apache.zookeeper.Watcher.Event.EventType -import org.apache.spark.Logging +import org.apache.spark.{SparkConf, Logging} import org.apache.spark.deploy.master.MasterMessages._ -private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: ActorRef, masterUrl: String) +private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: ActorRef, + masterUrl: String, conf: SparkConf) extends LeaderElectionAgent with SparkZooKeeperWatcher with Logging { - val WORKING_DIR = System.getProperty("spark.deploy.zookeeper.dir", "/spark") + "/leader_election" + val WORKING_DIR = conf.getOrElse("spark.deploy.zookeeper.dir", "/spark") + "/leader_election" private val watcher = new ZooKeeperWatcher() - private val zk = new SparkZooKeeperSession(this) + private val zk = new SparkZooKeeperSession(this, conf) private var status = LeadershipStatus.NOT_LEADER private var myLeaderFile: String = _ private var leaderUrl: String = _ diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala index 825344b3bb534..c55b720422040 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala @@ -17,19 +17,19 @@ package org.apache.spark.deploy.master -import org.apache.spark.Logging +import org.apache.spark.{SparkConf, Logging} import org.apache.zookeeper._ import akka.serialization.Serialization -class ZooKeeperPersistenceEngine(serialization: Serialization) +class ZooKeeperPersistenceEngine(serialization: Serialization, conf: SparkConf) extends PersistenceEngine with SparkZooKeeperWatcher with Logging { - val WORKING_DIR = System.getProperty("spark.deploy.zookeeper.dir", "/spark") + "/master_status" + val WORKING_DIR = conf.getOrElse("spark.deploy.zookeeper.dir", "/spark") + "/master_status" - val zk = new SparkZooKeeperSession(this) + val zk = new SparkZooKeeperSession(this, conf) zk.connect() diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index 9ab594b682c43..ead35662fc75a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -31,7 +31,7 @@ import org.apache.spark.util.{AkkaUtils, Utils} */ private[spark] class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { - val timeout = AkkaUtils.askTimeout + val timeout = AkkaUtils.askTimeout(master.conf) val host = Utils.localHostName() val port = requestedPort diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 87531b6719e1b..75a6e75c7840f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -25,23 +25,14 @@ import scala.collection.mutable.HashMap import scala.concurrent.duration._ import akka.actor._ -import akka.remote.{ DisassociatedEvent, RemotingLifecycleEvent} - -import org.apache.spark.{SparkException, Logging} +import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} +import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.deploy.{ExecutorDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.Master import org.apache.spark.deploy.worker.ui.WorkerWebUI import org.apache.spark.metrics.MetricsSystem -import org.apache.spark.util.{Utils, AkkaUtils} -import org.apache.spark.deploy.DeployMessages.WorkerStateResponse -import org.apache.spark.deploy.DeployMessages.RegisterWorkerFailed -import org.apache.spark.deploy.DeployMessages.KillExecutor -import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged -import org.apache.spark.deploy.DeployMessages.Heartbeat -import org.apache.spark.deploy.DeployMessages.RegisteredWorker -import org.apache.spark.deploy.DeployMessages.LaunchExecutor -import org.apache.spark.deploy.DeployMessages.RegisterWorker +import org.apache.spark.util.{AkkaUtils, Utils} /** * @param masterUrls Each url should look like spark://host:port. @@ -53,7 +44,8 @@ private[spark] class Worker( cores: Int, memory: Int, masterUrls: Array[String], - workDirPath: String = null) + workDirPath: String = null, + val conf: SparkConf) extends Actor with Logging { import context.dispatcher @@ -63,7 +55,7 @@ private[spark] class Worker( val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For worker and executor IDs // Send a heartbeat every (heartbeat timeout) / 4 milliseconds - val HEARTBEAT_MILLIS = System.getProperty("spark.worker.timeout", "60").toLong * 1000 / 4 + val HEARTBEAT_MILLIS = conf.getOrElse("spark.worker.timeout", "60").toLong * 1000 / 4 val REGISTRATION_TIMEOUT = 20.seconds val REGISTRATION_RETRIES = 3 @@ -92,7 +84,7 @@ private[spark] class Worker( var coresUsed = 0 var memoryUsed = 0 - val metricsSystem = MetricsSystem.createMetricsSystem("worker") + val metricsSystem = MetricsSystem.createMetricsSystem("worker", conf) val workerSource = new WorkerSource(this) def coresFree: Int = cores - coresUsed @@ -275,6 +267,7 @@ private[spark] class Worker( } private[spark] object Worker { + import org.apache.spark.SparkContext.globalConf def main(argStrings: Array[String]) { val args = new WorkerArguments(argStrings) val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort, args.cores, @@ -287,9 +280,10 @@ private[spark] object Worker { : (ActorSystem, Int) = { // The LocalSparkCluster runs multiple local sparkWorkerX actor systems val systemName = "sparkWorker" + workerNumber.map(_.toString).getOrElse("") - val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port) + val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port, + conf = globalConf) actorSystem.actorOf(Props(classOf[Worker], host, boundPort, webUiPort, cores, memory, - masterUrls, workDir), name = "Worker") + masterUrls, workDir, globalConf), name = "Worker") (actorSystem, boundPort) } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index 40d6bdb3fdbe9..ec47ba1b56768 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -22,7 +22,7 @@ import java.io.File import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.{Handler, Server} -import org.apache.spark.Logging +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.deploy.worker.Worker import org.apache.spark.ui.{JettyUtils, UIUtils} import org.apache.spark.ui.JettyUtils._ @@ -34,10 +34,10 @@ import org.apache.spark.util.{AkkaUtils, Utils} private[spark] class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[Int] = None) extends Logging { - val timeout = AkkaUtils.askTimeout + val timeout = AkkaUtils.askTimeout(worker.conf) val host = Utils.localHostName() val port = requestedPort.getOrElse( - System.getProperty("worker.ui.port", WorkerWebUI.DEFAULT_PORT).toInt) + worker.conf.getOrElse("worker.ui.port", WorkerWebUI.DEFAULT_PORT).toInt) var server: Option[Server] = None var boundPort: Option[Int] = None 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 debbdd4c444cc..c8319f6f6e692 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -22,7 +22,7 @@ import java.nio.ByteBuffer import akka.actor._ import akka.remote._ -import org.apache.spark.Logging +import org.apache.spark.{SparkContext, Logging} import org.apache.spark.TaskState.TaskState import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.util.{Utils, AkkaUtils} @@ -98,10 +98,10 @@ private[spark] object CoarseGrainedExecutorBackend { // Create a new ActorSystem to run the backend, because we can't create a SparkEnv / Executor // before getting started with all our system properties, etc val (actorSystem, boundPort) = AkkaUtils.createActorSystem("sparkExecutor", hostname, 0, - indestructible = true) + indestructible = true, conf = SparkContext.globalConf) // set it val sparkHostPort = hostname + ":" + boundPort - System.setProperty("spark.hostPort", sparkHostPort) +// conf.set("spark.hostPort", sparkHostPort) actorSystem.actorOf( Props(classOf[CoarseGrainedExecutorBackend], driverUrl, executorId, sparkHostPort, cores), name = "Executor") 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 0f19d7a96b085..70fc30e9931f3 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -57,17 +57,17 @@ private[spark] class Executor( // Make sure the local hostname we report matches the cluster scheduler's name for this host Utils.setCustomHostname(slaveHostname) - + val conf = new SparkConf(false) // Set spark.* system properties from executor arg for ((key, value) <- properties) { - System.setProperty(key, value) + conf.set(key, value) } // If we are in yarn mode, systems can have different disk layouts so we must set it // to what Yarn on this system said was available. This will be used later when SparkEnv // created. if (java.lang.Boolean.valueOf(System.getenv("SPARK_YARN_MODE"))) { - System.setProperty("spark.local.dir", getYarnLocalDirs()) + conf.set("spark.local.dir", getYarnLocalDirs()) } // Create our ClassLoader and set it on this thread @@ -108,7 +108,7 @@ private[spark] class Executor( // Initialize Spark environment (using system properties read above) private val env = { if (!isLocal) { - val _env = SparkEnv.createFromSystemProperties(executorId, slaveHostname, 0, + val _env = SparkEnv.createFromSystemProperties(executorId, slaveHostname, 0, conf, isDriver = false, isLocal = false) SparkEnv.set(_env) _env.metricsSystem.registerSource(executorSource) @@ -303,7 +303,7 @@ private[spark] class Executor( * new classes defined by the REPL as the user types code */ private def addReplClassLoaderIfNeeded(parent: ClassLoader): ClassLoader = { - val classUri = System.getProperty("spark.repl.class.uri") + val classUri = conf.getOrElse("spark.repl.class.uri", null) if (classUri != null) { logInfo("Using REPL class URI: " + classUri) try { diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index 570a979b56879..8ef5019b6c308 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -22,6 +22,7 @@ import java.io.{InputStream, OutputStream} import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} +import org.apache.spark.SparkConf /** @@ -37,15 +38,16 @@ trait CompressionCodec { private[spark] object CompressionCodec { - + import org.apache.spark.SparkContext.globalConf def createCodec(): CompressionCodec = { createCodec(System.getProperty( "spark.io.compression.codec", classOf[LZFCompressionCodec].getName)) } def createCodec(codecName: String): CompressionCodec = { - Class.forName(codecName, true, Thread.currentThread.getContextClassLoader) - .newInstance().asInstanceOf[CompressionCodec] + val ctor = Class.forName(codecName, true, Thread.currentThread.getContextClassLoader) + .getConstructor(classOf[SparkConf]) + ctor.newInstance(globalConf).asInstanceOf[CompressionCodec] } } @@ -53,7 +55,7 @@ private[spark] object CompressionCodec { /** * LZF implementation of [[org.apache.spark.io.CompressionCodec]]. */ -class LZFCompressionCodec extends CompressionCodec { +class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { override def compressedOutputStream(s: OutputStream): OutputStream = { new LZFOutputStream(s).setFinishBlockOnFlush(true) @@ -67,10 +69,10 @@ class LZFCompressionCodec extends CompressionCodec { * Snappy implementation of [[org.apache.spark.io.CompressionCodec]]. * Block size can be configured by spark.io.compression.snappy.block.size. */ -class SnappyCompressionCodec extends CompressionCodec { +class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec { override def compressedOutputStream(s: OutputStream): OutputStream = { - val blockSize = System.getProperty("spark.io.compression.snappy.block.size", "32768").toInt + val blockSize = conf.getOrElse("spark.io.compression.snappy.block.size", "32768").toInt new SnappyOutputStream(s, blockSize) } 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 bec0c83be8bea..ac29816f19f5a 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -24,7 +24,7 @@ import java.util.concurrent.TimeUnit import scala.collection.mutable -import org.apache.spark.Logging +import org.apache.spark.{SparkConf, Logging} import org.apache.spark.metrics.sink.{MetricsServlet, Sink} import org.apache.spark.metrics.source.Source @@ -62,10 +62,11 @@ import org.apache.spark.metrics.source.Source * * [options] is the specific property of this source or sink. */ -private[spark] class MetricsSystem private (val instance: String) extends Logging { +private[spark] class MetricsSystem private (val instance: String, + conf: SparkConf) extends Logging { initLogging() - val confFile = System.getProperty("spark.metrics.conf") + val confFile = conf.getOrElse("spark.metrics.conf", null) val metricsConfig = new MetricsConfig(Option(confFile)) val sinks = new mutable.ArrayBuffer[Sink] @@ -159,5 +160,6 @@ private[spark] object MetricsSystem { } } - def createMetricsSystem(instance: String): MetricsSystem = new MetricsSystem(instance) + def createMetricsSystem(instance: String, conf: SparkConf): MetricsSystem = + new MetricsSystem(instance, conf) } diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index 703bc6a9ca9fa..3e902f8ac57c6 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -37,7 +37,7 @@ import scala.concurrent.duration._ import org.apache.spark.util.Utils -private[spark] class ConnectionManager(port: Int) extends Logging { +private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Logging { class MessageStatus( val message: Message, @@ -54,22 +54,22 @@ private[spark] class ConnectionManager(port: Int) extends Logging { private val selector = SelectorProvider.provider.openSelector() private val handleMessageExecutor = new ThreadPoolExecutor( - System.getProperty("spark.core.connection.handler.threads.min","20").toInt, - System.getProperty("spark.core.connection.handler.threads.max","60").toInt, - System.getProperty("spark.core.connection.handler.threads.keepalive","60").toInt, TimeUnit.SECONDS, + conf.getOrElse("spark.core.connection.handler.threads.min", "20").toInt, + conf.getOrElse("spark.core.connection.handler.threads.max", "60").toInt, + conf.getOrElse("spark.core.connection.handler.threads.keepalive", "60").toInt, TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable]()) private val handleReadWriteExecutor = new ThreadPoolExecutor( - System.getProperty("spark.core.connection.io.threads.min","4").toInt, - System.getProperty("spark.core.connection.io.threads.max","32").toInt, - System.getProperty("spark.core.connection.io.threads.keepalive","60").toInt, TimeUnit.SECONDS, + conf.getOrElse("spark.core.connection.io.threads.min", "4").toInt, + conf.getOrElse("spark.core.connection.io.threads.max", "32").toInt, + conf.getOrElse("spark.core.connection.io.threads.keepalive", "60").toInt, TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable]()) // Use a different, yet smaller, thread pool - infrequently used with very short lived tasks : which should be executed asap private val handleConnectExecutor = new ThreadPoolExecutor( - System.getProperty("spark.core.connection.connect.threads.min","1").toInt, - System.getProperty("spark.core.connection.connect.threads.max","8").toInt, - System.getProperty("spark.core.connection.connect.threads.keepalive","60").toInt, TimeUnit.SECONDS, + conf.getOrElse("spark.core.connection.connect.threads.min", "1").toInt, + conf.getOrElse("spark.core.connection.connect.threads.max", "8").toInt, + conf.getOrElse("spark.core.connection.connect.threads.keepalive", "60").toInt, TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable]()) private val serverChannel = ServerSocketChannel.open() @@ -593,8 +593,10 @@ private[spark] class ConnectionManager(port: Int) extends Logging { private[spark] object ConnectionManager { + import SparkContext.globalConf + def main(args: Array[String]) { - val manager = new ConnectionManager(9999) + val manager = new ConnectionManager(9999, globalConf) manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { println("Received [" + msg + "] from [" + id + "]") None diff --git a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala index 781715108be63..4ca3cd390b95b 100644 --- a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala +++ b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala @@ -21,9 +21,9 @@ import java.nio.ByteBuffer import java.net.InetAddress private[spark] object ReceiverTest { - + import org.apache.spark.SparkContext.globalConf def main(args: Array[String]) { - val manager = new ConnectionManager(9999) + val manager = new ConnectionManager(9999, globalConf) println("Started connection manager with id = " + manager.id) manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { diff --git a/core/src/main/scala/org/apache/spark/network/SenderTest.scala b/core/src/main/scala/org/apache/spark/network/SenderTest.scala index 777574980fbc6..11c21fc1d5f87 100644 --- a/core/src/main/scala/org/apache/spark/network/SenderTest.scala +++ b/core/src/main/scala/org/apache/spark/network/SenderTest.scala @@ -21,7 +21,7 @@ import java.nio.ByteBuffer import java.net.InetAddress private[spark] object SenderTest { - + import org.apache.spark.SparkContext.globalConf def main(args: Array[String]) { if (args.length < 2) { @@ -33,7 +33,7 @@ private[spark] object SenderTest { val targetPort = args(1).toInt val targetConnectionManagerId = new ConnectionManagerId(targetHost, targetPort) - val manager = new ConnectionManager(0) + val manager = new ConnectionManager(0, globalConf) println("Started connection manager with id = " + manager.id) manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { diff --git a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala index b1e1576dadc1a..81b3104afd24c 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala @@ -23,20 +23,20 @@ import io.netty.buffer.ByteBuf import io.netty.channel.ChannelHandlerContext import io.netty.util.CharsetUtil -import org.apache.spark.Logging +import org.apache.spark.{SparkContext, SparkConf, Logging} import org.apache.spark.network.ConnectionManagerId import scala.collection.JavaConverters._ import org.apache.spark.storage.BlockId -private[spark] class ShuffleCopier extends Logging { +private[spark] class ShuffleCopier(conf: SparkConf) extends Logging { def getBlock(host: String, port: Int, blockId: BlockId, resultCollectCallback: (BlockId, Long, ByteBuf) => Unit) { val handler = new ShuffleCopier.ShuffleClientHandler(resultCollectCallback) - val connectTimeout = System.getProperty("spark.shuffle.netty.connect.timeout", "60000").toInt + val connectTimeout = conf.getOrElse("spark.shuffle.netty.connect.timeout", "60000").toInt val fc = new FileClient(handler, connectTimeout) try { @@ -107,7 +107,7 @@ private[spark] object ShuffleCopier extends Logging { val tasks = (for (i <- Range(0, threads)) yield { Executors.callable(new Runnable() { def run() { - val copier = new ShuffleCopier() + val copier = new ShuffleCopier(SparkContext.globalConf) copier.getBlock(host, port, blockId, echoResultCollectCallBack) } }) diff --git a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala index a712ef1c27093..9fbe0027480cf 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala @@ -75,6 +75,8 @@ class CheckpointRDD[T: ClassTag](sc: SparkContext, val checkpointPath: String) private[spark] object CheckpointRDD extends Logging { + import SparkContext.{globalConf => conf} + def splitIdToFile(splitId: Int): String = { "part-%05d".format(splitId) } @@ -92,7 +94,7 @@ private[spark] object CheckpointRDD extends Logging { throw new IOException("Checkpoint failed: temporary path " + tempOutputPath + " already exists") } - val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt + val bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt val fileOutputStream = if (blockSize < 0) { fs.create(tempOutputPath, false, bufferSize) @@ -122,7 +124,7 @@ private[spark] object CheckpointRDD extends Logging { def readFromFile[T](path: Path, context: TaskContext): Iterator[T] = { val env = SparkEnv.get val fs = path.getFileSystem(SparkHadoopUtil.get.newConfiguration()) - val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt + val bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt val fileInputStream = fs.open(path, bufferSize) val serializer = env.serializer.newInstance() val deserializeStream = serializer.deserializeStream(fileInputStream) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index ea45566ad1da9..f8b1a6932ec1b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -81,6 +81,7 @@ abstract class RDD[T: ClassTag]( def this(@transient oneParent: RDD[_]) = this(oneParent.context , List(new OneToOneDependency(oneParent))) + private[spark] def conf = sc.conf // ======================================================================= // Methods that should be implemented by subclasses of RDD // ======================================================================= diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 60927831a159a..3f55cd5642330 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -41,7 +41,7 @@ import org.apache.spark.storage.StorageLevel class JobLogger(val user: String, val logDirName: String) extends SparkListener with Logging { - def this() = this(System.getProperty("user.name", ""), + def this() = this(System.getProperty("user.name", ""), String.valueOf(System.currentTimeMillis())) private val logDir = diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala index 356fe56bf32c7..9002d33cda2c4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala @@ -20,7 +20,7 @@ package org.apache.spark.scheduler import java.io.{FileInputStream, InputStream} import java.util.{NoSuchElementException, Properties} -import org.apache.spark.Logging +import org.apache.spark.{SparkConf, Logging} import scala.xml.XML @@ -49,10 +49,10 @@ private[spark] class FIFOSchedulableBuilder(val rootPool: Pool) } } -private[spark] class FairSchedulableBuilder(val rootPool: Pool) +private[spark] class FairSchedulableBuilder(val rootPool: Pool, conf: SparkConf) extends SchedulableBuilder with Logging { - val schedulerAllocFile = Option(System.getProperty("spark.scheduler.allocation.file")) + val schedulerAllocFile = Option(conf.get("spark.scheduler.allocation.file")) val DEFAULT_SCHEDULER_FILE = "fairscheduler.xml" val FAIR_SCHEDULER_PROPERTIES = "spark.scheduler.pool" val DEFAULT_POOL_NAME = "default" diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala index 66ab8ea4cdf57..7e231ec44c71b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala @@ -49,11 +49,12 @@ private[spark] class ClusterScheduler(val sc: SparkContext) extends TaskScheduler with Logging { + val conf = sc.conf // How often to check for speculative tasks - val SPECULATION_INTERVAL = System.getProperty("spark.speculation.interval", "100").toLong + val SPECULATION_INTERVAL = conf.getOrElse("spark.speculation.interval", "100").toLong // Threshold above which we warn user initial TaskSet may be starved - val STARVATION_TIMEOUT = System.getProperty("spark.starvation.timeout", "15000").toLong + val STARVATION_TIMEOUT = conf.getOrElse("spark.starvation.timeout", "15000").toLong // ClusterTaskSetManagers are not thread safe, so any access to one should be synchronized // on this class. @@ -90,7 +91,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) var rootPool: Pool = null // default scheduler is FIFO val schedulingMode: SchedulingMode = SchedulingMode.withName( - System.getProperty("spark.scheduler.mode", "FIFO")) + conf.getOrElse("spark.scheduler.mode", "FIFO")) // This is a var so that we can reset it for testing purposes. private[spark] var taskResultGetter = new TaskResultGetter(sc.env, this) @@ -108,7 +109,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) case SchedulingMode.FIFO => new FIFOSchedulableBuilder(rootPool) case SchedulingMode.FAIR => - new FairSchedulableBuilder(rootPool) + new FairSchedulableBuilder(rootPool, conf) } } schedulableBuilder.buildPools() @@ -119,7 +120,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) override def start() { backend.start() - if (System.getProperty("spark.speculation", "false").toBoolean) { + if (conf.getOrElse("spark.speculation", "false").toBoolean) { logInfo("Starting speculative execution thread") import sc.env.actorSystem.dispatcher sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL milliseconds, diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala index bf494aa64dc19..398b0cefbf236 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -50,15 +50,16 @@ private[spark] class ClusterTaskSetManager( extends TaskSetManager with Logging { + val conf = sched.sc.conf // CPUs to request per task - val CPUS_PER_TASK = System.getProperty("spark.task.cpus", "1").toInt + val CPUS_PER_TASK = conf.getOrElse("spark.task.cpus", "1").toInt // Maximum times a task is allowed to fail before failing the job - val MAX_TASK_FAILURES = System.getProperty("spark.task.maxFailures", "4").toInt + val MAX_TASK_FAILURES = conf.getOrElse("spark.task.maxFailures", "4").toInt // Quantile of tasks at which to start speculation - val SPECULATION_QUANTILE = System.getProperty("spark.speculation.quantile", "0.75").toDouble - val SPECULATION_MULTIPLIER = System.getProperty("spark.speculation.multiplier", "1.5").toDouble + val SPECULATION_QUANTILE = conf.getOrElse("spark.speculation.quantile", "0.75").toDouble + val SPECULATION_MULTIPLIER = conf.getOrElse("spark.speculation.multiplier", "1.5").toDouble // Serializer for closures and tasks. val env = SparkEnv.get @@ -117,7 +118,7 @@ private[spark] class ClusterTaskSetManager( // How frequently to reprint duplicate exceptions in full, in milliseconds val EXCEPTION_PRINT_INTERVAL = - System.getProperty("spark.logging.exceptionPrintInterval", "10000").toLong + conf.getOrElse("spark.logging.exceptionPrintInterval", "10000").toLong // Map of recent exceptions (identified by string representation and top stack frame) to // duplicate count (how many times the same exception has appeared) and time the full exception @@ -677,14 +678,14 @@ private[spark] class ClusterTaskSetManager( } private def getLocalityWait(level: TaskLocality.TaskLocality): Long = { - val defaultWait = System.getProperty("spark.locality.wait", "3000") + val defaultWait = conf.getOrElse("spark.locality.wait", "3000") level match { case TaskLocality.PROCESS_LOCAL => - System.getProperty("spark.locality.wait.process", defaultWait).toLong + conf.getOrElse("spark.locality.wait.process", defaultWait).toLong case TaskLocality.NODE_LOCAL => - System.getProperty("spark.locality.wait.node", defaultWait).toLong + conf.getOrElse("spark.locality.wait.node", defaultWait).toLong case TaskLocality.RACK_LOCAL => - System.getProperty("spark.locality.wait.rack", defaultWait).toLong + conf.getOrElse("spark.locality.wait.rack", defaultWait).toLong case TaskLocality.ANY => 0L } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 7e22c843bf05e..40555903ac767 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -22,6 +22,7 @@ import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.concurrent.Await import scala.concurrent.duration._ +import scala.util.Try import akka.actor._ import akka.pattern.ask @@ -46,8 +47,8 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac { // Use an atomic variable to track total number of cores in the cluster for simplicity and speed var totalCoreCount = new AtomicInteger(0) - - private val timeout = AkkaUtils.askTimeout + val conf = scheduler.sc.conf + private val timeout = AkkaUtils.askTimeout(conf) class DriverActor(sparkProperties: Seq[(String, String)]) extends Actor { private val executorActor = new HashMap[String, ActorRef] @@ -61,7 +62,7 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) // Periodically revive offers to allow delay scheduling to work - val reviveInterval = System.getProperty("spark.scheduler.revive.interval", "1000").toLong + val reviveInterval = conf.getOrElse("spark.scheduler.revive.interval", "1000").toLong import context.dispatcher context.system.scheduler.schedule(0.millis, reviveInterval.millis, self, ReviveOffers) } @@ -162,7 +163,7 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac override def start() { val properties = new ArrayBuffer[(String, String)] - val iterator = System.getProperties.entrySet.iterator + val iterator = scheduler.sc.conf.getAllConfiguration while (iterator.hasNext) { val entry = iterator.next val (key, value) = (entry.getKey.toString, entry.getValue.toString) @@ -170,6 +171,7 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac properties += ((key, value)) } } + //TODO (prashant) send conf instead of properties driverActor = actorSystem.actorOf( Props(new DriverActor(properties)), name = CoarseGrainedSchedulerBackend.ACTOR_NAME) } @@ -208,7 +210,7 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac driverActor ! KillTask(taskId, executorId) } - override def defaultParallelism() = Option(System.getProperty("spark.default.parallelism")) + override def defaultParallelism() = Try(conf.get("spark.default.parallelism")).toOption .map(_.toInt).getOrElse(math.max(totalCoreCount.get(), 2)) // Called by subclasses when notified of a lost worker diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala index e8fecec4a64f3..d01329b2b34b1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala @@ -31,13 +31,13 @@ private[spark] class SimrSchedulerBackend( val tmpPath = new Path(driverFilePath + "_tmp") val filePath = new Path(driverFilePath) - val maxCores = System.getProperty("spark.simr.executor.cores", "1").toInt + val maxCores = conf.getOrElse("spark.simr.executor.cores", "1").toInt override def start() { super.start() val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"), + sc.conf.get("spark.driver.host"), sc.conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) val conf = new Configuration() 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 7127a72d6d1e5..d6b8ac2d57d53 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 @@ -36,14 +36,14 @@ private[spark] class SparkDeploySchedulerBackend( var stopping = false var shutdownCallback : (SparkDeploySchedulerBackend) => Unit = _ - val maxCores = System.getProperty("spark.cores.max", Int.MaxValue.toString).toInt + val maxCores = conf.getOrElse("spark.cores.max", Int.MaxValue.toString).toInt override def start() { super.start() // The endpoint for executors to talk to us val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"), + conf.get("spark.driver.host"), conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}") val command = Command( @@ -52,7 +52,7 @@ private[spark] class SparkDeploySchedulerBackend( val appDesc = new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome, "http://" + sc.ui.appUIAddress) - client = new Client(sc.env.actorSystem, masters, appDesc, this) + client = new Client(sc.env.actorSystem, masters, appDesc, this, conf) client.start() } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala index e68c5277135f6..ff6cc37f1d07b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala @@ -31,7 +31,8 @@ import org.apache.spark.util.Utils */ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: ClusterScheduler) extends Logging { - private val THREADS = System.getProperty("spark.resultGetter.threads", "4").toInt + + private val THREADS = sparkEnv.conf.getOrElse("spark.resultGetter.threads", "4").toInt private val getTaskResultExecutor = Utils.newDaemonFixedThreadPool( THREADS, "Result resolver thread") 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 84fe3094cc7ba..2a3b0e15f7fa4 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 @@ -61,7 +61,7 @@ private[spark] class CoarseMesosSchedulerBackend( var driver: SchedulerDriver = null // Maximum number of cores to acquire (TODO: we'll need more flexible controls here) - val maxCores = System.getProperty("spark.cores.max", Int.MaxValue.toString).toInt + val maxCores = conf.getOrElse("spark.cores.max", Int.MaxValue.toString).toInt // Cores we have acquired with each Mesos task ID val coresByTaskId = new HashMap[Int, Int] @@ -76,7 +76,7 @@ private[spark] class CoarseMesosSchedulerBackend( "Spark home is not set; set it through the spark.home system " + "property, the SPARK_HOME environment variable or the SparkContext constructor")) - val extraCoresPerSlave = System.getProperty("spark.mesos.extra.cores", "0").toInt + val extraCoresPerSlave = conf.getOrElse("spark.mesos.extra.cores", "0").toInt var nextMesosTaskId = 0 @@ -121,10 +121,10 @@ private[spark] class CoarseMesosSchedulerBackend( val command = CommandInfo.newBuilder() .setEnvironment(environment) val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - System.getProperty("spark.driver.host"), - System.getProperty("spark.driver.port"), + conf.get("spark.driver.host"), + conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) - val uri = System.getProperty("spark.executor.uri") + val uri = conf.get("spark.executor.uri") if (uri == null) { val runScript = new File(sparkHome, "spark-class").getCanonicalPath command.setValue( 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 50cbc2ca92e7e..9bb92b4f01d73 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 @@ -101,7 +101,7 @@ private[spark] class MesosSchedulerBackend( } val command = CommandInfo.newBuilder() .setEnvironment(environment) - val uri = System.getProperty("spark.executor.uri") + val uri = sc.conf.get("spark.executor.uri") if (uri == null) { command.setValue(new File(sparkHome, "spark-executor").getCanonicalPath) } else { @@ -341,5 +341,5 @@ private[spark] class MesosSchedulerBackend( } // TODO: query Mesos for number of cores - override def defaultParallelism() = System.getProperty("spark.default.parallelism", "8").toInt + override def defaultParallelism() = sc.conf.getOrElse("spark.default.parallelism", "8").toInt } diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala index 01e95162c0f70..6069c1db3a6c4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala @@ -80,6 +80,7 @@ private[spark] class LocalScheduler(val threads: Int, val maxFailures: Int, val with Logging { val env = SparkEnv.get + val conf = env.conf val attemptId = new AtomicInteger var dagScheduler: DAGScheduler = null @@ -91,7 +92,7 @@ private[spark] class LocalScheduler(val threads: Int, val maxFailures: Int, val var schedulableBuilder: SchedulableBuilder = null var rootPool: Pool = null val schedulingMode: SchedulingMode = SchedulingMode.withName( - System.getProperty("spark.scheduler.mode", "FIFO")) + conf.getOrElse("spark.scheduler.mode", "FIFO")) val activeTaskSets = new HashMap[String, LocalTaskSetManager] val taskIdToTaskSetId = new HashMap[Long, String] val taskSetTaskIds = new HashMap[String, HashSet[Long]] @@ -106,7 +107,7 @@ private[spark] class LocalScheduler(val threads: Int, val maxFailures: Int, val case SchedulingMode.FIFO => new FIFOSchedulableBuilder(rootPool) case SchedulingMode.FAIR => - new FairSchedulableBuilder(rootPool) + new FairSchedulableBuilder(rootPool, conf) } } schedulableBuilder.buildPools() diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index e748c2275d589..17cec81038fdc 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -25,18 +25,20 @@ import com.esotericsoftware.kryo.{KryoException, Kryo} import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput} import com.twitter.chill.{EmptyScalaKryoInstantiator, AllScalaRegistrar} -import org.apache.spark.{SerializableWritable, Logging} +import org.apache.spark.{SparkContext, SparkConf, SerializableWritable, Logging} import org.apache.spark.broadcast.HttpBroadcast import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage._ +import scala.util.Try /** * A Spark serializer that uses the [[https://code.google.com/p/kryo/ Kryo serialization library]]. */ class KryoSerializer extends org.apache.spark.serializer.Serializer with Logging { + private val conf = SparkContext.globalConf private val bufferSize = { - System.getProperty("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024 + conf.getOrElse("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024 } def newKryoOutput() = new KryoOutput(bufferSize) @@ -48,7 +50,7 @@ class KryoSerializer extends org.apache.spark.serializer.Serializer with Logging // Allow disabling Kryo reference tracking if user knows their object graphs don't have loops. // Do this before we invoke the user registrator so the user registrator can override this. - kryo.setReferences(System.getProperty("spark.kryo.referenceTracking", "true").toBoolean) + kryo.setReferences(conf.getOrElse("spark.kryo.referenceTracking", "true").toBoolean) for (cls <- KryoSerializer.toRegister) kryo.register(cls) @@ -58,7 +60,7 @@ class KryoSerializer extends org.apache.spark.serializer.Serializer with Logging // Allow the user to register their own classes by setting spark.kryo.registrator try { - Option(System.getProperty("spark.kryo.registrator")).foreach { regCls => + Try(conf.get("spark.kryo.registrator")).toOption.foreach { regCls => logDebug("Running user registrator: " + regCls) val reg = Class.forName(regCls, true, classLoader).newInstance().asInstanceOf[KryoRegistrator] reg.registerClasses(kryo) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index e51c5b30a3dab..ee2ae471a912a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -312,7 +312,7 @@ object BlockFetcherIterator { logDebug("Sending request for %d blocks (%s) from %s".format( req.blocks.size, Utils.bytesToString(req.size), req.address.host)) val cmId = new ConnectionManagerId(req.address.host, req.address.nettyPort) - val cpier = new ShuffleCopier + val cpier = new ShuffleCopier(blockManager.conf) cpier.getBlocks(cmId, req.blocks, putResult) logDebug("Sent request for remote blocks " + req.blocks + " from " + req.address.host ) } @@ -327,7 +327,7 @@ object BlockFetcherIterator { fetchRequestsSync.put(request) } - copiers = startCopiers(System.getProperty("spark.shuffle.copier.threads", "6").toInt) + copiers = startCopiers(conf.getOrElse("spark.shuffle.copier.threads", "6").toInt) logInfo("Started " + fetchRequestsSync.size + " remote gets in " + Utils.getUsedTimeMs(startTime)) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 19a025a32973a..ffd166e93a5d9 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -30,7 +30,7 @@ import scala.concurrent.duration._ import it.unimi.dsi.fastutil.io.{FastBufferedOutputStream, FastByteArrayOutputStream} -import org.apache.spark.{Logging, SparkEnv, SparkException} +import org.apache.spark.{SparkConf, Logging, SparkEnv, SparkException} import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ import org.apache.spark.serializer.Serializer @@ -43,12 +43,13 @@ private[spark] class BlockManager( actorSystem: ActorSystem, val master: BlockManagerMaster, val defaultSerializer: Serializer, - maxMemory: Long) + maxMemory: Long, + val conf: SparkConf) extends Logging { val shuffleBlockManager = new ShuffleBlockManager(this) val diskBlockManager = new DiskBlockManager(shuffleBlockManager, - System.getProperty("spark.local.dir", System.getProperty("java.io.tmpdir"))) + conf.getOrElse("spark.local.dir", System.getProperty("java.io.tmpdir"))) private val blockInfo = new TimeStampedHashMap[BlockId, BlockInfo] @@ -57,12 +58,12 @@ private[spark] class BlockManager( // If we use Netty for shuffle, start a new Netty-based shuffle sender service. private val nettyPort: Int = { - val useNetty = System.getProperty("spark.shuffle.use.netty", "false").toBoolean - val nettyPortConfig = System.getProperty("spark.shuffle.sender.port", "0").toInt + val useNetty = conf.getOrElse("spark.shuffle.use.netty", "false").toBoolean + val nettyPortConfig = conf.getOrElse("spark.shuffle.sender.port", "0").toInt if (useNetty) diskBlockManager.startShuffleBlockSender(nettyPortConfig) else 0 } - val connectionManager = new ConnectionManager(0) + val connectionManager = new ConnectionManager(0, conf) implicit val futureExecContext = connectionManager.futureExecContext val blockManagerId = BlockManagerId( @@ -71,14 +72,14 @@ private[spark] class BlockManager( // Max megabytes of data to keep in flight per reducer (to avoid over-allocating memory // for receiving shuffle outputs) val maxBytesInFlight = - System.getProperty("spark.reducer.maxMbInFlight", "48").toLong * 1024 * 1024 + conf.getOrElse("spark.reducer.maxMbInFlight", "48").toLong * 1024 * 1024 // Whether to compress broadcast variables that are stored - val compressBroadcast = System.getProperty("spark.broadcast.compress", "true").toBoolean + val compressBroadcast = conf.getOrElse("spark.broadcast.compress", "true").toBoolean // Whether to compress shuffle output that are stored - val compressShuffle = System.getProperty("spark.shuffle.compress", "true").toBoolean + val compressShuffle = conf.getOrElse("spark.shuffle.compress", "true").toBoolean // Whether to compress RDD partitions that are stored serialized - val compressRdds = System.getProperty("spark.rdd.compress", "false").toBoolean + val compressRdds = conf.getOrElse("spark.rdd.compress", "false").toBoolean val heartBeatFrequency = BlockManager.getHeartBeatFrequencyFromSystemProperties @@ -115,8 +116,8 @@ private[spark] class BlockManager( * Construct a BlockManager with a memory limit set based on system properties. */ def this(execId: String, actorSystem: ActorSystem, master: BlockManagerMaster, - serializer: Serializer) = { - this(execId, actorSystem, master, serializer, BlockManager.getMaxMemoryFromSystemProperties) + serializer: Serializer, conf: SparkConf) = { + this(execId, actorSystem, master, serializer, BlockManager.getMaxMemoryFromSystemProperties, conf) } /** @@ -439,7 +440,7 @@ private[spark] class BlockManager( : BlockFetcherIterator = { val iter = - if (System.getProperty("spark.shuffle.use.netty", "false").toBoolean) { + if (conf.getOrElse("spark.shuffle.use.netty", "false").toBoolean) { new BlockFetcherIterator.NettyBlockFetcherIterator(this, blocksByAddress, serializer) } else { new BlockFetcherIterator.BasicBlockFetcherIterator(this, blocksByAddress, serializer) @@ -465,7 +466,8 @@ private[spark] class BlockManager( def getDiskWriter(blockId: BlockId, file: File, serializer: Serializer, bufferSize: Int) : BlockObjectWriter = { val compressStream: OutputStream => OutputStream = wrapForCompression(blockId, _) - new DiskBlockObjectWriter(blockId, file, serializer, bufferSize, compressStream) + val syncWrites = conf.getOrElse("spark.shuffle.sync", "false").toBoolean + new DiskBlockObjectWriter(blockId, file, serializer, bufferSize, compressStream, syncWrites) } /** @@ -856,19 +858,19 @@ private[spark] class BlockManager( private[spark] object BlockManager extends Logging { - + import org.apache.spark.SparkContext.{globalConf => conf} val ID_GENERATOR = new IdGenerator def getMaxMemoryFromSystemProperties: Long = { - val memoryFraction = System.getProperty("spark.storage.memoryFraction", "0.66").toDouble + val memoryFraction = conf.getOrElse("spark.storage.memoryFraction", "0.66").toDouble (Runtime.getRuntime.maxMemory * memoryFraction).toLong } def getHeartBeatFrequencyFromSystemProperties: Long = - System.getProperty("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong / 4 + conf.getOrElse("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong / 4 def getDisableHeartBeatsForTesting: Boolean = - System.getProperty("spark.test.disableBlockManagerHeartBeat", "false").toBoolean + conf.getOrElse("spark.test.disableBlockManagerHeartBeat", "false").toBoolean /** * Attempt to clean up a ByteBuffer if it is memory-mapped. This uses an *unsafe* Sun API that diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index e1d68ef59296a..fde7d63a68e6b 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -23,19 +23,20 @@ import scala.concurrent.ExecutionContext.Implicits.global import akka.actor._ import akka.pattern.ask -import org.apache.spark.{Logging, SparkException} +import org.apache.spark.{SparkConf, Logging, SparkException} import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.AkkaUtils private[spark] -class BlockManagerMaster(var driverActor : Either[ActorRef, ActorSelection]) extends Logging { +class BlockManagerMaster(var driverActor : Either[ActorRef, ActorSelection], + conf: SparkConf) extends Logging { - val AKKA_RETRY_ATTEMPTS: Int = System.getProperty("spark.akka.num.retries", "3").toInt - val AKKA_RETRY_INTERVAL_MS: Int = System.getProperty("spark.akka.retry.wait", "3000").toInt + val AKKA_RETRY_ATTEMPTS: Int = conf.getOrElse("spark.akka.num.retries", "3").toInt + val AKKA_RETRY_INTERVAL_MS: Int = conf.getOrElse("spark.akka.retry.wait", "3000").toInt val DRIVER_AKKA_ACTOR_NAME = "BlockManagerMaster" - val timeout = AkkaUtils.askTimeout + val timeout = AkkaUtils.askTimeout(conf) /** Remove a dead executor from the driver actor. This is only called on the driver side. */ def removeExecutor(execId: String) { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 21022e1cfb086..05502e4451e24 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -27,7 +27,7 @@ import scala.concurrent.duration._ import akka.actor.{Actor, ActorRef, Cancellable} import akka.pattern.ask -import org.apache.spark.{Logging, SparkException} +import org.apache.spark.{SparkConf, Logging, SparkException} import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -36,7 +36,7 @@ import org.apache.spark.util.{AkkaUtils, Utils} * all slaves' block managers. */ private[spark] -class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { +class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Actor with Logging { // Mapping from block manager id to the block manager's information. private val blockManagerInfo = @@ -48,14 +48,14 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { // Mapping from block id to the set of block managers that have the block. private val blockLocations = new JHashMap[BlockId, mutable.HashSet[BlockManagerId]] - private val akkaTimeout = AkkaUtils.askTimeout + private val akkaTimeout = AkkaUtils.askTimeout(conf) initLogging() - val slaveTimeout = System.getProperty("spark.storage.blockManagerSlaveTimeoutMs", + val slaveTimeout = conf.getOrElse("spark.storage.blockManagerSlaveTimeoutMs", "" + (BlockManager.getHeartBeatFrequencyFromSystemProperties * 3)).toLong - val checkTimeoutInterval = System.getProperty("spark.storage.blockManagerTimeoutIntervalMs", + val checkTimeoutInterval = conf.getOrElse("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong var timeoutCheckingTask: Cancellable = null diff --git a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala index b4451fc7b8e56..61e63c60d56e3 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala @@ -74,7 +74,8 @@ class DiskBlockObjectWriter( file: File, serializer: Serializer, bufferSize: Int, - compressStream: OutputStream => OutputStream) + compressStream: OutputStream => OutputStream, + syncWrites: Boolean) extends BlockObjectWriter(blockId) with Logging { @@ -97,8 +98,6 @@ class DiskBlockObjectWriter( override def flush() = out.flush() } - private val syncWrites = System.getProperty("spark.shuffle.sync", "false").toBoolean - /** The file channel, used for repositioning / truncating the file. */ private var channel: FileChannel = null private var bs: OutputStream = null diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index fcd2e97982955..8f528babd4ece 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -38,7 +38,7 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD extends PathResolver with Logging { private val MAX_DIR_CREATION_ATTEMPTS: Int = 10 - private val subDirsPerLocalDir = System.getProperty("spark.diskStore.subDirectories", "64").toInt + private val subDirsPerLocalDir = shuffleManager.conf.getOrElse("spark.diskStore.subDirectories", "64").toInt // Create one local directory for each path mentioned in spark.local.dir; then, inside this // directory, create multiple subdirectories that we will hash files into, in order to avoid diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala index e828e1d1c5e7b..850d3178dd1f6 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala @@ -27,6 +27,8 @@ import org.apache.spark.serializer.Serializer import org.apache.spark.util.{MetadataCleanerType, MetadataCleaner, TimeStampedHashMap} import org.apache.spark.util.collection.{PrimitiveKeyOpenHashMap, PrimitiveVector} import org.apache.spark.storage.ShuffleBlockManager.ShuffleFileGroup +import scala.util.Try +import org.apache.spark.SparkConf /** A group of writers for a ShuffleMapTask, one writer per reducer. */ private[spark] trait ShuffleWriterGroup { @@ -59,12 +61,13 @@ private[spark] trait ShuffleWriterGroup { */ private[spark] class ShuffleBlockManager(blockManager: BlockManager) { + def conf = blockManager.conf // Turning off shuffle file consolidation causes all shuffle Blocks to get their own file. // TODO: Remove this once the shuffle file consolidation feature is stable. val consolidateShuffleFiles = - System.getProperty("spark.shuffle.consolidateFiles", "false").toBoolean + conf.getOrElse("spark.shuffle.consolidateFiles", "false").toBoolean - private val bufferSize = System.getProperty("spark.shuffle.file.buffer.kb", "100").toInt * 1024 + private val bufferSize = conf.getOrElse("spark.shuffle.file.buffer.kb", "100").toInt * 1024 /** * Contains all the state related to a particular shuffle. This includes a pool of unused diff --git a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala index a8db37ded1433..b3b38933933e0 100644 --- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala @@ -22,6 +22,7 @@ import akka.actor._ import java.util.concurrent.ArrayBlockingQueue import util.Random import org.apache.spark.serializer.KryoSerializer +import org.apache.spark.SparkContext /** * This class tests the BlockManager and MemoryStore for thread safety and @@ -91,11 +92,12 @@ private[spark] object ThreadingTest { def main(args: Array[String]) { System.setProperty("spark.kryoserializer.buffer.mb", "1") val actorSystem = ActorSystem("test") + val conf = SparkContext.globalConf val serializer = new KryoSerializer val blockManagerMaster = new BlockManagerMaster( - Left(actorSystem.actorOf(Props(new BlockManagerMasterActor(true))))) + Left(actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf)))), conf) val blockManager = new BlockManager( - "", actorSystem, blockManagerMaster, serializer, 1024 * 1024) + "", actorSystem, blockManagerMaster, serializer, 1024 * 1024, conf) val producers = (1 to numProducers).map(i => new ProducerThread(blockManager, i)) val consumers = producers.map(p => new ConsumerThread(blockManager, p.queue)) producers.foreach(_.start) diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index f1d86c0221fea..0ce8d9c8c449d 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -32,7 +32,7 @@ import org.apache.spark.util.Utils /** Top level user interface for Spark */ private[spark] class SparkUI(sc: SparkContext) extends Logging { val host = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(Utils.localHostName()) - val port = Option(System.getProperty("spark.ui.port")).getOrElse(SparkUI.DEFAULT_PORT).toInt + val port = sc.conf.getOrElse("spark.ui.port", SparkUI.DEFAULT_PORT).toInt var boundPort: Option[Int] = None var server: Option[Server] = None diff --git a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala index fcd1b518d099b..14751e8e8ef2f 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala @@ -30,6 +30,8 @@ import org.apache.spark.scheduler.SchedulingMode * Usage: ./run spark.ui.UIWorkloadGenerator [master] */ private[spark] object UIWorkloadGenerator { + + import SparkContext.{globalConf => conf} val NUM_PARTITIONS = 100 val INTER_JOB_WAIT_MS = 5000 @@ -43,7 +45,7 @@ private[spark] object UIWorkloadGenerator { val appName = "Spark UI Tester" if (schedulingMode == SchedulingMode.FAIR) { - System.setProperty("spark.scheduler.mode", "FAIR") + conf.set("spark.scheduler.mode", "FAIR") } val sc = new SparkContext(master, appName) diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala index c5bf2acc9ec95..b637d37517e0a 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala @@ -63,7 +63,7 @@ private[spark] class EnvironmentUI(sc: SparkContext) { UIUtils.listingTable(propertyHeaders, propertyRow, otherProperties, fixedWidth = true) val classPathEntries = classPathProperty._2 - .split(System.getProperty("path.separator", ":")) + .split(sc.conf.getOrElse("path.separator", ":")) .filterNot(e => e.isEmpty) .map(e => (e, "System Classpath")) val addedJars = sc.addedJars.iterator.toSeq.map{case (path, time) => (path, "Added By User")} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 6b854740d6a24..f01a1380b90b0 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -33,7 +33,7 @@ import org.apache.spark.scheduler._ */ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkListener { // How many stages to remember - val RETAINED_STAGES = System.getProperty("spark.ui.retained_stages", "1000").toInt + val RETAINED_STAGES = sc.conf.getOrElse("spark.ui.retained_stages", "1000").toInt val DEFAULT_POOL_NAME = "default" val stageIdToPool = new HashMap[Int, String]() diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 1c8b51b8bc1e1..76febd5702fa4 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -21,6 +21,7 @@ import scala.concurrent.duration.{Duration, FiniteDuration} import akka.actor.{ActorSystem, ExtendedActorSystem, IndestructibleActorSystem} import com.typesafe.config.ConfigFactory +import org.apache.spark.SparkConf /** * Various utility classes for working with Akka. @@ -37,22 +38,22 @@ private[spark] object AkkaUtils { * If indestructible is set to true, the Actor System will continue running in the event * of a fatal exception. This is used by [[org.apache.spark.executor.Executor]]. */ - def createActorSystem(name: String, host: String, port: Int, indestructible: Boolean = false) - : (ActorSystem, Int) = { + def createActorSystem(name: String, host: String, port: Int, indestructible: Boolean = false, + conf: SparkConf): (ActorSystem, Int) = { - val akkaThreads = System.getProperty("spark.akka.threads", "4").toInt - val akkaBatchSize = System.getProperty("spark.akka.batchSize", "15").toInt + val akkaThreads = conf.getOrElse("spark.akka.threads", "4").toInt + val akkaBatchSize = conf.getOrElse("spark.akka.batchSize", "15").toInt - val akkaTimeout = System.getProperty("spark.akka.timeout", "100").toInt + val akkaTimeout = conf.getOrElse("spark.akka.timeout", "100").toInt - val akkaFrameSize = System.getProperty("spark.akka.frameSize", "10").toInt + val akkaFrameSize = conf.getOrElse("spark.akka.frameSize", "10").toInt val lifecycleEvents = - if (System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off" + if (conf.getOrElse("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off" - val akkaHeartBeatPauses = System.getProperty("spark.akka.heartbeat.pauses", "600").toInt + val akkaHeartBeatPauses = conf.getOrElse("spark.akka.heartbeat.pauses", "600").toInt val akkaFailureDetector = - System.getProperty("spark.akka.failure-detector.threshold", "300.0").toDouble - val akkaHeartBeatInterval = System.getProperty("spark.akka.heartbeat.interval", "1000").toInt + conf.getOrElse("spark.akka.failure-detector.threshold", "300.0").toDouble + val akkaHeartBeatInterval = conf.getOrElse("spark.akka.heartbeat.interval", "1000").toInt val akkaConf = ConfigFactory.parseString( s""" @@ -87,7 +88,7 @@ private[spark] object AkkaUtils { } /** Returns the default Spark timeout to use for Akka ask operations. */ - def askTimeout: FiniteDuration = { - Duration.create(System.getProperty("spark.akka.askTimeout", "30").toLong, "seconds") + def askTimeout(conf: SparkConf): FiniteDuration = { + Duration.create(conf.getOrElse("spark.akka.askTimeout", "30").toLong, "seconds") } } diff --git a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala index 7b41ef89f1da8..bf71d17a21271 100644 --- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala @@ -18,7 +18,7 @@ package org.apache.spark.util import java.util.{TimerTask, Timer} -import org.apache.spark.Logging +import org.apache.spark.{SparkContext, Logging} /** @@ -66,21 +66,21 @@ object MetadataCleanerType extends Enumeration { } object MetadataCleaner { - + private val conf = SparkContext.globalConf // using only sys props for now : so that workers can also get to it while preserving earlier behavior. - def getDelaySeconds = System.getProperty("spark.cleaner.ttl", "-1").toInt + def getDelaySeconds = conf.getOrElse("spark.cleaner.ttl", "3500").toInt //TODO: this is to fix tests for time being def getDelaySeconds(cleanerType: MetadataCleanerType.MetadataCleanerType): Int = { - System.getProperty(MetadataCleanerType.systemProperty(cleanerType), getDelaySeconds.toString).toInt + conf.getOrElse(MetadataCleanerType.systemProperty(cleanerType), getDelaySeconds.toString).toInt } def setDelaySeconds(cleanerType: MetadataCleanerType.MetadataCleanerType, delay: Int) { - System.setProperty(MetadataCleanerType.systemProperty(cleanerType), delay.toString) + conf.set(MetadataCleanerType.systemProperty(cleanerType), delay.toString) } def setDelaySeconds(delay: Int, resetAll: Boolean = true) { // override for all ? - System.setProperty("spark.cleaner.ttl", delay.toString) + conf.set("spark.cleaner.ttl", delay.toString) if (resetAll) { for (cleanerType <- MetadataCleanerType.values) { System.clearProperty(MetadataCleanerType.systemProperty(cleanerType)) diff --git a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala index a25b37a2a96a9..1407c39bfbe96 100644 --- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala +++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala @@ -30,7 +30,7 @@ import java.lang.management.ManagementFactory import scala.collection.mutable.ArrayBuffer import it.unimi.dsi.fastutil.ints.IntOpenHashSet -import org.apache.spark.Logging +import org.apache.spark.{SparkConf, SparkContext, Logging} /** * Estimates the sizes of Java objects (number of bytes of memory they occupy), for use in @@ -41,6 +41,7 @@ import org.apache.spark.Logging */ private[spark] object SizeEstimator extends Logging { + private def conf = SparkContext.globalConf // Sizes of primitive types private val BYTE_SIZE = 1 private val BOOLEAN_SIZE = 1 @@ -90,8 +91,8 @@ private[spark] object SizeEstimator extends Logging { } private def getIsCompressedOops : Boolean = { - if (System.getProperty("spark.test.useCompressedOops") != null) { - return System.getProperty("spark.test.useCompressedOops").toBoolean + if (conf.getOrElse("spark.test.useCompressedOops", null) != null) { + return conf.get("spark.test.useCompressedOops").toBoolean } try { diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 3f7858d2dedc3..fd5888e5252fc 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -36,7 +36,7 @@ import org.apache.hadoop.fs.{Path, FileSystem, FileUtil} import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} import org.apache.spark.deploy.SparkHadoopUtil import java.nio.ByteBuffer -import org.apache.spark.{SparkException, Logging} +import org.apache.spark.{SparkContext, SparkException, Logging} /** @@ -44,6 +44,7 @@ import org.apache.spark.{SparkException, Logging} */ private[spark] object Utils extends Logging { + private lazy val conf = SparkContext.globalConf /** Serialize an object using Java serialization */ def serialize[T](o: T): Array[Byte] = { val bos = new ByteArrayOutputStream() @@ -312,7 +313,7 @@ private[spark] object Utils extends Logging { * multiple paths. */ def getLocalDir: String = { - System.getProperty("spark.local.dir", System.getProperty("java.io.tmpdir")).split(',')(0) + conf.getOrElse("spark.local.dir", System.getProperty("java.io.tmpdir")).split(',')(0) } /** @@ -398,7 +399,7 @@ private[spark] object Utils extends Logging { } def localHostPort(): String = { - val retval = System.getProperty("spark.hostPort", null) + val retval = conf.getOrElse("spark.hostPort", null) if (retval == null) { logErrorWithStack("spark.hostPort not set but invoking localHostPort") return localHostName() diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 271dc905bc386..10b8b441fde18 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.AkkaUtils class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { - + private val conf = new SparkConf test("compressSize") { assert(MapOutputTracker.compressSize(0L) === 0) assert(MapOutputTracker.compressSize(1L) === 1) @@ -48,14 +48,14 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { test("master start and stop") { val actorSystem = ActorSystem("test") - val tracker = new MapOutputTrackerMaster() + val tracker = new MapOutputTrackerMaster(conf) tracker.trackerActor = Left(actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker)))) tracker.stop() } test("master register and fetch") { val actorSystem = ActorSystem("test") - val tracker = new MapOutputTrackerMaster() + val tracker = new MapOutputTrackerMaster(conf) tracker.trackerActor = Left(actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker)))) tracker.registerShuffle(10, 2) val compressedSize1000 = MapOutputTracker.compressSize(1000L) @@ -74,7 +74,7 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { test("master register and unregister and fetch") { val actorSystem = ActorSystem("test") - val tracker = new MapOutputTrackerMaster() + val tracker = new MapOutputTrackerMaster(conf) tracker.trackerActor = Left(actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker)))) tracker.registerShuffle(10, 2) val compressedSize1000 = MapOutputTracker.compressSize(1000L) @@ -96,16 +96,16 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { test("remote fetch") { val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0) + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, conf = conf) System.setProperty("spark.driver.port", boundPort.toString) // Will be cleared by LocalSparkContext System.setProperty("spark.hostPort", hostname + ":" + boundPort) - val masterTracker = new MapOutputTrackerMaster() + val masterTracker = new MapOutputTrackerMaster(conf) masterTracker.trackerActor = Left(actorSystem.actorOf( Props(new MapOutputTrackerMasterActor(masterTracker)), "MapOutputTracker")) - val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0) - val slaveTracker = new MapOutputTracker() + val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = conf) + val slaveTracker = new MapOutputTracker(conf) slaveTracker.trackerActor = Right(slaveSystem.actorSelection( "akka.tcp://spark@localhost:" + boundPort + "/user/MapOutputTracker")) 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 7181333adfe68..4ecdde0001e51 100644 --- a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala @@ -19,17 +19,19 @@ package org.apache.spark.metrics import org.scalatest.{BeforeAndAfter, FunSuite} import org.apache.spark.deploy.master.MasterSource +import org.apache.spark.SparkConf class MetricsSystemSuite extends FunSuite with BeforeAndAfter { var filePath: String = _ - + var conf: SparkConf = null before { filePath = getClass.getClassLoader.getResource("test_metrics_system.properties").getFile() System.setProperty("spark.metrics.conf", filePath) + conf = new SparkConf } test("MetricsSystem with default config") { - val metricsSystem = MetricsSystem.createMetricsSystem("default") + val metricsSystem = MetricsSystem.createMetricsSystem("default", conf) val sources = metricsSystem.sources val sinks = metricsSystem.sinks @@ -39,7 +41,7 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { } test("MetricsSystem with sources add") { - val metricsSystem = MetricsSystem.createMetricsSystem("test") + val metricsSystem = MetricsSystem.createMetricsSystem("test", conf) val sources = metricsSystem.sources val sinks = metricsSystem.sinks diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 706d84a58b563..2aa259daf38b3 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -17,21 +17,14 @@ package org.apache.spark.scheduler -import scala.collection.mutable.{Map, HashMap} - -import org.scalatest.FunSuite -import org.scalatest.BeforeAndAfter - -import org.apache.spark.LocalSparkContext -import org.apache.spark.MapOutputTrackerMaster -import org.apache.spark.SparkContext -import org.apache.spark.Partition -import org.apache.spark.TaskContext -import org.apache.spark.{Dependency, ShuffleDependency, OneToOneDependency} -import org.apache.spark.{FetchFailed, Success, TaskEndReason} +import scala.Tuple2 +import scala.collection.mutable.{HashMap, Map} + +import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} +import org.scalatest.{BeforeAndAfter, FunSuite} /** * Tests for DAGScheduler. These tests directly call the event processing functions in DAGScheduler @@ -46,7 +39,7 @@ import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} * and capturing the resulting TaskSets from the mock TaskScheduler. */ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { - + val conf = new SparkConf /** Set of TaskSets the DAGScheduler has requested executed. */ val taskSets = scala.collection.mutable.Buffer[TaskSet]() val taskScheduler = new TaskScheduler() { @@ -74,7 +67,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont */ val cacheLocations = new HashMap[(Int, Int), Seq[BlockManagerId]] // stub out BlockManagerMaster.getLocations to use our cacheLocations - val blockManagerMaster = new BlockManagerMaster(null) { + val blockManagerMaster = new BlockManagerMaster(null, conf) { override def getLocations(blockIds: Array[BlockId]): Seq[Seq[BlockManagerId]] = { blockIds.map { _.asRDDId.map(id => (id.rddId -> id.splitIndex)).flatMap(key => cacheLocations.get(key)). @@ -99,7 +92,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont taskSets.clear() cacheLocations.clear() results.clear() - mapOutputTracker = new MapOutputTrackerMaster() + mapOutputTracker = new MapOutputTrackerMaster(conf) scheduler = new DAGScheduler(taskScheduler, mapOutputTracker, blockManagerMaster, sc.env) { override def runLocally(job: ActiveJob) { // don't bother with the thread while unit testing diff --git a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala index 002368ff554f7..dd122615adb42 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala @@ -95,7 +95,7 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) - val user = System.getProperty("user.name", SparkContext.SPARK_UNKNOWN_USER) + val user = System.getProperty("user.name", SparkContext.SPARK_UNKNOWN_USER) joblogger.getLogDir should be ("/tmp/spark-%s".format(user)) joblogger.getJobIDtoPrintWriter.size should be (1) diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala index 95d3553d91840..34d2e4cb8c7c5 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala @@ -169,7 +169,7 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() System.setProperty("spark.scheduler.allocation.file", xmlPath) val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) - val schedulableBuilder = new FairSchedulableBuilder(rootPool) + val schedulableBuilder = new FairSchedulableBuilder(rootPool, sc.conf) schedulableBuilder.buildPools() assert(rootPool.getSchedulableByName("default") != null) diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala index bb28a31a990bc..2bb827c022c74 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala @@ -81,8 +81,8 @@ class FakeClusterScheduler(sc: SparkContext, liveExecutors: (String, String)* /* class ClusterTaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { import TaskLocality.{ANY, PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL} - - val LOCALITY_WAIT = System.getProperty("spark.locality.wait", "3000").toLong + private val conf = new SparkConf + val LOCALITY_WAIT = conf.getOrElse("spark.locality.wait", "3000").toLong test("TaskSet with no preferences") { sc = new SparkContext("local", "test") diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala index 27c2d5336142f..618fae7c16083 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala @@ -21,7 +21,7 @@ import java.nio.ByteBuffer import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} -import org.apache.spark.{LocalSparkContext, SparkContext, SparkEnv} +import org.apache.spark.{SparkConf, LocalSparkContext, SparkContext, SparkEnv} import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, TaskResult} import org.apache.spark.storage.TaskResultBlockId diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 5b4d63b954cc8..4ef5538951a59 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -31,8 +31,10 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark.util.{SizeEstimator, Utils, AkkaUtils, ByteBufferInputStream} import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} +import org.apache.spark.{SparkConf, SparkContext} class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodTester { + private val conf = new SparkConf var store: BlockManager = null var store2: BlockManager = null var actorSystem: ActorSystem = null @@ -42,7 +44,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT var oldHeartBeat: String = null // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test - System.setProperty("spark.kryoserializer.buffer.mb", "1") + conf.set("spark.kryoserializer.buffer.mb", "1") val serializer = new KryoSerializer // Implicitly convert strings to BlockIds for test clarity. @@ -50,22 +52,23 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT def rdd(rddId: Int, splitId: Int) = RDDBlockId(rddId, splitId) before { - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("test", "localhost", 0) + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("test", "localhost", 0, conf = conf) this.actorSystem = actorSystem - System.setProperty("spark.driver.port", boundPort.toString) - System.setProperty("spark.hostPort", "localhost:" + boundPort) + conf.set("spark.driver.port", boundPort.toString) + conf.set("spark.hostPort", "localhost:" + boundPort) master = new BlockManagerMaster( - Left(actorSystem.actorOf(Props(new BlockManagerMasterActor(true))))) + Left(actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf)))), conf) // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case - oldArch = System.setProperty("os.arch", "amd64") - oldOops = System.setProperty("spark.test.useCompressedOops", "true") - oldHeartBeat = System.setProperty("spark.storage.disableBlockManagerHeartBeat", "true") + System.setProperty("os.arch", "amd64") + conf.set("os.arch", "amd64") + conf.set("spark.test.useCompressedOops", "true") + conf.set("spark.storage.disableBlockManagerHeartBeat", "true") val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() // Set some value ... - System.setProperty("spark.hostPort", Utils.localHostName() + ":" + 1111) + conf.set("spark.hostPort", Utils.localHostName() + ":" + 1111) } after { @@ -86,13 +89,13 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT master = null if (oldArch != null) { - System.setProperty("os.arch", oldArch) + conf.set("os.arch", oldArch) } else { System.clearProperty("os.arch") } if (oldOops != null) { - System.setProperty("spark.test.useCompressedOops", oldOops) + conf.set("spark.test.useCompressedOops", oldOops) } else { System.clearProperty("spark.test.useCompressedOops") } @@ -133,7 +136,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("master + 1 manager interaction") { - store = new BlockManager("", actorSystem, master, serializer, 2000) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -163,8 +166,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("master + 2 managers interaction") { - store = new BlockManager("exec1", actorSystem, master, serializer, 2000) - store2 = new BlockManager("exec2", actorSystem, master, new KryoSerializer, 2000) + store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf) + store2 = new BlockManager("exec2", actorSystem, master, new KryoSerializer, 2000, conf) val peers = master.getPeers(store.blockManagerId, 1) assert(peers.size === 1, "master did not return the other manager as a peer") @@ -179,7 +182,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("removing block") { - store = new BlockManager("", actorSystem, master, serializer, 2000) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -227,7 +230,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("removing rdd") { - store = new BlockManager("", actorSystem, master, serializer, 2000) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -261,7 +264,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT test("reregistration on heart beat") { val heartBeat = PrivateMethod[Unit]('heartBeat) - store = new BlockManager("", actorSystem, master, serializer, 2000) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf) val a1 = new Array[Byte](400) store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) @@ -277,7 +280,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("reregistration on block update") { - store = new BlockManager("", actorSystem, master, serializer, 2000) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) @@ -296,7 +299,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT test("reregistration doesn't dead lock") { val heartBeat = PrivateMethod[Unit]('heartBeat) - store = new BlockManager("", actorSystem, master, serializer, 2000) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf) val a1 = new Array[Byte](400) val a2 = List(new Array[Byte](400)) @@ -333,7 +336,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU storage") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -352,7 +355,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU storage with serialization") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -371,7 +374,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU for partitions of same RDD") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -390,7 +393,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU for partitions of multiple RDDs") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) store.putSingle(rdd(0, 1), new Array[Byte](400), StorageLevel.MEMORY_ONLY) store.putSingle(rdd(0, 2), new Array[Byte](400), StorageLevel.MEMORY_ONLY) store.putSingle(rdd(1, 1), new Array[Byte](400), StorageLevel.MEMORY_ONLY) @@ -413,7 +416,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("on-disk storage") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -426,7 +429,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -441,7 +444,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage with getLocalBytes") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -456,7 +459,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage with serialization") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -471,7 +474,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage with serialization and getLocalBytes") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -486,7 +489,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("LRU with mixed storage levels") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -511,7 +514,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU with streams") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) val list1 = List(new Array[Byte](200), new Array[Byte](200)) val list2 = List(new Array[Byte](200), new Array[Byte](200)) val list3 = List(new Array[Byte](200), new Array[Byte](200)) @@ -535,7 +538,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("LRU with mixed storage levels and streams") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) val list1 = List(new Array[Byte](200), new Array[Byte](200)) val list2 = List(new Array[Byte](200), new Array[Byte](200)) val list3 = List(new Array[Byte](200), new Array[Byte](200)) @@ -581,7 +584,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("overly large block") { - store = new BlockManager("", actorSystem, master, serializer, 500) + store = new BlockManager("", actorSystem, master, serializer, 500, conf) store.putSingle("a1", new Array[Byte](1000), StorageLevel.MEMORY_ONLY) assert(store.getSingle("a1") === None, "a1 was in store") store.putSingle("a2", new Array[Byte](1000), StorageLevel.MEMORY_AND_DISK) @@ -591,53 +594,53 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT test("block compression") { try { - System.setProperty("spark.shuffle.compress", "true") - store = new BlockManager("exec1", actorSystem, master, serializer, 2000) + conf.set("spark.shuffle.compress", "true") + store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf) store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) <= 100, "shuffle_0_0_0 was not compressed") store.stop() store = null - System.setProperty("spark.shuffle.compress", "false") - store = new BlockManager("exec2", actorSystem, master, serializer, 2000) + conf.set("spark.shuffle.compress", "false") + store = new BlockManager("exec2", actorSystem, master, serializer, 2000, conf) store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) >= 1000, "shuffle_0_0_0 was compressed") store.stop() store = null - System.setProperty("spark.broadcast.compress", "true") - store = new BlockManager("exec3", actorSystem, master, serializer, 2000) + conf.set("spark.broadcast.compress", "true") + store = new BlockManager("exec3", actorSystem, master, serializer, 2000, conf) store.putSingle(BroadcastBlockId(0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(BroadcastBlockId(0)) <= 100, "broadcast_0 was not compressed") store.stop() store = null - System.setProperty("spark.broadcast.compress", "false") - store = new BlockManager("exec4", actorSystem, master, serializer, 2000) + conf.set("spark.broadcast.compress", "false") + store = new BlockManager("exec4", actorSystem, master, serializer, 2000, conf) store.putSingle(BroadcastBlockId(0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(BroadcastBlockId(0)) >= 1000, "broadcast_0 was compressed") store.stop() store = null - System.setProperty("spark.rdd.compress", "true") - store = new BlockManager("exec5", actorSystem, master, serializer, 2000) + conf.set("spark.rdd.compress", "true") + store = new BlockManager("exec5", actorSystem, master, serializer, 2000, conf) store.putSingle(rdd(0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(rdd(0, 0)) <= 100, "rdd_0_0 was not compressed") store.stop() store = null - System.setProperty("spark.rdd.compress", "false") - store = new BlockManager("exec6", actorSystem, master, serializer, 2000) + conf.set("spark.rdd.compress", "false") + store = new BlockManager("exec6", actorSystem, master, serializer, 2000, conf) store.putSingle(rdd(0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(rdd(0, 0)) >= 1000, "rdd_0_0 was compressed") store.stop() store = null // Check that any other block types are also kept uncompressed - store = new BlockManager("exec7", actorSystem, master, serializer, 2000) + store = new BlockManager("exec7", actorSystem, master, serializer, 2000, conf) store.putSingle("other_block", new Array[Byte](1000), StorageLevel.MEMORY_ONLY) assert(store.memoryStore.getSize("other_block") >= 1000, "other_block was compressed") store.stop() @@ -651,7 +654,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT test("block store put failure") { // Use Java serializer so we can create an unserializable error. - store = new BlockManager("", actorSystem, master, new JavaSerializer, 1200) + store = new BlockManager("", actorSystem, master, new JavaSerializer, 1200, conf) // The put should fail since a1 is not serializable. class UnserializableClass diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index 070982e798f34..f940448abd5fc 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -17,15 +17,18 @@ package org.apache.spark.storage -import java.io.{FileWriter, File} +import java.io.{File, FileWriter} import scala.collection.mutable import com.google.common.io.Files +import org.apache.spark.SparkConf import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} +import scala.util.Try +import akka.actor.{Props, ActorSelection, ActorSystem} class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with BeforeAndAfterAll { - + private val testConf = new SparkConf val rootDir0 = Files.createTempDir() rootDir0.deleteOnExit() val rootDir1 = Files.createTempDir() @@ -36,10 +39,11 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with Before // This suite focuses primarily on consolidation features, // so we coerce consolidation if not already enabled. val consolidateProp = "spark.shuffle.consolidateFiles" - val oldConsolidate = Option(System.getProperty(consolidateProp)) - System.setProperty(consolidateProp, "true") + val oldConsolidate = Try(testConf.get(consolidateProp)).toOption + testConf.set(consolidateProp, "true") val shuffleBlockManager = new ShuffleBlockManager(null) { + override def conf = testConf.clone var idToSegmentMap = mutable.Map[ShuffleBlockId, FileSegment]() override def getBlockLocation(id: ShuffleBlockId) = idToSegmentMap(id) } diff --git a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala index 5aff26f9fcb29..a5facd5bbd261 100644 --- a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.util import org.scalatest.FunSuite import org.scalatest.BeforeAndAfterAll import org.scalatest.PrivateMethodTester +import org.apache.spark.SparkContext class DummyClass1 {} @@ -139,7 +140,8 @@ class SizeEstimatorSuite test("64-bit arch with no compressed oops") { val arch = System.setProperty("os.arch", "amd64") val oops = System.setProperty("spark.test.useCompressedOops", "false") - + SparkContext.globalConf.set("os.arch", "amd64") + SparkContext.globalConf.set("spark.test.useCompressedOops", "false") val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala index 72b5c7b88e19b..12c430be27c2c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala @@ -36,16 +36,18 @@ object WikipediaPageRank { System.err.println("Usage: WikipediaPageRank ") System.exit(-1) } - - System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - System.setProperty("spark.kryo.registrator", classOf[PRKryoRegistrator].getName) + val sparkConf = new SparkConf() + sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + sparkConf.set("spark.kryo.registrator", classOf[PRKryoRegistrator].getName) val inputFile = args(0) val threshold = args(1).toDouble val numPartitions = args(2).toInt val host = args(3) val usePartitioner = args(4).toBoolean - val sc = new SparkContext(host, "WikipediaPageRank") + + sparkConf.setMasterUrl(host).setAppName("WikipediaPageRank") + val sc = new SparkContext(sparkConf) // Parse the Wikipedia page data into a graph val input = sc.textFile(inputFile) diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala index ddf6855325e73..5bf0b7a24a7f5 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala @@ -34,15 +34,19 @@ object WikipediaPageRankStandalone { System.err.println("Usage: WikipediaPageRankStandalone ") System.exit(-1) } + val sparkConf = new SparkConf() + sparkConf.set("spark.serializer", "spark.bagel.examples.WPRSerializer") - System.setProperty("spark.serializer", "spark.bagel.examples.WPRSerializer") val inputFile = args(0) val threshold = args(1).toDouble val numIterations = args(2).toInt val host = args(3) val usePartitioner = args(4).toBoolean - val sc = new SparkContext(host, "WikipediaPageRankStandalone") + + sparkConf.setMasterUrl(host).setAppName("WikipediaPageRankStandalone") + + val sc = new SparkContext(sparkConf) val input = sc.textFile(inputFile) val partitioner = new HashPartitioner(sc.defaultParallelism) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala index 50e3f9639cd5b..2402409e6ef87 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala @@ -26,6 +26,7 @@ import akka.actor.ActorRef import akka.actor.Props import akka.actor.actorRef2Scala +import org.apache.spark.SparkConf import org.apache.spark.streaming.Seconds import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.StreamingContext.toPairDStreamFunctions @@ -116,7 +117,7 @@ object FeederActor { val Seq(host, port) = args.toSeq - val actorSystem = AkkaUtils.createActorSystem("test", host, port.toInt)._1 + val actorSystem = AkkaUtils.createActorSystem("test", host, port.toInt, conf = new SparkConf)._1 val feeder = actorSystem.actorOf(Props[FeederActor], "FeederActor") println("Feeder started as:" + feeder) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index 36853acab514d..2f2d106f86876 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -578,14 +578,13 @@ object ALS { val implicitPrefs = if (args.length >= 7) args(6).toBoolean else false val alpha = if (args.length >= 8) args(7).toDouble else 1 val blocks = if (args.length == 9) args(8).toInt else -1 - - System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - System.setProperty("spark.kryo.registrator", classOf[ALSRegistrator].getName) - System.setProperty("spark.kryo.referenceTracking", "false") - System.setProperty("spark.kryoserializer.buffer.mb", "8") - System.setProperty("spark.locality.wait", "10000") - val sc = new SparkContext(master, "ALS") + sc.conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + sc.conf.set("spark.kryo.registrator", classOf[ALSRegistrator].getName) + sc.conf.set("spark.kryo.referenceTracking", "false") + sc.conf.set("spark.kryoserializer.buffer.mb", "8") + sc.conf.set("spark.locality.wait", "10000") + val ratings = sc.textFile(ratingsFile).map { line => val fields = line.split(',') Rating(fields(0).toInt, fields(1).toInt, fields(2).toDouble) diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index eeeca3ea8a33e..433268a1dd59c 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -61,13 +61,13 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e private var amClient: AMRMClient[ContainerRequest] = _ // Default to numWorkers * 2, with minimum of 3 - private val maxNumWorkerFailures = System.getProperty("spark.yarn.max.worker.failures", + private val maxNumWorkerFailures = conf.getOrElse("spark.yarn.max.worker.failures", math.max(args.numWorkers * 2, 3).toString()).toInt def run() { // Setup the directories so things go to YARN approved directories rather // than user specified and /tmp. - System.setProperty("spark.local.dir", getLocalDirs()) + conf.set("spark.local.dir", getLocalDirs()) // Use priority 30 as it's higher then HDFS. It's same priority as MapReduce is using. ShutdownHookManager.get().addShutdownHook(new AppMasterShutdownHook(this), 30) @@ -138,10 +138,10 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e logInfo("Waiting for Spark driver to be reachable.") var driverUp = false var tries = 0 - val numTries = System.getProperty("spark.yarn.applicationMaster.waitTries", "10").toInt + val numTries = conf.getOrElse("spark.yarn.applicationMaster.waitTries", "10").toInt while (!driverUp && tries < numTries) { - val driverHost = System.getProperty("spark.driver.host") - val driverPort = System.getProperty("spark.driver.port") + val driverHost = conf.get("spark.driver.host") + val driverPort = conf.get("spark.driver.port") try { val socket = new Socket(driverHost, driverPort.toInt) socket.close() @@ -199,7 +199,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e ApplicationMaster.sparkContextRef.synchronized { var numTries = 0 val waitTime = 10000L - val maxNumTries = System.getProperty("spark.yarn.ApplicationMaster.waitTries", "10").toInt + val maxNumTries = conf.getOrElse("spark.yarn.ApplicationMaster.waitTries", "10").toInt while (ApplicationMaster.sparkContextRef.get() == null && numTries < maxNumTries) { logInfo("Waiting for Spark context initialization ... " + numTries) numTries = numTries + 1 @@ -265,7 +265,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // we want to be reasonably responsive without causing too many requests to RM. val schedulerInterval = - System.getProperty("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong + conf.getOrElse("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong // must be <= timeoutInterval / 2. val interval = math.min(timeoutInterval / 2, schedulerInterval) @@ -343,7 +343,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e private def cleanupStagingDir() { var stagingDirPath: Path = null try { - val preserveFiles = System.getProperty("spark.yarn.preserve.staging.files", "false").toBoolean + val preserveFiles = conf.getOrElse("spark.yarn.preserve.staging.files", "false").toBoolean if (!preserveFiles) { stagingDirPath = new Path(System.getenv("SPARK_YARN_STAGING_DIR")) if (stagingDirPath == null) { diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 94678815e806a..a322f60864c65 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -244,7 +244,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } } val dst = new Path(fs.getHomeDirectory(), appStagingDir) - val replication = System.getProperty("spark.yarn.submit.file.replication", "3").toShort + val replication = conf.getOrElse("spark.yarn.submit.file.replication", "3").toShort if (UserGroupInformation.isSecurityEnabled()) { val dstFs = dst.getFileSystem(conf) @@ -499,7 +499,7 @@ object Client { Path.SEPARATOR + LOG4J_PROP) } // Normally the users app.jar is last in case conflicts with spark jars - val userClasspathFirst = System.getProperty("spark.yarn.user.classpath.first", "false") + val userClasspathFirst = conf.getOrElse("spark.yarn.user.classpath.first", "false") .toBoolean if (userClasspathFirst) { Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 70be15d0a3999..41ac292249ef4 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -35,7 +35,7 @@ class ClientArguments(val args: Array[String]) { var workerMemory = 1024 // MB var workerCores = 1 var numWorkers = 2 - var amQueue = System.getProperty("QUEUE", "default") + var amQueue = conf.getOrElse("QUEUE", "default") var amMemory: Int = 512 // MB var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster" var appName: String = "Spark" diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala index bc31bb2eb0901..f7d73f0d83f47 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala @@ -136,8 +136,8 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte Thread.sleep(100) } } - System.setProperty("spark.driver.host", driverHost) - System.setProperty("spark.driver.port", driverPort.toString) + conf.set("spark.driver.host", driverHost) + conf.set("spark.driver.port", driverPort.toString) val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME) diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index c27257cda4e55..71d1cbd416f5a 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -254,8 +254,8 @@ private[yarn] class YarnAllocationHandler( } else { val workerId = workerIdCounter.incrementAndGet().toString val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - System.getProperty("spark.driver.host"), - System.getProperty("spark.driver.port"), + conf.get("spark.driver.host"), + conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) logInfo("Launching container %s for on host %s".format(containerId, workerHostname)) diff --git a/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index b206780c7806e..6feaaff014256 100644 --- a/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -52,8 +52,8 @@ private[spark] class YarnClientSchedulerBackend( if (workerNumber == null) workerNumber = defaultWorkerNumber - val driverHost = System.getProperty("spark.driver.host") - val driverPort = System.getProperty("spark.driver.port") + val driverHost = conf.get("spark.driver.host") + val driverPort = conf.get("spark.driver.port") val hostport = driverHost + ":" + driverPort val argsArray = Array[String]( diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index ab96cfa18be16..ffb54a24ac7ec 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -244,7 +244,8 @@ object SparkBuild extends Build { "com.codahale.metrics" % "metrics-ganglia" % "3.0.0", "com.codahale.metrics" % "metrics-graphite" % "3.0.0", "com.twitter" %% "chill" % "0.3.1", - "com.twitter" % "chill-java" % "0.3.1" + "com.twitter" % "chill-java" % "0.3.1", + "com.typesafe" % "config" % "1.0.2" ) ) diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index 523fd1222dd71..b2f499e637d3d 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -930,9 +930,6 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, def createSparkContext(): SparkContext = { val uri = System.getenv("SPARK_EXECUTOR_URI") - if (uri != null) { - System.setProperty("spark.executor.uri", uri) - } val master = this.master match { case Some(m) => m case None => { @@ -942,6 +939,10 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } val jars = SparkILoop.getAddedJars.map(new java.io.File(_).getAbsolutePath) sparkContext = new SparkContext(master, "Spark shell", System.getenv("SPARK_HOME"), jars) + if (uri != null) { + sparkContext.conf.set("spark.executor.uri", uri) + } + sparkContext.conf.set("spark.repl.class.uri", intp.classServer.uri) echo("Created spark context..") sparkContext } diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala index e1455ef8a110a..0d412e44786b3 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala @@ -34,10 +34,8 @@ import scala.tools.reflect.StdRuntimeTags._ import scala.util.control.ControlThrowable import util.stackTraceString -import org.apache.spark.HttpServer +import org.apache.spark.{SparkContext, HttpServer, SparkEnv, Logging} import org.apache.spark.util.Utils -import org.apache.spark.SparkEnv -import org.apache.spark.Logging // /** directory to save .class files to */ // private class ReplVirtualDirectory(out: JPrintWriter) extends VirtualDirectory("((memory))", None) { @@ -91,7 +89,7 @@ import org.apache.spark.Logging /** Local directory to save .class files too */ val outputDir = { val tmp = System.getProperty("java.io.tmpdir") - val rootDir = System.getProperty("spark.repl.classdir", tmp) + val rootDir = SparkContext.globalConf.getOrElse("spark.repl.classdir", tmp) Utils.createTempDir(rootDir) } if (SPARK_DEBUG_REPL) { @@ -112,7 +110,6 @@ import org.apache.spark.Logging // Start the classServer and store its URI in a spark system property // (which will be passed to executors so that they can connect to it) classServer.start() - System.setProperty("spark.repl.class.uri", classServer.uri) if (SPARK_DEBUG_REPL) { echo("Class server started, URI = " + classServer.uri) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index 9271914eb536a..b8e1427a21c97 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -34,7 +34,7 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) extends Logging with Serializable { val master = ssc.sc.master val framework = ssc.sc.appName - val sparkHome = ssc.sc.sparkHome + val sparkHome = ssc.sc.getSparkHome.getOrElse(null) val jars = ssc.sc.jars val environment = ssc.sc.environment val graph = ssc.graph @@ -42,6 +42,7 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) val checkpointDuration = ssc.checkpointDuration val pendingTimes = ssc.scheduler.jobManager.getPendingTimes() val delaySeconds = MetadataCleaner.getDelaySeconds + val sparkConf = ssc.sc.conf def validate() { assert(master != null, "Checkpoint.master is null") diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala index ed892e33e6c34..1d23713c80ff1 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala @@ -26,7 +26,7 @@ class Scheduler(ssc: StreamingContext) extends Logging { initLogging() - val concurrentJobs = System.getProperty("spark.streaming.concurrentJobs", "1").toInt + val concurrentJobs = ssc.sc.conf.getOrElse("spark.streaming.concurrentJobs", "1").toInt val jobManager = new JobManager(ssc, concurrentJobs) val checkpointWriter = if (ssc.checkpointDuration != null && ssc.checkpointDir != null) { new CheckpointWriter(ssc.checkpointDir) @@ -34,7 +34,7 @@ class Scheduler(ssc: StreamingContext) extends Logging { null } - val clockClass = System.getProperty( + val clockClass = ssc.sc.conf.getOrElse( "spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock") val clock = Class.forName(clockClass).newInstance().asInstanceOf[Clock] val timer = new RecurringTimer(clock, ssc.graph.batchDuration.milliseconds, @@ -73,7 +73,7 @@ class Scheduler(ssc: StreamingContext) extends Logging { // or if the property is defined set it to that time if (clock.isInstanceOf[ManualClock]) { val lastTime = ssc.initialCheckpoint.checkpointTime.milliseconds - val jumpTime = System.getProperty("spark.streaming.manualClock.jump", "0").toLong + val jumpTime = ssc.sc.conf.getOrElse("spark.streaming.manualClock.jump", "0").toLong clock.asInstanceOf[ManualClock].setTime(lastTime + jumpTime) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index d2c4fdee657f7..76744223e12d9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -115,7 +115,7 @@ class StreamingContext private ( protected[streaming] val sc: SparkContext = { if (isCheckpointPresent) { - new SparkContext(cp_.master, cp_.framework, cp_.sparkHome, cp_.jars, cp_.environment) + new SparkContext(cp_.sparkConf, cp_.environment) } else { sc_ } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala index d5ae8aef9278a..8bf761b8cb09f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala @@ -175,8 +175,8 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging /** A helper actor that communicates with the NetworkInputTracker */ private class NetworkReceiverActor extends Actor { logInfo("Attempting to register with tracker") - val ip = System.getProperty("spark.driver.host", "localhost") - val port = System.getProperty("spark.driver.port", "7077").toInt + val ip = env.conf.getOrElse("spark.driver.host", "localhost") + val port = env.conf.getOrElse("spark.driver.port", "7077").toInt val url = "akka.tcp://spark@%s:%s/user/NetworkInputTracker".format(ip, port) val tracker = env.actorSystem.actorSelection(url) val timeout = 5.seconds @@ -213,7 +213,7 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging case class Block(id: BlockId, buffer: ArrayBuffer[T], metadata: Any = null) val clock = new SystemClock() - val blockInterval = System.getProperty("spark.streaming.blockInterval", "200").toLong + val blockInterval = env.conf.getOrElse("spark.streaming.blockInterval", "200").toLong val blockIntervalTimer = new RecurringTimer(clock, blockInterval, updateCurrentBuffer) val blockStorageLevel = storageLevel val blocksForPushing = new ArrayBlockingQueue[Block](1000) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index e81287b44e51f..315bd5443c67b 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -42,7 +42,7 @@ import org.apache.spark.streaming.util.ManualClock */ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") + conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") before { FileUtils.deleteDirectory(new File(checkpointDir)) @@ -69,7 +69,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { assert(batchDuration === Milliseconds(500), "batchDuration for this test must be 1 second") - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") + conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") val stateStreamCheckpointInterval = Seconds(1) @@ -135,13 +135,13 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { // Restart stream computation from the new checkpoint file to see whether that file has // correct checkpoint data + conf.set("spark.streaming.manualClock.jump", (batchDuration.milliseconds * 7).toString) ssc = new StreamingContext(checkpointDir) stateStream = ssc.graph.getOutputStreams().head.dependencies.head.dependencies.head logInfo("Restored data of state stream = \n[" + stateStream.generatedRDDs.mkString("\n") + "]") assert(!stateStream.generatedRDDs.isEmpty, "No restored RDDs in state stream after recovery from second failure") // Adjust manual clock time as if it is being restarted after a delay - System.setProperty("spark.streaming.manualClock.jump", (batchDuration.milliseconds * 7).toString) ssc.start() advanceTimeWithRealDelay(ssc, 4) ssc.stop() diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 7dc82decefd6e..da8f135dd75a6 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -53,7 +53,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { override def checkpointDir = "checkpoint" before { - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") + conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") } after { @@ -68,7 +68,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { testServer.start() // Set up the streaming context and input streams - val ssc = new StreamingContext(master, framework, batchDuration) + val ssc = new StreamingContext(new SparkContext(conf), batchDuration) val networkStream = ssc.socketTextStream("localhost", testServer.port, StorageLevel.MEMORY_AND_DISK) val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String ]] val outputStream = new TestOutputStream(networkStream, outputBuffer) @@ -113,7 +113,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { test("flume input stream") { // Set up the streaming context and input streams - val ssc = new StreamingContext(master, framework, batchDuration) + val ssc = new StreamingContext(new SparkContext(conf), batchDuration) val flumeStream = ssc.flumeStream("localhost", testPort, StorageLevel.MEMORY_AND_DISK) val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]] with SynchronizedBuffer[Seq[SparkFlumeEvent]] @@ -162,11 +162,11 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { test("file input stream") { // Disable manual clock as FileInputDStream does not work with manual clock - System.clearProperty("spark.streaming.clock") + conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock") // Set up the streaming context and input streams val testDir = Files.createTempDir() - val ssc = new StreamingContext(master, framework, batchDuration) + val ssc = new StreamingContext(new SparkContext(conf), batchDuration) val fileStream = ssc.textFileStream(testDir.toString) val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] def output = outputBuffer.flatMap(x => x) @@ -207,7 +207,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { FileUtils.deleteDirectory(testDir) // Enable manual clock back again for other tests - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") + conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") } @@ -218,7 +218,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { testServer.start() // Set up the streaming context and input streams - val ssc = new StreamingContext(master, framework, batchDuration) + val ssc = new StreamingContext(new SparkContext(conf), batchDuration) val networkStream = ssc.actorStream[String](Props(new TestActor(port)), "TestActor", StorageLevel.MEMORY_AND_DISK) //Had to pass the local value of port to prevent from closing over entire scope val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] @@ -262,7 +262,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { } test("kafka input stream") { - val ssc = new StreamingContext(master, framework, batchDuration) + val ssc = new StreamingContext(new SparkContext(conf), batchDuration) val topics = Map("my-topic" -> 1) val test1 = ssc.kafkaStream("localhost:12345", "group", topics) val test2 = ssc.kafkaStream("localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK) @@ -285,7 +285,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { MultiThreadTestReceiver.haveAllThreadsFinished = false // set up the network stream using the test receiver - val ssc = new StreamingContext(master, framework, batchDuration) + val ssc = new StreamingContext(new SparkContext(conf), batchDuration) val networkStream = ssc.networkStream[Int](testReceiver) val countStream = networkStream.count val outputBuffer = new ArrayBuffer[Seq[Long]] with SynchronizedBuffer[Seq[Long]] diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala index 2f34e812a1009..d1cab0c6092c5 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -28,7 +28,7 @@ import java.io.{ObjectInputStream, IOException} import org.scalatest.{BeforeAndAfter, FunSuite} -import org.apache.spark.Logging +import org.apache.spark.{SparkContext, SparkConf, Logging} import org.apache.spark.rdd.RDD /** @@ -130,6 +130,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { // Whether to actually wait in real time before changing manual clock def actuallyWait = false + def conf = new SparkConf().setMasterUrl(master).setAppName(framework).set("spark.cleaner.ttl", "3600") /** * Set up required DStreams to test the DStream operation using the two sequences * of input collections. @@ -139,9 +140,9 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { operation: DStream[U] => DStream[V], numPartitions: Int = numInputPartitions ): StreamingContext = { - + val sc = new SparkContext(conf) // Create StreamingContext - val ssc = new StreamingContext(master, framework, batchDuration) + val ssc = new StreamingContext(sc, batchDuration) if (checkpointDir != null) { ssc.checkpoint(checkpointDir) } @@ -165,9 +166,9 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { input2: Seq[Seq[V]], operation: (DStream[U], DStream[V]) => DStream[W] ): StreamingContext = { - + val sc = new SparkContext(conf) // Create StreamingContext - val ssc = new StreamingContext(master, framework, batchDuration) + val ssc = new StreamingContext(sc, batchDuration) if (checkpointDir != null) { ssc.checkpoint(checkpointDir) } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 240ed8b32ae93..1dd38dd13edaf 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -58,13 +58,13 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e YarnConfiguration.DEFAULT_RM_AM_MAX_RETRIES) private var isLastAMRetry: Boolean = true // default to numWorkers * 2, with minimum of 3 - private val maxNumWorkerFailures = System.getProperty("spark.yarn.max.worker.failures", + private val maxNumWorkerFailures = conf.getOrElse("spark.yarn.max.worker.failures", math.max(args.numWorkers * 2, 3).toString()).toInt def run() { // Setup the directories so things go to yarn approved directories rather // then user specified and /tmp. - System.setProperty("spark.local.dir", getLocalDirs()) + conf.set("spark.local.dir", getLocalDirs()) // Use priority 30 as its higher then HDFS. Its same priority as MapReduce is using. ShutdownHookManager.get().addShutdownHook(new AppMasterShutdownHook(this), 30) @@ -165,10 +165,10 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e logInfo("Waiting for spark driver to be reachable.") var driverUp = false var tries = 0 - val numTries = System.getProperty("spark.yarn.applicationMaster.waitTries", "10").toInt + val numTries = conf.getOrElse("spark.yarn.applicationMaster.waitTries", "10").toInt while(!driverUp && tries < numTries) { - val driverHost = System.getProperty("spark.driver.host") - val driverPort = System.getProperty("spark.driver.port") + val driverHost = conf.get("spark.driver.host") + val driverPort = conf.get("spark.driver.port") try { val socket = new Socket(driverHost, driverPort.toInt) socket.close() @@ -226,7 +226,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e ApplicationMaster.sparkContextRef.synchronized { var count = 0 val waitTime = 10000L - val numTries = System.getProperty("spark.yarn.ApplicationMaster.waitTries", "10").toInt + val numTries = conf.getOrElse("spark.yarn.ApplicationMaster.waitTries", "10").toInt while (ApplicationMaster.sparkContextRef.get() == null && count < numTries) { logInfo("Waiting for spark context initialization ... " + count) count = count + 1 @@ -294,7 +294,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // we want to be reasonably responsive without causing too many requests to RM. val schedulerInterval = - System.getProperty("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong + conf.getOrElse("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong // must be <= timeoutInterval / 2. val interval = math.min(timeoutInterval / 2, schedulerInterval) @@ -377,7 +377,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e private def cleanupStagingDir() { var stagingDirPath: Path = null try { - val preserveFiles = System.getProperty("spark.yarn.preserve.staging.files", "false").toBoolean + val preserveFiles = conf.getOrElse("spark.yarn.preserve.staging.files", "false").toBoolean if (!preserveFiles) { stagingDirPath = new Path(System.getenv("SPARK_YARN_STAGING_DIR")) if (stagingDirPath == null) { diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 79dd03806523b..29892e98e3dd9 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -230,7 +230,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } } val dst = new Path(fs.getHomeDirectory(), appStagingDir) - val replication = System.getProperty("spark.yarn.submit.file.replication", "3").toShort + val replication = conf.getOrElse("spark.yarn.submit.file.replication", "3").toShort if (UserGroupInformation.isSecurityEnabled()) { val dstFs = dst.getFileSystem(conf) @@ -461,7 +461,7 @@ object Client { def main(argStrings: Array[String]) { // Set an env variable indicating we are running in YARN mode. // Note that anything with SPARK prefix gets propagated to all (remote) processes - System.setProperty("SPARK_YARN_MODE", "true") + conf.set("SPARK_YARN_MODE", "true") val args = new ClientArguments(argStrings) @@ -483,7 +483,7 @@ object Client { Path.SEPARATOR + LOG4J_PROP) } // Normally the users app.jar is last in case conflicts with spark jars - val userClasspathFirst = System.getProperty("spark.yarn.user.classpath.first", "false") + val userClasspathFirst = conf.getOrElse("spark.yarn.user.classpath.first", "false") .toBoolean if (userClasspathFirst) { Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index b3a7886d93226..617289f568382 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -33,7 +33,7 @@ class ClientArguments(val args: Array[String]) { var workerMemory = 1024 var workerCores = 1 var numWorkers = 2 - var amQueue = System.getProperty("QUEUE", "default") + var amQueue = conf.getOrElse("QUEUE", "default") var amMemory: Int = 512 var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster" var appName: String = "Spark" diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala index 69038844bbe43..c1e79cbe665a5 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala @@ -162,8 +162,8 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte Thread.sleep(100) } } - System.setProperty("spark.driver.host", driverHost) - System.setProperty("spark.driver.port", driverPort.toString) + conf.set("spark.driver.host", driverHost) + conf.set("spark.driver.port", driverPort.toString) val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 9ab20735299c0..4c9fee5695564 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -239,7 +239,7 @@ private[yarn] class YarnAllocationHandler( // (workerIdCounter) val workerId = workerIdCounter.incrementAndGet().toString val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"), + conf.get("spark.driver.host"), conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) logInfo("launching container on " + containerId + " host " + workerHostname) diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index b206780c7806e..6feaaff014256 100644 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -52,8 +52,8 @@ private[spark] class YarnClientSchedulerBackend( if (workerNumber == null) workerNumber = defaultWorkerNumber - val driverHost = System.getProperty("spark.driver.host") - val driverPort = System.getProperty("spark.driver.port") + val driverHost = conf.get("spark.driver.host") + val driverPort = conf.get("spark.driver.port") val hostport = driverHost + ":" + driverPort val argsArray = Array[String](