Skip to content

Commit

Permalink
Merge pull request apache#309 from mateiz/conf2
Browse files Browse the repository at this point in the history
SPARK-544. Migrate configuration to a SparkConf class

This is still a work in progress based on Prashant and Evan's code. So far I've done the following:

- Got rid of global SparkContext.globalConf
- Passed SparkConf to serializers and compression codecs
- Made SparkConf public instead of private[spark]
- Improved API of SparkContext and SparkConf
- Switched executor environment vars to be passed through SparkConf
- Fixed some places that were still using system properties
- Fixed some tests, though others are still failing

This still fails several tests in core, repl and streaming, likely due to properties not being set or cleared correctly (some of the tests run fine in isolation). But the API at least is hopefully ready for review. Unfortunately there was a lot of global stuff before due to a "SparkContext.globalConf" method that let you set a "default" configuration of sorts, which meant I had to make some pretty big changes.
  • Loading branch information
pwendell committed Jan 2, 2014
2 parents c1d928a + 7e8d2e8 commit 3713f81
Show file tree
Hide file tree
Showing 140 changed files with 1,731 additions and 941 deletions.
8 changes: 4 additions & 4 deletions core/src/main/scala/org/apache/spark/Accumulators.scala
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ class Accumulable[R, T] (
@transient initialValue: R,
param: AccumulableParam[R, T])
extends Serializable {

val id = Accumulators.newId
@transient private var value_ = initialValue // Current value on master
val zero = param.zero(initialValue) // Zero value to be passed to workers
Expand Down Expand Up @@ -113,7 +113,7 @@ class Accumulable[R, T] (
def setValue(newValue: R) {
this.value = newValue
}

// Called by Java when deserializing an object
private def readObject(in: ObjectInputStream) {
in.defaultReadObject()
Expand Down Expand Up @@ -177,7 +177,7 @@ class GrowableAccumulableParam[R <% Growable[T] with TraversableOnce[T] with Ser
def zero(initialValue: R): R = {
// We need to clone initialValue, but it's hard to specify that R should also be Cloneable.
// Instead we'll serialize it to a buffer and load it back.
val ser = new JavaSerializer().newInstance()
val ser = new JavaSerializer(new SparkConf(false)).newInstance()
val copy = ser.deserialize[R](ser.serialize(initialValue))
copy.clear() // In case it contained stuff
copy
Expand Down Expand Up @@ -215,7 +215,7 @@ private object Accumulators {
val originals = Map[Long, Accumulable[_, _]]()
val localAccums = Map[Thread, Map[Long, Accumulable[_, _]]]()
var lastId: Long = 0

def newId: Long = synchronized {
lastId += 1
return lastId
Expand Down
11 changes: 6 additions & 5 deletions core/src/main/scala/org/apache/spark/MapOutputTracker.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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] = _
Expand All @@ -65,7 +65,7 @@ private[spark] class MapOutputTracker extends Logging {
protected val epochLock = new java.lang.Object

private val metadataCleaner =
new MetadataCleaner(MetadataCleanerType.MAP_OUTPUT_TRACKER, this.cleanup)
new MetadataCleaner(MetadataCleanerType.MAP_OUTPUT_TRACKER, this.cleanup, conf)

// Send a message to the trackerActor and get its result within a default timeout, or
// throw a SparkException if this fails.
Expand Down Expand Up @@ -129,7 +129,7 @@ private[spark] class MapOutputTracker extends Logging {
if (fetchedStatuses == null) {
// We won the race to fetch the output locs; do so
logInfo("Doing the fetch; tracker actor = " + trackerActor)
val hostPort = Utils.localHostPort()
val hostPort = Utils.localHostPort(conf)
// This try-finally prevents hangs due to timeouts:
try {
val fetchedBytes =
Expand Down Expand Up @@ -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
Expand Down
4 changes: 2 additions & 2 deletions core/src/main/scala/org/apache/spark/Partitioner.scala
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ object Partitioner {
for (r <- bySize if r.partitioner != None) {
return r.partitioner.get
}
if (System.getProperty("spark.default.parallelism") != null) {
if (rdd.context.conf.contains("spark.default.parallelism")) {
return new HashPartitioner(rdd.context.defaultParallelism)
} else {
return new HashPartitioner(bySize.head.partitions.size)
Expand Down Expand Up @@ -90,7 +90,7 @@ class HashPartitioner(partitions: Int) extends Partitioner {
class RangePartitioner[K <% Ordered[K]: ClassTag, V](
partitions: Int,
@transient rdd: RDD[_ <: Product2[K,V]],
private val ascending: Boolean = true)
private val ascending: Boolean = true)
extends Partitioner {

// An array of upper bounds for the first (partitions - 1) partitions
Expand Down
189 changes: 189 additions & 0 deletions core/src/main/scala/org/apache/spark/SparkConf.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,189 @@
package org.apache.spark

import scala.collection.JavaConverters._
import scala.collection.mutable.HashMap

import com.typesafe.config.ConfigFactory

/**
* Configuration for a Spark application. Used to set various Spark parameters as key-value pairs.
*
* Most of the time, you would create a SparkConf object with `new SparkConf()`, which will load
* values from both the `spark.*` Java system properties and any `spark.conf` on your application's
* classpath (if it has one). In this case, system properties take priority over `spark.conf`, and
* any parameters you set directly on the `SparkConf` object take priority over both of those.
*
* For unit tests, you can also call `new SparkConf(false)` to skip loading external settings and
* get the same configuration no matter what is on the classpath.
*
* All setter methods in this class support chaining. For example, you can write
* `new SparkConf().setMaster("local").setAppName("My app")`.
*
* Note that once a SparkConf object is passed to Spark, it is cloned and can no longer be modified
* by the user. Spark does not support modifying the configuration at runtime.
*
* @param loadDefaults whether to load values from the system properties and classpath
*/
class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable {

/** Create a SparkConf that loads defaults from system properties and the classpath */
def this() = this(true)

private val settings = new HashMap[String, String]()

if (loadDefaults) {
ConfigFactory.invalidateCaches()
val typesafeConfig = ConfigFactory.systemProperties()
.withFallback(ConfigFactory.parseResources("spark.conf"))
for (e <- typesafeConfig.entrySet().asScala if e.getKey.startsWith("spark.")) {
settings(e.getKey) = e.getValue.unwrapped.toString
}
}

/** Set a configuration variable. */
def set(key: String, value: String): SparkConf = {
if (key == null) {
throw new NullPointerException("null key")
}
if (value == null) {
throw new NullPointerException("null value")
}
settings(key) = value
this
}

/**
* The master URL to connect to, such as "local" to run locally with one thread, "local[4]" to
* run locally with 4 cores, or "spark://master:7077" to run on a Spark standalone cluster.
*/
def setMaster(master: String): SparkConf = {
set("spark.master", master)
}

/** Set a name for your application. Shown in the Spark web UI. */
def setAppName(name: String): SparkConf = {
set("spark.app.name", name)
}

/** Set JAR files to distribute to the cluster. */
def setJars(jars: Seq[String]): SparkConf = {
set("spark.jars", jars.mkString(","))
}

/** Set JAR files to distribute to the cluster. (Java-friendly version.) */
def setJars(jars: Array[String]): SparkConf = {
setJars(jars.toSeq)
}

/**
* Set an environment variable to be used when launching executors for this application.
* These variables are stored as properties of the form spark.executorEnv.VAR_NAME
* (for example spark.executorEnv.PATH) but this method makes them easier to set.
*/
def setExecutorEnv(variable: String, value: String): SparkConf = {
set("spark.executorEnv." + variable, value)
}

/**
* Set multiple environment variables to be used when launching executors.
* These variables are stored as properties of the form spark.executorEnv.VAR_NAME
* (for example spark.executorEnv.PATH) but this method makes them easier to set.
*/
def setExecutorEnv(variables: Seq[(String, String)]): SparkConf = {
for ((k, v) <- variables) {
setExecutorEnv(k, v)
}
this
}

/**
* Set multiple environment variables to be used when launching executors.
* (Java-friendly version.)
*/
def setExecutorEnv(variables: Array[(String, String)]): SparkConf = {
setExecutorEnv(variables.toSeq)
}

/**
* Set the location where Spark is installed on worker nodes.
*/
def setSparkHome(home: String): SparkConf = {
set("spark.home", home)
}

/** Set multiple parameters together */
def setAll(settings: Traversable[(String, String)]) = {
this.settings ++= settings
this
}

/** Set a parameter if it isn't already configured */
def setIfMissing(key: String, value: String): SparkConf = {
if (!settings.contains(key)) {
settings(key) = value
}
this
}

/** Remove a parameter from the configuration */
def remove(key: String): SparkConf = {
settings.remove(key)
this
}

/** Get a parameter; throws a NoSuchElementException if it's not set */
def get(key: String): String = {
settings.getOrElse(key, throw new NoSuchElementException(key))
}

/** Get a parameter, falling back to a default if not set */
def get(key: String, defaultValue: String): String = {
settings.getOrElse(key, defaultValue)
}

/** Get a parameter as an Option */
def getOption(key: String): Option[String] = {
settings.get(key)
}

/** Get all parameters as a list of pairs */
def getAll: Array[(String, String)] = settings.clone().toArray

/** Get a parameter as an integer, falling back to a default if not set */
def getInt(key: String, defaultValue: Int): Int = {
getOption(key).map(_.toInt).getOrElse(defaultValue)
}

/** Get a parameter as a long, falling back to a default if not set */
def getLong(key: String, defaultValue: Long): Long = {
getOption(key).map(_.toLong).getOrElse(defaultValue)
}

/** Get a parameter as a double, falling back to a default if not set */
def getDouble(key: String, defaultValue: Double): Double = {
getOption(key).map(_.toDouble).getOrElse(defaultValue)
}

/** Get all executor environment variables set on this SparkConf */
def getExecutorEnv: Seq[(String, String)] = {
val prefix = "spark.executorEnv."
getAll.filter{case (k, v) => k.startsWith(prefix)}
.map{case (k, v) => (k.substring(prefix.length), v)}
}

/** Does the configuration contain a given parameter? */
def contains(key: String): Boolean = settings.contains(key)

/** Copy this object */
override def clone: SparkConf = {
new SparkConf(false).setAll(settings)
}

/**
* Return a string listing all keys and values, one per line. This is useful to print the
* configuration out for debugging.
*/
def toDebugString: String = {
settings.toArray.sorted.map{case (k, v) => k + "=" + v}.mkString("\n")
}
}
Loading

0 comments on commit 3713f81

Please sign in to comment.