You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by pw...@apache.org on 2014/01/02 06:29:39 UTC

[01/33] spark-544, introducing SparkConf and related configuration overhaul.

Updated Branches:
  refs/heads/master c1d928a89 -> 3713f8129


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
----------------------------------------------------------------------
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 70be15d..41ac292 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"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
----------------------------------------------------------------------
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 bc31bb2..f7d73f0 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)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
----------------------------------------------------------------------
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 c27257c..71d1cbd 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))

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
----------------------------------------------------------------------
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 b206780..6feaaff 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](

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/project/SparkBuild.scala
----------------------------------------------------------------------
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index ab96cfa..ffb54a2 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"
       )
   )
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala
----------------------------------------------------------------------
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 523fd12..b2f499e 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
   }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala
----------------------------------------------------------------------
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 e1455ef..0d412e4 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)
       }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
----------------------------------------------------------------------
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 9271914..b8e1427 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")

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala
----------------------------------------------------------------------
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 ed892e3..1d23713 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)
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
----------------------------------------------------------------------
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 d2c4fde..7674422 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_
     }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala
----------------------------------------------------------------------
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 d5ae8ae..8bf761b 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)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
----------------------------------------------------------------------
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 e81287b..315bd54 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()

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
----------------------------------------------------------------------
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 7dc82de..da8f135 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]]

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
----------------------------------------------------------------------
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 2f34e81..d1cab0c 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)
     }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
----------------------------------------------------------------------
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 240ed8b..1dd38dd 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) {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
----------------------------------------------------------------------
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 79dd038..29892e9 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.$() + 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
----------------------------------------------------------------------
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 b3a7886..617289f 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"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
----------------------------------------------------------------------
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 6903884..c1e79cb 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)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
----------------------------------------------------------------------
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 9ab2073..4c9fee5 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)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
----------------------------------------------------------------------
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 b206780..6feaaff 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](


[14/33] git commit: Add SparkConf support in Python

Posted by pw...@apache.org.
Add SparkConf support in Python


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/cd00225d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/cd00225d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/cd00225d

Branch: refs/heads/master
Commit: cd00225db9b90fc845fd1458831bdd9d014d1bb6
Parents: 1c11f54
Author: Matei Zaharia <ma...@databricks.com>
Authored: Sun Dec 29 14:03:39 2013 -0500
Committer: Matei Zaharia <ma...@databricks.com>
Committed: Sun Dec 29 14:03:39 2013 -0500

----------------------------------------------------------------------
 .../main/scala/org/apache/spark/SparkConf.scala |  14 ++-
 .../scala/org/apache/spark/SparkContext.scala   |   2 +-
 python/pyspark/__init__.py                      |   5 +-
 python/pyspark/conf.py                          | 113 +++++++++++++++++++
 python/pyspark/context.py                       |  40 +++++--
 python/pyspark/java_gateway.py                  |   1 +
 6 files changed, 158 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/cd00225d/core/src/main/scala/org/apache/spark/SparkConf.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
index bd24cd1..670c8b4 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -74,13 +74,21 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable {
     this
   }
 
-  /** Set an environment variable to be used when launching executors for this application. */
+  /**
+   * 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 = {
     settings("spark.executorEnv." + variable) = value
     this
   }
 
-  /** Set multiple environment variables to be used when launching executors. */
+  /**
+   * 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)
@@ -135,7 +143,7 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable {
   }
 
   /** Get all parameters as a list of pairs */
-  def getAll: Seq[(String, String)] = settings.clone().toSeq
+  def getAll: Array[(String, String)] = settings.clone().toArray
 
   /** Get a parameter, falling back to a default if not set */
   def getOrElse(k: String, defaultValue: String): String = {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/cd00225d/core/src/main/scala/org/apache/spark/SparkContext.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 0567f7f..c109ff9 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -78,7 +78,7 @@ class SparkContext(
    * @param conf a [[org.apache.spark.SparkConf]] object specifying other Spark parameters
    */
   def this(master: String, appName: String, conf: SparkConf) =
-    this(conf.setMaster(master).setAppName(appName))
+    this(conf.clone().setMaster(master).setAppName(appName))
 
   /**
    * Alternative constructor that allows setting common Spark properties directly

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/cd00225d/python/pyspark/__init__.py
----------------------------------------------------------------------
diff --git a/python/pyspark/__init__.py b/python/pyspark/__init__.py
index 1f35f6f..f1b95ac 100644
--- a/python/pyspark/__init__.py
+++ b/python/pyspark/__init__.py
@@ -28,6 +28,8 @@ Public classes:
         A broadcast variable that gets reused across tasks.
     - L{Accumulator<pyspark.accumulators.Accumulator>}
         An "add-only" shared variable that tasks can only add values to.
+    - L{SparkConf<pyspark.conf.SparkConf}
+        Configuration for a Spark application.
     - L{SparkFiles<pyspark.files.SparkFiles>}
         Access files shipped with jobs.
     - L{StorageLevel<pyspark.storagelevel.StorageLevel>}
@@ -38,10 +40,11 @@ import os
 sys.path.insert(0, os.path.join(os.environ["SPARK_HOME"], "python/lib/py4j0.7.egg"))
 
 
+from pyspark.conf import SparkConf
 from pyspark.context import SparkContext
 from pyspark.rdd import RDD
 from pyspark.files import SparkFiles
 from pyspark.storagelevel import StorageLevel
 
 
-__all__ = ["SparkContext", "RDD", "SparkFiles", "StorageLevel"]
+__all__ = ["SparkConf", "SparkContext", "RDD", "SparkFiles", "StorageLevel"]

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/cd00225d/python/pyspark/conf.py
----------------------------------------------------------------------
diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py
new file mode 100644
index 0000000..56e615c
--- /dev/null
+++ b/python/pyspark/conf.py
@@ -0,0 +1,113 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+"""
+>>> from pyspark.conf import SparkConf
+>>> from pyspark.context import SparkContext
+>>> conf = SparkConf()
+>>> conf.setMaster("local").setAppName("My app")
+<pyspark.conf.SparkConf object at ...>
+>>> conf.get("spark.master")
+u'local'
+>>> conf.get("spark.appName")
+u'My app'
+>>> sc = SparkContext(conf=conf)
+>>> sc.master
+u'local'
+>>> sc.appName
+u'My app'
+>>> sc.sparkHome == None
+True
+
+>>> conf = SparkConf()
+>>> conf.setSparkHome("/path")
+<pyspark.conf.SparkConf object at ...>
+>>> conf.get("spark.home")
+u'/path'
+>>> conf.setExecutorEnv("VAR1", "value1")
+<pyspark.conf.SparkConf object at ...>
+>>> conf.setExecutorEnv(pairs = [("VAR3", "value3"), ("VAR4", "value4")])
+<pyspark.conf.SparkConf object at ...>
+>>> conf.get("spark.executorEnv.VAR1")
+u'value1'
+>>> sorted(conf.getAll(), key=lambda p: p[0])
+[(u'spark.executorEnv.VAR1', u'value1'), (u'spark.executorEnv.VAR3', u'value3'), (u'spark.executorEnv.VAR4', u'value4'), (u'spark.home', u'/path')]
+"""
+
+
+class SparkConf(object):
+    def __init__(self, loadDefaults=False):
+        from pyspark.context import SparkContext
+        SparkContext._ensure_initialized()
+        self._jconf = SparkContext._jvm.SparkConf(loadDefaults)
+
+    def set(self, key, value):
+        self._jconf.set(key, value)
+        return self
+
+    def setMaster(self, value):
+        self._jconf.setMaster(value)
+        return self
+
+    def setAppName(self, value):
+        self._jconf.setAppName(value)
+        return self
+
+    def setSparkHome(self, value):
+        self._jconf.setSparkHome(value)
+        return self
+
+    def setExecutorEnv(self, key=None, value=None, pairs=None):
+        if (key != None and pairs != None) or (key == None and pairs == None):
+            raise Exception("Either pass one key-value pair or a list of pairs")
+        elif key != None:
+            self._jconf.setExecutorEnv(key, value)
+        elif pairs != None:
+            for (k, v) in pairs:
+                self._jconf.setExecutorEnv(k, v)
+        return self
+
+    def setAll(self, pairs):
+        for (k, v) in pairs:
+            self._jconf.set(k, v)
+        return self
+
+    def get(self, key):
+        return self._jconf.get(key)
+
+    def getOrElse(self, key, defaultValue):
+        return self._jconf.getOrElse(key, defaultValue)
+
+    def getAll(self):
+        pairs = []
+        for elem in self._jconf.getAll():
+            pairs.append((elem._1(), elem._2()))
+        return pairs
+
+    def contains(self, key):
+        return self._jconf.contains(key)
+
+
+def _test():
+    import doctest
+    (failure_count, test_count) = doctest.testmod(optionflags=doctest.ELLIPSIS)
+    if failure_count:
+        exit(-1)
+
+
+if __name__ == "__main__":
+    _test()

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/cd00225d/python/pyspark/context.py
----------------------------------------------------------------------
diff --git a/python/pyspark/context.py b/python/pyspark/context.py
index c0645b2..97c1526 100644
--- a/python/pyspark/context.py
+++ b/python/pyspark/context.py
@@ -24,6 +24,7 @@ from tempfile import NamedTemporaryFile
 from pyspark import accumulators
 from pyspark.accumulators import Accumulator
 from pyspark.broadcast import Broadcast
+from pyspark.conf import SparkConf
 from pyspark.files import SparkFiles
 from pyspark.java_gateway import launch_gateway
 from pyspark.serializers import PickleSerializer, BatchedSerializer, MUTF8Deserializer
@@ -49,14 +50,14 @@ class SparkContext(object):
     _python_includes = None # zip and egg files that need to be added to PYTHONPATH
 
 
-    def __init__(self, master, jobName, sparkHome=None, pyFiles=None,
-        environment=None, batchSize=1024, serializer=PickleSerializer()):
+    def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None,
+        environment=None, batchSize=1024, serializer=PickleSerializer(), conf=None):
         """
         Create a new SparkContext.
 
         @param master: Cluster URL to connect to
                (e.g. mesos://host:port, spark://host:port, local[4]).
-        @param jobName: A name for your job, to display on the cluster web UI
+        @param appName: A name for your job, to display on the cluster web UI.
         @param sparkHome: Location where Spark is installed on cluster nodes.
         @param pyFiles: Collection of .zip or .py files to send to the cluster
                and add to PYTHONPATH.  These can be paths on the local file
@@ -67,6 +68,7 @@ class SparkContext(object):
                Java object.  Set 1 to disable batching or -1 to use an
                unlimited batch size.
         @param serializer: The serializer for RDDs.
+        @param conf: A L{SparkConf} object setting Spark properties.
 
 
         >>> from pyspark.context import SparkContext
@@ -79,10 +81,7 @@ class SparkContext(object):
         """
         SparkContext._ensure_initialized(self)
 
-        self.master = master
-        self.jobName = jobName
-        self.sparkHome = sparkHome or None # None becomes null in Py4J
-        self.environment = environment or {}
+        self.conf = conf or SparkConf()
         self._batchSize = batchSize  # -1 represents an unlimited batch size
         self._unbatched_serializer = serializer
         if batchSize == 1:
@@ -91,10 +90,26 @@ class SparkContext(object):
             self.serializer = BatchedSerializer(self._unbatched_serializer,
                                                 batchSize)
 
+        # Set parameters passed directly on our conf; these operations will be no-ops
+        # if the parameters were None
+        self.conf.setMaster(master)
+        self.conf.setAppName(appName)
+        self.conf.setSparkHome(sparkHome)
+        environment = environment or {}
+        for key, value in environment.iteritems():
+            self.conf.setExecutorEnv(key, value)
+
+        if not self.conf.contains("spark.master"):
+            raise Exception("A master URL must be set in your configuration")
+        if not self.conf.contains("spark.appName"):
+            raise Exception("An application name must be set in your configuration")
+
+        self.master = self.conf.get("spark.master")
+        self.appName = self.conf.get("spark.appName")
+        self.sparkHome = self.conf.getOrElse("spark.home", None)
+
         # Create the Java SparkContext through Py4J
-        empty_string_array = self._gateway.new_array(self._jvm.String, 0)
-        self._jsc = self._jvm.JavaSparkContext(master, jobName, sparkHome,
-                                              empty_string_array)
+        self._jsc = self._jvm.JavaSparkContext(self.conf._jconf)
 
         # Create a single Accumulator in Java that we'll send all our updates through;
         # they will be passed back to us through a TCP server
@@ -105,6 +120,7 @@ class SparkContext(object):
                 self._jvm.PythonAccumulatorParam(host, port))
 
         self.pythonExec = os.environ.get("PYSPARK_PYTHON", 'python')
+
         # Broadcast's __reduce__ method stores Broadcast instances here.
         # This allows other code to determine which Broadcast instances have
         # been pickled, so it can determine which Java broadcast objects to
@@ -143,8 +159,8 @@ class SparkContext(object):
     @classmethod
     def setSystemProperty(cls, key, value):
         """
-        Set a system property, such as spark.executor.memory. This must be
-        invoked before instantiating SparkContext.
+        Set a Java system property, such as spark.executor.memory. This must
+        must be invoked before instantiating SparkContext.
         """
         SparkContext._ensure_initialized()
         SparkContext._jvm.java.lang.System.setProperty(key, value)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/cd00225d/python/pyspark/java_gateway.py
----------------------------------------------------------------------
diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py
index e615c1e..128f078 100644
--- a/python/pyspark/java_gateway.py
+++ b/python/pyspark/java_gateway.py
@@ -60,6 +60,7 @@ def launch_gateway():
     # Connect to the gateway
     gateway = JavaGateway(GatewayClient(port=port), auto_convert=False)
     # Import the classes used by PySpark
+    java_import(gateway.jvm, "org.apache.spark.SparkConf")
     java_import(gateway.jvm, "org.apache.spark.api.java.*")
     java_import(gateway.jvm, "org.apache.spark.api.python.*")
     java_import(gateway.jvm, "scala.Tuple2")


[10/33] git commit: Fix CheckpointSuite test failures

Posted by pw...@apache.org.
Fix CheckpointSuite test failures


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/a8f31638
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/a8f31638
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/a8f31638

Branch: refs/heads/master
Commit: a8f316386a429c6d73e3e3824ea6eb28b0381cb5
Parents: 578bd1f
Author: Matei Zaharia <ma...@databricks.com>
Authored: Sat Dec 28 21:26:43 2013 -0500
Committer: Matei Zaharia <ma...@databricks.com>
Committed: Sat Dec 28 21:26:43 2013 -0500

----------------------------------------------------------------------
 .../scala/org/apache/spark/streaming/CheckpointSuite.scala    | 7 +++----
 1 file changed, 3 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/a8f31638/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
----------------------------------------------------------------------
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 315bd54..2a41ec0 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
@@ -42,8 +42,6 @@ import org.apache.spark.streaming.util.ManualClock
  */
 class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
 
-  conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
-
   before {
     FileUtils.deleteDirectory(new File(checkpointDir))
   }
@@ -135,13 +133,14 @@ 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
+    // Adjust manual clock time as if it is being restarted after a delay; this is a hack because
+    // we modify the conf object, but it works for this one property
+    ssc.conf.set("spark.streaming.manualClock.jump", (batchDuration.milliseconds * 7).toString)
     ssc.start()
     advanceTimeWithRealDelay(ssc, 4)
     ssc.stop()


[03/33] git commit: spark-544, introducing SparkConf and related configuration overhaul.

Posted by pw...@apache.org.
spark-544, introducing SparkConf and related configuration overhaul.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/2573add9
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/2573add9
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/2573add9

Branch: refs/heads/master
Commit: 2573add94cf920a88f74d80d8ea94218d812704d
Parents: 0bc57c5
Author: Prashant Sharma <pr...@imaginea.com>
Authored: Tue Dec 24 18:30:31 2013 +0530
Committer: Prashant Sharma <sc...@gmail.com>
Committed: Wed Dec 25 00:09:36 2013 +0530

----------------------------------------------------------------------
 .../org/apache/spark/MapOutputTracker.scala     |   7 +-
 .../scala/org/apache/spark/Partitioner.scala    |   4 +-
 .../main/scala/org/apache/spark/SparkConf.scala |  71 ++++++++++
 .../scala/org/apache/spark/SparkContext.scala   | 140 ++++++++++---------
 .../main/scala/org/apache/spark/SparkEnv.scala  |  44 +++---
 .../org/apache/spark/api/python/PythonRDD.scala |   6 +-
 .../org/apache/spark/broadcast/Broadcast.scala  |   6 +-
 .../spark/broadcast/BroadcastFactory.scala      |   4 +-
 .../apache/spark/broadcast/HttpBroadcast.scala  |  14 +-
 .../spark/broadcast/TorrentBroadcast.scala      |   9 +-
 .../spark/deploy/ApplicationDescription.scala   |   2 +-
 .../apache/spark/deploy/SparkHadoopUtil.scala   |   3 +-
 .../org/apache/spark/deploy/client/Client.scala |  13 +-
 .../apache/spark/deploy/client/TestClient.scala |   7 +-
 .../org/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 +-
 .../org/apache/spark/deploy/worker/Worker.scala |  28 ++--
 .../spark/deploy/worker/ui/WorkerWebUI.scala    |   6 +-
 .../executor/CoarseGrainedExecutorBackend.scala |   6 +-
 .../org/apache/spark/executor/Executor.scala    |  10 +-
 .../org/apache/spark/io/CompressionCodec.scala  |  14 +-
 .../apache/spark/metrics/MetricsSystem.scala    |  10 +-
 .../spark/network/ConnectionManager.scala       |  24 ++--
 .../org/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 +
 .../org/apache/spark/scheduler/JobLogger.scala  |   2 +-
 .../spark/scheduler/SchedulableBuilder.scala    |   6 +-
 .../scheduler/cluster/ClusterScheduler.scala    |  11 +-
 .../cluster/ClusterTaskSetManager.scala         |  19 +--
 .../cluster/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 +-
 .../spark/scheduler/local/LocalScheduler.scala  |   5 +-
 .../spark/serializer/KryoSerializer.scala       |  10 +-
 .../spark/storage/BlockFetcherIterator.scala    |   4 +-
 .../org/apache/spark/storage/BlockManager.scala |  38 ++---
 .../spark/storage/BlockManagerMaster.scala      |  11 +-
 .../spark/storage/BlockManagerMasterActor.scala |  10 +-
 .../spark/storage/BlockObjectWriter.scala       |   5 +-
 .../apache/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 +-
 .../org/apache/spark/ui/env/EnvironmentUI.scala |   2 +-
 .../spark/ui/jobs/JobProgressListener.scala     |   2 +-
 .../scala/org/apache/spark/util/AkkaUtils.scala |  25 ++--
 .../org/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 ++-
 .../apache/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 +-
 .../apache/spark/util/SizeEstimatorSuite.scala  |   4 +-
 .../examples/bagel/WikipediaPageRank.scala      |  10 +-
 .../bagel/WikipediaPageRankStandalone.scala     |   8 +-
 .../streaming/examples/ActorWordCount.scala     |   3 +-
 .../apache/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 +-
 .../org/apache/spark/streaming/Checkpoint.scala |   3 +-
 .../org/apache/spark/streaming/Scheduler.scala  |   6 +-
 .../spark/streaming/StreamingContext.scala      |   2 +-
 .../streaming/dstream/NetworkInputDStream.scala |   6 +-
 .../spark/streaming/CheckpointSuite.scala       |   6 +-
 .../spark/streaming/InputStreamsSuite.scala     |  18 +--
 .../apache/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(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
index ccffcc3..4520edb 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

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/Partitioner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala
index bcec41c..04c1eed 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)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/SparkConf.scala
----------------------------------------------------------------------
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 0000000..9a4eefa
--- /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
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/SparkContext.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index a0f794e..4300b07 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(
     "<driver>",
-    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 = "<unknown>"
 
+  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)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/SparkEnv.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala
index 826f5c2..78e4ae2 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)
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
----------------------------------------------------------------------
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 ca42c76..d6eacfe 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
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala
----------------------------------------------------------------------
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 43c1829..be99d22 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
       }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala
----------------------------------------------------------------------
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 68bff75..fb161ce 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
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala
----------------------------------------------------------------------
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 47db720..cecb8c2 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)
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
----------------------------------------------------------------------
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 073a0a5..4a3801d 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)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala
----------------------------------------------------------------------
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 19d393a..dda43dc 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", "<unknown>")
+  val user = System.getProperty("user.name",  "<unknown>")
 
   override def toString: String = "ApplicationDescription(" + name + ")"
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
----------------------------------------------------------------------
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 fc1537f..1c979ac 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]

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/deploy/client/Client.scala
----------------------------------------------------------------------
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 953755e..9bbd635 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 {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
----------------------------------------------------------------------
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 5b62d3b..426cf52 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()
   }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
----------------------------------------------------------------------
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 eebd079..2c162c4 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)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala
----------------------------------------------------------------------
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 9d89b45..7ce83f9 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)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala
----------------------------------------------------------------------
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 6cc7fd2..79d95b1 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

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala
----------------------------------------------------------------------
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 7d535b0..df5bb36 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 = _

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
----------------------------------------------------------------------
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 825344b..c55b720 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()
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
----------------------------------------------------------------------
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 9ab594b..ead3566 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
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
----------------------------------------------------------------------
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 87531b6..75a6e75 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)
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
----------------------------------------------------------------------
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 40d6bdb..ec47ba1 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

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
----------------------------------------------------------------------
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 debbdd4..c8319f6 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")

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/executor/Executor.scala
----------------------------------------------------------------------
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 0f19d7a..70fc30e 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 {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
----------------------------------------------------------------------
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 570a979..8ef5019 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)
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
----------------------------------------------------------------------
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 bec0c83..ac29816 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)
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala
----------------------------------------------------------------------
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 703bc6a..3e902f8 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

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala
----------------------------------------------------------------------
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 7817151..4ca3cd3 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) => { 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/network/SenderTest.scala
----------------------------------------------------------------------
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 7775749..11c21fc 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) => { 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala
----------------------------------------------------------------------
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 b1e1576..81b3104 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)
         }
       })

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
----------------------------------------------------------------------
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 a712ef1..9fbe002 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)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/rdd/RDD.scala
----------------------------------------------------------------------
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 ea45566..f8b1a69 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
   // =======================================================================

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala
----------------------------------------------------------------------
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 6092783..3f55cd5 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", "<unknown>"),
+  def this() = this(System.getProperty("user.name",  "<unknown>"),
     String.valueOf(System.currentTimeMillis()))
 
   private val logDir =

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala
----------------------------------------------------------------------
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 356fe56..9002d33 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"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala
----------------------------------------------------------------------
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 66ab8ea..7e231ec 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,

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala
----------------------------------------------------------------------
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 bf494aa..398b0ce 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
     }


[22/33] git commit: Properly show Spark properties on web UI, and change app name property

Posted by pw...@apache.org.
Properly show Spark properties on web UI, and change app name property


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/994f080f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/994f080f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/994f080f

Branch: refs/heads/master
Commit: 994f080f8ae3372366e6004600ba791c8a372ff0
Parents: eaa8a68
Author: Matei Zaharia <ma...@databricks.com>
Authored: Sun Dec 29 22:19:33 2013 -0500
Committer: Matei Zaharia <ma...@databricks.com>
Committed: Sun Dec 29 22:19:33 2013 -0500

----------------------------------------------------------------------
 core/src/main/scala/org/apache/spark/SparkConf.scala   |  2 +-
 .../src/main/scala/org/apache/spark/SparkContext.scala |  4 ++--
 .../scala/org/apache/spark/ui/env/EnvironmentUI.scala  | 13 ++++++++-----
 .../test/scala/org/apache/spark/SparkConfSuite.scala   |  2 +-
 python/pyspark/conf.py                                 |  2 +-
 python/pyspark/context.py                              |  4 ++--
 6 files changed, 15 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/994f080f/core/src/main/scala/org/apache/spark/SparkConf.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
index 8cecaff..ae52de4 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -54,7 +54,7 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable {
   /** Set a name for your application. Shown in the Spark web UI. */
   def setAppName(name: String): SparkConf = {
     if (name != null) {
-      settings("spark.appName") = name
+      settings("spark.app.name") = name
     }
     this
   }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/994f080f/core/src/main/scala/org/apache/spark/SparkContext.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 6f54fa7..810ed18 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -112,7 +112,7 @@ class SparkContext(
   if (!conf.contains("spark.master")) {
     throw new SparkException("A master URL must be set in your configuration")
   }
-  if (!conf.contains("spark.appName")) {
+  if (!conf.contains("spark.app.name")) {
     throw new SparkException("An application must be set in your configuration")
   }
 
@@ -127,7 +127,7 @@ class SparkContext(
   }
 
   val master = conf.get("spark.master")
-  val appName = conf.get("spark.appName")
+  val appName = conf.get("spark.app.name")
 
   val isLocal = (master == "local" || master.startsWith("local["))
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/994f080f/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala
----------------------------------------------------------------------
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 91fa00a..6b4602f 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
@@ -48,12 +48,15 @@ private[spark] class EnvironmentUI(sc: SparkContext) {
     def jvmTable =
       UIUtils.listingTable(Seq("Name", "Value"), jvmRow, jvmInformation, fixedWidth = true)
 
-    val properties = System.getProperties.iterator.toSeq
-    val classPathProperty = properties.find { case (k, v) =>
-      k.contains("java.class.path")
+    val sparkProperties = sc.conf.getAll.sorted
+
+    val systemProperties = System.getProperties.iterator.toSeq
+    val classPathProperty = systemProperties.find { case (k, v) =>
+      k == "java.class.path"
     }.getOrElse(("", ""))
-    val sparkProperties = properties.filter(_._1.startsWith("spark")).sorted
-    val otherProperties = properties.diff(sparkProperties :+ classPathProperty).sorted
+    val otherProperties = systemProperties.filter { case (k, v) =>
+      k != "java.class.path" && !k.startsWith("spark.")
+    }.sorted
 
     val propertyHeaders = Seq("Name", "Value")
     def propertyRow(kv: (String, String)) = <tr><td>{kv._1}</td><td>{kv._2}</td></tr>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/994f080f/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
index aaf0b80..77c7b82 100644
--- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
@@ -46,7 +46,7 @@ class SparkConfSuite extends FunSuite with LocalSparkContext {
     conf.setExecutorEnv(Seq(("VAR2", "value2"), ("VAR3", "value3")))
 
     assert(conf.get("spark.master") === "local[3]")
-    assert(conf.get("spark.appName") === "My app")
+    assert(conf.get("spark.app.name") === "My app")
     assert(conf.get("spark.home") === "/path")
     assert(conf.get("spark.jars") === "a.jar,b.jar")
     assert(conf.get("spark.executorEnv.VAR1") === "value1")

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/994f080f/python/pyspark/conf.py
----------------------------------------------------------------------
diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py
index cf98b0e..c07dd88 100644
--- a/python/pyspark/conf.py
+++ b/python/pyspark/conf.py
@@ -23,7 +23,7 @@
 <pyspark.conf.SparkConf object at ...>
 >>> conf.get("spark.master")
 u'local'
->>> conf.get("spark.appName")
+>>> conf.get("spark.app.name")
 u'My app'
 >>> sc = SparkContext(conf=conf)
 >>> sc.master

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/994f080f/python/pyspark/context.py
----------------------------------------------------------------------
diff --git a/python/pyspark/context.py b/python/pyspark/context.py
index 8b02802..12ac029 100644
--- a/python/pyspark/context.py
+++ b/python/pyspark/context.py
@@ -104,13 +104,13 @@ class SparkContext(object):
         # Check that we have at least the required parameters
         if not self.conf.contains("spark.master"):
             raise Exception("A master URL must be set in your configuration")
-        if not self.conf.contains("spark.appName"):
+        if not self.conf.contains("spark.app.name"):
             raise Exception("An application name must be set in your configuration")
 
         # Read back our properties from the conf in case we loaded some of them from
         # the classpath or an external config file
         self.master = self.conf.get("spark.master")
-        self.appName = self.conf.get("spark.appName")
+        self.appName = self.conf.get("spark.app.name")
         self.sparkHome = self.conf.getOrElse("spark.home", None)
         for (k, v) in self.conf.getAll():
             if k.startswith("spark.executorEnv."):


[09/33] git commit: Fix test failures due to setting / clearing clock type in Streaming

Posted by pw...@apache.org.
Fix test failures due to setting / clearing clock type in Streaming


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/578bd1fc
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/578bd1fc
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/578bd1fc

Branch: refs/heads/master
Commit: 578bd1fc28513eb84002c604000250f5cff9b815
Parents: 5bbe738
Author: Matei Zaharia <ma...@databricks.com>
Authored: Sat Dec 28 21:21:06 2013 -0500
Committer: Matei Zaharia <ma...@databricks.com>
Committed: Sat Dec 28 21:21:06 2013 -0500

----------------------------------------------------------------------
 .../java/org/apache/spark/streaming/JavaAPISuite.java  |  7 ++++---
 .../apache/spark/streaming/BasicOperationsSuite.scala  | 13 ++++++++-----
 .../org/apache/spark/streaming/TestSuiteBase.scala     |  1 +
 .../apache/spark/streaming/WindowOperationsSuite.scala |  3 +--
 4 files changed, 14 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/578bd1fc/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
----------------------------------------------------------------------
diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
index daeb99f..a1db099 100644
--- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
+++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
@@ -62,13 +62,14 @@ public class JavaAPISuite implements Serializable {
 
   @Before
   public void setUp() {
-      System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock");
-      ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000));
+    System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock");
+    ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000));
     ssc.checkpoint("checkpoint");
   }
 
   @After
   public void tearDown() {
+    System.clearProperty("spark.streaming.clock");
     ssc.stop();
     ssc = null;
 
@@ -101,7 +102,7 @@ public class JavaAPISuite implements Serializable {
         Arrays.asList("hello", "world"),
         Arrays.asList("goodnight", "moon"));
 
-   List<List<Integer>> expected = Arrays.asList(
+    List<List<Integer>> expected = Arrays.asList(
         Arrays.asList(5,5),
         Arrays.asList(9,4));
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/578bd1fc/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
----------------------------------------------------------------------
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
index 259ef16..60e986c 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
@@ -23,14 +23,13 @@ import org.apache.spark.rdd.RDD
 import org.apache.spark.SparkContext._
 
 import util.ManualClock
+import org.apache.spark.{SparkContext, SparkConf}
 
 class BasicOperationsSuite extends TestSuiteBase {
 
-  override def framework() = "BasicOperationsSuite"
+  override def framework = "BasicOperationsSuite"
 
-  before {
-    System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
-  }
+  conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
 
   after {
     // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
@@ -387,7 +386,11 @@ class BasicOperationsSuite extends TestSuiteBase {
   }
 
   test("slice") {
-    val ssc = new StreamingContext("local[2]", "BasicOperationSuite", Seconds(1))
+    val conf2 = new SparkConf()
+      .setMaster("local[2]")
+      .setAppName("BasicOperationsSuite")
+      .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
+    val ssc = new StreamingContext(new SparkContext(conf2), Seconds(1))
     val input = Seq(Seq(1), Seq(2), Seq(3), Seq(4))
     val stream = new TestInputStream[Int](ssc, input, 2)
     ssc.registerInputStream(stream)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/578bd1fc/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
----------------------------------------------------------------------
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 a265284..3dd6718 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
@@ -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
 
+  // A SparkConf to use in tests. Can be modified before calling setupStreams to configure things.
   val conf = new SparkConf()
     .setMaster(master)
     .setAppName(framework)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/578bd1fc/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala
----------------------------------------------------------------------
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala
index f50e05c..3242c4c 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala
@@ -18,11 +18,10 @@
 package org.apache.spark.streaming
 
 import org.apache.spark.streaming.StreamingContext._
-import collection.mutable.ArrayBuffer
 
 class WindowOperationsSuite extends TestSuiteBase {
 
-  System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
+  conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
 
   override def framework = "WindowOperationsSuite"
 


[12/33] git commit: Fix other failing tests

Posted by pw...@apache.org.
Fix other failing tests


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/20631348
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/20631348
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/20631348

Branch: refs/heads/master
Commit: 20631348d198ba52059f278c1b415c3a80a95b81
Parents: 0900d5c
Author: Matei Zaharia <ma...@databricks.com>
Authored: Sat Dec 28 23:17:58 2013 -0500
Committer: Matei Zaharia <ma...@databricks.com>
Committed: Sat Dec 28 23:17:58 2013 -0500

----------------------------------------------------------------------
 .../org/apache/spark/io/CompressionCodec.scala     |  2 +-
 .../org/apache/spark/SharedSparkContext.scala      |  4 +++-
 .../apache/spark/metrics/MetricsSystemSuite.scala  |  4 ++--
 .../spark/serializer/KryoSerializerSuite.scala     | 17 ++---------------
 .../apache/spark/streaming/StreamingContext.scala  | 14 +++++++-------
 .../streaming/api/java/JavaStreamingContext.scala  |  9 +++++++++
 .../org/apache/spark/streaming/JavaAPISuite.java   |  9 ++++++---
 .../apache/spark/streaming/CheckpointSuite.scala   |  8 --------
 8 files changed, 30 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/20631348/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
----------------------------------------------------------------------
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 2040268..075a18b 100644
--- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
+++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
@@ -46,7 +46,7 @@ private[spark] object CompressionCodec {
   def createCodec(conf: SparkConf, codecName: String): CompressionCodec = {
     val ctor = Class.forName(codecName, true, Thread.currentThread.getContextClassLoader)
       .getConstructor(classOf[SparkConf])
-      ctor.newInstance(conf).asInstanceOf[CompressionCodec]
+    ctor.newInstance(conf).asInstanceOf[CompressionCodec]
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/20631348/core/src/test/scala/org/apache/spark/SharedSparkContext.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/SharedSparkContext.scala b/core/src/test/scala/org/apache/spark/SharedSparkContext.scala
index 288aa14..c650ef4 100644
--- a/core/src/test/scala/org/apache/spark/SharedSparkContext.scala
+++ b/core/src/test/scala/org/apache/spark/SharedSparkContext.scala
@@ -27,8 +27,10 @@ trait SharedSparkContext extends BeforeAndAfterAll { self: Suite =>
 
   def sc: SparkContext = _sc
 
+  var conf = new SparkConf(false)
+
   override def beforeAll() {
-    _sc = new SparkContext("local", "test")
+    _sc = new SparkContext("local", "test", conf)
     super.beforeAll()
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/20631348/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala
----------------------------------------------------------------------
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 4ecdde0..71a2c6c 100644
--- a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala
+++ b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala
@@ -24,10 +24,10 @@ 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
+    conf = new SparkConf(false).set("spark.metrics.conf", filePath)
   }
 
   test("MetricsSystem with default config") {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/20631348/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
index 33b0148..d23e014 100644
--- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
@@ -26,7 +26,8 @@ import org.apache.spark.{SparkConf, SharedSparkContext}
 import org.apache.spark.serializer.KryoTest._
 
 class KryoSerializerSuite extends FunSuite with SharedSparkContext {
-  val conf = new SparkConf(false)
+  conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
+  conf.set("spark.kryo.registrator", classOf[MyRegistrator].getName)
 
   test("basic types") {
     val ser = new KryoSerializer(conf).newInstance()
@@ -127,8 +128,6 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext {
   }
 
   test("custom registrator") {
-    System.setProperty("spark.kryo.registrator", classOf[MyRegistrator].getName)
-
     val ser = new KryoSerializer(conf).newInstance()
     def check[T](t: T) {
       assert(ser.deserialize[T](ser.serialize(t)) === t)
@@ -188,18 +187,6 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext {
         .fold(new ClassWithoutNoArgConstructor(10))((t1, t2) => new ClassWithoutNoArgConstructor(t1.x + t2.x)).x
     assert(10 + control.sum === result)
   }
-
-  override def beforeAll() {
-    System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
-    System.setProperty("spark.kryo.registrator", classOf[MyRegistrator].getName)
-    super.beforeAll()
-  }
-
-  override def afterAll() {
-    super.afterAll()
-    System.clearProperty("spark.kryo.registrator")
-    System.clearProperty("spark.serializer")
-  }
 }
 
 object KryoTest {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/20631348/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
----------------------------------------------------------------------
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 9d2033f..286ec28 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
@@ -63,8 +63,8 @@ class StreamingContext private (
 
   /**
    * Create a StreamingContext using an existing SparkContext.
-   * @param sparkContext Existing SparkContext
-   * @param batchDuration The time interval at which streaming data will be divided into batches
+   * @param sparkContext existing SparkContext
+   * @param batchDuration the time interval at which streaming data will be divided into batches
    */
   def this(sparkContext: SparkContext, batchDuration: Duration) = {
     this(sparkContext, null, batchDuration)
@@ -72,8 +72,8 @@ class StreamingContext private (
 
   /**
    * Create a StreamingContext by providing the configuration necessary for a new SparkContext.
-   * @param conf A standard Spark application configuration
-   * @param batchDuration The time interval at which streaming data will be divided into batches
+   * @param conf a [[org.apache.spark.SparkConf]] object specifying Spark parameters
+   * @param batchDuration the time interval at which streaming data will be divided into batches
    */
   def this(conf: SparkConf, batchDuration: Duration) = {
     this(StreamingContext.createNewSparkContext(conf), null, batchDuration)
@@ -81,9 +81,9 @@ class StreamingContext private (
 
   /**
    * Create a StreamingContext by providing the details necessary for creating a new SparkContext.
-   * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
-   * @param appName A name for your job, to display on the cluster web UI
-   * @param batchDuration The time interval at which streaming data will be divided into batches
+   * @param master cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
+   * @param appName a name for your job, to display on the cluster web UI
+   * @param batchDuration the time interval at which streaming data will be divided into batches
    */
   def this(
       master: String,

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/20631348/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
index 80dcf87..5842a7c 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
@@ -39,6 +39,7 @@ import org.apache.spark.api.java.function.{Function => JFunction, Function2 => J
 import org.apache.spark.api.java.{JavaPairRDD, JavaSparkContext, JavaRDD}
 import org.apache.spark.streaming._
 import org.apache.spark.streaming.dstream._
+import org.apache.spark.SparkConf
 
 /**
  * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic
@@ -123,6 +124,14 @@ class JavaStreamingContext(val ssc: StreamingContext) {
     this(new StreamingContext(sparkContext.sc, batchDuration))
 
   /**
+   * Creates a StreamingContext using an existing SparkContext.
+   * @param conf A Spark application configuration
+   * @param batchDuration The time interval at which streaming data will be divided into batches
+   */
+  def this(conf: SparkConf, batchDuration: Duration) =
+    this(new StreamingContext(conf, batchDuration))
+
+  /**
    * Re-creates a StreamingContext from a checkpoint file.
    * @param path Path either to the directory that was specified as the checkpoint directory, or
    *             to the checkpoint file 'graph' or 'graph.bk'.

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/20631348/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
----------------------------------------------------------------------
diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
index a1db099..d53d433 100644
--- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
+++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
@@ -25,6 +25,7 @@ import com.google.common.io.Files;
 import kafka.serializer.StringDecoder;
 
 import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.spark.SparkConf;
 import org.apache.spark.streaming.api.java.JavaDStreamLike;
 import org.junit.After;
 import org.junit.Assert;
@@ -62,14 +63,16 @@ public class JavaAPISuite implements Serializable {
 
   @Before
   public void setUp() {
-    System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock");
-    ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000));
+    SparkConf conf = new SparkConf()
+        .setMaster("local[2]")
+        .setAppName("test")
+        .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock");
+    ssc = new JavaStreamingContext(conf, new Duration(1000));
     ssc.checkpoint("checkpoint");
   }
 
   @After
   public void tearDown() {
-    System.clearProperty("spark.streaming.clock");
     ssc.stop();
     ssc = null;
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/20631348/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
----------------------------------------------------------------------
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 2a41ec0..ca230fd 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
@@ -201,10 +201,6 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
   // It also tests whether batches, whose processing was incomplete due to the
   // failure, are re-processed or not.
   test("recovery with file input stream") {
-    // Disable manual clock as FileInputDStream does not work with manual clock
-    val clockProperty = System.getProperty("spark.streaming.clock")
-    System.clearProperty("spark.streaming.clock")
-
     // Set up the streaming context and input streams
     val testDir = Files.createTempDir()
     var ssc = new StreamingContext(master, framework, Seconds(1))
@@ -301,10 +297,6 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
     )
     // To ensure that all the inputs were received correctly
     assert(expectedOutput.last === output.last)
-
-    // Enable manual clock back again for other tests
-    if (clockProperty != null)
-      System.setProperty("spark.streaming.clock", clockProperty)
   }
 
 


[08/33] git commit: Fix Executor not getting properties in local mode

Posted by pw...@apache.org.
Fix Executor not getting properties in local mode


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/5bbe7386
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/5bbe7386
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/5bbe7386

Branch: refs/heads/master
Commit: 5bbe73864eea78b76448ce42a7af847dad73b269
Parents: a16c52e
Author: Matei Zaharia <ma...@databricks.com>
Authored: Sat Dec 28 17:31:58 2013 -0500
Committer: Matei Zaharia <ma...@databricks.com>
Committed: Sat Dec 28 17:31:58 2013 -0500

----------------------------------------------------------------------
 core/src/main/scala/org/apache/spark/SparkConf.scala              | 2 +-
 .../scala/org/apache/spark/scheduler/local/LocalScheduler.scala   | 3 ++-
 2 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/5bbe7386/core/src/main/scala/org/apache/spark/SparkConf.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
index 185ddb1..bd24cd1 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -28,7 +28,7 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable {
   if (loadDefaults) {
     val typesafeConfig = ConfigFactory.systemProperties()
       .withFallback(ConfigFactory.parseResources("spark.conf"))
-    for (e <- typesafeConfig.entrySet().asScala) {
+    for (e <- typesafeConfig.entrySet().asScala if e.getKey.startsWith("spark.")) {
       settings(e.getKey) = e.getValue.unwrapped.toString
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/5bbe7386/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala
----------------------------------------------------------------------
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 8498cff..7c173e3 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
@@ -50,7 +50,8 @@ private[spark]
 class LocalActor(localScheduler: LocalScheduler, private var freeCores: Int)
   extends Actor with Logging {
 
-  val executor = new Executor("localhost", "localhost", Seq.empty, isLocal = true)
+  val executor = new Executor(
+    "localhost", "localhost", localScheduler.sc.conf.getAll, isLocal = true)
 
   def receive = {
     case LocalReviveOffers =>


[19/33] git commit: Fix a change that was lost during merge

Posted by pw...@apache.org.
Fix a change that was lost during merge


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/1ee7f5ae
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/1ee7f5ae
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/1ee7f5ae

Branch: refs/heads/master
Commit: 1ee7f5aee451eb5686f1c5ef251d1ebdddb0e847
Parents: 0bd1900
Author: Matei Zaharia <ma...@databricks.com>
Authored: Sun Dec 29 18:15:46 2013 -0500
Committer: Matei Zaharia <ma...@databricks.com>
Committed: Sun Dec 29 18:15:46 2013 -0500

----------------------------------------------------------------------
 .../scala/org/apache/spark/scheduler/local/LocalBackend.scala     | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1ee7f5ae/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala
index 4edc6a0..897d47a 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala
@@ -47,7 +47,8 @@ private[spark] class LocalActor(
   private val localExecutorId = "localhost"
   private val localExecutorHostname = "localhost"
 
-  val executor = new Executor(localExecutorId, localExecutorHostname, Seq.empty, isLocal = true)
+  val executor = new Executor(
+    localExecutorId, localExecutorHostname, scheduler.conf.getAll, isLocal = true)
 
   def receive = {
     case ReviveOffers =>


[23/33] Updated docs for SparkConf and handled review comments

Posted by pw...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0fa58097/docs/job-scheduling.md
----------------------------------------------------------------------
diff --git a/docs/job-scheduling.md b/docs/job-scheduling.md
index dbcb9ae..5951155 100644
--- a/docs/job-scheduling.md
+++ b/docs/job-scheduling.md
@@ -32,12 +32,12 @@ Resource allocation can be configured as follows, based on the cluster type:
 
 * **Standalone mode:** By default, applications submitted to the standalone mode cluster will run in
   FIFO (first-in-first-out) order, and each application will try to use all available nodes. You can limit
-  the number of nodes an application uses by setting the `spark.cores.max` system property in it. This
+  the number of nodes an application uses by setting the `spark.cores.max` configuration property in it. This
   will allow multiple users/applications to run concurrently. For example, you might launch a long-running
   server that uses 10 cores, and allow users to launch shells that use 20 cores each.
   Finally, in addition to controlling cores, each application's `spark.executor.memory` setting controls
   its memory use.
-* **Mesos:** To use static partitioning on Mesos, set the `spark.mesos.coarse` system property to `true`,
+* **Mesos:** To use static partitioning on Mesos, set the `spark.mesos.coarse` configuration property to `true`,
   and optionally set `spark.cores.max` to limit each application's resource share as in the standalone mode.
   You should also set `spark.executor.memory` to control the executor memory.
 * **YARN:** The `--num-workers` option to the Spark YARN client controls how many workers it will allocate
@@ -78,11 +78,13 @@ of cluster resources. This means that short jobs submitted while a long job is r
 resources right away and still get good response times, without waiting for the long job to finish. This
 mode is best for multi-user settings.
 
-To enable the fair scheduler, simply set the `spark.scheduler.mode` to `FAIR` before creating
+To enable the fair scheduler, simply set the `spark.scheduler.mode` property to `FAIR` when configuring
 a SparkContext:
 
 {% highlight scala %}
-System.setProperty("spark.scheduler.mode", "FAIR")
+val conf = new SparkConf().setMaster(...).setAppName(...)
+conf.set("spark.scheduler.mode", "FAIR")
+val sc = new SparkContext(conf)
 {% endhighlight %}
 
 ## Fair Scheduler Pools
@@ -98,8 +100,8 @@ adding the `spark.scheduler.pool` "local property" to the SparkContext in the th
 This is done as follows:
 
 {% highlight scala %}
-// Assuming context is your SparkContext variable
-context.setLocalProperty("spark.scheduler.pool", "pool1")
+// Assuming sc is your SparkContext variable
+sc.setLocalProperty("spark.scheduler.pool", "pool1")
 {% endhighlight %}
 
 After setting this local property, _all_ jobs submitted within this thread (by calls in this thread
@@ -108,7 +110,7 @@ it easy to have a thread run multiple jobs on behalf of the same user. If you'd
 pool that a thread is associated with, simply call:
 
 {% highlight scala %}
-context.setLocalProperty("spark.scheduler.pool", null)
+sc.setLocalProperty("spark.scheduler.pool", null)
 {% endhighlight %}
 
 ## Default Behavior of Pools
@@ -138,10 +140,11 @@ properties:
   of the cluster. By default, each pool's `minShare` is 0.
 
 The pool properties can be set by creating an XML file, similar to `conf/fairscheduler.xml.template`,
-and setting the `spark.scheduler.allocation.file` property:
+and setting a `spark.scheduler.allocation.file` property in your
+[SparkConf](configuration.html#spark-properties).
 
 {% highlight scala %}
-System.setProperty("spark.scheduler.allocation.file", "/path/to/file")
+conf.set("spark.scheduler.allocation.file", "/path/to/file")
 {% endhighlight %}
 
 The format of the XML file is simply a `<pool>` element for each pool, with different elements

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0fa58097/docs/monitoring.md
----------------------------------------------------------------------
diff --git a/docs/monitoring.md b/docs/monitoring.md
index 5ed0474..0d5eb70 100644
--- a/docs/monitoring.md
+++ b/docs/monitoring.md
@@ -32,7 +32,8 @@ Spark has a configurable metrics system based on the
 This allows users to report Spark metrics to a variety of sinks including HTTP, JMX, and CSV 
 files. The metrics system is configured via a configuration file that Spark expects to be present 
 at `$SPARK_HOME/conf/metrics.conf`. A custom file location can be specified via the 
-`spark.metrics.conf` Java system property. Spark's metrics are decoupled into different 
+`spark.metrics.conf` [configuration property](configuration.html#spark-properties).
+Spark's metrics are decoupled into different 
 _instances_ corresponding to Spark components. Within each instance, you can configure a 
 set of sinks to which metrics are reported. The following instances are currently supported:
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0fa58097/docs/python-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md
index 55e39b1..96f93e2 100644
--- a/docs/python-programming-guide.md
+++ b/docs/python-programming-guide.md
@@ -131,15 +131,16 @@ sc = SparkContext("local", "App Name", pyFiles=['MyFile.py', 'lib.zip', 'app.egg
 Files listed here will be added to the `PYTHONPATH` and shipped to remote worker machines.
 Code dependencies can be added to an existing SparkContext using its `addPyFile()` method.
 
-You can set [system properties](configuration.html#system-properties)
-using `SparkContext.setSystemProperty()` class method *before*
-instantiating SparkContext. For example, to set the amount of memory
-per executor process:
+You can set [configuration properties](configuration.html#spark-properties) by passing a
+[SparkConf](api/pyspark/pyspark.conf.SparkConf-class.html) object to SparkContext:
 
 {% highlight python %}
-from pyspark import SparkContext
-SparkContext.setSystemProperty('spark.executor.memory', '2g')
-sc = SparkContext("local", "App Name")
+from pyspark import SparkConf, SparkContext
+conf = (SparkConf()
+         .setMaster("local")
+         .setAppName("My app")
+         .set("spark.executor.memory", "1g"))
+sc = SparkContext(conf = conf)
 {% endhighlight %}
 
 # API Docs

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0fa58097/docs/quick-start.md
----------------------------------------------------------------------
diff --git a/docs/quick-start.md b/docs/quick-start.md
index 8f782db..bdf127d 100644
--- a/docs/quick-start.md
+++ b/docs/quick-start.md
@@ -124,7 +124,7 @@ object SimpleApp {
 }
 {% endhighlight %}
 
-This program simply counts the number of lines containing 'a' and the number containing 'b' in the Spark README. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. Unlike the earlier examples with the Spark shell, which initializes its own SparkContext, we initialize a SparkContext as part of the proogram. We pass the SparkContext constructor four arguments, the type of scheduler we want to use (in this case, a local scheduler), a name for the application, the directory where Spark is installed, and a name for the jar file containing the application's code. The final two arguments are needed in a distributed setting, where Spark is running across several nodes, so we include them for completeness. Spark will automatically ship the jar files you list to slave nodes.
+This program just counts the number of lines containing 'a' and the number containing 'b' in the Spark README. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. Unlike the earlier examples with the Spark shell, which initializes its own SparkContext, we initialize a SparkContext as part of the proogram. We pass the SparkContext constructor four arguments, the type of scheduler we want to use (in this case, a local scheduler), a name for the application, the directory where Spark is installed, and a name for the jar file containing the application's code. The final two arguments are needed in a distributed setting, where Spark is running across several nodes, so we include them for completeness. Spark will automatically ship the jar files you list to slave nodes.
 
 This file depends on the Spark API, so we'll also include an sbt configuration file, `simple.sbt` which explains that Spark is a dependency. This file also adds a repository that Spark depends on:
 
@@ -193,7 +193,7 @@ public class SimpleApp {
 }
 {% endhighlight %}
 
-This program simply counts the number of lines containing 'a' and the number containing 'b' in a system log file. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. As with the Scala example, we initialize a SparkContext, though we use the special `JavaSparkContext` class to get a Java-friendly one. We also create RDDs (represented by `JavaRDD`) and run transformations on them. Finally, we pass functions to Spark by creating classes that extend `spark.api.java.function.Function`. The [Java programming guide](java-programming-guide.html) describes these differences in more detail.
+This program just counts the number of lines containing 'a' and the number containing 'b' in a text file. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. As with the Scala example, we initialize a SparkContext, though we use the special `JavaSparkContext` class to get a Java-friendly one. We also create RDDs (represented by `JavaRDD`) and run transformations on them. Finally, we pass functions to Spark by creating classes that extend `spark.api.java.function.Function`. The [Java programming guide](java-programming-guide.html) describes these differences in more detail.
 
 To build the program, we also write a Maven `pom.xml` file that lists Spark as a dependency. Note that Spark artifacts are tagged with a Scala version.
 
@@ -224,11 +224,11 @@ To build the program, we also write a Maven `pom.xml` file that lists Spark as a
 If you also wish to read data from Hadoop's HDFS, you will also need to add a dependency on `hadoop-client` for your version of HDFS:
 
 {% highlight xml %}
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-client</artifactId>
-      <version>...</version>
-    </dependency>
+<dependency>
+  <groupId>org.apache.hadoop</groupId>
+  <artifactId>hadoop-client</artifactId>
+  <version>...</version>
+</dependency>
 {% endhighlight %}
 
 We lay out these files according to the canonical Maven directory structure:
@@ -270,7 +270,7 @@ print "Lines with a: %i, lines with b: %i" % (numAs, numBs)
 {% endhighlight %}
 
 
-This program simply counts the number of lines containing 'a' and the number containing 'b' in a system log file.
+This program just counts the number of lines containing 'a' and the number containing 'b' in a text file.
 Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. 
 As with the Scala and Java examples, we use a SparkContext to create RDDs.
 We can pass Python functions to Spark, which are automatically serialized along with any variables that they reference.
@@ -301,14 +301,38 @@ assembly jar (or "uber" jar) containing your code and its dependencies. Both
 have assembly plugins. When creating assembly jars, list Spark 
 itself as a `provided` dependency; it need not be bundled since it is 
 already present on the slaves. Once you have an assembled jar, 
-add it to the SparkContext as shown here. It is also possible to submit 
-your dependent jars one-by-one when creating a SparkContext.
+add it to the SparkContext as shown here. It is also possible to add
+your dependent jars one-by-one using the `addJar` method of `SparkContext`.
+
+For Python, you can use the `pyFiles` argument of SparkContext
+or its `addPyFile` method to add `.py`, `.zip` or `.egg` files to be distributed.
 
 ### Setting Configuration Options
-Spark includes several configuration options which influence the behavior
-of your application. These should be set as 
-[JVM system properties](configuration.html#system-properties) in your 
-program. The options will be captured and shipped to all slave nodes.
+Spark includes several [configuration options](configuration.html#spark-properties)
+that influence the behavior of your application.
+These should be set by building a [SparkConf](api/core/index.html#org.apache.spark.SparkConf)
+object and passing it to the SparkContext constructor.
+For example, in Java and Scala, you can do:
+
+{% highlight scala %}
+import org.apache.spark.{SparkConf, SparkContext}
+val conf = new SparkConf()
+             .setMaster("local")
+             .setAppName("My application")
+             .set("spark.executor.memory", "1g")
+val sc = new SparkContext(conf)
+{% endhighlight %}
+
+Or in Python:
+
+{% highlight scala %}
+from pyspark import SparkConf, SparkContext
+conf = SparkConf()
+conf.setMaster("local")
+conf.setAppName("My application")
+conf.set("spark.executor.memory", "1g"))
+sc = SparkContext(conf = conf)
+{% endhighlight %}
 
 ### Accessing Hadoop Filesystems
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0fa58097/docs/running-on-mesos.md
----------------------------------------------------------------------
diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md
index 322ff58..68259f0 100644
--- a/docs/running-on-mesos.md
+++ b/docs/running-on-mesos.md
@@ -15,15 +15,16 @@ Spark can run on clusters managed by [Apache Mesos](http://mesos.apache.org/). F
    * `export MESOS_NATIVE_LIBRARY=<path to libmesos.so>`. This path is usually `<prefix>/lib/libmesos.so` (where the prefix is `/usr/local` by default, see above). Also, on Mac OS X, the library is called `libmesos.dylib` instead of `libmesos.so`.
    * `export SPARK_EXECUTOR_URI=<path to spark-{{site.SPARK_VERSION}}.tar.gz uploaded above>`.
    * `export MASTER=mesos://HOST:PORT` where HOST:PORT is the host and port (default: 5050) of your Mesos master (or `zk://...` if using Mesos with ZooKeeper).
-8. To run a Spark application against the cluster, when you create your `SparkContext`, pass the string `mesos://HOST:PORT` as the first parameter. In addition, you'll need to set the `spark.executor.uri` property. For example:
+8. To run a Spark application against the cluster, when you create your `SparkContext`, pass the string `mesos://HOST:PORT` as the master URL. In addition, you'll need to set the `spark.executor.uri` property. For example:
 
 {% highlight scala %}
-System.setProperty("spark.executor.uri", "<path to spark-{{site.SPARK_VERSION}}.tar.gz uploaded above>")
-val sc = new SparkContext("mesos://HOST:5050", "App Name", ...)
+val conf = new SparkConf()
+  .setMaster("mesos://HOST:5050")
+  .setAppName("My app")
+  .set("spark.executor.uri", "<path to spark-{{site.SPARK_VERSION}}.tar.gz uploaded above>")
+val sc = new SparkContext(conf)
 {% endhighlight %}
 
-If you want to run Spark on Amazon EC2, you can use the Spark [EC2 launch scripts](ec2-scripts.html), which provide an easy way to launch a cluster with Mesos, Spark, and HDFS pre-configured. This will get you a cluster in about five minutes without any configuration on your part.
-
 # Mesos Run Modes
 
 Spark can run over Mesos in two modes: "fine-grained" and "coarse-grained". In fine-grained mode, which is the default,
@@ -34,17 +35,15 @@ launch only *one* long-running Spark task on each Mesos machine, and dynamically
 it. The benefit is much lower startup overhead, but at the cost of reserving the Mesos resources for the complete duration
 of the application.
 
-To run in coarse-grained mode, set the `spark.mesos.coarse` system property to true *before* creating your SparkContext:
+To run in coarse-grained mode, set the `spark.mesos.coarse` property in your [SparkConf](configuration.html#spark-properties):
 
 {% highlight scala %}
-System.setProperty("spark.mesos.coarse", "true")
-val sc = new SparkContext("mesos://HOST:5050", "App Name", ...)
+conf.set("spark.mesos.coarse", "true")
 {% endhighlight %}
 
 In addition, for coarse-grained mode, you can control the maximum number of resources Spark will acquire. By default,
 it will acquire *all* cores in the cluster (that get offered by Mesos), which only makes sense if you run just one
-application at a time. You can cap the maximum number of cores using `System.setProperty("spark.cores.max", "10")` (for example).
-Again, this must be done *before* initializing a SparkContext.
+application at a time. You can cap the maximum number of cores using `conf.set("spark.cores.max", "10")` (for example).
 
 
 # Running Alongside Hadoop

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0fa58097/docs/scala-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md
index 56d2a3a..1db255c 100644
--- a/docs/scala-programming-guide.md
+++ b/docs/scala-programming-guide.md
@@ -49,6 +49,9 @@ This is done through the following constructor:
 new SparkContext(master, appName, [sparkHome], [jars])
 {% endhighlight %}
 
+or through `new SparkContext(conf)`, which takes a [SparkConf](api/core/index.html#org.apache.spark.SparkConf)
+object for more advanced configuration.
+
 The `master` parameter is a string specifying a [Spark or Mesos cluster URL](#master-urls) to connect to, or a special "local" string to run in local mode, as described below. `appName` is a name for your application, which will be shown in the cluster web UI. Finally, the last two parameters are needed to deploy your code to a cluster if running in distributed mode, as described later.
 
 In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the context connects to using the `MASTER` environment variable, and you can add JARs to the classpath with the `ADD_JARS` variable. For example, to run `spark-shell` on four cores, use
@@ -94,7 +97,6 @@ If you want to run your application on a cluster, you will need to specify the t
 
 If you run `spark-shell` on a cluster, you can add JARs to it by specifying the `ADD_JARS` environment variable before you launch it.  This variable should contain a comma-separated list of JARs. For example, `ADD_JARS=a.jar,b.jar ./spark-shell` will launch a shell with `a.jar` and `b.jar` on its classpath. In addition, any new classes you define in the shell will automatically be distributed.
 
-
 # Resilient Distributed Datasets (RDDs)
 
 Spark revolves around the concept of a _resilient distributed dataset_ (RDD), which is a fault-tolerant collection of elements that can be operated on in parallel. There are currently two types of RDDs: *parallelized collections*, which take an existing Scala collection and run functions on it in parallel, and *Hadoop datasets*, which run functions on each record of a file in Hadoop distributed file system or any other storage system supported by Hadoop. Both types of RDDs can be operated on through the same methods.

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0fa58097/docs/spark-standalone.md
----------------------------------------------------------------------
diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md
index b822265..f7f0b78 100644
--- a/docs/spark-standalone.md
+++ b/docs/spark-standalone.md
@@ -154,11 +154,18 @@ You can also pass an option `-c <numCores>` to control the number of cores that
 
 The standalone cluster mode currently only supports a simple FIFO scheduler across applications.
 However, to allow multiple concurrent users, you can control the maximum number of resources each
-application will acquire.
+application will use.
 By default, it will acquire *all* cores in the cluster, which only makes sense if you just run one
-application at a time. You can cap the number of cores using
-`System.setProperty("spark.cores.max", "10")` (for example).
-This value must be set *before* initializing your SparkContext.
+application at a time. You can cap the number of cores by setting `spark.cores.max` in your
+[SparkConf](configuration.html#spark-properties). For example:
+
+{% highlight scala %}
+val conf = new SparkConf()
+             .setMaster(...)
+             .setAppName(...)
+             .set("spark.cores.max", "10")
+val sc = new SparkContext(conf)
+{% endhighlight %}
 
 
 # Monitoring and Logging

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0fa58097/docs/streaming-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md
index 82f42e0..6f97db6 100644
--- a/docs/streaming-programming-guide.md
+++ b/docs/streaming-programming-guide.md
@@ -326,7 +326,7 @@ Getting the best performance of a Spark Streaming application on a cluster requi
 There are a number of optimizations that can be done in Spark to minimize the processing time of each batch. These have been discussed in detail in [Tuning Guide](tuning.html). This section highlights some of the most important ones.
 
 ### Level of Parallelism
-Cluster resources maybe under-utilized if the number of parallel tasks used in any stage of the computation is not high enough. For example, for distributed reduce operations like `reduceByKey` and `reduceByKeyAndWindow`, the default number of parallel tasks is 8. You can pass the level of parallelism as an argument (see the [`PairDStreamFunctions`](api/streaming/index.html#org.apache.spark.PairDStreamFunctions) documentation), or set the system property `spark.default.parallelism` to change the default.
+Cluster resources maybe under-utilized if the number of parallel tasks used in any stage of the computation is not high enough. For example, for distributed reduce operations like `reduceByKey` and `reduceByKeyAndWindow`, the default number of parallel tasks is 8. You can pass the level of parallelism as an argument (see the [`PairDStreamFunctions`](api/streaming/index.html#org.apache.spark.PairDStreamFunctions) documentation), or set the [config property](configuration.html#spark-properties) `spark.default.parallelism` to change the default.
 
 ### Data Serialization
 The overhead of data serialization can be significant, especially when sub-second batch sizes are to be achieved. There are two aspects to it.
@@ -349,7 +349,7 @@ For a Spark Streaming application running on a cluster to be stable, the process
 A good approach to figure out the right batch size for your application is to test it with a conservative batch size (say, 5-10 seconds) and a low data rate. To verify whether the system is able to keep up with data rate, you can check the value of the end-to-end delay experienced by each processed batch (in the Spark master logs, find the line having the phrase "Total delay"). If the delay is maintained to be less than the batch size, then system is stable. Otherwise, if the delay is continuously increasing, it means that the system is unable to keep up and it therefore unstable. Once you have an idea of a stable configuration, you can try increasing the data rate and/or reducing the batch size. Note that momentary increase in the delay due to temporary data rate increases maybe fine as long as the delay reduces back to a low value (i.e., less than batch size).
 
 ## 24/7 Operation
-By default, Spark does not forget any of the metadata (RDDs generated, stages processed, etc.). But for a Spark Streaming application to operate 24/7, it is necessary for Spark to do periodic cleanup of it metadata. This can be enabled by setting the Java system property `spark.cleaner.ttl` to the number of seconds you want any metadata to persist. For example, setting `spark.cleaner.ttl` to 600 would cause Spark periodically cleanup all metadata and persisted RDDs that are older than 10 minutes. Note, that this property needs to be set before the SparkContext is created.
+By default, Spark does not forget any of the metadata (RDDs generated, stages processed, etc.). But for a Spark Streaming application to operate 24/7, it is necessary for Spark to do periodic cleanup of it metadata. This can be enabled by setting the [config property](configuration.html#spark-properties) `spark.cleaner.ttl` to the number of seconds you want any metadata to persist. For example, setting `spark.cleaner.ttl` to 600 would cause Spark periodically cleanup all metadata and persisted RDDs that are older than 10 minutes. Note, that this property needs to be set before the SparkContext is created.
 
 This value is closely tied with any window operation that is being used. Any window operation would require the input data to be persisted in memory for at least the duration of the window. Hence it is necessary to set the delay to at least the value of the largest window operation used in the Spark Streaming application. If this delay is set too low, the application will throw an exception saying so.
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0fa58097/docs/tuning.md
----------------------------------------------------------------------
diff --git a/docs/tuning.md b/docs/tuning.md
index a4be188..bbb8700 100644
--- a/docs/tuning.md
+++ b/docs/tuning.md
@@ -38,14 +38,15 @@ in your operations) and performance. It provides two serialization libraries:
   `Serializable` types and requires you to *register* the classes you'll use in the program in advance
   for best performance.
 
-You can switch to using Kryo by calling `System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer")`
-*before* creating your SparkContext. This setting configures the serializer used for not only shuffling data between worker
+You can switch to using Kryo by initializing your job with a [SparkConf](configuration.html#spark-properties)
+and calling `conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")`.
+This setting configures the serializer used for not only shuffling data between worker
 nodes but also when serializing RDDs to disk.  The only reason Kryo is not the default is because of the custom
 registration requirement, but we recommend trying it in any network-intensive application.
 
 Finally, to register your classes with Kryo, create a public class that extends
 [`org.apache.spark.serializer.KryoRegistrator`](api/core/index.html#org.apache.spark.serializer.KryoRegistrator) and set the
-`spark.kryo.registrator` system property to point to it, as follows:
+`spark.kryo.registrator` config property to point to it, as follows:
 
 {% highlight scala %}
 import com.esotericsoftware.kryo.Kryo
@@ -58,17 +59,17 @@ class MyRegistrator extends KryoRegistrator {
   }
 }
 
-// Make sure to set these properties *before* creating a SparkContext!
-System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
-System.setProperty("spark.kryo.registrator", "mypackage.MyRegistrator")
-val sc = new SparkContext(...)
+val conf = new SparkConf().setMaster(...).setAppName(...)
+conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
+conf.set("spark.kryo.registrator", "mypackage.MyRegistrator")
+val sc = new SparkContext(conf)
 {% endhighlight %}
 
 The [Kryo documentation](http://code.google.com/p/kryo/) describes more advanced
 registration options, such as adding custom serialization code.
 
 If your objects are large, you may also need to increase the `spark.kryoserializer.buffer.mb`
-system property. The default is 2, but this value needs to be large enough to hold the *largest*
+config property. The default is 2, but this value needs to be large enough to hold the *largest*
 object you will serialize.
 
 Finally, if you don't register your classes, Kryo will still work, but it will have to store the
@@ -165,7 +166,7 @@ cache RDDs. This means that 33% of memory is available for any objects created d
 
 In case your tasks slow down and you find that your JVM is garbage-collecting frequently or running out of
 memory, lowering this value will help reduce the memory consumption. To change this to say 50%, you can call
-`System.setProperty("spark.storage.memoryFraction", "0.5")`. Combined with the use of serialized caching,
+`conf.set("spark.storage.memoryFraction", "0.5")` on your SparkConf. Combined with the use of serialized caching,
 using a smaller cache should be sufficient to mitigate most of the garbage collection problems.
 In case you are interested in further tuning the Java GC, continue reading below.
 
@@ -219,7 +220,7 @@ enough. Spark automatically sets the number of "map" tasks to run on each file a
 distributed "reduce" operations, such as `groupByKey` and `reduceByKey`, it uses the largest
 parent RDD's number of partitions. You can pass the level of parallelism as a second argument
 (see the [`spark.PairRDDFunctions`](api/core/index.html#org.apache.spark.rdd.PairRDDFunctions) documentation),
-or set the system property `spark.default.parallelism` to change the default.
+or set the config property `spark.default.parallelism` to change the default.
 In general, we recommend 2-3 tasks per CPU core in your cluster.
 
 ## Memory Usage of Reduce Tasks

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0fa58097/python/pyspark/conf.py
----------------------------------------------------------------------
diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py
index c07dd88..9dcdcfa 100644
--- a/python/pyspark/conf.py
+++ b/python/pyspark/conf.py
@@ -44,6 +44,11 @@ u'/path'
 <pyspark.conf.SparkConf object at ...>
 >>> conf.get("spark.executorEnv.VAR1")
 u'value1'
+>>> print conf.toDebugString()
+spark.executorEnv.VAR1=value1
+spark.executorEnv.VAR3=value3
+spark.executorEnv.VAR4=value4
+spark.home=/path
 >>> sorted(conf.getAll(), key=lambda p: p[0])
 [(u'spark.executorEnv.VAR1', u'value1'), (u'spark.executorEnv.VAR3', u'value3'), (u'spark.executorEnv.VAR4', u'value4'), (u'spark.home', u'/path')]
 """
@@ -67,6 +72,9 @@ class SparkConf(object):
 
     All setter methods in this class support chaining. For example,
     you can write C{conf.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.
     """
 
     def __init__(self, loadDefaults=True, _jvm=None):
@@ -74,7 +82,9 @@ class SparkConf(object):
         Create a new Spark configuration.
 
         @param loadDefaults: whether to load values from Java system
-               properties and classpath (true by default)
+               properties and classpath (True by default)
+        @param _jvm: internal parameter used to pass a handle to the
+               Java VM; does not need to be set by users
         """
         from pyspark.context import SparkContext
         SparkContext._ensure_initialized()
@@ -97,10 +107,7 @@ class SparkConf(object):
         return self
 
     def setSparkHome(self, value):
-        """
-        Set path where Spark is installed on worker nodes (needed for some
-        deployment modes).
-        """
+        """Set path where Spark is installed on worker nodes."""
         self._jconf.setSparkHome(value)
         return self
 
@@ -144,6 +151,13 @@ class SparkConf(object):
         """Does this configuration contain a given key?"""
         return self._jconf.contains(key)
 
+    def toDebugString(self):
+        """
+        Returns a printable version of the configuration, as a list of
+        key=value pairs, one per line.
+        """
+        return self._jconf.toDebugString()
+
 
 def _test():
     import doctest

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0fa58097/python/pyspark/context.py
----------------------------------------------------------------------
diff --git a/python/pyspark/context.py b/python/pyspark/context.py
index 12ac029..ee2f5cb 100644
--- a/python/pyspark/context.py
+++ b/python/pyspark/context.py
@@ -83,7 +83,7 @@ class SparkContext(object):
         SparkContext._ensure_initialized(self)
 
         self.environment = environment or {}
-        self.conf = conf or SparkConf(_jvm=self._jvm)
+        self._conf = conf or SparkConf(_jvm=self._jvm)
         self._batchSize = batchSize  # -1 represents an unlimited batch size
         self._unbatched_serializer = serializer
         if batchSize == 1:
@@ -94,31 +94,31 @@ class SparkContext(object):
 
         # Set parameters passed directly to us on the conf; these operations will be
         # no-ops if the parameters were None
-        self.conf.setMaster(master)
-        self.conf.setAppName(appName)
-        self.conf.setSparkHome(sparkHome)
+        self._conf.setMaster(master)
+        self._conf.setAppName(appName)
+        self._conf.setSparkHome(sparkHome)
         if environment:
             for key, value in environment.iteritems():
-                self.conf.setExecutorEnv(key, value)
+                self._conf.setExecutorEnv(key, value)
 
         # Check that we have at least the required parameters
-        if not self.conf.contains("spark.master"):
+        if not self._conf.contains("spark.master"):
             raise Exception("A master URL must be set in your configuration")
-        if not self.conf.contains("spark.app.name"):
+        if not self._conf.contains("spark.app.name"):
             raise Exception("An application name must be set in your configuration")
 
         # Read back our properties from the conf in case we loaded some of them from
         # the classpath or an external config file
-        self.master = self.conf.get("spark.master")
-        self.appName = self.conf.get("spark.app.name")
-        self.sparkHome = self.conf.getOrElse("spark.home", None)
-        for (k, v) in self.conf.getAll():
+        self.master = self._conf.get("spark.master")
+        self.appName = self._conf.get("spark.app.name")
+        self.sparkHome = self._conf.getOrElse("spark.home", None)
+        for (k, v) in self._conf.getAll():
             if k.startswith("spark.executorEnv."):
                 varName = k[len("spark.executorEnv."):]
                 self.environment[varName] = v
 
         # Create the Java SparkContext through Py4J
-        self._jsc = self._jvm.JavaSparkContext(self.conf._jconf)
+        self._jsc = self._jvm.JavaSparkContext(self._conf._jconf)
 
         # Create a single Accumulator in Java that we'll send all our updates through;
         # they will be passed back to us through a TCP server


[33/33] git commit: Merge pull request #309 from mateiz/conf2

Posted by pw...@apache.org.
Merge pull request #309 from mateiz/conf2

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.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/3713f812
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/3713f812
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/3713f812

Branch: refs/heads/master
Commit: 3713f8129a618a633a7aca8c944960c3e7ac9d3b
Parents: c1d928a 7e8d2e8
Author: Patrick Wendell <pw...@gmail.com>
Authored: Wed Jan 1 21:29:12 2014 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Wed Jan 1 21:29:12 2014 -0800

----------------------------------------------------------------------
 .../scala/org/apache/spark/Accumulators.scala   |   8 +-
 .../org/apache/spark/MapOutputTracker.scala     |  11 +-
 .../scala/org/apache/spark/Partitioner.scala    |   4 +-
 .../main/scala/org/apache/spark/SparkConf.scala | 189 +++++++++++++++
 .../scala/org/apache/spark/SparkContext.scala   | 241 +++++++++++--------
 .../main/scala/org/apache/spark/SparkEnv.scala  |  54 +++--
 .../spark/api/java/JavaSparkContext.scala       |  21 +-
 .../org/apache/spark/api/python/PythonRDD.scala |   4 +-
 .../org/apache/spark/broadcast/Broadcast.scala  |   8 +-
 .../spark/broadcast/BroadcastFactory.scala      |   4 +-
 .../apache/spark/broadcast/HttpBroadcast.scala  |  43 ++--
 .../spark/broadcast/TorrentBroadcast.scala      |  45 ++--
 .../spark/deploy/FaultToleranceTest.scala       |   4 +-
 .../apache/spark/deploy/LocalSparkCluster.scala |   7 +-
 .../apache/spark/deploy/SparkHadoopUtil.scala   |  14 +-
 .../org/apache/spark/deploy/client/Client.scala |  13 +-
 .../apache/spark/deploy/client/TestClient.scala |  10 +-
 .../org/apache/spark/deploy/master/Master.scala |  41 ++--
 .../spark/deploy/master/MasterArguments.scala   |  11 +-
 .../deploy/master/SparkZooKeeperSession.scala   |   7 +-
 .../master/ZooKeeperLeaderElectionAgent.scala   |   9 +-
 .../master/ZooKeeperPersistenceEngine.scala     |   8 +-
 .../spark/deploy/master/ui/MasterWebUI.scala    |   2 +-
 .../org/apache/spark/deploy/worker/Worker.scala |  34 ++-
 .../spark/deploy/worker/ui/WorkerWebUI.scala    |   6 +-
 .../executor/CoarseGrainedExecutorBackend.scala |   6 +-
 .../org/apache/spark/executor/Executor.scala    |  21 +-
 .../org/apache/spark/io/CompressionCodec.scala  |  19 +-
 .../apache/spark/metrics/MetricsSystem.scala    |  10 +-
 .../spark/network/ConnectionManager.scala       |  22 +-
 .../org/apache/spark/network/ReceiverTest.scala |  12 +-
 .../org/apache/spark/network/SenderTest.scala   |  16 +-
 .../spark/network/netty/ShuffleCopier.scala     |  10 +-
 .../org/apache/spark/rdd/CheckpointRDD.scala    |   5 +-
 .../org/apache/spark/rdd/CoGroupedRDD.scala     |   2 +-
 .../main/scala/org/apache/spark/rdd/RDD.scala   |   1 +
 .../org/apache/spark/rdd/ShuffledRDD.scala      |   2 +-
 .../org/apache/spark/rdd/SubtractedRDD.scala    |   2 +-
 .../apache/spark/scheduler/DAGScheduler.scala   |   3 +-
 .../spark/scheduler/InputFormatInfo.scala       |  14 +-
 .../org/apache/spark/scheduler/ResultTask.scala |   4 +-
 .../spark/scheduler/SchedulableBuilder.scala    |   6 +-
 .../spark/scheduler/SchedulerBackend.scala      |   3 -
 .../apache/spark/scheduler/ShuffleMapTask.scala |   6 +-
 .../spark/scheduler/TaskResultGetter.scala      |   3 +-
 .../spark/scheduler/TaskSchedulerImpl.scala     |  20 +-
 .../apache/spark/scheduler/TaskSetManager.scala |  18 +-
 .../cluster/CoarseGrainedSchedulerBackend.scala |  20 +-
 .../cluster/SimrSchedulerBackend.scala          |   4 +-
 .../cluster/SparkDeploySchedulerBackend.scala   |   8 +-
 .../mesos/CoarseMesosSchedulerBackend.scala     |  14 +-
 .../cluster/mesos/MesosSchedulerBackend.scala   |   8 +-
 .../spark/scheduler/local/LocalBackend.scala    |   3 +-
 .../spark/serializer/JavaSerializer.scala       |   3 +-
 .../spark/serializer/KryoSerializer.scala       |  14 +-
 .../spark/serializer/SerializerManager.scala    |  12 +-
 .../spark/storage/BlockFetcherIterator.scala    |   4 +-
 .../org/apache/spark/storage/BlockManager.scala |  58 ++---
 .../spark/storage/BlockManagerMaster.scala      |  11 +-
 .../spark/storage/BlockManagerMasterActor.scala |  14 +-
 .../spark/storage/BlockObjectWriter.scala       |   5 +-
 .../apache/spark/storage/DiskBlockManager.scala |   2 +-
 .../spark/storage/ShuffleBlockManager.scala     |  10 +-
 .../spark/storage/StoragePerfTester.scala       |   2 +-
 .../apache/spark/storage/ThreadingTest.scala    |   8 +-
 .../scala/org/apache/spark/ui/SparkUI.scala     |   4 +-
 .../apache/spark/ui/UIWorkloadGenerator.scala   |  17 +-
 .../org/apache/spark/ui/env/EnvironmentUI.scala |  15 +-
 .../spark/ui/jobs/JobProgressListener.scala     |   4 +-
 .../scala/org/apache/spark/util/AkkaUtils.scala |  25 +-
 .../org/apache/spark/util/MetadataCleaner.scala |  35 ++-
 .../org/apache/spark/util/SizeEstimator.scala   |  14 +-
 .../scala/org/apache/spark/util/Utils.scala     |  25 +-
 core/src/test/resources/spark.conf              |   8 +
 .../apache/spark/MapOutputTrackerSuite.scala    |  16 +-
 .../org/apache/spark/SharedSparkContext.scala   |   4 +-
 .../scala/org/apache/spark/SparkConfSuite.scala | 110 +++++++++
 .../apache/spark/io/CompressionCodecSuite.scala |   8 +-
 .../spark/metrics/MetricsSystemSuite.scala      |   8 +-
 .../spark/scheduler/ClusterSchedulerSuite.scala |   2 +-
 .../spark/scheduler/DAGSchedulerSuite.scala     |  23 +-
 .../apache/spark/scheduler/JobLoggerSuite.scala |   2 +-
 .../spark/scheduler/TaskResultGetterSuite.scala |   6 +-
 .../spark/scheduler/TaskSetManagerSuite.scala   |   4 +-
 .../spark/serializer/KryoSerializerSuite.scala  |  29 +--
 .../spark/storage/BlockManagerSuite.scala       |  97 ++++----
 .../spark/storage/DiskBlockManagerSuite.scala   |  18 +-
 .../apache/spark/util/SizeEstimatorSuite.scala  |   2 +-
 docs/_config.yml                                |   2 +-
 docs/configuration.md                           |  71 ++++--
 docs/css/bootstrap.min.css                      |   2 +-
 docs/job-scheduling.md                          |  21 +-
 docs/monitoring.md                              |   3 +-
 docs/python-programming-guide.md                |  15 +-
 docs/quick-start.md                             |  52 ++--
 docs/running-on-mesos.md                        |  19 +-
 docs/scala-programming-guide.md                 |   4 +-
 docs/spark-standalone.md                        |  15 +-
 docs/streaming-programming-guide.md             |   4 +-
 docs/tuning.md                                  |  21 +-
 .../examples/bagel/WikipediaPageRank.scala      |  10 +-
 .../bagel/WikipediaPageRankStandalone.scala     |   8 +-
 .../streaming/examples/ActorWordCount.scala     |   3 +-
 .../apache/spark/mllib/recommendation/ALS.scala |  13 +-
 .../spark/deploy/yarn/ApplicationMaster.scala   |  52 ++--
 .../org/apache/spark/deploy/yarn/Client.scala   |  40 +--
 .../spark/deploy/yarn/ClientArguments.scala     |   2 +-
 .../spark/deploy/yarn/WorkerLauncher.scala      |   4 +-
 .../deploy/yarn/YarnAllocationHandler.scala     |   4 +-
 .../cluster/YarnClientSchedulerBackend.scala    |   4 +-
 project/SparkBuild.scala                        |   1 +
 python/epydoc.conf                              |   2 +-
 python/pyspark/__init__.py                      |  32 ++-
 python/pyspark/broadcast.py                     |  11 +
 python/pyspark/conf.py                          | 171 +++++++++++++
 python/pyspark/context.py                       |  59 +++--
 python/pyspark/java_gateway.py                  |   1 +
 python/run-tests                                |   3 +-
 .../org/apache/spark/repl/SparkILoop.scala      |  17 +-
 .../org/apache/spark/repl/SparkIMain.scala      |   7 +-
 .../org/apache/spark/streaming/Checkpoint.scala |  18 +-
 .../org/apache/spark/streaming/DStream.scala    |   2 +-
 .../spark/streaming/StreamingContext.scala      |  55 +++--
 .../api/java/JavaStreamingContext.scala         |   9 +
 .../streaming/dstream/NetworkInputDStream.scala |   6 +-
 .../streaming/scheduler/JobGenerator.scala      |   8 +-
 .../streaming/scheduler/JobScheduler.scala      |   4 +-
 .../spark/streaming/util/RawTextSender.scala    |   4 +-
 .../apache/spark/streaming/JavaAPISuite.java    |  10 +-
 .../spark/streaming/BasicOperationsSuite.scala  |   8 +-
 .../spark/streaming/CheckpointSuite.scala       |  15 +-
 .../spark/streaming/InputStreamsSuite.scala     |  18 +-
 .../apache/spark/streaming/TestSuiteBase.scala  |  34 +--
 .../spark/streaming/WindowOperationsSuite.scala |   1 -
 .../spark/deploy/yarn/ApplicationMaster.scala   |  64 ++---
 .../org/apache/spark/deploy/yarn/Client.scala   |  52 ++--
 .../spark/deploy/yarn/ClientArguments.scala     |   2 +-
 .../spark/deploy/yarn/WorkerLauncher.scala      |   4 +-
 .../deploy/yarn/YarnAllocationHandler.scala     |   2 +-
 .../cluster/YarnClientSchedulerBackend.scala    |   4 +-
 140 files changed, 1731 insertions(+), 941 deletions(-)
----------------------------------------------------------------------



[26/33] git commit: Merge remote-tracking branch 'apache/master' into conf2

Posted by pw...@apache.org.
Merge remote-tracking branch 'apache/master' into conf2

Conflicts:
	core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
	streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
	streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/ba9338f1
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/ba9338f1
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/ba9338f1

Branch: refs/heads/master
Commit: ba9338f104ccc71d4f342a3f96624a9b36895f48
Parents: 0fa5809 63b411d
Author: Matei Zaharia <ma...@databricks.com>
Authored: Tue Dec 31 18:23:14 2013 -0500
Committer: Matei Zaharia <ma...@databricks.com>
Committed: Tue Dec 31 18:23:14 2013 -0500

----------------------------------------------------------------------
 .../apache/spark/network/netty/FileClient.java  |  32 ++--
 .../netty/FileClientChannelInitializer.java     |   6 +-
 .../spark/network/netty/FileClientHandler.java  |  12 +-
 .../apache/spark/network/netty/FileServer.java  |  29 ++--
 .../netty/FileServerChannelInitializer.java     |   3 +-
 .../spark/network/netty/FileServerHandler.java  |  18 ++-
 .../scala/org/apache/spark/SparkContext.scala   |  23 +--
 .../spark/api/java/JavaSparkContext.scala       |  15 +-
 .../org/apache/spark/rdd/CheckpointRDD.scala    |  32 ++--
 .../apache/spark/rdd/RDDCheckpointData.scala    |  15 +-
 .../apache/spark/scheduler/DAGScheduler.scala   |   2 +-
 .../org/apache/spark/scheduler/JobLogger.scala  |   2 +-
 .../apache/spark/scheduler/SparkListener.scala  |  20 ++-
 .../spark/scheduler/SparkListenerBus.scala      |   2 +-
 .../apache/spark/scheduler/TaskSetManager.scala |   6 -
 .../spark/ui/jobs/JobProgressListener.scala     |   2 +-
 .../scala/org/apache/spark/JavaAPISuite.java    |   4 +-
 .../apache/spark/scheduler/JobLoggerSuite.scala |   2 +-
 .../spark/scheduler/SparkListenerSuite.scala    |   2 +-
 pom.xml                                         |   2 +-
 project/SparkBuild.scala                        |   2 +-
 python/pyspark/context.py                       |   9 +-
 python/pyspark/tests.py                         |   4 +-
 .../org/apache/spark/streaming/Checkpoint.scala |  53 ++++---
 .../spark/streaming/StreamingContext.scala      |  31 ++--
 .../api/java/JavaStreamingContext.scala         |  13 +-
 .../streaming/dstream/FileInputDStream.scala    | 153 +++++++++++--------
 .../streaming/scheduler/JobGenerator.scala      |  66 +++++---
 .../spark/streaming/CheckpointSuite.scala       |  44 +++---
 .../spark/streaming/InputStreamsSuite.scala     |   2 +-
 30 files changed, 347 insertions(+), 259 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ba9338f1/core/src/main/scala/org/apache/spark/SparkContext.scala
----------------------------------------------------------------------
diff --cc core/src/main/scala/org/apache/spark/SparkContext.scala
index 8134ce7,7514ce5..fbc7a78
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@@ -19,15 -19,14 +19,15 @@@ package org.apache.spar
  
  import java.io._
  import java.net.URI
- import java.util.Properties
+ import java.util.{UUID, Properties}
  import java.util.concurrent.atomic.AtomicInteger
  
 -import scala.collection.Map
 +import scala.collection.{Map, Set, immutable}
  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

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ba9338f1/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ba9338f1/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
----------------------------------------------------------------------
diff --cc core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
index 2897c4b,293a7d1..172ba6b
--- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
@@@ -118,10 -125,14 +124,14 @@@ private[spark] object CheckpointRDD ext
      }
    }
  
-   def readFromFile[T](path: Path, context: TaskContext): Iterator[T] = {
+   def readFromFile[T](
+       path: Path,
+       broadcastedConf: Broadcast[SerializableWritable[Configuration]],
+       context: TaskContext
+     ): Iterator[T] = {
      val env = SparkEnv.get
-     val fs = path.getFileSystem(SparkHadoopUtil.get.newConfiguration())
+     val fs = path.getFileSystem(broadcastedConf.value.value)
 -    val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt
 +    val bufferSize = env.conf.getOrElse("spark.buffer.size", "65536").toInt
      val fileInputStream = fs.open(path, bufferSize)
      val serializer = env.serializer.newInstance()
      val deserializeStream = serializer.deserializeStream(fileInputStream)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ba9338f1/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ba9338f1/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
----------------------------------------------------------------------
diff --cc core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
index d752e6f,7929051..b99664a
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
@@@ -114,13 -112,9 +114,9 @@@ private[spark] class TaskSetManager
    // Task index, start and finish time for each task attempt (indexed by task ID)
    val taskInfos = new HashMap[Long, TaskInfo]
  
-   // Did the TaskSet fail?
-   var failed = false
-   var causeOfFailure = ""
- 
    // 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

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ba9338f1/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ba9338f1/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ba9338f1/project/SparkBuild.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ba9338f1/python/pyspark/context.py
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ba9338f1/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
----------------------------------------------------------------------
diff --cc streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
index 35e23c1,4960a85..af44327
--- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
@@@ -21,12 -21,13 +21,13 @@@ import java.io.
  import java.util.concurrent.Executors
  import java.util.concurrent.RejectedExecutionException
  
- import org.apache.hadoop.fs.Path
+ import org.apache.hadoop.fs.{FileSystem, Path}
  import org.apache.hadoop.conf.Configuration
  
 -import org.apache.spark.Logging
 +import org.apache.spark.{SparkConf, Logging}
  import org.apache.spark.io.CompressionCodec
  import org.apache.spark.util.MetadataCleaner
+ import org.apache.spark.deploy.SparkHadoopUtil
  
  
  private[streaming]
@@@ -54,23 -55,21 +55,23 @@@ class Checkpoint(@transient ssc: Stream
  
  
  /**
-  * Convenience class to speed up the writing of graph checkpoint to file
+  * Convenience class to handle the writing of graph checkpoint to file
   */
  private[streaming]
- class CheckpointWriter(conf: SparkConf, checkpointDir: String) extends Logging {
 -class CheckpointWriter(checkpointDir: String, hadoopConf: Configuration) extends Logging {
++class CheckpointWriter(conf: SparkConf, checkpointDir: String, hadoopConf: Configuration)
++  extends Logging
++{
    val file = new Path(checkpointDir, "graph")
-   // The file to which we actually write - and then "move" to file.
-   private val writeFile = new Path(file.getParent, file.getName + ".next")
-   private val bakFile = new Path(file.getParent, file.getName + ".bk")
- 
-   private var stopped = false
- 
-   val hadoopConf = new Configuration()
-   var fs = file.getFileSystem(hadoopConf)
-   val maxAttempts = 3
+   val MAX_ATTEMPTS = 3
    val executor = Executors.newFixedThreadPool(1)
+   val compressionCodec = CompressionCodec.createCodec()
+   // The file to which we actually write - and then "move" to file
+   val writeFile = new Path(file.getParent, file.getName + ".next")
+   // The file to which existing checkpoint is backed up (i.e. "moved")
+   val bakFile = new Path(file.getParent, file.getName + ".bk")
  
-   private val compressionCodec = CompressionCodec.createCodec(conf)
+   private var stopped = false
+   private var fs_ : FileSystem = _
  
    // Removed code which validates whether there is only one CheckpointWriter per path 'file' since
    // I did not notice any errors - reintroduce it ?
@@@ -141,12 -151,12 +153,12 @@@
  private[streaming]
  object CheckpointReader extends Logging {
  
 -  def read(path: String): Checkpoint = {
 +  def read(conf: SparkConf, path: String): Checkpoint = {
      val fs = new Path(path).getFileSystem(new Configuration())
-     val attempts = Seq(
-       new Path(path, "graph"), new Path(path, "graph.bk"), new Path(path), new Path(path + ".bk"))
+     val attempts = Seq(new Path(path, "graph"), new Path(path, "graph.bk"),
+       new Path(path), new Path(path + ".bk"))
  
 -    val compressionCodec = CompressionCodec.createCodec()
 +    val compressionCodec = CompressionCodec.createCodec(conf)
  
      attempts.foreach(file => {
        if (fs.exists(file)) {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ba9338f1/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ba9338f1/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ba9338f1/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
----------------------------------------------------------------------
diff --cc streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
index dbd0841,921a33a..e448211
--- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
@@@ -32,14 -39,23 +39,23 @@@ class JobGenerator(jobScheduler: JobSch
    initLogging()
  
    val ssc = jobScheduler.ssc
-   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,
-     longTime => generateJobs(new Time(longTime)))
    val graph = ssc.graph
+   val eventProcessorActor = ssc.env.actorSystem.actorOf(Props(new Actor {
+     def receive = {
+       case event: JobGeneratorEvent =>
+         logDebug("Got event of type " + event.getClass.getName)
+         processEvent(event)
+     }
+   }))
+   val clock = {
 -    val clockClass = System.getProperty(
++    val clockClass = ssc.sc.conf.getOrElse(
+       "spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock")
 -    Class.forName(clockClass).newInstance().asInstanceOf[Clock]
++    val clock = Class.forName(clockClass).newInstance().asInstanceOf[Clock]
+   }
+   val timer = new RecurringTimer(clock, ssc.graph.batchDuration.milliseconds,
+     longTime => eventProcessorActor ! GenerateJobs(new Time(longTime)))
    lazy val checkpointWriter = if (ssc.checkpointDuration != null && ssc.checkpointDir != null) {
-     new CheckpointWriter(ssc.conf, ssc.checkpointDir)
 -    new CheckpointWriter(ssc.checkpointDir, ssc.sparkContext.hadoopConfiguration)
++    new CheckpointWriter(ssc.conf, ssc.checkpointDir, ssc.sparkContext.hadoopConfiguration)
    } else {
      null
    }
@@@ -52,10 -66,9 +66,9 @@@
      } else {
        startFirstTime()
      }
-     logInfo("JobGenerator started")
    }
 -  
 +
-   def stop() = synchronized {
+   def stop() {
      timer.stop()
      if (checkpointWriter != null) checkpointWriter.stop()
      ssc.graph.stop()

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ba9338f1/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
----------------------------------------------------------------------
diff --cc streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
index c60a3f5,4e25c95..8dc80ac
--- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
@@@ -63,10 -57,10 +57,10 @@@ class CheckpointSuite extends TestSuite
  
      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)
- 
+     val fs = FileSystem.getLocal(new Configuration())
      // this ensure checkpointing occurs at least once
      val firstNumBatches = (stateStreamCheckpointInterval / batchDuration).toLong * 2
      val secondNumBatches = firstNumBatches
@@@ -132,11 -129,11 +129,12 @@@
      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")
+     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)
 +    // Adjust manual clock time as if it is being restarted after a delay; this is a hack because
 +    // we modify the conf object, but it works for this one property
 +    ssc.conf.set("spark.streaming.manualClock.jump", (batchDuration.milliseconds * 7).toString)
      ssc.start()
      advanceTimeWithRealDelay(ssc, 4)
      ssc.stop()

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ba9338f1/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
----------------------------------------------------------------------


[25/33] git commit: Updated docs for SparkConf and handled review comments

Posted by pw...@apache.org.
Updated docs for SparkConf and handled review comments


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/0fa58097
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/0fa58097
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/0fa58097

Branch: refs/heads/master
Commit: 0fa5809768cf60ec62b4277f04e23a44dc1582e2
Parents: 994f080
Author: Matei Zaharia <ma...@databricks.com>
Authored: Mon Dec 30 22:17:28 2013 -0500
Committer: Matei Zaharia <ma...@databricks.com>
Committed: Mon Dec 30 22:17:28 2013 -0500

----------------------------------------------------------------------
 .../scala/org/apache/spark/Partitioner.scala    |  2 +-
 .../main/scala/org/apache/spark/SparkConf.scala | 31 +++++----
 .../scala/org/apache/spark/SparkContext.scala   | 18 +++--
 .../main/scala/org/apache/spark/SparkEnv.scala  | 13 ++--
 .../spark/api/java/JavaSparkContext.scala       |  6 ++
 .../spark/deploy/FaultToleranceTest.scala       |  4 +-
 .../org/apache/spark/executor/Executor.scala    |  2 +-
 .../scala/org/apache/spark/util/Utils.scala     | 10 +--
 core/src/test/resources/spark.conf              |  2 +
 docs/_config.yml                                |  2 +-
 docs/configuration.md                           | 71 +++++++++++++++-----
 docs/css/bootstrap.min.css                      |  2 +-
 docs/job-scheduling.md                          | 21 +++---
 docs/monitoring.md                              |  3 +-
 docs/python-programming-guide.md                | 15 +++--
 docs/quick-start.md                             | 52 ++++++++++----
 docs/running-on-mesos.md                        | 19 +++---
 docs/scala-programming-guide.md                 |  4 +-
 docs/spark-standalone.md                        | 15 +++--
 docs/streaming-programming-guide.md             |  4 +-
 docs/tuning.md                                  | 21 +++---
 python/pyspark/conf.py                          | 24 +++++--
 python/pyspark/context.py                       | 24 +++----
 23 files changed, 241 insertions(+), 124 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0fa58097/core/src/main/scala/org/apache/spark/Partitioner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala
index 7cb545a..31b0773 100644
--- a/core/src/main/scala/org/apache/spark/Partitioner.scala
+++ b/core/src/main/scala/org/apache/spark/Partitioner.scala
@@ -52,7 +52,7 @@ object Partitioner {
     for (r <- bySize if r.partitioner != None) {
       return r.partitioner.get
     }
-    if (rdd.context.conf.getOrElse("spark.default.parallelism", null) != null) {
+    if (rdd.context.conf.contains("spark.default.parallelism")) {
       return new HashPartitioner(rdd.context.defaultParallelism)
     } else {
       return new HashPartitioner(bySize.head.partitions.size)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0fa58097/core/src/main/scala/org/apache/spark/SparkConf.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
index ae52de4..96239cf 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -16,6 +16,12 @@ import com.typesafe.config.ConfigFactory
  * 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 {
@@ -69,10 +75,7 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable {
 
   /** Set JAR files to distribute to the cluster. (Java-friendly version.) */
   def setJars(jars: Array[String]): SparkConf = {
-    if (!jars.isEmpty) {
-      settings("spark.jars") = jars.mkString(",")
-    }
-    this
+    setJars(jars.toSeq)
   }
 
   /**
@@ -102,15 +105,11 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable {
    * (Java-friendly version.)
    */
   def setExecutorEnv(variables: Array[(String, String)]): SparkConf = {
-    for ((k, v) <- variables) {
-      setExecutorEnv(k, v)
-    }
-    this
+    setExecutorEnv(variables.toSeq)
   }
 
   /**
-   * Set the location where Spark is installed on worker nodes. This is only needed on Mesos if
-   * you are not using `spark.executor.uri` to disseminate the Spark binary distribution.
+   * Set the location where Spark is installed on worker nodes.
    */
   def setSparkHome(home: String): SparkConf = {
     if (home != null) {
@@ -154,8 +153,8 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable {
   /** Get all executor environment variables set on this SparkConf */
   def getExecutorEnv: Seq[(String, String)] = {
     val prefix = "spark.executorEnv."
-    getAll.filter(pair => pair._1.startsWith(prefix))
-          .map(pair => (pair._1.substring(prefix.length), pair._2))
+    getAll.filter{case (k, v) => k.startsWith(prefix)}
+          .map{case (k, v) => (k.substring(prefix.length), v)}
   }
 
   /** Does the configuration contain a given parameter? */
@@ -165,4 +164,12 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable {
   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")
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0fa58097/core/src/main/scala/org/apache/spark/SparkContext.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 810ed18..8134ce7 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -55,14 +55,14 @@ 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 conf_ a Spark Config object describing the application configuration. Any settings in
+ * @param config a Spark Config object describing the application configuration. Any settings in
  *   this config overrides the default configs as well as system properties.
  * @param preferredNodeLocationData used in YARN mode to select nodes to launch containers on. Can
  *   be generated using [[org.apache.spark.scheduler.InputFormatInfo.computePreferredLocations]]
  *   from a list of input files or InputFormats for the application.
  */
 class SparkContext(
-    conf_ : SparkConf,
+    config: SparkConf,
     // 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. It contains
     // a map from hostname to a list of input format splits on the host.
@@ -107,7 +107,13 @@ class SparkContext(
       preferredNodeLocationData)
   }
 
-  val conf = conf_.clone()
+  private[spark] val conf = config.clone()
+
+  /**
+   * Return a copy of this SparkContext's configuration. The configuration ''cannot'' be
+   * changed at runtime.
+   */
+  def getConf: SparkConf = conf.clone()
 
   if (!conf.contains("spark.master")) {
     throw new SparkException("A master URL must be set in your configuration")
@@ -135,11 +141,11 @@ class SparkContext(
   initLogging()
 
   // Create the Spark execution environment (cache, map output tracker, etc)
-  private[spark] val env = SparkEnv.createFromSystemProperties(
+  private[spark] val env = SparkEnv.create(
+    conf,
     "<driver>",
     conf.get("spark.driver.host"),
     conf.get("spark.driver.port").toInt,
-    conf,
     isDriver = true,
     isLocal = isLocal)
   SparkEnv.set(env)
@@ -730,7 +736,7 @@ class SparkContext(
    * (in that order of preference). If neither of these is set, return None.
    */
   private[spark] def getSparkHome(): Option[String] = {
-    if (conf.getOrElse("spark.home", null) != null) {
+    if (conf.contains("spark.home")) {
       Some(conf.get("spark.home"))
     } else if (System.getenv("SPARK_HOME") != null) {
       Some(System.getenv("SPARK_HOME"))

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0fa58097/core/src/main/scala/org/apache/spark/SparkEnv.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala
index 34fad3e..d06af8e 100644
--- a/core/src/main/scala/org/apache/spark/SparkEnv.scala
+++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala
@@ -40,7 +40,7 @@ import com.google.common.collect.MapMaker
  * objects needs to have the right SparkEnv set. You can get the current environment with
  * SparkEnv.get (e.g. after creating a SparkContext) and set it with SparkEnv.set.
  */
-class SparkEnv (
+class SparkEnv private[spark] (
     val executorId: String,
     val actorSystem: ActorSystem,
     val serializerManager: SerializerManager,
@@ -63,7 +63,7 @@ class SparkEnv (
   // (e.g., HadoopFileRDD uses this to cache JobConfs and InputFormats).
   private[spark] val hadoopJobMetadata = new MapMaker().softValues().makeMap[String, Any]()
 
-  def stop() {
+  private[spark] def stop() {
     pythonWorkers.foreach { case(key, worker) => worker.stop() }
     httpFileServer.stop()
     mapOutputTracker.stop()
@@ -79,6 +79,7 @@ class SparkEnv (
     //actorSystem.awaitTermination()
   }
 
+  private[spark]
   def createPythonWorker(pythonExec: String, envVars: Map[String, String]): java.net.Socket = {
     synchronized {
       val key = (pythonExec, envVars)
@@ -111,11 +112,11 @@ object SparkEnv extends Logging {
 	  env.get()
   }
 
-  def createFromSystemProperties(
+  private[spark] def create(
+      conf: SparkConf,
       executorId: String,
       hostname: String,
       port: Int,
-      conf: SparkConf,
       isDriver: Boolean,
       isLocal: Boolean): SparkEnv = {
 
@@ -129,7 +130,7 @@ object SparkEnv extends Logging {
     }
 
     // set only if unset until now.
-    if (conf.getOrElse("spark.hostPort",  null) == null) {
+    if (!conf.contains("spark.hostPort")) {
       if (!isDriver){
         // unexpected
         Utils.logErrorWithStack("Unexpected NOT to have spark.hostPort set")
@@ -216,7 +217,7 @@ object SparkEnv extends Logging {
     }
 
     // Warn about deprecated spark.cache.class property
-    if (conf.getOrElse("spark.cache.class", null) != null) {
+    if (conf.contains("spark.cache.class")) {
       logWarning("The spark.cache.class property is no longer being used! Specify storage " +
         "levels using the RDD.persist() method instead.")
     }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0fa58097/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
index e03cf9d..d6aeed7 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
@@ -418,6 +418,12 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
       implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
     new JavaRDD(sc.checkpointFile(path))
   }
+
+  /**
+   * Return a copy of this JavaSparkContext's configuration. The configuration ''cannot'' be
+   * changed at runtime.
+   */
+  def getConf: SparkConf = sc.getConf
 }
 
 object JavaSparkContext {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0fa58097/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala
index 0aa8852..4dfb19e 100644
--- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala
@@ -190,7 +190,7 @@ private[spark] object FaultToleranceTest extends App with Logging {
   /** Creates a SparkContext, which constructs a Client to interact with our cluster. */
   def createClient() = {
     if (sc != null) { sc.stop() }
-    // Counter-hack: Because of a hack in SparkEnv#createFromSystemProperties() that changes this
+    // Counter-hack: Because of a hack in SparkEnv#create() that changes this
     // property, we need to reset it.
     System.setProperty("spark.driver.port", "0")
     sc = new SparkContext(getMasterUrls(masters), "fault-tolerance", containerSparkHome)
@@ -417,4 +417,4 @@ private[spark] object Docker extends Logging {
     "docker ps -l -q".!(ProcessLogger(line => id = line))
     new DockerId(id)
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0fa58097/core/src/main/scala/org/apache/spark/executor/Executor.scala
----------------------------------------------------------------------
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 a6eabc4..2400154 100644
--- a/core/src/main/scala/org/apache/spark/executor/Executor.scala
+++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala
@@ -109,7 +109,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, conf,
+      val _env = SparkEnv.create(conf, executorId, slaveHostname, 0,
         isDriver = false, isLocal = false)
       SparkEnv.set(_env)
       _env.metricsSystem.registerSource(executorSource)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0fa58097/core/src/main/scala/org/apache/spark/util/Utils.scala
----------------------------------------------------------------------
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 b6b89cc..ca3320b 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -397,12 +397,11 @@ private[spark] object Utils extends Logging {
   }
 
   def localHostPort(conf: SparkConf): String = {
-    val retval = conf.getOrElse("spark.hostPort",  null)
+    val retval = conf.getOrElse("spark.hostPort", null)
     if (retval == null) {
       logErrorWithStack("spark.hostPort not set but invoking localHostPort")
       return localHostName()
     }
-
     retval
   }
 
@@ -414,9 +413,12 @@ private[spark] object Utils extends Logging {
     assert(hostPort.indexOf(':') != -1, message)
   }
 
-  // Used by DEBUG code : remove when all testing done
   def logErrorWithStack(msg: String) {
-    try { throw new Exception } catch { case ex: Exception => { logError(msg, ex) } }
+    try {
+      throw new Exception
+    } catch {
+      case ex: Exception => logError(msg, ex)
+    }
   }
 
   // Typically, this will be of order of number of nodes in cluster

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0fa58097/core/src/test/resources/spark.conf
----------------------------------------------------------------------
diff --git a/core/src/test/resources/spark.conf b/core/src/test/resources/spark.conf
index 6c99bdc..aa4e751 100644
--- a/core/src/test/resources/spark.conf
+++ b/core/src/test/resources/spark.conf
@@ -1,3 +1,5 @@
+# A simple spark.conf file used only in our unit tests
+
 spark.test.intTestProperty = 1
 
 spark.test {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0fa58097/docs/_config.yml
----------------------------------------------------------------------
diff --git a/docs/_config.yml b/docs/_config.yml
index 02067f9..11d18f0 100644
--- a/docs/_config.yml
+++ b/docs/_config.yml
@@ -4,7 +4,7 @@ markdown: kramdown
 # These allow the documentation to be updated with nerw releases
 # of Spark, Scala, and Mesos.
 SPARK_VERSION: 0.9.0-incubating-SNAPSHOT
-SPARK_VERSION_SHORT: 0.9.0-SNAPSHOT
+SPARK_VERSION_SHORT: 0.9.0
 SCALA_VERSION: 2.10
 MESOS_VERSION: 0.13.0
 SPARK_ISSUE_TRACKER_URL: https://spark-project.atlassian.net

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0fa58097/docs/configuration.md
----------------------------------------------------------------------
diff --git a/docs/configuration.md b/docs/configuration.md
index 677d182..567aba0 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -3,26 +3,37 @@ layout: global
 title: Spark Configuration
 ---
 
-Spark provides three main locations to configure the system:
+Spark provides three locations to configure the system:
 
-* [Java system properties](#system-properties), which control internal configuration parameters and can be set
-  either programmatically (by calling `System.setProperty` *before* creating a `SparkContext`) or through
-  JVM arguments.
-* [Environment variables](#environment-variables) for configuring per-machine settings such as the IP address,
-  which can be set in the `conf/spark-env.sh` script.
-* [Logging configuration](#configuring-logging), which is done through `log4j.properties`.
+* [Spark properties](#spark-properties) control most application parameters and can be set by passing
+  a [SparkConf](api/core/index.html#org.apache.spark.SparkConf) object to SparkContext, or through Java
+  system properties.
+* [Environment variables](#environment-variables) can be used to set per-machine settings, such as
+  the IP address, through the `conf/spark-env.sh` script on each node.
+* [Logging](#configuring-logging) can be configured through `log4j.properties`.
 
 
-# System Properties
+# Spark Properties
 
-To set a system property for configuring Spark, you need to either pass it with a -D flag to the JVM (for example `java -Dspark.cores.max=5 MyProgram`) or call `System.setProperty` in your code *before* creating your Spark context, as follows:
+Spark properties control most application settings and are configured separately for each application.
+The preferred way to set them is by passing a [SparkConf](api/core/index.html#org.apache.spark.SparkConf)
+class to your SparkContext constructor.
+Alternatively, Spark will also load them from Java system properties (for compatibility with old versions
+of Spark) and from a [`spark.conf` file](#configuration-files) on your classpath.
+
+SparkConf lets you configure most of the common properties to initialize a cluster (e.g., master URL and
+application name), as well as arbitrary key-value pairs through the `set()` method. For example, we could
+initialize an application as follows:
 
 {% highlight scala %}
-System.setProperty("spark.cores.max", "5")
-val sc = new SparkContext(...)
+val conf = new SparkConf()
+             .setMaster("local")
+             .setAppName("My application")
+             .set("spark.executor.memory", "1g")
+val sc = new SparkContext(conf)
 {% endhighlight %}
 
-Most of the configurable system properties control internal settings that have reasonable default values. However,
+Most of the properties control internal settings that have reasonable default values. However,
 there are at least five properties that you will commonly want to control:
 
 <table class="table">
@@ -385,11 +396,40 @@ Apart from these, the following properties are also available, and may be useful
 </tr>
 </table>
 
+## Viewing Spark Properties
+
+The application web UI at `http://<driver>:4040` lists Spark properties in the "Environment" tab.
+This is a useful place to check to make sure that your properties have been set correctly.
+
+## Configuration Files
+
+You can also configure Spark properties through a `spark.conf` file on your Java classpath.
+Because these properties are usually application-specific, we recommend putting this fine *only* on your
+application's classpath, and not in a global Spark classpath.
+
+The `spark.conf` file uses Typesafe Config's [HOCON format](https://github.com/typesafehub/config#json-superset),
+which is a superset of Java properties files and JSON. For example, the following is a simple config file:
+
+{% highlight awk %}
+# Comments are allowed
+spark.executor.memory = 512m
+spark.serializer = org.apache.spark.serializer.KryoSerializer
+{% endhighlight %}
+
+The format also allows hierarchical nesting, as follows:
+
+{% highlight awk %}
+spark.akka {
+  threads = 8
+  timeout = 200
+}
+{% endhighlight %}
+
 # Environment Variables
 
-Certain Spark settings can also be configured through environment variables, which are read from the `conf/spark-env.sh`
+Certain Spark settings can be configured through environment variables, which are read from the `conf/spark-env.sh`
 script in the directory where Spark is installed (or `conf/spark-env.cmd` on Windows). These variables are meant to be for machine-specific settings, such
-as library search paths. While Java system properties can also be set here, for application settings, we recommend setting
+as library search paths. While Spark properties can also be set there through `SPARK_JAVA_OPTS`, for per-application settings, we recommend setting
 these properties within the application instead of in `spark-env.sh` so that different applications can use different
 settings.
 
@@ -406,7 +446,8 @@ The following variables can be set in `spark-env.sh`:
    Note that applications can also add dependencies for themselves through `SparkContext.addJar` -- we recommend
    doing that when possible.
 * `SPARK_JAVA_OPTS`, to add JVM options. This includes Java options like garbage collector settings and any system
-   properties that you'd like to pass with `-D` (e.g., `-Dspark.local.dir=/disk1,/disk2`). 
+   properties that you'd like to pass with `-D`. One use case is to set some Spark properties differently on this
+   machine, e.g., `-Dspark.local.dir=/disk1,/disk2`.
 * Options for the Spark [standalone cluster scripts](spark-standalone.html#cluster-launch-scripts), such as number of cores
   to use on each machine and maximum memory.
 


[15/33] git commit: Fix some other Python tests due to initializing JVM in a different way

Posted by pw...@apache.org.
Fix some other Python tests due to initializing JVM in a different way

The test in context.py created two different instances of the
SparkContext class by copying "globals", so that some tests can have a
global "sc" object and others can try initializing their own contexts.
This led to two JVM gateways being created since SparkConf also looked
at pyspark.context.SparkContext to get the JVM.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/615fb649
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/615fb649
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/615fb649

Branch: refs/heads/master
Commit: 615fb649d66b13371927a051d249433d746c5f19
Parents: cd00225
Author: Matei Zaharia <ma...@databricks.com>
Authored: Sun Dec 29 14:31:45 2013 -0500
Committer: Matei Zaharia <ma...@databricks.com>
Committed: Sun Dec 29 14:32:05 2013 -0500

----------------------------------------------------------------------
 python/pyspark/conf.py    |  5 +++--
 python/pyspark/context.py | 23 +++++++++++++++--------
 python/run-tests          |  1 +
 3 files changed, 19 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/615fb649/python/pyspark/conf.py
----------------------------------------------------------------------
diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py
index 56e615c..eb7a6c1 100644
--- a/python/pyspark/conf.py
+++ b/python/pyspark/conf.py
@@ -50,10 +50,11 @@ u'value1'
 
 
 class SparkConf(object):
-    def __init__(self, loadDefaults=False):
+    def __init__(self, loadDefaults=True, _jvm=None):
         from pyspark.context import SparkContext
         SparkContext._ensure_initialized()
-        self._jconf = SparkContext._jvm.SparkConf(loadDefaults)
+        _jvm = _jvm or SparkContext._jvm
+        self._jconf = _jvm.SparkConf(loadDefaults)
 
     def set(self, key, value):
         self._jconf.set(key, value)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/615fb649/python/pyspark/context.py
----------------------------------------------------------------------
diff --git a/python/pyspark/context.py b/python/pyspark/context.py
index 97c1526..9d75c2b 100644
--- a/python/pyspark/context.py
+++ b/python/pyspark/context.py
@@ -81,7 +81,8 @@ class SparkContext(object):
         """
         SparkContext._ensure_initialized(self)
 
-        self.conf = conf or SparkConf()
+        self.environment = environment or {}
+        self.conf = conf or SparkConf(_jvm=self._jvm)
         self._batchSize = batchSize  # -1 represents an unlimited batch size
         self._unbatched_serializer = serializer
         if batchSize == 1:
@@ -90,23 +91,30 @@ class SparkContext(object):
             self.serializer = BatchedSerializer(self._unbatched_serializer,
                                                 batchSize)
 
-        # Set parameters passed directly on our conf; these operations will be no-ops
-        # if the parameters were None
+        # Set parameters passed directly to us on the conf; these operations will be
+        # no-ops if the parameters were None
         self.conf.setMaster(master)
         self.conf.setAppName(appName)
         self.conf.setSparkHome(sparkHome)
-        environment = environment or {}
-        for key, value in environment.iteritems():
-            self.conf.setExecutorEnv(key, value)
+        if environment:
+            for key, value in environment.iteritems():
+                self.conf.setExecutorEnv(key, value)
 
+        # Check that we have at least the required parameters
         if not self.conf.contains("spark.master"):
             raise Exception("A master URL must be set in your configuration")
         if not self.conf.contains("spark.appName"):
             raise Exception("An application name must be set in your configuration")
 
+        # Read back our properties from the conf in case we loaded some of them from
+        # the classpath or an external config file
         self.master = self.conf.get("spark.master")
         self.appName = self.conf.get("spark.appName")
         self.sparkHome = self.conf.getOrElse("spark.home", None)
+        for (k, v) in self.conf.getAll():
+            if k.startswith("spark.executorEnv."):
+                varName = k[len("spark.executorEnv."):]
+                self.environment[varName] = v
 
         # Create the Java SparkContext through Py4J
         self._jsc = self._jvm.JavaSparkContext(self.conf._jconf)
@@ -147,8 +155,7 @@ class SparkContext(object):
             if not SparkContext._gateway:
                 SparkContext._gateway = launch_gateway()
                 SparkContext._jvm = SparkContext._gateway.jvm
-                SparkContext._writeToFile = \
-                    SparkContext._jvm.PythonRDD.writeToFile
+                SparkContext._writeToFile = SparkContext._jvm.PythonRDD.writeToFile
 
             if instance:
                 if SparkContext._active_spark_context and SparkContext._active_spark_context != instance:

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/615fb649/python/run-tests
----------------------------------------------------------------------
diff --git a/python/run-tests b/python/run-tests
index d4dad67..a0898b3 100755
--- a/python/run-tests
+++ b/python/run-tests
@@ -35,6 +35,7 @@ function run_test() {
 
 run_test "pyspark/rdd.py"
 run_test "pyspark/context.py"
+run_test "pyspark/conf.py"
 run_test "-m doctest pyspark/broadcast.py"
 run_test "-m doctest pyspark/accumulators.py"
 run_test "-m doctest pyspark/serializers.py"


[05/33] Various fixes to configuration code

Posted by pw...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala
----------------------------------------------------------------------
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 398b0ce..a46b16b 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
@@ -52,14 +52,14 @@ private[spark] class ClusterTaskSetManager(
 {
   val conf = sched.sc.conf
   // CPUs to request per task
-  val CPUS_PER_TASK = conf.getOrElse("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 = conf.getOrElse("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 = conf.getOrElse("spark.speculation.quantile",  "0.75").toDouble
-  val SPECULATION_MULTIPLIER = conf.getOrElse("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
@@ -118,7 +118,7 @@ private[spark] class ClusterTaskSetManager(
 
   // How frequently to reprint duplicate exceptions in full, in milliseconds
   val EXCEPTION_PRINT_INTERVAL =
-    conf.getOrElse("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
@@ -678,7 +678,7 @@ private[spark] class ClusterTaskSetManager(
   }
 
   private def getLocalityWait(level: TaskLocality.TaskLocality): Long = {
-    val defaultWait = conf.getOrElse("spark.locality.wait",  "3000")
+    val defaultWait = conf.getOrElse("spark.locality.wait", "3000")
     level match {
       case TaskLocality.PROCESS_LOCAL =>
         conf.getOrElse("spark.locality.wait.process",  defaultWait).toLong

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
----------------------------------------------------------------------
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 4055590..156b01b 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
@@ -62,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 = conf.getOrElse("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)
     }
@@ -118,7 +118,7 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac
         removeExecutor(executorId, reason)
         sender ! true
 
-      case DisassociatedEvent(_, address, _) => 
+      case DisassociatedEvent(_, address, _) =>
         addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client disassociated"))
 
     }
@@ -163,10 +163,7 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac
 
   override def start() {
     val properties = new ArrayBuffer[(String, String)]
-    val iterator = scheduler.sc.conf.getAllConfiguration
-    while (iterator.hasNext) {
-      val entry = iterator.next
-      val (key, value) = (entry.getKey.toString, entry.getValue.toString)
+    for ((key, value) <- scheduler.sc.conf.getAll) {
       if (key.startsWith("spark.") && !key.equals("spark.hostPort")) {
         properties += ((key, value))
       }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala
index 5367218..65d3fc8 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala
@@ -31,7 +31,4 @@ private[spark] trait SchedulerBackend {
   def defaultParallelism(): Int
 
   def killTask(taskId: Long, executorId: String): Unit = throw new UnsupportedOperationException
-
-  // Memory used by each executor (in megabytes)
-  protected val executorMemory: Int = SparkContext.executorMemoryRequested
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala
----------------------------------------------------------------------
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 d01329b..d74f000 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,7 +31,7 @@ private[spark] class SimrSchedulerBackend(
   val tmpPath = new Path(driverFilePath + "_tmp")
   val filePath = new Path(driverFilePath)
 
-  val maxCores = conf.getOrElse("spark.simr.executor.cores",  "1").toInt
+  val maxCores = conf.getOrElse("spark.simr.executor.cores", "1").toInt
 
   override def start() {
     super.start()

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
----------------------------------------------------------------------
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 d6b8ac2..de69e32 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
@@ -49,7 +49,7 @@ private[spark] class SparkDeploySchedulerBackend(
     val command = Command(
       "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs)
     val sparkHome = sc.getSparkHome().getOrElse(null)
-    val appDesc = new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome,
+    val appDesc = new ApplicationDescription(appName, maxCores, sc.executorMemory, command, sparkHome,
         "http://" + sc.ui.appUIAddress)
 
     client = new Client(sc.env.actorSystem, masters, appDesc, this, conf)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala
----------------------------------------------------------------------
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 ff6cc37..319c91b 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
@@ -32,7 +32,7 @@ import org.apache.spark.util.Utils
 private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: ClusterScheduler)
   extends Logging {
 
-  private val THREADS = sparkEnv.conf.getOrElse("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")
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
----------------------------------------------------------------------
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 2a3b0e1..1695374 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
@@ -76,7 +76,7 @@ private[spark] class CoarseMesosSchedulerBackend(
     "Spark home is not set; set it through the spark.home system " +
     "property, the SPARK_HOME environment variable or the SparkContext constructor"))
 
-  val extraCoresPerSlave = conf.getOrElse("spark.mesos.extra.cores",  "0").toInt
+  val extraCoresPerSlave = conf.getOrElse("spark.mesos.extra.cores", "0").toInt
 
   var nextMesosTaskId = 0
 
@@ -176,7 +176,7 @@ private[spark] class CoarseMesosSchedulerBackend(
         val slaveId = offer.getSlaveId.toString
         val mem = getResource(offer.getResourcesList, "mem")
         val cpus = getResource(offer.getResourcesList, "cpus").toInt
-        if (totalCoresAcquired < maxCores && mem >= executorMemory && cpus >= 1 &&
+        if (totalCoresAcquired < maxCores && mem >= sc.executorMemory && cpus >= 1 &&
             failuresBySlaveId.getOrElse(slaveId, 0) < MAX_SLAVE_FAILURES &&
             !slaveIdsWithExecutors.contains(slaveId)) {
           // Launch an executor on the slave
@@ -192,7 +192,7 @@ private[spark] class CoarseMesosSchedulerBackend(
             .setCommand(createCommand(offer, cpusToUse + extraCoresPerSlave))
             .setName("Task " + taskId)
             .addResources(createResource("cpus", cpusToUse))
-            .addResources(createResource("mem", executorMemory))
+            .addResources(createResource("mem", sc.executorMemory))
             .build()
           d.launchTasks(offer.getId, Collections.singletonList(task), filters)
         } else {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
----------------------------------------------------------------------
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 9bb92b4..8dfd4d5 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
@@ -114,7 +114,7 @@ private[spark] class MesosSchedulerBackend(
     val memory = Resource.newBuilder()
       .setName("mem")
       .setType(Value.Type.SCALAR)
-      .setScalar(Value.Scalar.newBuilder().setValue(executorMemory).build())
+      .setScalar(Value.Scalar.newBuilder().setValue(sc.executorMemory).build())
       .build()
     ExecutorInfo.newBuilder()
       .setExecutorId(ExecutorID.newBuilder().setValue(execId).build())
@@ -199,7 +199,7 @@ private[spark] class MesosSchedulerBackend(
         def enoughMemory(o: Offer) = {
           val mem = getResource(o.getResourcesList, "mem")
           val slaveId = o.getSlaveId.getValue
-          mem >= executorMemory || slaveIdsWithExecutors.contains(slaveId)
+          mem >= sc.executorMemory || slaveIdsWithExecutors.contains(slaveId)
         }
 
         for ((offer, index) <- offers.zipWithIndex if enoughMemory(offer)) {
@@ -341,5 +341,5 @@ private[spark] class MesosSchedulerBackend(
   }
 
   // TODO: query Mesos for number of cores
-  override def defaultParallelism() = sc.conf.getOrElse("spark.default.parallelism",  "8").toInt
+  override def defaultParallelism() = sc.conf.getOrElse("spark.default.parallelism", "8").toInt
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala
----------------------------------------------------------------------
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 6069c1d..8498cff 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
@@ -92,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(
-    conf.getOrElse("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]]

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala
index 4de8161..5d3d436 100644
--- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala
@@ -21,6 +21,7 @@ import java.io._
 import java.nio.ByteBuffer
 
 import org.apache.spark.util.ByteBufferInputStream
+import org.apache.spark.SparkConf
 
 private[spark] class JavaSerializationStream(out: OutputStream) extends SerializationStream {
   val objOut = new ObjectOutputStream(out)
@@ -77,6 +78,6 @@ private[spark] class JavaSerializerInstance extends SerializerInstance {
 /**
  * A Spark serializer that uses Java's built-in serialization.
  */
-class JavaSerializer extends Serializer {
+class JavaSerializer(conf: SparkConf) extends Serializer {
   def newInstance(): SerializerInstance = new JavaSerializerInstance
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
----------------------------------------------------------------------
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 17cec81..2367f3f 100644
--- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
@@ -25,20 +25,21 @@ 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.{SparkContext, SparkConf, SerializableWritable, Logging}
+import org.apache.spark._
 import org.apache.spark.broadcast.HttpBroadcast
 import org.apache.spark.scheduler.MapStatus
 import org.apache.spark.storage._
 import scala.util.Try
+import org.apache.spark.storage.PutBlock
+import org.apache.spark.storage.GetBlock
+import org.apache.spark.storage.GotBlock
 
 /**
  * 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
+class KryoSerializer(conf: SparkConf) extends org.apache.spark.serializer.Serializer with Logging {
   private val bufferSize = {
-    conf.getOrElse("spark.kryoserializer.buffer.mb",  "2").toInt * 1024 * 1024
+    conf.getOrElse("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024
   }
 
   def newKryoOutput() = new KryoOutput(bufferSize)
@@ -50,7 +51,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(conf.getOrElse("spark.kryo.referenceTracking",  "true").toBoolean)
+    kryo.setReferences(conf.getOrElse("spark.kryo.referenceTracking", "true").toBoolean)
 
     for (cls <- KryoSerializer.toRegister) kryo.register(cls)
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala
index 2955986..2246527 100644
--- a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala
@@ -18,6 +18,7 @@
 package org.apache.spark.serializer
 
 import java.util.concurrent.ConcurrentHashMap
+import org.apache.spark.SparkConf
 
 
 /**
@@ -32,12 +33,12 @@ private[spark] class SerializerManager {
 
   def default = _default
 
-  def setDefault(clsName: String): Serializer = {
-    _default = get(clsName)
+  def setDefault(clsName: String, conf: SparkConf): Serializer = {
+    _default = get(clsName, conf)
     _default
   }
 
-  def get(clsName: String): Serializer = {
+  def get(clsName: String, conf: SparkConf): Serializer = {
     if (clsName == null) {
       default
     } else {
@@ -51,8 +52,9 @@ private[spark] class SerializerManager {
         serializer = serializers.get(clsName)
         if (serializer == null) {
           val clsLoader = Thread.currentThread.getContextClassLoader
-          serializer =
-            Class.forName(clsName, true, clsLoader).newInstance().asInstanceOf[Serializer]
+          val cls = Class.forName(clsName, true, clsLoader)
+          val constructor = cls.getConstructor(classOf[SparkConf])
+          serializer = constructor.newInstance(conf).asInstanceOf[Serializer]
           serializers.put(clsName, serializer)
         }
         serializer

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala
----------------------------------------------------------------------
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 ee2ae47..3b25f68 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala
@@ -327,7 +327,7 @@ object BlockFetcherIterator {
         fetchRequestsSync.put(request)
       }
 
-      copiers = startCopiers(conf.getOrElse("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))
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
----------------------------------------------------------------------
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 ffd166e..16ee208 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
@@ -58,8 +58,8 @@ private[spark] class BlockManager(
 
   // If we use Netty for shuffle, start a new Netty-based shuffle sender service.
   private val nettyPort: Int = {
-    val useNetty = conf.getOrElse("spark.shuffle.use.netty",  "false").toBoolean
-    val nettyPortConfig = conf.getOrElse("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
   }
 
@@ -72,18 +72,18 @@ 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 =
-    conf.getOrElse("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 = conf.getOrElse("spark.broadcast.compress",  "true").toBoolean
+  val compressBroadcast = conf.getOrElse("spark.broadcast.compress", "true").toBoolean
   // Whether to compress shuffle output that are stored
-  val compressShuffle = conf.getOrElse("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 = conf.getOrElse("spark.rdd.compress",  "false").toBoolean
+  val compressRdds = conf.getOrElse("spark.rdd.compress", "false").toBoolean
 
-  val heartBeatFrequency = BlockManager.getHeartBeatFrequencyFromSystemProperties
+  val heartBeatFrequency = BlockManager.getHeartBeatFrequency(conf)
 
-  val hostPort = Utils.localHostPort()
+  val hostPort = Utils.localHostPort(conf)
 
   val slaveActor = actorSystem.actorOf(Props(new BlockManagerSlaveActor(this)),
     name = "BlockManagerActor" + BlockManager.ID_GENERATOR.next)
@@ -101,8 +101,11 @@ private[spark] class BlockManager(
 
   var heartBeatTask: Cancellable = null
 
-  private val metadataCleaner = new MetadataCleaner(MetadataCleanerType.BLOCK_MANAGER, this.dropOldNonBroadcastBlocks)
-  private val broadcastCleaner = new MetadataCleaner(MetadataCleanerType.BROADCAST_VARS, this.dropOldBroadcastBlocks)
+  private val metadataCleaner = new MetadataCleaner(
+    MetadataCleanerType.BLOCK_MANAGER, this.dropOldNonBroadcastBlocks, conf)
+  private val broadcastCleaner = new MetadataCleaner(
+    MetadataCleanerType.BROADCAST_VARS, this.dropOldBroadcastBlocks, conf)
+
   initialize()
 
   // The compression codec to use. Note that the "lazy" val is necessary because we want to delay
@@ -110,14 +113,14 @@ private[spark] class BlockManager(
   // program could be using a user-defined codec in a third party jar, which is loaded in
   // Executor.updateDependencies. When the BlockManager is initialized, user level jars hasn't been
   // loaded yet.
-  private lazy val compressionCodec: CompressionCodec = CompressionCodec.createCodec()
+  private lazy val compressionCodec: CompressionCodec = CompressionCodec.createCodec(conf)
 
   /**
    * Construct a BlockManager with a memory limit set based on system properties.
    */
   def this(execId: String, actorSystem: ActorSystem, master: BlockManagerMaster,
            serializer: Serializer, conf: SparkConf) = {
-    this(execId, actorSystem, master, serializer, BlockManager.getMaxMemoryFromSystemProperties, conf)
+    this(execId, actorSystem, master, serializer, BlockManager.getMaxMemory(conf), conf)
   }
 
   /**
@@ -127,7 +130,7 @@ private[spark] class BlockManager(
   private def initialize() {
     master.registerBlockManager(blockManagerId, maxMemory, slaveActor)
     BlockManagerWorker.startBlockManagerWorker(this)
-    if (!BlockManager.getDisableHeartBeatsForTesting) {
+    if (!BlockManager.getDisableHeartBeatsForTesting(conf)) {
       heartBeatTask = actorSystem.scheduler.schedule(0.seconds, heartBeatFrequency.milliseconds) {
         heartBeat()
       }
@@ -440,7 +443,7 @@ private[spark] class BlockManager(
       : BlockFetcherIterator = {
 
     val iter =
-      if (conf.getOrElse("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)
@@ -466,7 +469,7 @@ private[spark] class BlockManager(
   def getDiskWriter(blockId: BlockId, file: File, serializer: Serializer, bufferSize: Int)
     : BlockObjectWriter = {
     val compressStream: OutputStream => OutputStream = wrapForCompression(blockId, _)
-    val syncWrites = conf.getOrElse("spark.shuffle.sync",  "false").toBoolean
+    val syncWrites = conf.getOrElse("spark.shuffle.sync", "false").toBoolean
     new DiskBlockObjectWriter(blockId, file, serializer, bufferSize, compressStream, syncWrites)
   }
 
@@ -858,19 +861,18 @@ 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 = conf.getOrElse("spark.storage.memoryFraction",  "0.66").toDouble
+  def getMaxMemory(conf: SparkConf): Long = {
+    val memoryFraction = conf.getOrElse("spark.storage.memoryFraction", "0.66").toDouble
     (Runtime.getRuntime.maxMemory * memoryFraction).toLong
   }
 
-  def getHeartBeatFrequencyFromSystemProperties: Long =
-    conf.getOrElse("spark.storage.blockManagerTimeoutIntervalMs",  "60000").toLong / 4
+  def getHeartBeatFrequency(conf: SparkConf): Long =
+    conf.getOrElse("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong / 4
 
-  def getDisableHeartBeatsForTesting: Boolean =
-    conf.getOrElse("spark.test.disableBlockManagerHeartBeat",  "false").toBoolean
+  def getDisableHeartBeatsForTesting(conf: SparkConf): Boolean =
+    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

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala
----------------------------------------------------------------------
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 fde7d63..8e4a88b 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala
@@ -31,8 +31,8 @@ private[spark]
 class BlockManagerMaster(var driverActor : Either[ActorRef, ActorSelection],
     conf: SparkConf) extends Logging {
 
-  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 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"
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
----------------------------------------------------------------------
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 05502e4..73a1da2 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
@@ -53,7 +53,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act
   initLogging()
 
   val slaveTimeout = conf.getOrElse("spark.storage.blockManagerSlaveTimeoutMs",
-    "" + (BlockManager.getHeartBeatFrequencyFromSystemProperties * 3)).toLong
+    "" + (BlockManager.getHeartBeatFrequency(conf) * 3)).toLong
 
   val checkTimeoutInterval = conf.getOrElse("spark.storage.blockManagerTimeoutIntervalMs",
     "60000").toLong
@@ -61,7 +61,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act
   var timeoutCheckingTask: Cancellable = null
 
   override def preStart() {
-    if (!BlockManager.getDisableHeartBeatsForTesting) {
+    if (!BlockManager.getDisableHeartBeatsForTesting(conf)) {
       import context.dispatcher
       timeoutCheckingTask = context.system.scheduler.schedule(
         0.seconds, checkTimeoutInterval.milliseconds, self, ExpireDeadHosts)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
----------------------------------------------------------------------
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 8f528ba..7697092 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 = shuffleManager.conf.getOrElse("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

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala
----------------------------------------------------------------------
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 850d317..f592df2 100644
--- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala
@@ -62,12 +62,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 =
-    conf.getOrElse("spark.shuffle.consolidateFiles",  "false").toBoolean
+    conf.getOrElse("spark.shuffle.consolidateFiles", "false").toBoolean
 
-  private val bufferSize = conf.getOrElse("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
@@ -82,8 +83,8 @@ class ShuffleBlockManager(blockManager: BlockManager) {
   type ShuffleId = Int
   private val shuffleStates = new TimeStampedHashMap[ShuffleId, ShuffleState]
 
-  private
-  val metadataCleaner = new MetadataCleaner(MetadataCleanerType.SHUFFLE_BLOCK_MANAGER, this.cleanup)
+  private val metadataCleaner =
+    new MetadataCleaner(MetadataCleanerType.SHUFFLE_BLOCK_MANAGER, this.cleanup, conf)
 
   def forMapTask(shuffleId: Int, mapId: Int, numBuckets: Int, serializer: Serializer) = {
     new ShuffleWriterGroup {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala b/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala
index d52b3d8..40734aa 100644
--- a/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala
+++ b/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala
@@ -56,7 +56,7 @@ object StoragePerfTester {
 
     def writeOutputBytes(mapId: Int, total: AtomicLong) = {
       val shuffle = blockManager.shuffleBlockManager.forMapTask(1, mapId, numOutputSplits,
-        new KryoSerializer())
+        new KryoSerializer(sc.conf))
       val writers = shuffle.writers
       for (i <- 1 to recordsPerMap) {
         writers(i % numOutputSplits).write(writeData)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala
----------------------------------------------------------------------
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 b3b3893..dca98c6 100644
--- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala
+++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala
@@ -22,7 +22,7 @@ import akka.actor._
 import java.util.concurrent.ArrayBlockingQueue
 import util.Random
 import org.apache.spark.serializer.KryoSerializer
-import org.apache.spark.SparkContext
+import org.apache.spark.{SparkConf, SparkContext}
 
 /**
  * This class tests the BlockManager and MemoryStore for thread safety and
@@ -92,8 +92,8 @@ 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 conf = new SparkConf()
+    val serializer = new KryoSerializer(conf)
     val blockManagerMaster = new BlockManagerMaster(
       Left(actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf)))), conf)
     val blockManager = new BlockManager(

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
----------------------------------------------------------------------
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 14751e8..58d47a2 100644
--- a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
+++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
@@ -19,7 +19,7 @@ package org.apache.spark.ui
 
 import scala.util.Random
 
-import org.apache.spark.SparkContext
+import org.apache.spark.{SparkConf, SparkContext}
 import org.apache.spark.SparkContext._
 import org.apache.spark.scheduler.SchedulingMode
 
@@ -31,7 +31,6 @@ import org.apache.spark.scheduler.SchedulingMode
  */
 private[spark] object UIWorkloadGenerator {
 
-  import SparkContext.{globalConf => conf}
   val NUM_PARTITIONS = 100
   val INTER_JOB_WAIT_MS = 5000
 
@@ -40,14 +39,14 @@ private[spark] object UIWorkloadGenerator {
       println("usage: ./spark-class org.apache.spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]")
       System.exit(1)
     }
-    val master = args(0)
-    val schedulingMode = SchedulingMode.withName(args(1))
-    val appName = "Spark UI Tester"
 
+    val conf = new SparkConf().setMaster(args(0)).setAppName("Spark UI tester")
+
+    val schedulingMode = SchedulingMode.withName(args(1))
     if (schedulingMode == SchedulingMode.FAIR) {
-      conf.set("spark.scheduler.mode",  "FAIR")
+      conf.set("spark.scheduler.mode", "FAIR")
     }
-    val sc = new SparkContext(master, appName)
+    val sc = new SparkContext(conf)
 
     def setProperties(s: String) = {
       if(schedulingMode == SchedulingMode.FAIR) {
@@ -57,11 +56,11 @@ private[spark] object UIWorkloadGenerator {
     }
 
     val baseData = sc.makeRDD(1 to NUM_PARTITIONS * 10, NUM_PARTITIONS)
-    def nextFloat() = (new Random()).nextFloat()
+    def nextFloat() = new Random().nextFloat()
 
     val jobs = Seq[(String, () => Long)](
       ("Count", baseData.count),
-      ("Cache and Count", baseData.map(x => x).cache.count),
+      ("Cache and Count", baseData.map(x => x).cache().count),
       ("Single Shuffle", baseData.map(x => (x % 10, x)).reduceByKey(_ + _).count),
       ("Entirely failed phase", baseData.map(x => throw new Exception).count),
       ("Partially failed phase", {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala
----------------------------------------------------------------------
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 b637d37..91fa00a 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(sc.conf.getOrElse("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")}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
----------------------------------------------------------------------
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 f01a138..6ff8e9f 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 = sc.conf.getOrElse("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]()
@@ -105,7 +105,7 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList
     val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashSet[StageInfo]())
     stages += stage
   }
-  
+
   override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized {
     val sid = taskStart.task.stageId
     val tasksActive = stageIdToTasksActive.getOrElseUpdate(sid, new HashSet[TaskInfo]())

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
----------------------------------------------------------------------
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 76febd5..58b26f7 100644
--- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
+++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
@@ -41,19 +41,19 @@ private[spark] object AkkaUtils {
   def createActorSystem(name: String, host: String, port: Int, indestructible: Boolean = false,
     conf: SparkConf): (ActorSystem, Int) = {
 
-    val akkaThreads   = conf.getOrElse("spark.akka.threads",  "4").toInt
-    val akkaBatchSize = conf.getOrElse("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 = conf.getOrElse("spark.akka.timeout",  "100").toInt
+    val akkaTimeout = conf.getOrElse("spark.akka.timeout", "100").toInt
 
-    val akkaFrameSize = conf.getOrElse("spark.akka.frameSize",  "10").toInt
+    val akkaFrameSize = conf.getOrElse("spark.akka.frameSize", "10").toInt
     val lifecycleEvents =
-      if (conf.getOrElse("spark.akka.logLifecycleEvents",  "false").toBoolean) "on" else "off"
+      if (conf.getOrElse("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off"
 
-    val akkaHeartBeatPauses = conf.getOrElse("spark.akka.heartbeat.pauses",  "600").toInt
+    val akkaHeartBeatPauses = conf.getOrElse("spark.akka.heartbeat.pauses", "600").toInt
     val akkaFailureDetector =
-      conf.getOrElse("spark.akka.failure-detector.threshold",  "300.0").toDouble
-    val akkaHeartBeatInterval = conf.getOrElse("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"""
@@ -89,6 +89,6 @@ private[spark] object AkkaUtils {
 
   /** Returns the default Spark timeout to use for Akka ask operations. */
   def askTimeout(conf: SparkConf): FiniteDuration = {
-    Duration.create(conf.getOrElse("spark.akka.askTimeout",  "30").toLong, "seconds")
+    Duration.create(conf.getOrElse("spark.akka.askTimeout", "30").toLong, "seconds")
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
----------------------------------------------------------------------
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 bf71d17..431d888 100644
--- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
+++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
@@ -18,16 +18,21 @@
 package org.apache.spark.util
 
 import java.util.{TimerTask, Timer}
-import org.apache.spark.{SparkContext, Logging}
+import org.apache.spark.{SparkConf, SparkContext, Logging}
 
 
 /**
  * Runs a timer task to periodically clean up metadata (e.g. old files or hashtable entries)
  */
-class MetadataCleaner(cleanerType: MetadataCleanerType.MetadataCleanerType, cleanupFunc: (Long) => Unit) extends Logging {
+class MetadataCleaner(
+    cleanerType: MetadataCleanerType.MetadataCleanerType,
+    cleanupFunc: (Long) => Unit,
+    conf: SparkConf)
+  extends Logging
+{
   val name = cleanerType.toString
 
-  private val delaySeconds = MetadataCleaner.getDelaySeconds
+  private val delaySeconds = MetadataCleaner.getDelaySeconds(conf)
   private val periodSeconds = math.max(10, delaySeconds / 10)
   private val timer = new Timer(name + " cleanup timer", true)
 
@@ -65,22 +70,28 @@ object MetadataCleanerType extends Enumeration {
   def systemProperty(which: MetadataCleanerType.MetadataCleanerType) = "spark.cleaner.ttl." + which.toString
 }
 
+// TODO: This mutates a Conf to set properties right now, which is kind of ugly when used in the
+// initialization of StreamingContext. It's okay for users trying to configure stuff themselves.
 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 = conf.getOrElse("spark.cleaner.ttl",  "3500").toInt //TODO: this is to fix tests for time being
+  def getDelaySeconds(conf: SparkConf) = {
+    conf.getOrElse("spark.cleaner.ttl", "3500").toInt
+  }
 
-  def getDelaySeconds(cleanerType: MetadataCleanerType.MetadataCleanerType): Int = {
-    conf.getOrElse(MetadataCleanerType.systemProperty(cleanerType),  getDelaySeconds.toString).toInt
+  def getDelaySeconds(conf: SparkConf, cleanerType: MetadataCleanerType.MetadataCleanerType): Int =
+  {
+    conf.getOrElse(MetadataCleanerType.systemProperty(cleanerType), getDelaySeconds(conf).toString)
+      .toInt
   }
 
-  def setDelaySeconds(cleanerType: MetadataCleanerType.MetadataCleanerType, delay: Int) {
+  def setDelaySeconds(conf: SparkConf, cleanerType: MetadataCleanerType.MetadataCleanerType,
+      delay: Int)
+  {
     conf.set(MetadataCleanerType.systemProperty(cleanerType),  delay.toString)
   }
 
-  def setDelaySeconds(delay: Int, resetAll: Boolean = true) {
+  def setDelaySeconds(conf: SparkConf, delay: Int, resetAll: Boolean = true) {
     // override for all ?
-    conf.set("spark.cleaner.ttl",  delay.toString)
+    conf.set("spark.cleaner.ttl", delay.toString)
     if (resetAll) {
       for (cleanerType <- MetadataCleanerType.values) {
         System.clearProperty(MetadataCleanerType.systemProperty(cleanerType))

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala
----------------------------------------------------------------------
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 1407c39..bddb3bb 100644
--- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala
+++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala
@@ -30,10 +30,10 @@ import java.lang.management.ManagementFactory
 import scala.collection.mutable.ArrayBuffer
 
 import it.unimi.dsi.fastutil.ints.IntOpenHashSet
-import org.apache.spark.{SparkConf, SparkContext, Logging}
+import org.apache.spark.{SparkEnv, SparkConf, SparkContext, Logging}
 
 /**
- * Estimates the sizes of Java objects (number of bytes of memory they occupy), for use in 
+ * Estimates the sizes of Java objects (number of bytes of memory they occupy), for use in
  * memory-aware caches.
  *
  * Based on the following JavaWorld article:
@@ -41,7 +41,6 @@ import org.apache.spark.{SparkConf, SparkContext, 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,9 +89,11 @@ private[spark] object SizeEstimator extends Logging {
     classInfos.put(classOf[Object], new ClassInfo(objectSize, Nil))
   }
 
-  private def getIsCompressedOops : Boolean = {
-    if (conf.getOrElse("spark.test.useCompressedOops", null) != null) {
-      return conf.get("spark.test.useCompressedOops").toBoolean 
+  private def getIsCompressedOops: Boolean = {
+    // This is only used by tests to override the detection of compressed oops. The test
+    // actually uses a system property instead of a SparkConf, so we'll stick with that.
+    if (System.getProperty("spark.test.useCompressedOops") != null) {
+      return System.getProperty("spark.test.useCompressedOops").toBoolean
     }
 
     try {
@@ -104,7 +105,7 @@ private[spark] object SizeEstimator extends Logging {
       val getVMMethod = hotSpotMBeanClass.getDeclaredMethod("getVMOption",
           Class.forName("java.lang.String"))
 
-      val bean = ManagementFactory.newPlatformMXBeanProxy(server, 
+      val bean = ManagementFactory.newPlatformMXBeanProxy(server,
         hotSpotMBeanName, hotSpotMBeanClass)
       // TODO: We could use reflection on the VMOption returned ?
       return getVMMethod.invoke(bean, "UseCompressedOops").toString.contains("true")
@@ -252,7 +253,7 @@ private[spark] object SizeEstimator extends Logging {
     if (info != null) {
       return info
     }
-    
+
     val parent = getClassInfo(cls.getSuperclass)
     var shellSize = parent.shellSize
     var pointerFields = parent.pointerFields

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/util/Utils.scala
----------------------------------------------------------------------
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 fd5888e..b6b89cc 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -36,15 +36,13 @@ 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.{SparkContext, SparkException, Logging}
+import org.apache.spark.{SparkConf, SparkContext, SparkException, Logging}
 
 
 /**
  * Various utility methods used by Spark.
  */
 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()
@@ -240,9 +238,9 @@ private[spark] object Utils extends Logging {
    * Throws SparkException if the target file already exists and has different contents than
    * the requested file.
    */
-  def fetchFile(url: String, targetDir: File) {
+  def fetchFile(url: String, targetDir: File, conf: SparkConf) {
     val filename = url.split("/").last
-    val tempDir = getLocalDir
+    val tempDir = getLocalDir(conf)
     val tempFile =  File.createTempFile("fetchFileTemp", null, new File(tempDir))
     val targetFile = new File(targetDir, filename)
     val uri = new URI(url)
@@ -312,7 +310,7 @@ private[spark] object Utils extends Logging {
    * return a single directory, even though the spark.local.dir property might be a list of
    * multiple paths.
    */
-  def getLocalDir: String = {
+  def getLocalDir(conf: SparkConf): String = {
     conf.getOrElse("spark.local.dir",  System.getProperty("java.io.tmpdir")).split(',')(0)
   }
 
@@ -398,7 +396,7 @@ private[spark] object Utils extends Logging {
     InetAddress.getByName(address).getHostName
   }
 
-  def localHostPort(): String = {
+  def localHostPort(conf: SparkConf): String = {
     val retval = conf.getOrElse("spark.hostPort",  null)
     if (retval == null) {
       logErrorWithStack("spark.hostPort not set but invoking localHostPort")
@@ -838,7 +836,7 @@ private[spark] object Utils extends Logging {
     }
   }
 
-  /** 
+  /**
    * Timing method based on iterations that permit JVM JIT optimization.
    * @param numIters number of iterations
    * @param f function to be executed

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala
index ab81bfb..8d75460 100644
--- a/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala
+++ b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala
@@ -20,9 +20,11 @@ package org.apache.spark.io
 import java.io.{ByteArrayInputStream, ByteArrayOutputStream}
 
 import org.scalatest.FunSuite
+import org.apache.spark.SparkConf
 
 
 class CompressionCodecSuite extends FunSuite {
+  val conf = new SparkConf(false)
 
   def testCodec(codec: CompressionCodec) {
     // Write 1000 integers to the output stream, compressed.
@@ -43,19 +45,19 @@ class CompressionCodecSuite extends FunSuite {
   }
 
   test("default compression codec") {
-    val codec = CompressionCodec.createCodec()
+    val codec = CompressionCodec.createCodec(conf)
     assert(codec.getClass === classOf[LZFCompressionCodec])
     testCodec(codec)
   }
 
   test("lzf compression codec") {
-    val codec = CompressionCodec.createCodec(classOf[LZFCompressionCodec].getName)
+    val codec = CompressionCodec.createCodec(conf, classOf[LZFCompressionCodec].getName)
     assert(codec.getClass === classOf[LZFCompressionCodec])
     testCodec(codec)
   }
 
   test("snappy compression codec") {
-    val codec = CompressionCodec.createCodec(classOf[SnappyCompressionCodec].getName)
+    val codec = CompressionCodec.createCodec(conf, classOf[SnappyCompressionCodec].getName)
     assert(codec.getClass === classOf[SnappyCompressionCodec])
     testCodec(codec)
   }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala
----------------------------------------------------------------------
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 2bb827c..3711382 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
@@ -82,7 +82,7 @@ class FakeClusterScheduler(sc: SparkContext, liveExecutors: (String, String)* /*
 class ClusterTaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging {
   import TaskLocality.{ANY, PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL}
   private val conf = new SparkConf
-  val LOCALITY_WAIT = conf.getOrElse("spark.locality.wait",  "3000").toLong
+  val LOCALITY_WAIT = conf.getOrElse("spark.locality.wait", "3000").toLong
 
   test("TaskSet with no preferences") {
     sc = new SparkContext("local", "test")

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
index c016c51..33b0148 100644
--- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
@@ -22,12 +22,14 @@ import scala.collection.mutable
 import com.esotericsoftware.kryo.Kryo
 
 import org.scalatest.FunSuite
-import org.apache.spark.SharedSparkContext
+import org.apache.spark.{SparkConf, SharedSparkContext}
 import org.apache.spark.serializer.KryoTest._
 
 class KryoSerializerSuite extends FunSuite with SharedSparkContext {
+  val conf = new SparkConf(false)
+
   test("basic types") {
-    val ser = (new KryoSerializer).newInstance()
+    val ser = new KryoSerializer(conf).newInstance()
     def check[T](t: T) {
       assert(ser.deserialize[T](ser.serialize(t)) === t)
     }
@@ -57,7 +59,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext {
   }
 
   test("pairs") {
-    val ser = (new KryoSerializer).newInstance()
+    val ser = new KryoSerializer(conf).newInstance()
     def check[T](t: T) {
       assert(ser.deserialize[T](ser.serialize(t)) === t)
     }
@@ -81,7 +83,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext {
   }
 
   test("Scala data structures") {
-    val ser = (new KryoSerializer).newInstance()
+    val ser = new KryoSerializer(conf).newInstance()
     def check[T](t: T) {
       assert(ser.deserialize[T](ser.serialize(t)) === t)
     }
@@ -104,7 +106,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext {
   }
 
   test("ranges") {
-    val ser = (new KryoSerializer).newInstance()
+    val ser = new KryoSerializer(conf).newInstance()
     def check[T](t: T) {
       assert(ser.deserialize[T](ser.serialize(t)) === t)
       // Check that very long ranges don't get written one element at a time
@@ -127,7 +129,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext {
   test("custom registrator") {
     System.setProperty("spark.kryo.registrator", classOf[MyRegistrator].getName)
 
-    val ser = (new KryoSerializer).newInstance()
+    val ser = new KryoSerializer(conf).newInstance()
     def check[T](t: T) {
       assert(ser.deserialize[T](ser.serialize(t)) === t)
     }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
----------------------------------------------------------------------
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 4ef5538..a0fc344 100644
--- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
@@ -34,7 +34,7 @@ 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
+  private val conf = new SparkConf(false)
   var store: BlockManager = null
   var store2: BlockManager = null
   var actorSystem: ActorSystem = null
@@ -45,7 +45,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
 
   // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test
   conf.set("spark.kryoserializer.buffer.mb", "1")
-  val serializer = new KryoSerializer
+  val serializer = new KryoSerializer(conf)
 
   // Implicitly convert strings to BlockIds for test clarity.
   implicit def StringToBlockId(value: String): BlockId = new TestBlockId(value)
@@ -167,7 +167,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
 
   test("master + 2 managers interaction") {
     store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf)
-    store2 = new BlockManager("exec2", actorSystem, master, new KryoSerializer, 2000, conf)
+    store2 = new BlockManager("exec2", actorSystem, master, new KryoSerializer(conf), 2000, conf)
 
     val peers = master.getPeers(store.blockManagerId, 1)
     assert(peers.size === 1, "master did not return the other manager as a peer")
@@ -654,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("<driver>", actorSystem, master, new JavaSerializer, 1200, conf)
+    store = new BlockManager("<driver>", actorSystem, master, new JavaSerializer(conf), 1200, conf)
 
     // The put should fail since a1 is not serializable.
     class UnserializableClass

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala
----------------------------------------------------------------------
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 a5facd5..11ebdc3 100644
--- a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala
@@ -140,8 +140,6 @@ 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()
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala
----------------------------------------------------------------------
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 12c430b..4c0de46 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
@@ -37,7 +37,7 @@ object WikipediaPageRank {
       System.exit(-1)
     }
     val sparkConf = new SparkConf()
-    sparkConf.set("spark.serializer",  "org.apache.spark.serializer.KryoSerializer")
+    sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
     sparkConf.set("spark.kryo.registrator",  classOf[PRKryoRegistrator].getName)
 
     val inputFile = args(0)
@@ -46,7 +46,7 @@ object WikipediaPageRank {
     val host = args(3)
     val usePartitioner = args(4).toBoolean
 
-    sparkConf.setMasterUrl(host).setAppName("WikipediaPageRank")
+    sparkConf.setMaster(host).setAppName("WikipediaPageRank")
     val sc = new SparkContext(sparkConf)
 
     // Parse the Wikipedia page data into a graph

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala
----------------------------------------------------------------------
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 5bf0b7a..2cf273a 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
@@ -35,7 +35,7 @@ object WikipediaPageRankStandalone {
       System.exit(-1)
     }
     val sparkConf = new SparkConf()
-    sparkConf.set("spark.serializer",  "spark.bagel.examples.WPRSerializer")
+    sparkConf.set("spark.serializer", "spark.bagel.examples.WPRSerializer")
 
 
     val inputFile = args(0)
@@ -44,7 +44,7 @@ object WikipediaPageRankStandalone {
     val host = args(3)
     val usePartitioner = args(4).toBoolean
 
-    sparkConf.setMasterUrl(host).setAppName("WikipediaPageRankStandalone")
+    sparkConf.setMaster(host).setAppName("WikipediaPageRankStandalone")
 
     val sc = new SparkContext(sparkConf)
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala
----------------------------------------------------------------------
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 2f2d106..8b27ecf 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
@@ -579,12 +579,12 @@ object ALS {
     val alpha = if (args.length >= 8) args(7).toDouble else 1
     val blocks = if (args.length == 9) args(8).toInt else -1
     val sc = new SparkContext(master, "ALS")
-    sc.conf.set("spark.serializer",  "org.apache.spark.serializer.KryoSerializer")
+    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")
-    
+    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)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
----------------------------------------------------------------------
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 433268a..91e35e2 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
@@ -45,7 +45,7 @@ import org.apache.spark.util.Utils
 class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) extends Logging {
 
   def this(args: ApplicationMasterArguments) = this(args, new Configuration())
-  
+
   private var rpc: YarnRPC = YarnRPC.create(conf)
   private val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
   private var appAttemptId: ApplicationAttemptId = _
@@ -81,12 +81,12 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     // Workaround until hadoop moves to something which has
     // https://issues.apache.org/jira/browse/HADOOP-8406 - fixed in (2.0.2-alpha but no 0.23 line)
     // org.apache.hadoop.io.compress.CompressionCodecFactory.getCodecClasses(conf)
-    
+
     ApplicationMaster.register(this)
 
     // Start the user's JAR
     userThread = startUserClass()
-    
+
     // This a bit hacky, but we need to wait until the spark.driver.port property has
     // been set by the Thread executing the user class.
     waitForSparkMaster()
@@ -99,7 +99,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     // Allocate all containers
     allocateWorkers()
 
-    // Wait for the user class to Finish     
+    // Wait for the user class to Finish
     userThread.join()
 
     System.exit(0)
@@ -119,7 +119,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     }
     localDirs
   }
-  
+
   private def getApplicationAttemptId(): ApplicationAttemptId = {
     val envs = System.getenv()
     val containerIdString = envs.get(ApplicationConstants.Environment.CONTAINER_ID.name())
@@ -128,17 +128,17 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     logInfo("ApplicationAttemptId: " + appAttemptId)
     appAttemptId
   }
-  
+
   private def registerApplicationMaster(): RegisterApplicationMasterResponse = {
     logInfo("Registering the ApplicationMaster")
     amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress)
   }
-  
+
   private def waitForSparkMaster() {
     logInfo("Waiting for Spark driver to be reachable.")
     var driverUp = false
     var tries = 0
-    val numTries = conf.getOrElse("spark.yarn.applicationMaster.waitTries",  "10").toInt
+    val numTries = conf.getOrElse("spark.yarn.applicationMaster.waitTries", "10").toInt
     while (!driverUp && tries < numTries) {
       val driverHost = conf.get("spark.driver.host")
       val driverPort = conf.get("spark.driver.port")
@@ -199,7 +199,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
       ApplicationMaster.sparkContextRef.synchronized {
         var numTries = 0
         val waitTime = 10000L
-        val maxNumTries = conf.getOrElse("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
@@ -214,7 +214,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
             yarnConf,
             amClient,
             appAttemptId,
-            args, 
+            args,
             sparkContext.preferredNodeLocationData)
         } else {
           logWarning("Unable to retrieve SparkContext inspite of waiting for %d, maxNumTries = %d".
@@ -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 =
-        conf.getOrElse("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)
@@ -314,11 +314,11 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     for (container <- containers) {
       logInfo("Launching shell command on a new container."
         + ", containerId=" + container.getId()
-        + ", containerNode=" + container.getNodeId().getHost() 
+        + ", containerNode=" + container.getNodeId().getHost()
         + ":" + container.getNodeId().getPort()
         + ", containerNodeURI=" + container.getNodeHttpAddress()
         + ", containerState" + container.getState()
-        + ", containerResourceMemory"  
+        + ", containerResourceMemory"
         + container.getResource().getMemory())
     }
   }
@@ -338,12 +338,12 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
   }
 
   /**
-   * Clean up the staging directory. 
+   * Clean up the staging directory.
    */
-  private def cleanupStagingDir() { 
+  private def cleanupStagingDir() {
     var stagingDirPath: Path = null
     try {
-      val preserveFiles = conf.getOrElse("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) {
@@ -359,7 +359,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     }
   }
 
-  // The shutdown hook that runs when a signal is received AND during normal close of the JVM. 
+  // The shutdown hook that runs when a signal is received AND during normal close of the JVM.
   class AppMasterShutdownHook(appMaster: ApplicationMaster) extends Runnable {
 
     def run() {
@@ -415,18 +415,18 @@ object ApplicationMaster {
     // Note that this will unfortunately not properly clean up the staging files because it gets
     // called too late, after the filesystem is already shutdown.
     if (modified) {
-      Runtime.getRuntime().addShutdownHook(new Thread with Logging { 
+      Runtime.getRuntime().addShutdownHook(new Thread with Logging {
         // This is not only logs, but also ensures that log system is initialized for this instance
         // when we are actually 'run'-ing.
         logInfo("Adding shutdown hook for context " + sc)
-        override def run() { 
-          logInfo("Invoking sc stop from shutdown hook") 
-          sc.stop() 
+        override def run() {
+          logInfo("Invoking sc stop from shutdown hook")
+          sc.stop()
           // Best case ...
           for (master <- applicationMasters) {
             master.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED)
           }
-        } 
+        }
       } )
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
----------------------------------------------------------------------
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 a322f60..963b5b8 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
@@ -40,7 +40,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.apache.hadoop.yarn.ipc.YarnRPC
 import org.apache.hadoop.yarn.util.{Apps, Records}
 
-import org.apache.spark.Logging 
+import org.apache.spark.Logging
 import org.apache.spark.util.Utils
 import org.apache.spark.deploy.SparkHadoopUtil
 
@@ -150,7 +150,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
         queueInfo.getChildQueues.size))
   }
 
-  def verifyClusterResources(app: GetNewApplicationResponse) = { 
+  def verifyClusterResources(app: GetNewApplicationResponse) = {
     val maxMem = app.getMaximumResourceCapability().getMemory()
     logInfo("Max mem capabililty of a single resource in this cluster " + maxMem)
 
@@ -221,7 +221,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
       FileUtil.copy(remoteFs, originalPath, fs, newPath, false, conf)
       fs.setReplication(newPath, replication)
       if (setPerms) fs.setPermission(newPath, new FsPermission(APP_FILE_PERMISSION))
-    } 
+    }
     // Resolve any symlinks in the URI path so using a "current" symlink to point to a specific
     // version shows the specific version in the distributed cache configuration
     val qualPath = fs.makeQualified(newPath)
@@ -244,7 +244,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
       }
     }
     val dst = new Path(fs.getHomeDirectory(), appStagingDir)
-    val replication = conf.getOrElse("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)
@@ -269,7 +269,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
         }
         val setPermissions = if (destName.equals(Client.APP_JAR)) true else false
         val destPath = copyRemoteFile(dst, new Path(localURI), replication, setPermissions)
-        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, 
+        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE,
           destName, statCache)
       }
     }
@@ -283,7 +283,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
         val destPath = copyRemoteFile(dst, localPath, replication)
         // Only add the resource to the Spark ApplicationMaster.
         val appMasterOnly = true
-        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, 
+        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE,
           linkname, statCache, appMasterOnly)
       }
     }
@@ -295,7 +295,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
         val localPath = new Path(localURI)
         val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName())
         val destPath = copyRemoteFile(dst, localPath, replication)
-        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, 
+        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE,
           linkname, statCache)
       }
     }
@@ -307,7 +307,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
         val localPath = new Path(localURI)
         val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName())
         val destPath = copyRemoteFile(dst, localPath, replication)
-        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, 
+        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE,
           linkname, statCache)
       }
     }
@@ -317,7 +317,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
   }
 
   def setupLaunchEnv(
-      localResources: HashMap[String, LocalResource], 
+      localResources: HashMap[String, LocalResource],
       stagingDir: String): HashMap[String, String] = {
     logInfo("Setting up the launch environment")
     val log4jConfLocalRes = localResources.getOrElse(Client.LOG4J_PROP, null)
@@ -406,11 +406,11 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     }
 
     val commands = List[String](
-      javaCommand + 
+      javaCommand +
       " -server " +
       JAVA_OPTS +
       " " + args.amClass +
-      " --class " + args.userClass + 
+      " --class " + args.userClass +
       " --jar " + args.userJar +
       userArgsToString(args) +
       " --worker-memory " + args.workerMemory +
@@ -436,7 +436,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     super.submitApplication(appContext)
   }
 
-  def monitorApplication(appId: ApplicationId): Boolean = {  
+  def monitorApplication(appId: ApplicationId): Boolean = {
     while (true) {
       Thread.sleep(1000)
       val report = super.getApplicationReport(appId)
@@ -458,7 +458,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
 
       val state = report.getYarnApplicationState()
       val dsStatus = report.getFinalApplicationStatus()
-      if (state == YarnApplicationState.FINISHED || 
+      if (state == YarnApplicationState.FINISHED ||
         state == YarnApplicationState.FAILED ||
         state == YarnApplicationState.KILLED) {
         return true
@@ -495,25 +495,25 @@ object Client {
     Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$())
     // If log4j present, ensure ours overrides all others
     if (addLog4j) {
-      Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + 
+      Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
         Path.SEPARATOR + LOG4J_PROP)
     }
     // Normally the users app.jar is last in case conflicts with spark jars
-    val userClasspathFirst = conf.getOrElse("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.$() + 
+      Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
         Path.SEPARATOR + APP_JAR)
     }
-    Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + 
+    Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
       Path.SEPARATOR + SPARK_JAR)
     Client.populateHadoopClasspath(conf, env)
 
     if (!userClasspathFirst) {
-      Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + 
+      Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
         Path.SEPARATOR + APP_JAR)
     }
-    Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + 
+    Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
       Path.SEPARATOR + "*")
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
----------------------------------------------------------------------
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 41ac292..1a9bb97 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 = conf.getOrElse("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"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala
----------------------------------------------------------------------
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 b2f499e..f108c70 100644
--- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala
@@ -35,6 +35,7 @@ import java.lang.{Class => jClass}
 import scala.reflect.api.{Mirror, TypeCreator, Universe => ApiUniverse}
 
 import org.apache.spark.Logging
+import org.apache.spark.SparkConf
 import org.apache.spark.SparkContext
 
 /** The Scala interactive shell.  It provides a read-eval-print loop
@@ -929,7 +930,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
   }
 
   def createSparkContext(): SparkContext = {
-    val uri = System.getenv("SPARK_EXECUTOR_URI")
+    val execUri = System.getenv("SPARK_EXECUTOR_URI")
     val master = this.master match {
       case Some(m) => m
       case None => {
@@ -938,11 +939,16 @@ 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)
+    val conf = new SparkConf()
+      .setMaster(master)
+      .setAppName("Spark shell")
+      .setSparkHome(System.getenv("SPARK_HOME"))
+      .setJars(jars)
+      .set("spark.repl.class.uri", intp.classServer.uri)
+    if (execUri != null) {
+      conf.set("spark.executor.uri", execUri)
     }
-    sparkContext.conf.set("spark.repl.class.uri",  intp.classServer.uri)
+    sparkContext = new SparkContext(conf)
     echo("Created spark context..")
     sparkContext
   }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala
----------------------------------------------------------------------
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 0d412e4..a993083 100644
--- a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala
@@ -34,7 +34,7 @@ import scala.tools.reflect.StdRuntimeTags._
 import scala.util.control.ControlThrowable
 import util.stackTraceString
 
-import org.apache.spark.{SparkContext, HttpServer, SparkEnv, Logging}
+import org.apache.spark.{HttpServer, SparkConf, Logging}
 import org.apache.spark.util.Utils
 
 // /** directory to save .class files to */
@@ -89,7 +89,7 @@ import org.apache.spark.util.Utils
       /** Local directory to save .class files too */
       val outputDir = {
         val tmp = System.getProperty("java.io.tmpdir")
-        val rootDir = SparkContext.globalConf.getOrElse("spark.repl.classdir",  tmp)
+        val rootDir = new SparkConf().getOrElse("spark.repl.classdir",  tmp)
         Utils.createTempDir(rootDir)
       }
       if (SPARK_DEBUG_REPL) {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
----------------------------------------------------------------------
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 b8e1427..f106bba 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
@@ -24,7 +24,7 @@ import java.util.concurrent.RejectedExecutionException
 import org.apache.hadoop.fs.Path
 import org.apache.hadoop.conf.Configuration
 
-import org.apache.spark.Logging
+import org.apache.spark.{SparkConf, Logging}
 import org.apache.spark.io.CompressionCodec
 import org.apache.spark.util.MetadataCleaner
 
@@ -36,12 +36,11 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time)
   val framework = ssc.sc.appName
   val sparkHome = ssc.sc.getSparkHome.getOrElse(null)
   val jars = ssc.sc.jars
-  val environment = ssc.sc.environment
   val graph = ssc.graph
   val checkpointDir = ssc.checkpointDir
   val checkpointDuration = ssc.checkpointDuration
   val pendingTimes = ssc.scheduler.jobManager.getPendingTimes()
-  val delaySeconds = MetadataCleaner.getDelaySeconds
+  val delaySeconds = MetadataCleaner.getDelaySeconds(ssc.conf)
   val sparkConf = ssc.sc.conf
 
   def validate() {
@@ -58,7 +57,7 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time)
  * Convenience class to speed up the writing of graph checkpoint to file
  */
 private[streaming]
-class CheckpointWriter(checkpointDir: String) extends Logging {
+class CheckpointWriter(conf: SparkConf, checkpointDir: String) extends Logging {
   val file = new Path(checkpointDir, "graph")
   // The file to which we actually write - and then "move" to file.
   private val writeFile = new Path(file.getParent, file.getName + ".next")
@@ -66,14 +65,14 @@ class CheckpointWriter(checkpointDir: String) extends Logging {
 
   private var stopped = false
 
-  val conf = new Configuration()
-  var fs = file.getFileSystem(conf)
+  val hadoopConf = new Configuration()
+  var fs = file.getFileSystem(hadoopConf)
   val maxAttempts = 3
   val executor = Executors.newFixedThreadPool(1)
 
-  private val compressionCodec = CompressionCodec.createCodec()
+  private val compressionCodec = CompressionCodec.createCodec(conf)
 
-  // Removed code which validates whether there is only one CheckpointWriter per path 'file' since 
+  // Removed code which validates whether there is only one CheckpointWriter per path 'file' since
   // I did not notice any errors - reintroduce it ?
 
   class CheckpointWriteHandler(checkpointTime: Time, bytes: Array[Byte]) extends Runnable {
@@ -142,11 +141,12 @@ class CheckpointWriter(checkpointDir: String) extends Logging {
 private[streaming]
 object CheckpointReader extends Logging {
 
-  def read(path: String): Checkpoint = {
+  def read(conf: SparkConf, path: String): Checkpoint = {
     val fs = new Path(path).getFileSystem(new Configuration())
-    val attempts = Seq(new Path(path, "graph"), new Path(path, "graph.bk"), new Path(path), new Path(path + ".bk"))
+    val attempts = Seq(
+      new Path(path, "graph"), new Path(path, "graph.bk"), new Path(path), new Path(path + ".bk"))
 
-    val compressionCodec = CompressionCodec.createCodec()
+    val compressionCodec = CompressionCodec.createCodec(conf)
 
     attempts.foreach(file => {
       if (fs.exists(file)) {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala
index 329d2b5..8005202 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala
@@ -213,7 +213,7 @@ abstract class DStream[T: ClassTag] (
         checkpointDuration + "). Please set it to higher than " + checkpointDuration + "."
     )
 
-    val metadataCleanerDelay = MetadataCleaner.getDelaySeconds
+    val metadataCleanerDelay = MetadataCleaner.getDelaySeconds(ssc.conf)
     logInfo("metadataCleanupDelay = " + metadataCleanerDelay)
     assert(
       metadataCleanerDelay < 0 || rememberDuration.milliseconds < metadataCleanerDelay * 1000,


[13/33] git commit: Fix Python use of getLocalDir

Posted by pw...@apache.org.
Fix Python use of getLocalDir


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/1c11f54a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/1c11f54a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/1c11f54a

Branch: refs/heads/master
Commit: 1c11f54a9b7340ccfa7bf7236fbcd210b77ae0a8
Parents: 2063134
Author: Matei Zaharia <ma...@databricks.com>
Authored: Sun Dec 29 00:11:36 2013 -0500
Committer: Matei Zaharia <ma...@databricks.com>
Committed: Sun Dec 29 00:11:36 2013 -0500

----------------------------------------------------------------------
 python/pyspark/context.py | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1c11f54a/python/pyspark/context.py
----------------------------------------------------------------------
diff --git a/python/pyspark/context.py b/python/pyspark/context.py
index 0604f68..c0645b2 100644
--- a/python/pyspark/context.py
+++ b/python/pyspark/context.py
@@ -121,7 +121,7 @@ class SparkContext(object):
             self.addPyFile(path)
 
         # Create a temporary directory inside spark.local.dir:
-        local_dir = self._jvm.org.apache.spark.util.Utils.getLocalDir()
+        local_dir = self._jvm.org.apache.spark.util.Utils.getLocalDir(self._jsc.sc().conf())
         self._temp_dir = \
             self._jvm.org.apache.spark.util.Utils.createTempDir(local_dir).getAbsolutePath()
 


[29/33] git commit: Merge remote-tracking branch 'apache/master' into conf2

Posted by pw...@apache.org.
Merge remote-tracking branch 'apache/master' into conf2

Conflicts:
	core/src/main/scala/org/apache/spark/SparkContext.scala
	core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
	core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/45ff8f41
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/45ff8f41
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/45ff8f41

Branch: refs/heads/master
Commit: 45ff8f413d9959b7f464176cd20dc56db3f711af
Parents: 0e5b2ad c1d928a
Author: Matei Zaharia <ma...@databricks.com>
Authored: Wed Jan 1 21:25:00 2014 -0500
Committer: Matei Zaharia <ma...@databricks.com>
Committed: Wed Jan 1 21:25:00 2014 -0500

----------------------------------------------------------------------
 .../org/apache/spark/default-log4j.properties   |  8 ++++
 .../scala/org/apache/spark/HttpServer.scala     |  1 +
 .../main/scala/org/apache/spark/Logging.scala   | 41 +++++++++++++++++---
 .../scala/org/apache/spark/SparkContext.scala   |  3 --
 .../org/apache/spark/executor/Executor.scala    |  2 -
 .../apache/spark/metrics/MetricsConfig.scala    |  1 -
 .../apache/spark/metrics/MetricsSystem.scala    |  1 -
 .../spark/storage/BlockManagerMasterActor.scala |  2 -
 .../spark/storage/BlockManagerWorker.scala      |  3 --
 .../spark/storage/BlockMessageArray.scala       |  2 -
 .../org/apache/spark/CheckpointSuite.scala      |  2 -
 spark-class                                     |  9 ++++-
 .../org/apache/spark/streaming/DStream.scala    |  2 -
 .../apache/spark/streaming/DStreamGraph.scala   |  1 -
 .../spark/streaming/StreamingContext.scala      |  2 -
 .../streaming/dstream/NetworkInputDStream.scala |  2 -
 .../streaming/scheduler/JobGenerator.scala      |  2 -
 .../streaming/scheduler/JobScheduler.scala      |  2 -
 .../streaming/util/MasterFailureTest.scala      |  3 --
 19 files changed, 52 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/45ff8f41/core/src/main/scala/org/apache/spark/SparkContext.scala
----------------------------------------------------------------------
diff --cc core/src/main/scala/org/apache/spark/SparkContext.scala
index fbc7a78,d3903ce..46874c4
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@@ -55,99 -68,43 +55,96 @@@ 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 environment Environment variables to set on worker nodes.
 + * @param config a Spark Config object describing the application configuration. Any settings in
 + *   this config overrides the default configs as well as system properties.
 + * @param preferredNodeLocationData used in YARN mode to select nodes to launch containers on. Can
 + *   be generated using [[org.apache.spark.scheduler.InputFormatInfo.computePreferredLocations]]
 + *   from a list of input files or InputFormats for the application.
   */
  class SparkContext(
 -    val master: String,
 -    val appName: String,
 -    val sparkHome: String = null,
 -    val jars: Seq[String] = Nil,
 -    val environment: Map[String, String] = Map(),
 +    config: SparkConf,
      // 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())
 +    // too. This is typically generated from InputFormatInfo.computePreferredLocations. It contains
 +    // a map from hostname to a list of input format splits on the host.
 +    val preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map())
    extends Logging {
  
 -  // Set Spark driver host and port system properties
 -  if (System.getProperty("spark.driver.host") == null) {
 -    System.setProperty("spark.driver.host", Utils.localHostName())
 +  /**
 +   * Alternative constructor that allows setting common Spark properties directly
 +   *
 +   * @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 conf a [[org.apache.spark.SparkConf]] object specifying other Spark parameters
 +   */
 +  def this(master: String, appName: String, conf: SparkConf) =
 +    this(conf.clone().setMaster(master).setAppName(appName))
 +
 +  /**
 +   * Alternative constructor that allows setting common Spark properties directly
 +   *
 +   * @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: Map[String, Set[SplitInfo]] = Map()) =
 +  {
 +    this(
 +      new SparkConf()
 +        .setMaster(master)
 +        .setAppName(appName)
 +        .setJars(jars)
 +        .setExecutorEnv(environment.toSeq)
 +        .setSparkHome(sparkHome),
 +      preferredNodeLocationData)
    }
 -  if (System.getProperty("spark.driver.port") == null) {
 -    System.setProperty("spark.driver.port", "0")
 +
 +  private[spark] val conf = config.clone()
 +
 +  /**
 +   * Return a copy of this SparkContext's configuration. The configuration ''cannot'' be
 +   * changed at runtime.
 +   */
 +  def getConf: SparkConf = conf.clone()
 +
 +  if (!conf.contains("spark.master")) {
 +    throw new SparkException("A master URL must be set in your configuration")
 +  }
 +  if (!conf.contains("spark.app.name")) {
 +    throw new SparkException("An application must be set in your configuration")
 +  }
 +
 +  // Set Spark driver host and port system properties
 +  conf.setIfMissing("spark.driver.host", Utils.localHostName())
 +  conf.setIfMissing("spark.driver.port", "0")
 +
 +  val jars: Seq[String] = if (conf.contains("spark.jars")) {
 +    conf.get("spark.jars").split(",").filter(_.size != 0)
 +  } else {
 +    null
    }
  
 +  val master = conf.get("spark.master")
 +  val appName = conf.get("spark.app.name")
 +
    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(
 +  private[spark] val env = SparkEnv.create(
 +    conf,
      "<driver>",
 -    System.getProperty("spark.driver.host"),
 -    System.getProperty("spark.driver.port").toInt,
 -    true,
 -    isLocal)
 +    conf.get("spark.driver.host"),
 +    conf.get("spark.driver.port").toInt,
 +    isDriver = true,
 +    isLocal = isLocal)
    SparkEnv.set(env)
  
    // Used to store a URL for each static file/jar together with the file's local timestamp

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/45ff8f41/core/src/main/scala/org/apache/spark/executor/Executor.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/45ff8f41/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
----------------------------------------------------------------------
diff --cc core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
index ac29816,8e038ce..0e41c73
--- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
+++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
@@@ -62,11 -62,9 +62,10 @@@ import org.apache.spark.metrics.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]

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/45ff8f41/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
----------------------------------------------------------------------
diff --cc core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
index 73a1da2,e0eb02c..dbbeeb3
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
@@@ -48,14 -48,12 +48,12 @@@ class BlockManagerMasterActor(val isLoc
    // 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",
 -    "" + (BlockManager.getHeartBeatFrequencyFromSystemProperties * 3)).toLong
 +  val slaveTimeout = conf.getOrElse("spark.storage.blockManagerSlaveTimeoutMs",
 +    "" + (BlockManager.getHeartBeatFrequency(conf) * 3)).toLong
  
 -  val checkTimeoutInterval = System.getProperty("spark.storage.blockManagerTimeoutIntervalMs",
 +  val checkTimeoutInterval = conf.getOrElse("spark.storage.blockManagerTimeoutIntervalMs",
      "60000").toLong
  
    var timeoutCheckingTask: Cancellable = null

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/45ff8f41/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/45ff8f41/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
----------------------------------------------------------------------
diff --cc streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
index 0566704,659da26..304986f
--- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
@@@ -101,10 -92,8 +101,8 @@@ class StreamingContext private 
     * @param path Path either to the directory that was specified as the checkpoint directory, or
     *             to the checkpoint file 'graph' or 'graph.bk'.
     */
 -  def this(path: String) = this(null, CheckpointReader.read(path), null)
 +  def this(path: String) = this(null, CheckpointReader.read(new SparkConf(), path), null)
  
-   initLogging()
- 
    if (sc_ == null && cp_ == null) {
      throw new Exception("Spark Streaming cannot be initialized with " +
        "both SparkContext and checkpoint as null")

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/45ff8f41/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/45ff8f41/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/45ff8f41/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala
----------------------------------------------------------------------
diff --cc streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala
index 7fd8d41,488cc2f..651cdaa
--- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala
@@@ -30,10 -30,8 +30,8 @@@ import org.apache.spark.streaming.
  private[streaming]
  class JobScheduler(val ssc: StreamingContext) extends Logging {
  
-   initLogging()
- 
    val jobSets = new ConcurrentHashMap[Time, JobSet]
 -  val numConcurrentJobs = System.getProperty("spark.streaming.concurrentJobs", "1").toInt
 +  val numConcurrentJobs = ssc.conf.getOrElse("spark.streaming.concurrentJobs", "1").toInt
    val executor = Executors.newFixedThreadPool(numConcurrentJobs)
    val generator = new JobGenerator(this)
    val listenerBus = new StreamingListenerBus()


[31/33] git commit: Fixed two uses of conf.get with no default value in Mesos

Posted by pw...@apache.org.
Fixed two uses of conf.get with no default value in Mesos


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/0f606073
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/0f606073
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/0f606073

Branch: refs/heads/master
Commit: 0f6060733da83a862038fd397875cdb49d8c144d
Parents: e2c6864
Author: Matei Zaharia <ma...@databricks.com>
Authored: Wed Jan 1 22:09:42 2014 -0500
Committer: Matei Zaharia <ma...@databricks.com>
Committed: Wed Jan 1 22:09:42 2014 -0500

----------------------------------------------------------------------
 .../scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala      | 2 +-
 .../spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala      | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0f606073/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
----------------------------------------------------------------------
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 d247fa4..0881152 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
@@ -125,7 +125,7 @@ private[spark] class CoarseMesosSchedulerBackend(
       conf.get("spark.driver.host"),
       conf.get("spark.driver.port"),
       CoarseGrainedSchedulerBackend.ACTOR_NAME)
-    val uri = conf.get("spark.executor.uri")
+    val uri = conf.get("spark.executor.uri", null)
     if (uri == null) {
       val runScript = new File(sparkHome, "spark-class").getCanonicalPath
       command.setValue(

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0f606073/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
----------------------------------------------------------------------
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 c20fc41..bb278fb 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
@@ -100,7 +100,7 @@ private[spark] class MesosSchedulerBackend(
     }
     val command = CommandInfo.newBuilder()
       .setEnvironment(environment)
-    val uri = sc.conf.get("spark.executor.uri")
+    val uri = sc.conf.get("spark.executor.uri", null)
     if (uri == null) {
       command.setValue(new File(sparkHome, "spark-executor").getCanonicalPath)
     } else {


[20/33] git commit: Added tests for SparkConf and fixed a bug

Posted by pw...@apache.org.
Added tests for SparkConf and fixed a bug

Typesafe Config caches system properties the first time it's invoked
by default, ignoring later changes unless you do something special


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/11540b79
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/11540b79
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/11540b79

Branch: refs/heads/master
Commit: 11540b798d622f3883cb40b20cc30ea7d894790a
Parents: 1ee7f5a
Author: Matei Zaharia <ma...@databricks.com>
Authored: Sun Dec 29 18:44:06 2013 -0500
Committer: Matei Zaharia <ma...@databricks.com>
Committed: Sun Dec 29 18:44:06 2013 -0500

----------------------------------------------------------------------
 .../main/scala/org/apache/spark/SparkConf.scala |   1 +
 core/src/test/resources/spark.conf              |   6 +
 .../scala/org/apache/spark/SparkConfSuite.scala | 110 +++++++++++++++++++
 3 files changed, 117 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/11540b79/core/src/main/scala/org/apache/spark/SparkConf.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
index 670c8b4..8cecaff 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -26,6 +26,7 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable {
   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.")) {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/11540b79/core/src/test/resources/spark.conf
----------------------------------------------------------------------
diff --git a/core/src/test/resources/spark.conf b/core/src/test/resources/spark.conf
new file mode 100644
index 0000000..6c99bdc
--- /dev/null
+++ b/core/src/test/resources/spark.conf
@@ -0,0 +1,6 @@
+spark.test.intTestProperty = 1
+
+spark.test {
+  stringTestProperty = "hi"
+  listTestProperty = ["a", "b"]
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/11540b79/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
new file mode 100644
index 0000000..aaf0b80
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
@@ -0,0 +1,110 @@
+package org.apache.spark
+
+import org.scalatest.FunSuite
+
+class SparkConfSuite extends FunSuite with LocalSparkContext {
+  // This test uses the spark.conf in core/src/test/resources, which has a few test properties
+  test("loading from spark.conf") {
+    val conf = new SparkConf()
+    assert(conf.get("spark.test.intTestProperty") === "1")
+    assert(conf.get("spark.test.stringTestProperty") === "hi")
+    // NOTE: we don't use list properties yet, but when we do, we'll have to deal with this syntax
+    assert(conf.get("spark.test.listTestProperty") === "[a, b]")
+  }
+
+  // This test uses the spark.conf in core/src/test/resources, which has a few test properties
+  test("system properties override spark.conf") {
+    try {
+      System.setProperty("spark.test.intTestProperty", "2")
+      val conf = new SparkConf()
+      assert(conf.get("spark.test.intTestProperty") === "2")
+      assert(conf.get("spark.test.stringTestProperty") === "hi")
+    } finally {
+      System.clearProperty("spark.test.intTestProperty")
+    }
+  }
+
+  test("initializing without loading defaults") {
+    try {
+      System.setProperty("spark.test.intTestProperty", "2")
+      val conf = new SparkConf(false)
+      assert(!conf.contains("spark.test.intTestProperty"))
+      assert(!conf.contains("spark.test.stringTestProperty"))
+    } finally {
+      System.clearProperty("spark.test.intTestProperty")
+    }
+  }
+
+  test("named set methods") {
+    val conf = new SparkConf(false)
+
+    conf.setMaster("local[3]")
+    conf.setAppName("My app")
+    conf.setSparkHome("/path")
+    conf.setJars(Seq("a.jar", "b.jar"))
+    conf.setExecutorEnv("VAR1", "value1")
+    conf.setExecutorEnv(Seq(("VAR2", "value2"), ("VAR3", "value3")))
+
+    assert(conf.get("spark.master") === "local[3]")
+    assert(conf.get("spark.appName") === "My app")
+    assert(conf.get("spark.home") === "/path")
+    assert(conf.get("spark.jars") === "a.jar,b.jar")
+    assert(conf.get("spark.executorEnv.VAR1") === "value1")
+    assert(conf.get("spark.executorEnv.VAR2") === "value2")
+    assert(conf.get("spark.executorEnv.VAR3") === "value3")
+
+    // Test the Java-friendly versions of these too
+    conf.setJars(Array("c.jar", "d.jar"))
+    conf.setExecutorEnv(Array(("VAR4", "value4"), ("VAR5", "value5")))
+    assert(conf.get("spark.jars") === "c.jar,d.jar")
+    assert(conf.get("spark.executorEnv.VAR4") === "value4")
+    assert(conf.get("spark.executorEnv.VAR5") === "value5")
+  }
+
+  test("basic get and set") {
+    val conf = new SparkConf(false)
+    assert(conf.getAll.toSet === Set())
+    conf.set("k1", "v1")
+    conf.setAll(Seq(("k2", "v2"), ("k3", "v3")))
+    assert(conf.getAll.toSet === Set(("k1", "v1"), ("k2", "v2"), ("k3", "v3")))
+    conf.set("k1", "v4")
+    conf.setAll(Seq(("k2", "v5"), ("k3", "v6")))
+    assert(conf.getAll.toSet === Set(("k1", "v4"), ("k2", "v5"), ("k3", "v6")))
+    assert(conf.contains("k1"), "conf did not contain k1")
+    assert(!conf.contains("k4"), "conf contained k4")
+    assert(conf.get("k1") === "v4")
+    intercept[Exception] { conf.get("k4") }
+    assert(conf.getOrElse("k4", "not found") === "not found")
+    assert(conf.getOption("k1") === Some("v4"))
+    assert(conf.getOption("k4") === None)
+  }
+
+  test("creating SparkContext without master and app name") {
+    val conf = new SparkConf(false)
+    intercept[SparkException] { sc = new SparkContext(conf) }
+  }
+
+  test("creating SparkContext without master") {
+    val conf = new SparkConf(false).setAppName("My app")
+    intercept[SparkException] { sc = new SparkContext(conf) }
+  }
+
+  test("creating SparkContext without app name") {
+    val conf = new SparkConf(false).setMaster("local")
+    intercept[SparkException] { sc = new SparkContext(conf) }
+  }
+
+  test("creating SparkContext with both master and app name") {
+    val conf = new SparkConf(false).setMaster("local").setAppName("My app")
+    sc = new SparkContext(conf)
+    assert(sc.master === "local")
+    assert(sc.appName === "My app")
+  }
+
+  test("SparkContext property overriding") {
+    val conf = new SparkConf(false).setMaster("local").setAppName("My app")
+    sc = new SparkContext("local[2]", "My other app", conf)
+    assert(sc.master === "local[2]")
+    assert(sc.appName === "My other app")
+  }
+}


[16/33] git commit: Add Python docs about SparkConf

Posted by pw...@apache.org.
Add Python docs about SparkConf


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/58c6fa20
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/58c6fa20
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/58c6fa20

Branch: refs/heads/master
Commit: 58c6fa2041b99160f254b17c2b71de9d82c53f8c
Parents: 615fb64
Author: Matei Zaharia <ma...@databricks.com>
Authored: Sun Dec 29 14:46:59 2013 -0500
Committer: Matei Zaharia <ma...@databricks.com>
Committed: Sun Dec 29 14:46:59 2013 -0500

----------------------------------------------------------------------
 python/pyspark/conf.py    | 42 ++++++++++++++++++++++++++++++++++++++++++
 python/pyspark/context.py |  3 ++-
 2 files changed, 44 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/58c6fa20/python/pyspark/conf.py
----------------------------------------------------------------------
diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py
index eb7a6c1..a79f348 100644
--- a/python/pyspark/conf.py
+++ b/python/pyspark/conf.py
@@ -50,29 +50,62 @@ u'value1'
 
 
 class SparkConf(object):
+    """
+    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
+    C{SparkConf()}, which will load values from `spark.*` Java system
+    properties and any `spark.conf` on your application's classpath.
+    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 C{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 C{conf.setMaster("local").setAppName("My app")}.
+    """
+
     def __init__(self, loadDefaults=True, _jvm=None):
+        """
+        Create a new Spark configuration.
+
+        @param loadDefaults: whether to load values from Java system
+               properties and classpath (true by default)
+        """
         from pyspark.context import SparkContext
         SparkContext._ensure_initialized()
         _jvm = _jvm or SparkContext._jvm
         self._jconf = _jvm.SparkConf(loadDefaults)
 
     def set(self, key, value):
+        """Set a configuration property."""
         self._jconf.set(key, value)
         return self
 
     def setMaster(self, value):
+        """Set master URL to connect to."""
         self._jconf.setMaster(value)
         return self
 
     def setAppName(self, value):
+        """Set application name."""
         self._jconf.setAppName(value)
         return self
 
     def setSparkHome(self, value):
+        """
+        Set path where Spark is installed on worker nodes (needed for some
+        deployment modes).
+        """
         self._jconf.setSparkHome(value)
         return self
 
     def setExecutorEnv(self, key=None, value=None, pairs=None):
+        """Set an environment variable to be passed to executors."""
         if (key != None and pairs != None) or (key == None and pairs == None):
             raise Exception("Either pass one key-value pair or a list of pairs")
         elif key != None:
@@ -83,23 +116,32 @@ class SparkConf(object):
         return self
 
     def setAll(self, pairs):
+        """
+        Set multiple parameters, passed as a list of key-value pairs.
+
+        @param pairs: list of key-value pairs to set
+        """
         for (k, v) in pairs:
             self._jconf.set(k, v)
         return self
 
     def get(self, key):
+        """Get the configured value for some key, if set."""
         return self._jconf.get(key)
 
     def getOrElse(self, key, defaultValue):
+        """Get the value for some key, or return a default otherwise."""
         return self._jconf.getOrElse(key, defaultValue)
 
     def getAll(self):
+        """Get all values as a list of key-value pairs."""
         pairs = []
         for elem in self._jconf.getAll():
             pairs.append((elem._1(), elem._2()))
         return pairs
 
     def contains(self, key):
+        """Does this configuration contain a given key?"""
         return self._jconf.contains(key)
 
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/58c6fa20/python/pyspark/context.py
----------------------------------------------------------------------
diff --git a/python/pyspark/context.py b/python/pyspark/context.py
index 9d75c2b..1244a14 100644
--- a/python/pyspark/context.py
+++ b/python/pyspark/context.py
@@ -53,7 +53,8 @@ class SparkContext(object):
     def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None,
         environment=None, batchSize=1024, serializer=PickleSerializer(), conf=None):
         """
-        Create a new SparkContext.
+        Create a new SparkContext. At least the master and app name should be set,
+        either through the named parameters here or through C{conf}.
 
         @param master: Cluster URL to connect to
                (e.g. mesos://host:port, spark://host:port, local[4]).


[06/33] git commit: Various fixes to configuration code

Posted by pw...@apache.org.
Various fixes to configuration code

- Got rid of global SparkContext.globalConf
- Pass 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).


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/642029e7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/642029e7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/642029e7

Branch: refs/heads/master
Commit: 642029e7f43322f84abe4f7f36bb0b1b95d8101d
Parents: 2573add
Author: Matei Zaharia <ma...@databricks.com>
Authored: Sat Dec 28 17:13:15 2013 -0500
Committer: Matei Zaharia <ma...@databricks.com>
Committed: Sat Dec 28 17:13:15 2013 -0500

----------------------------------------------------------------------
 .../scala/org/apache/spark/Accumulators.scala   |   8 +-
 .../org/apache/spark/MapOutputTracker.scala     |   4 +-
 .../scala/org/apache/spark/Partitioner.scala    |   6 +-
 .../main/scala/org/apache/spark/SparkConf.scala | 158 +++++++++++++++----
 .../scala/org/apache/spark/SparkContext.scala   | 138 ++++++++--------
 .../main/scala/org/apache/spark/SparkEnv.scala  |  11 +-
 .../spark/api/java/JavaSparkContext.scala       |  15 +-
 .../org/apache/spark/api/python/PythonRDD.scala |   6 +-
 .../apache/spark/broadcast/HttpBroadcast.scala  |  33 ++--
 .../spark/broadcast/TorrentBroadcast.scala      |  38 ++---
 .../spark/deploy/ApplicationDescription.scala   |   2 +-
 .../apache/spark/deploy/LocalSparkCluster.scala |   7 +-
 .../apache/spark/deploy/SparkHadoopUtil.scala   |  14 +-
 .../apache/spark/deploy/client/TestClient.scala |   9 +-
 .../org/apache/spark/deploy/master/Master.scala |  36 +++--
 .../deploy/master/SparkZooKeeperSession.scala   |   2 +-
 .../master/ZooKeeperLeaderElectionAgent.scala   |   2 +-
 .../master/ZooKeeperPersistenceEngine.scala     |   2 +-
 .../org/apache/spark/deploy/worker/Worker.scala |  14 +-
 .../executor/CoarseGrainedExecutorBackend.scala |   4 +-
 .../org/apache/spark/executor/Executor.scala    |  17 +-
 .../org/apache/spark/io/CompressionCodec.scala  |  13 +-
 .../spark/network/ConnectionManager.scala       |   4 +-
 .../org/apache/spark/network/ReceiverTest.scala |  12 +-
 .../org/apache/spark/network/SenderTest.scala   |  16 +-
 .../spark/network/netty/ShuffleCopier.scala     |   6 +-
 .../org/apache/spark/rdd/CheckpointRDD.scala    |   7 +-
 .../org/apache/spark/rdd/CoGroupedRDD.scala     |   2 +-
 .../org/apache/spark/rdd/ShuffledRDD.scala      |   2 +-
 .../org/apache/spark/rdd/SubtractedRDD.scala    |   2 +-
 .../apache/spark/scheduler/DAGScheduler.scala   |   5 +-
 .../spark/scheduler/InputFormatInfo.scala       |  14 +-
 .../org/apache/spark/scheduler/JobLogger.scala  |   2 +-
 .../org/apache/spark/scheduler/ResultTask.scala |   4 +-
 .../spark/scheduler/SchedulableBuilder.scala    |   2 +-
 .../apache/spark/scheduler/ShuffleMapTask.scala |   6 +-
 .../scheduler/cluster/ClusterScheduler.scala    |   8 +-
 .../cluster/ClusterTaskSetManager.scala         |  12 +-
 .../cluster/CoarseGrainedSchedulerBackend.scala |   9 +-
 .../scheduler/cluster/SchedulerBackend.scala    |   3 -
 .../cluster/SimrSchedulerBackend.scala          |   2 +-
 .../cluster/SparkDeploySchedulerBackend.scala   |   2 +-
 .../scheduler/cluster/TaskResultGetter.scala    |   2 +-
 .../mesos/CoarseMesosSchedulerBackend.scala     |   6 +-
 .../cluster/mesos/MesosSchedulerBackend.scala   |   6 +-
 .../spark/scheduler/local/LocalScheduler.scala  |   2 +-
 .../spark/serializer/JavaSerializer.scala       |   3 +-
 .../spark/serializer/KryoSerializer.scala       |  13 +-
 .../spark/serializer/SerializerManager.scala    |  12 +-
 .../spark/storage/BlockFetcherIterator.scala    |   2 +-
 .../org/apache/spark/storage/BlockManager.scala |  46 +++---
 .../spark/storage/BlockManagerMaster.scala      |   4 +-
 .../spark/storage/BlockManagerMasterActor.scala |   4 +-
 .../apache/spark/storage/DiskBlockManager.scala |   2 +-
 .../spark/storage/ShuffleBlockManager.scala     |   9 +-
 .../spark/storage/StoragePerfTester.scala       |   2 +-
 .../apache/spark/storage/ThreadingTest.scala    |   6 +-
 .../apache/spark/ui/UIWorkloadGenerator.scala   |  17 +-
 .../org/apache/spark/ui/env/EnvironmentUI.scala |   2 +-
 .../spark/ui/jobs/JobProgressListener.scala     |   4 +-
 .../scala/org/apache/spark/util/AkkaUtils.scala |  18 +--
 .../org/apache/spark/util/MetadataCleaner.scala |  33 ++--
 .../org/apache/spark/util/SizeEstimator.scala   |  17 +-
 .../scala/org/apache/spark/util/Utils.scala     |  14 +-
 .../apache/spark/io/CompressionCodecSuite.scala |   8 +-
 .../cluster/ClusterTaskSetManagerSuite.scala    |   2 +-
 .../spark/serializer/KryoSerializerSuite.scala  |  14 +-
 .../spark/storage/BlockManagerSuite.scala       |   8 +-
 .../apache/spark/util/SizeEstimatorSuite.scala  |   2 -
 .../examples/bagel/WikipediaPageRank.scala      |   4 +-
 .../bagel/WikipediaPageRankStandalone.scala     |   4 +-
 .../apache/spark/mllib/recommendation/ALS.scala |  10 +-
 .../spark/deploy/yarn/ApplicationMaster.scala   |  44 +++---
 .../org/apache/spark/deploy/yarn/Client.scala   |  38 ++---
 .../spark/deploy/yarn/ClientArguments.scala     |   2 +-
 .../org/apache/spark/repl/SparkILoop.scala      |  16 +-
 .../org/apache/spark/repl/SparkIMain.scala      |   4 +-
 .../org/apache/spark/streaming/Checkpoint.scala |  22 +--
 .../org/apache/spark/streaming/DStream.scala    |   2 +-
 .../org/apache/spark/streaming/Scheduler.scala  |  10 +-
 .../spark/streaming/StreamingContext.scala      |  25 +--
 .../streaming/dstream/NetworkInputDStream.scala |   6 +-
 .../spark/streaming/util/RawTextSender.scala    |   4 +-
 .../spark/streaming/InputStreamsSuite.scala     |   6 +-
 .../apache/spark/streaming/TestSuiteBase.scala  |   6 +-
 .../spark/deploy/yarn/ApplicationMaster.scala   |  56 +++----
 .../org/apache/spark/deploy/yarn/Client.scala   |  50 +++---
 .../spark/deploy/yarn/ClientArguments.scala     |   2 +-
 88 files changed, 692 insertions(+), 536 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/Accumulators.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala
index 6e922a6..5f73d23 100644
--- a/core/src/main/scala/org/apache/spark/Accumulators.scala
+++ b/core/src/main/scala/org/apache/spark/Accumulators.scala
@@ -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
@@ -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()
@@ -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
@@ -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

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
index 4520edb..cdae167 100644
--- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
+++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
@@ -65,7 +65,7 @@ private[spark] class MapOutputTracker(conf: SparkConf) 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.
@@ -129,7 +129,7 @@ private[spark] class MapOutputTracker(conf: SparkConf) 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 =

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/Partitioner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala
index 04c1eed..7cb545a 100644
--- a/core/src/main/scala/org/apache/spark/Partitioner.scala
+++ b/core/src/main/scala/org/apache/spark/Partitioner.scala
@@ -32,8 +32,6 @@ 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.
    *
@@ -54,7 +52,7 @@ object Partitioner {
     for (r <- bySize if r.partitioner != None) {
       return r.partitioner.get
     }
-    if (conf.getOrElse("spark.default.parallelism", null) != null) {
+    if (rdd.context.conf.getOrElse("spark.default.parallelism", null) != null) {
       return new HashPartitioner(rdd.context.defaultParallelism)
     } else {
       return new HashPartitioner(bySize.head.partitions.size)
@@ -92,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

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/SparkConf.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
index 9a4eefa..185ddb1 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -1,71 +1,159 @@
 package org.apache.spark
 
-import scala.collection.JavaConversions._
-import scala.collection.concurrent.TrieMap
+import scala.collection.JavaConverters._
+import scala.collection.mutable.HashMap
 
 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]()
+/**
+ * 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.
+ *
+ * @param loadDefaults whether to load values from the system properties and classpath
+ */
+class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable {
 
-  if (loadClasspathRes && !config.entrySet().isEmpty) {
-    for (e <- config.entrySet()) {
-      configMap += ((e.getKey, e.getValue.unwrapped().toString))
+  /** 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) {
+    val typesafeConfig = ConfigFactory.systemProperties()
+      .withFallback(ConfigFactory.parseResources("spark.conf"))
+    for (e <- typesafeConfig.entrySet().asScala) {
+      settings(e.getKey) = e.getValue.unwrapped.toString
     }
   }
 
-  def setMasterUrl(master: String) = {
-    if (master != null)
-      configMap += (("spark.master", master))
+  /** Set a configuration variable. */
+  def set(key: String, value: String): SparkConf = {
+    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 = {
+    if (master != null) {
+      settings("spark.master") = master
+    }
     this
   }
 
-  def setAppName(name: String) = {
-    if (name != null)
-      configMap += (("spark.appName", name))
+  /** Set a name for your application. Shown in the Spark web UI. */
+  def setAppName(name: String): SparkConf = {
+    if (name != null) {
+      settings("spark.appName") = name
+    }
     this
   }
 
-  def setJars(jars: Seq[String]) = {
-    if (!jars.isEmpty)
-      configMap += (("spark.jars", jars.mkString(",")))
+  /** Set JAR files to distribute to the cluster. */
+  def setJars(jars: Seq[String]): SparkConf = {
+    if (!jars.isEmpty) {
+      settings("spark.jars") = jars.mkString(",")
+    }
     this
   }
 
-  def set(k: String, value: String) = {
-    configMap += ((k, value))
+  /** Set JAR files to distribute to the cluster. (Java-friendly version.) */
+  def setJars(jars: Array[String]): SparkConf = {
+    if (!jars.isEmpty) {
+      settings("spark.jars") = jars.mkString(",")
+    }
     this
   }
 
-  def setSparkHome(home: String) = {
-    if (home != null)
-      configMap += (("spark.home", home))
+  /** Set an environment variable to be used when launching executors for this application. */
+  def setExecutorEnv(variable: String, value: String): SparkConf = {
+    settings("spark.executorEnv." + variable) = value
     this
   }
 
-  def set(map: Seq[(String, String)]) = {
-    if (map != null && !map.isEmpty)
-      configMap ++= map
+  /** Set multiple environment variables to be used when launching executors. */
+  def setExecutorEnv(variables: Seq[(String, String)]): SparkConf = {
+    for ((k, v) <- variables) {
+      setExecutorEnv(k, v)
+    }
     this
   }
 
-  def get(k: String): String = {
-    configMap(k)
+  /**
+   * Set multiple environment variables to be used when launching executors.
+   * (Java-friendly version.)
+   */
+  def setExecutorEnv(variables: Array[(String, String)]): SparkConf = {
+    for ((k, v) <- variables) {
+      setExecutorEnv(k, v)
+    }
+    this
   }
 
-  def getAllConfiguration = configMap.clone.entrySet().iterator
+  /**
+   * Set the location where Spark is installed on worker nodes. This is only needed on Mesos if
+   * you are not using `spark.executor.uri` to disseminate the Spark binary distribution.
+   */
+  def setSparkHome(home: String): SparkConf = {
+    if (home != null) {
+      settings("spark.home") = home
+    }
+    this
+  }
 
+  /** 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
+  }
+
+  /** Get a parameter; throws an exception if it's not set */
+  def get(key: String): String = {
+    settings(key)
+  }
+
+  /** 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: Seq[(String, String)] = settings.clone().toSeq
+
+  /** Get a parameter, falling back to a default if not set */
   def getOrElse(k: String, defaultValue: String): String = {
-    configMap.getOrElse(k, defaultValue)
+    settings.getOrElse(k, defaultValue)
   }
 
-  override def clone: SparkConf = {
-    val conf = new SparkConf(false)
-    conf.set(configMap.toSeq)
-    conf
+  /** Get all executor environment variables set on this SparkConf */
+  def getExecutorEnv: Seq[(String, String)] = {
+    val prefix = "spark.executorEnv."
+    getAll.filter(pair => pair._1.startsWith(prefix))
+          .map(pair => (pair._1.substring(prefix.length), pair._2))
   }
 
+  /** 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)
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/SparkContext.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 4300b07..0567f7f 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -22,8 +22,7 @@ import java.net.URI
 import java.util.Properties
 import java.util.concurrent.atomic.AtomicInteger
 
-import scala.collection.{Map, immutable}
-import scala.collection.JavaConversions._
+import scala.collection.{Map, Set, immutable}
 import scala.collection.generic.Growable
 
 import scala.collection.mutable.{ArrayBuffer, HashMap}
@@ -57,23 +56,32 @@ 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 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.
+ * @param conf_ a Spark Config object describing the application configuration. Any settings in
+ *   this config overrides the default configs as well as system properties.
+ * @param preferredNodeLocationData used in YARN mode to select nodes to launch containers on. Can
+ *   be generated using [[org.apache.spark.scheduler.InputFormatInfo.computePreferredLocations]]
+ *   from a list of input files or InputFormats for the application.
  */
 class SparkContext(
-    val conf: SparkConf,
-    val environment: Map[String, String] = Map(),
+    conf_ : SparkConf,
     // 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]] = immutable.Map())
+    // too. This is typically generated from InputFormatInfo.computePreferredLocations. It contains
+    // a map from hostname to a list of input format splits on the host.
+    val preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map())
   extends Logging {
 
   /**
-   * 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.
+   * Alternative constructor that allows setting common Spark properties directly
+   *
+   * @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 conf a [[org.apache.spark.SparkConf]] object specifying other Spark parameters
+   */
+  def this(master: String, appName: String, conf: SparkConf) =
+    this(conf.setMaster(master).setAppName(appName))
+
+  /**
+   * Alternative constructor that allows setting common Spark properties directly
    *
    * @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.
@@ -82,24 +90,42 @@ class SparkContext(
    *             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)
+  def this(
+      master: String,
+      appName: String,
+      sparkHome: String = null,
+      jars: Seq[String] = Nil,
+      environment: Map[String, String] = Map(),
+      preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map()) =
+  {
+    this(
+      new SparkConf()
+        .setMaster(master)
+        .setAppName(appName)
+        .setJars(jars)
+        .setExecutorEnv(environment.toSeq)
+        .setSparkHome(sparkHome),
+      preferredNodeLocationData)
+  }
 
-  // Set Spark driver host and port system properties
-  Try(conf.get("spark.driver.host"))
-    .getOrElse(conf.set("spark.driver.host",  Utils.localHostName()))
+  val conf = conf_.clone()
+
+  if (!conf.contains("spark.master")) {
+    throw new SparkException("A master URL must be set in your configuration")
+  }
+  if (!conf.contains("spark.appName")) {
+    throw new SparkException("An application must be set in your configuration")
+  }
 
-  Try(conf.get("spark.driver.port"))
-    .getOrElse(conf.set("spark.driver.port",  "0"))
+  // Set Spark driver host and port system properties
+  conf.setIfMissing("spark.driver.host", Utils.localHostName())
+  conf.setIfMissing("spark.driver.port", "0")
 
-  val jars: Seq[String] = if (conf.getOrElse("spark.jars", null) != null) {
-    conf.get("spark.jars").split(",")
-  } else null
+  val jars: Seq[String] = if (conf.contains("spark.jars")) {
+    conf.get("spark.jars").split(",").filter(_.size != 0)
+  } else {
+    null
+  }
 
   val master = conf.get("spark.master")
   val appName = conf.get("spark.appName")
@@ -115,8 +141,8 @@ class SparkContext(
     conf.get("spark.driver.host"),
     conf.get("spark.driver.port").toInt,
     conf,
-    true,
-    isLocal)
+    isDriver = true,
+    isLocal = isLocal)
   SparkEnv.set(env)
 
   // Used to store a URL for each static file/jar together with the file's local timestamp
@@ -125,7 +151,8 @@ class SparkContext(
 
   // Keeps track of all persisted RDDs
   private[spark] val persistentRdds = new TimeStampedHashMap[Int, RDD[_]]
-  private[spark] val metadataCleaner = new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup)
+  private[spark] val metadataCleaner =
+    new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup, conf)
 
   // Initialize the Spark UI
   private[spark] val ui = new SparkUI(this)
@@ -135,9 +162,14 @@ class SparkContext(
 
   // Add each JAR given through the constructor
   if (jars != null) {
-    jars.foreach { addJar(_) }
+    jars.foreach(addJar)
   }
 
+  private[spark] val executorMemory = conf.getOption("spark.executor.memory")
+    .orElse(Option(System.getenv("SPARK_MEM")))
+    .map(Utils.memoryStringToMb)
+    .getOrElse(512)
+
   // Environment variables to pass to our executors
   private[spark] val executorEnvs = HashMap[String, String]()
   // Note: SPARK_MEM is included for Mesos, but overwritten for standalone mode in ExecutorRunner
@@ -148,10 +180,8 @@ class SparkContext(
     }
   }
   // Since memory can be set with a system property too, use that
-  executorEnvs("SPARK_MEM") = SparkContext.executorMemoryRequested + "m"
-  if (environment != null) {
-    executorEnvs ++= environment
-  }
+  executorEnvs("SPARK_MEM") = executorMemory + "m"
+  executorEnvs ++= conf.getExecutorEnv
 
   // Set SPARK_USER for user who is running SparkContext.
   val sparkUser = Option {
@@ -183,12 +213,12 @@ class SparkContext(
       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) =>
+    conf.getAll.foreach { case (key, value) =>
       if (key.startsWith("spark.hadoop.")) {
         hadoopConf.set(key.substring("spark.hadoop.".length), value)
       }
     }
-    val bufferSize = conf.getOrElse("spark.buffer.size",  "65536")
+    val bufferSize = conf.getOrElse("spark.buffer.size", "65536")
     hadoopConf.set("io.file.buffer.size", bufferSize)
     hadoopConf
   }
@@ -200,7 +230,7 @@ class SparkContext(
     override protected def childValue(parent: Properties): Properties = new Properties(parent)
   }
 
-  private[spark] def getLocalProperties(): Properties = localProperties.get()
+  private[spark] def getLocalProperties: Properties = localProperties.get()
 
   private[spark] def setLocalProperties(props: Properties) {
     localProperties.set(props)
@@ -533,7 +563,7 @@ class SparkContext(
     // Fetch the file locally in case a job is executed locally.
     // Jobs that run through LocalScheduler will already fetch the required dependencies,
     // but jobs run in DAGScheduler.runLocally() will not so we must fetch the files here.
-    Utils.fetchFile(path, new File(SparkFiles.getRootDirectory))
+    Utils.fetchFile(path, new File(SparkFiles.getRootDirectory), conf)
 
     logInfo("Added file " + path + " at " + key + " with timestamp " + addedFiles(key))
   }
@@ -915,14 +945,6 @@ object SparkContext {
 
   private[spark] val SPARK_UNKNOWN_USER = "<unknown>"
 
-  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
@@ -1031,18 +1053,10 @@ object SparkContext {
   /** Find the JAR that contains the class of a particular object */
   def jarOfObject(obj: AnyRef): Seq[String] = jarOfClass(obj.getClass)
 
-  /** 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
-    Try(globalConf.get("spark.executor.memory")).toOption
-      .orElse(Option(System.getenv("SPARK_MEM")))
-      .map(Utils.memoryStringToMb)
-      .getOrElse(512)
-  }
-
   // Creates a task scheduler based on a given master URL. Extracted for testing.
-  private
-  def createTaskScheduler(sc: SparkContext, master: String, appName: String): TaskScheduler = {
+  private def createTaskScheduler(sc: SparkContext, master: String, appName: String)
+      : TaskScheduler =
+  {
     // Regular expression used for local[N] master format
     val LOCAL_N_REGEX = """local\[([0-9]+)\]""".r
     // Regular expression for local[N, maxRetries], used in tests with failing tasks
@@ -1076,10 +1090,10 @@ object SparkContext {
       case LOCAL_CLUSTER_REGEX(numSlaves, coresPerSlave, memoryPerSlave) =>
         // Check to make sure memory requested <= memoryPerSlave. Otherwise Spark will just hang.
         val memoryPerSlaveInt = memoryPerSlave.toInt
-        if (SparkContext.executorMemoryRequested > memoryPerSlaveInt) {
+        if (sc.executorMemory > memoryPerSlaveInt) {
           throw new SparkException(
             "Asked to launch cluster with %d MB RAM / worker but requested %d MB/worker".format(
-              memoryPerSlaveInt, SparkContext.executorMemoryRequested))
+              memoryPerSlaveInt, sc.executorMemory))
         }
 
         val scheduler = new ClusterScheduler(sc)
@@ -1137,7 +1151,7 @@ object SparkContext {
       case mesosUrl @ MESOS_REGEX(_) =>
         MesosNativeLibrary.load()
         val scheduler = new ClusterScheduler(sc)
-        val coarseGrained = globalConf.getOrElse("spark.mesos.coarse",  "false").toBoolean
+        val coarseGrained = sc.conf.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)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/SparkEnv.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala
index 78e4ae2..34fad3e 100644
--- a/core/src/main/scala/org/apache/spark/SparkEnv.scala
+++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala
@@ -107,7 +107,7 @@ object SparkEnv extends Logging {
   /**
    * Returns the ThreadLocal SparkEnv.
    */
-  def getThreadLocal : SparkEnv = {
+  def getThreadLocal: SparkEnv = {
 	  env.get()
   }
 
@@ -150,18 +150,19 @@ object SparkEnv extends Logging {
     val serializerManager = new SerializerManager
 
     val serializer = serializerManager.setDefault(
-      conf.getOrElse("spark.serializer",  "org.apache.spark.serializer.JavaSerializer"))
+      conf.getOrElse("spark.serializer", "org.apache.spark.serializer.JavaSerializer"), conf)
 
     val closureSerializer = serializerManager.get(
-      conf.getOrElse("spark.closure.serializer",  "org.apache.spark.serializer.JavaSerializer"))
+      conf.getOrElse("spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer"),
+      conf)
 
     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 = conf.getOrElse("spark.driver.host",  "localhost")
-        val driverPort: Int = conf.getOrElse("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)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
index acf328a..e03cf9d 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
@@ -29,17 +29,22 @@ import org.apache.hadoop.mapred.JobConf
 import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
 import com.google.common.base.Optional
 
-import org.apache.spark.{Accumulable, AccumulableParam, Accumulator, AccumulatorParam, SparkContext}
+import org.apache.spark._
 import org.apache.spark.SparkContext.IntAccumulatorParam
 import org.apache.spark.SparkContext.DoubleAccumulatorParam
 import org.apache.spark.broadcast.Broadcast
 import org.apache.spark.rdd.RDD
+import scala.Tuple2
 
 /**
  * A Java-friendly version of [[org.apache.spark.SparkContext]] that returns [[org.apache.spark.api.java.JavaRDD]]s and
  * works with Java collections instead of Scala ones.
  */
 class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWorkaround {
+  /**
+   * @param conf a [[org.apache.spark.SparkConf]] object specifying Spark parameters
+   */
+  def this(conf: SparkConf) = this(new SparkContext(conf))
 
   /**
    * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
@@ -50,6 +55,14 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
   /**
    * @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 conf a [[org.apache.spark.SparkConf]] object specifying other Spark parameters
+   */
+  def this(master: String, appName: String, conf: SparkConf) =
+    this(conf.setMaster(master).setAppName(appName))
+
+  /**
+   * @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 The SPARK_HOME directory on the slave nodes
    * @param jarFile JAR file to send to the cluster. This can be a path on the local file system
    *                or an HDFS, HTTP, HTTPS, or FTP URL.

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
----------------------------------------------------------------------
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 d6eacfe..05fd824 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 = conf.getOrElse("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 = conf.getOrElse("spark.buffer.size",  "65536").toInt
+  val bufferSize = SparkEnv.get.conf.getOrElse("spark.buffer.size", "65536").toInt
 
   override def zero(value: JList[Array[Byte]]): JList[Array[Byte]] = new JArrayList
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala
----------------------------------------------------------------------
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 cecb8c2..47528bc 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala
@@ -31,7 +31,7 @@ import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedH
 
 private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long)
   extends Broadcast[T](id) with Logging with Serializable {
-  
+
   def value = value_
 
   def blockId = BroadcastBlockId(id)
@@ -40,7 +40,7 @@ private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolea
     SparkEnv.get.blockManager.putSingle(blockId, value_, StorageLevel.MEMORY_AND_DISK, false)
   }
 
-  if (!isLocal) { 
+  if (!isLocal) {
     HttpBroadcast.write(id, value_)
   }
 
@@ -81,41 +81,48 @@ private object HttpBroadcast extends Logging {
   private var serverUri: String = null
   private var server: HttpServer = null
 
+  // TODO: This shouldn't be a global variable so that multiple SparkContexts can coexist
   private val files = new TimeStampedHashSet[String]
-  private val cleaner = new MetadataCleaner(MetadataCleanerType.HTTP_BROADCAST, cleanup)
+  private var cleaner: MetadataCleaner = null
 
-  private val httpReadTimeout = TimeUnit.MILLISECONDS.convert(5,TimeUnit.MINUTES).toInt
+  private val httpReadTimeout = TimeUnit.MILLISECONDS.convert(5, TimeUnit.MINUTES).toInt
 
-  private lazy val compressionCodec = CompressionCodec.createCodec()
+  private var compressionCodec: CompressionCodec = null
 
   def initialize(isDriver: Boolean, conf: SparkConf) {
     synchronized {
       if (!initialized) {
-        bufferSize = conf.getOrElse("spark.buffer.size",  "65536").toInt
-        compress = conf.getOrElse("spark.broadcast.compress",  "true").toBoolean
+        bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt
+        compress = conf.getOrElse("spark.broadcast.compress", "true").toBoolean
         if (isDriver) {
-          createServer()
+          createServer(conf)
           conf.set("spark.httpBroadcast.uri",  serverUri)
         }
         serverUri = conf.get("spark.httpBroadcast.uri")
+        cleaner = new MetadataCleaner(MetadataCleanerType.HTTP_BROADCAST, cleanup, conf)
+        compressionCodec = CompressionCodec.createCodec(conf)
         initialized = true
       }
     }
   }
-  
+
   def stop() {
     synchronized {
       if (server != null) {
         server.stop()
         server = null
       }
+      if (cleaner != null) {
+        cleaner.cancel()
+        cleaner = null
+      }
+      compressionCodec = null
       initialized = false
-      cleaner.cancel()
     }
   }
 
-  private def createServer() {
-    broadcastDir = Utils.createTempDir(Utils.getLocalDir)
+  private def createServer(conf: SparkConf) {
+    broadcastDir = Utils.createTempDir(Utils.getLocalDir(conf))
     server = new HttpServer(broadcastDir)
     server.start()
     serverUri = server.uri
@@ -143,7 +150,7 @@ private object HttpBroadcast extends Logging {
     val in = {
       val httpConnection = new URL(url).openConnection()
       httpConnection.setReadTimeout(httpReadTimeout)
-      val inputStream = httpConnection.getInputStream()
+      val inputStream = httpConnection.getInputStream
       if (compress) {
         compressionCodec.compressedInputStream(inputStream)
       } else {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
----------------------------------------------------------------------
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 4a3801d..00ec3b9 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
@@ -83,13 +83,13 @@ extends Broadcast[T](id) with Logging with Serializable {
         case None =>
           val start = System.nanoTime
           logInfo("Started reading broadcast variable " + id)
-          
+
           // Initialize @transient variables that will receive garbage values from the master.
           resetWorkerVariables()
 
           if (receiveBroadcast(id)) {
             value_ = TorrentBroadcast.unBlockifyObject[T](arrayOfBlocks, totalBytes, totalBlocks)
-            
+
             // Store the merged copy in cache so that the next worker doesn't need to rebuild it.
             // This creates a tradeoff between memory usage and latency.
             // Storing copy doubles the memory footprint; not storing doubles deserialization cost.
@@ -122,14 +122,14 @@ extends Broadcast[T](id) with Logging with Serializable {
     while (attemptId > 0 && totalBlocks == -1) {
       TorrentBroadcast.synchronized {
         SparkEnv.get.blockManager.getSingle(metaId) match {
-          case Some(x) => 
+          case Some(x) =>
             val tInfo = x.asInstanceOf[TorrentInfo]
             totalBlocks = tInfo.totalBlocks
             totalBytes = tInfo.totalBytes
             arrayOfBlocks = new Array[TorrentBlock](totalBlocks)
             hasBlocks = 0
-          
-          case None => 
+
+          case None =>
             Thread.sleep(500)
         }
       }
@@ -145,13 +145,13 @@ extends Broadcast[T](id) with Logging with Serializable {
       val pieceId = BroadcastHelperBlockId(broadcastId, "piece" + pid)
       TorrentBroadcast.synchronized {
         SparkEnv.get.blockManager.getSingle(pieceId) match {
-          case Some(x) => 
+          case Some(x) =>
             arrayOfBlocks(pid) = x.asInstanceOf[TorrentBlock]
             hasBlocks += 1
             SparkEnv.get.blockManager.putSingle(
               pieceId, arrayOfBlocks(pid), StorageLevel.MEMORY_AND_DISK, true)
-          
-          case None => 
+
+          case None =>
             throw new SparkException("Failed to get " + pieceId + " of " + broadcastId)
         }
       }
@@ -175,13 +175,13 @@ extends Logging {
       }
     }
   }
-  
+
   def stop() {
     initialized = false
   }
 
-  lazy val BLOCK_SIZE = conf.getOrElse("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)
     val bais = new ByteArrayInputStream(byteArray)
@@ -210,7 +210,7 @@ extends Logging {
   }
 
   def unBlockifyObject[T](arrayOfBlocks: Array[TorrentBlock],
-                            totalBytes: Int, 
+                            totalBytes: Int,
                             totalBlocks: Int): T = {
     var retByteArray = new Array[Byte](totalBytes)
     for (i <- 0 until totalBlocks) {
@@ -223,22 +223,22 @@ extends Logging {
 }
 
 private[spark] case class TorrentBlock(
-    blockID: Int, 
-    byteArray: Array[Byte]) 
+    blockID: Int,
+    byteArray: Array[Byte])
   extends Serializable
 
 private[spark] case class TorrentInfo(
     @transient arrayOfBlocks : Array[TorrentBlock],
-    totalBlocks: Int, 
-    totalBytes: Int) 
+    totalBlocks: Int,
+    totalBytes: Int)
   extends Serializable {
-  
-  @transient var hasBlocks = 0 
+
+  @transient var hasBlocks = 0
 }
 
 private[spark] class TorrentBroadcastFactory
   extends BroadcastFactory {
-  
+
   def initialize(isDriver: Boolean, conf: SparkConf) { TorrentBroadcast.initialize(isDriver, conf) }
 
   def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) =

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala
----------------------------------------------------------------------
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 dda43dc..19d393a 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",  "<unknown>")
+  val user = System.getProperty("user.name", "<unknown>")
 
   override def toString: String = "ApplicationDescription(" + name + ")"
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala
index 59d12a3..ffc0cb0 100644
--- a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala
@@ -22,7 +22,7 @@ import akka.actor.ActorSystem
 import org.apache.spark.deploy.worker.Worker
 import org.apache.spark.deploy.master.Master
 import org.apache.spark.util.Utils
-import org.apache.spark.Logging
+import org.apache.spark.{SparkConf, Logging}
 
 import scala.collection.mutable.ArrayBuffer
 
@@ -43,7 +43,8 @@ class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: I
     logInfo("Starting a local Spark cluster with " + numWorkers + " workers.")
 
     /* Start the Master */
-    val (masterSystem, masterPort, _) = Master.startSystemAndActor(localHostname, 0, 0)
+    val conf = new SparkConf(false)
+    val (masterSystem, masterPort, _) = Master.startSystemAndActor(localHostname, 0, 0, conf)
     masterActorSystems += masterSystem
     val masterUrl = "spark://" + localHostname + ":" + masterPort
     val masters = Array(masterUrl)
@@ -55,7 +56,7 @@ class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: I
       workerActorSystems += workerSystem
     }
 
-    return masters
+    masters
   }
 
   def stop() {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
----------------------------------------------------------------------
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 1c979ac..4f402c1 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
@@ -34,10 +34,10 @@ class SparkHadoopUtil {
   UserGroupInformation.setConfiguration(conf)
 
   def runAsUser(user: String)(func: () => Unit) {
-    // if we are already running as the user intended there is no reason to do the doAs. It 
+    // if we are already running as the user intended there is no reason to do the doAs. It
     // will actually break secure HDFS access as it doesn't fill in the credentials. Also if
-    // the user is UNKNOWN then we shouldn't be creating a remote unknown user 
-    // (this is actually the path spark on yarn takes) since SPARK_USER is initialized only 
+    // the user is UNKNOWN then we shouldn't be creating a remote unknown user
+    // (this is actually the path spark on yarn takes) since SPARK_USER is initialized only
     // in SparkContext.
     val currentUser = Option(System.getProperty("user.name")).
       getOrElse(SparkContext.SPARK_UNKNOWN_USER)
@@ -67,12 +67,14 @@ class SparkHadoopUtil {
 }
 
 object SparkHadoopUtil {
-  import SparkContext.{globalConf => conf}
+
   private val hadoop = {
-    val yarnMode = java.lang.Boolean.valueOf(conf.getOrElse("SPARK_YARN_MODE",  System.getenv("SPARK_YARN_MODE")))
+    val yarnMode = java.lang.Boolean.valueOf(System.getenv("SPARK_YARN_MODE"))
     if (yarnMode) {
       try {
-        Class.forName("org.apache.spark.deploy.yarn.YarnSparkHadoopUtil").newInstance.asInstanceOf[SparkHadoopUtil]
+        Class.forName("org.apache.spark.deploy.yarn.YarnSparkHadoopUtil")
+          .newInstance()
+          .asInstanceOf[SparkHadoopUtil]
       } catch {
        case th: Throwable => throw new SparkException("Unable to load YARN support", th)
       }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
----------------------------------------------------------------------
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 426cf52..ef649fd 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.{SparkContext, Logging}
+import org.apache.spark.{SparkConf, SparkContext, Logging}
 import org.apache.spark.deploy.{Command, ApplicationDescription}
 
 private[spark] object TestClient {
@@ -46,11 +46,12 @@ private[spark] object TestClient {
   def main(args: Array[String]) {
     val url = args(0)
     val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0,
-      conf = SparkContext.globalConf)
+      conf = new SparkConf)
     val desc = new ApplicationDescription(
-      "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), "dummy-spark-home", "ignored")
+      "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, SparkContext.globalConf)
+    val client = new Client(actorSystem, Array(url), desc, listener, new SparkConf)
     client.start()
     actorSystem.awaitTermination()
   }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
----------------------------------------------------------------------
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 2c162c4..9c89e36 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.{SparkContext, Logging, SparkException}
+import org.apache.spark.{SparkConf, SparkContext, Logging, SparkException}
 import org.apache.spark.deploy.{ApplicationDescription, ExecutorState}
 import org.apache.spark.deploy.DeployMessages._
 import org.apache.spark.deploy.master.MasterMessages._
@@ -38,14 +38,16 @@ import org.apache.spark.metrics.MetricsSystem
 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
+  import context.dispatcher   // to use Akka's scheduler.schedule()
+
+  val conf = new SparkConf
+
   val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss")  // For application IDs
-  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")
+  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]
@@ -86,7 +88,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 = conf.getOrElse("spark.deploy.spreadOut",  "true").toBoolean
+  val spreadOutApps = conf.getOrElse("spark.deploy.spreadOut", "true").toBoolean
 
   override def preStart() {
     logInfo("Starting Spark master at " + masterUrl)
@@ -495,7 +497,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
         removeWorker(worker)
       } else {
         if (worker.lastHeartbeat < currentTime - ((REAPER_ITERATIONS + 1) * WORKER_TIMEOUT))
-          workers -= worker // we've seen this DEAD worker in the UI, etc. for long enough; cull it 
+          workers -= worker // we've seen this DEAD worker in the UI, etc. for long enough; cull it
       }
     }
   }
@@ -507,8 +509,9 @@ private[spark] object Master {
   val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r
 
   def main(argStrings: Array[String]) {
-    val args = new MasterArguments(argStrings, SparkContext.globalConf)
-    val (actorSystem, _, _) = startSystemAndActor(args.host, args.port, args.webUiPort)
+    val conf = new SparkConf
+    val args = new MasterArguments(argStrings, conf)
+    val (actorSystem, _, _) = startSystemAndActor(args.host, args.port, args.webUiPort, conf)
     actorSystem.awaitTermination()
   }
 
@@ -522,11 +525,12 @@ private[spark] object Master {
     }
   }
 
-  def startSystemAndActor(host: String, port: Int, webUiPort: Int): (ActorSystem, Int, Int) = {
-    val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port,
-      conf = SparkContext.globalConf)
+  def startSystemAndActor(host: String, port: Int, webUiPort: Int, conf: SparkConf)
+      : (ActorSystem, Int, Int) =
+  {
+    val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port, conf = conf)
     val actor = actorSystem.actorOf(Props(classOf[Master], host, boundPort, webUiPort), actorName)
-    val timeout = AkkaUtils.askTimeout(SparkContext.globalConf)
+    val timeout = AkkaUtils.askTimeout(conf)
     val respFuture = actor.ask(RequestWebUIPort)(timeout)
     val resp = Await.result(respFuture, timeout).asInstanceOf[WebUIPortResponse]
     (actorSystem, boundPort, resp.webUIBoundPort)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala
----------------------------------------------------------------------
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 79d95b1..60c7a7c 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
@@ -37,7 +37,7 @@ import org.apache.spark.{SparkConf, Logging}
  */
 private[spark] class SparkZooKeeperSession(zkWatcher: SparkZooKeeperWatcher,
     conf: SparkConf) extends Logging {
-  val ZK_URL = conf.getOrElse("spark.deploy.zookeeper.url",  "")
+  val ZK_URL = conf.getOrElse("spark.deploy.zookeeper.url", "")
 
   val ZK_ACL = ZooDefs.Ids.OPEN_ACL_UNSAFE
   val ZK_TIMEOUT_MILLIS = 30000

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala
----------------------------------------------------------------------
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 df5bb36..a61597b 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
@@ -28,7 +28,7 @@ private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: ActorRef,
     masterUrl: String, conf: SparkConf)
   extends LeaderElectionAgent with SparkZooKeeperWatcher with Logging  {
 
-  val WORKING_DIR = conf.getOrElse("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, conf)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
----------------------------------------------------------------------
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 c55b720..245a558 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
@@ -27,7 +27,7 @@ class ZooKeeperPersistenceEngine(serialization: Serialization, conf: SparkConf)
   with SparkZooKeeperWatcher
   with Logging
 {
-  val WORKING_DIR = conf.getOrElse("spark.deploy.zookeeper.dir",  "/spark") + "/master_status"
+  val WORKING_DIR = conf.getOrElse("spark.deploy.zookeeper.dir", "/spark") + "/master_status"
 
   val zk = new SparkZooKeeperSession(this, conf)
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
----------------------------------------------------------------------
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 75a6e75..f844fcb 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
@@ -55,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 = conf.getOrElse("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
@@ -267,7 +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,
@@ -276,14 +276,16 @@ private[spark] object Worker {
   }
 
   def startSystemAndActor(host: String, port: Int, webUiPort: Int, cores: Int, memory: Int,
-    masterUrls: Array[String], workDir: String, workerNumber: Option[Int] = None)
-    : (ActorSystem, Int) = {
+      masterUrls: Array[String], workDir: String, workerNumber: Option[Int] = None)
+      : (ActorSystem, Int) =
+  {
     // The LocalSparkCluster runs multiple local sparkWorkerX actor systems
+    val conf = new SparkConf
     val systemName = "sparkWorker" + workerNumber.map(_.toString).getOrElse("")
     val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port,
-      conf = globalConf)
+      conf = conf)
     actorSystem.actorOf(Props(classOf[Worker], host, boundPort, webUiPort, cores, memory,
-      masterUrls, workDir, globalConf), name = "Worker")
+      masterUrls, workDir, conf), name = "Worker")
     (actorSystem, boundPort)
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
----------------------------------------------------------------------
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 c8319f6..53a2b94 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.{SparkContext, Logging}
+import org.apache.spark.{SparkConf, SparkContext, Logging}
 import org.apache.spark.TaskState.TaskState
 import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._
 import org.apache.spark.util.{Utils, AkkaUtils}
@@ -98,7 +98,7 @@ 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, conf = SparkContext.globalConf)
+      indestructible = true, conf = new SparkConf)
     // set it
     val sparkHostPort = hostname + ":" + boundPort
 //    conf.set("spark.hostPort",  sparkHostPort)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/executor/Executor.scala
----------------------------------------------------------------------
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 70fc30e..a6eabc4 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,18 @@ private[spark] class Executor(
 
   // Make sure the local hostname we report matches the cluster scheduler's name for this host
   Utils.setCustomHostname(slaveHostname)
+
+  // Set spark.* properties from executor arg
   val conf = new SparkConf(false)
-  // Set spark.* system properties from executor arg
-  for ((key, value) <- properties) {
-    conf.set(key,  value)
-  }
+  conf.setAll(properties)
 
   // 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"))) {
-    conf.set("spark.local.dir",  getYarnLocalDirs())
+  if (java.lang.Boolean.valueOf(
+      System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE"))))
+  {
+    conf.set("spark.local.dir", getYarnLocalDirs())
   }
 
   // Create our ClassLoader and set it on this thread
@@ -331,12 +332,12 @@ private[spark] class Executor(
       // Fetch missing dependencies
       for ((name, timestamp) <- newFiles if currentFiles.getOrElse(name, -1L) < timestamp) {
         logInfo("Fetching " + name + " with timestamp " + timestamp)
-        Utils.fetchFile(name, new File(SparkFiles.getRootDirectory))
+        Utils.fetchFile(name, new File(SparkFiles.getRootDirectory), conf)
         currentFiles(name) = timestamp
       }
       for ((name, timestamp) <- newJars if currentJars.getOrElse(name, -1L) < timestamp) {
         logInfo("Fetching " + name + " with timestamp " + timestamp)
-        Utils.fetchFile(name, new File(SparkFiles.getRootDirectory))
+        Utils.fetchFile(name, new File(SparkFiles.getRootDirectory), conf)
         currentJars(name) = timestamp
         // Add it to our class loader
         val localName = name.split("/").last

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
----------------------------------------------------------------------
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 8ef5019..2040268 100644
--- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
+++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
@@ -22,7 +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
+import org.apache.spark.{SparkEnv, SparkConf}
 
 
 /**
@@ -38,16 +38,15 @@ trait CompressionCodec {
 
 
 private[spark] object CompressionCodec {
-  import org.apache.spark.SparkContext.globalConf
-  def createCodec(): CompressionCodec = {
-    createCodec(System.getProperty(
+  def createCodec(conf: SparkConf): CompressionCodec = {
+    createCodec(conf, conf.getOrElse(
       "spark.io.compression.codec", classOf[LZFCompressionCodec].getName))
   }
 
-  def createCodec(codecName: String): CompressionCodec = {
+  def createCodec(conf: SparkConf, codecName: String): CompressionCodec = {
     val ctor = Class.forName(codecName, true, Thread.currentThread.getContextClassLoader)
       .getConstructor(classOf[SparkConf])
-      ctor.newInstance(globalConf).asInstanceOf[CompressionCodec]
+      ctor.newInstance(conf).asInstanceOf[CompressionCodec]
   }
 }
 
@@ -72,7 +71,7 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec {
 class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec {
 
   override def compressedOutputStream(s: OutputStream): OutputStream = {
-    val blockSize = conf.getOrElse("spark.io.compression.snappy.block.size",  "32768").toInt
+    val blockSize = conf.getOrElse("spark.io.compression.snappy.block.size", "32768").toInt
     new SnappyOutputStream(s, blockSize)
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala
----------------------------------------------------------------------
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 3e902f8..697096f 100644
--- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala
+++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala
@@ -593,10 +593,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi
 
 private[spark] object ConnectionManager {
 
-  import SparkContext.globalConf
-
   def main(args: Array[String]) {
-    val manager = new ConnectionManager(9999, globalConf)
+    val manager = new ConnectionManager(9999, new SparkConf)
     manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => {
       println("Received [" + msg + "] from [" + id + "]")
       None

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala
----------------------------------------------------------------------
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 4ca3cd3..1c9d603 100644
--- a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala
+++ b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala
@@ -19,19 +19,19 @@ package org.apache.spark.network
 
 import java.nio.ByteBuffer
 import java.net.InetAddress
+import org.apache.spark.SparkConf
 
 private[spark] object ReceiverTest {
-  import org.apache.spark.SparkContext.globalConf
   def main(args: Array[String]) {
-    val manager = new ConnectionManager(9999, globalConf)
+    val manager = new ConnectionManager(9999, new SparkConf)
     println("Started connection manager with id = " + manager.id)
-    
-    manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { 
+
+    manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => {
       /*println("Received [" + msg + "] from [" + id + "] at " + System.currentTimeMillis)*/
-      val buffer = ByteBuffer.wrap("response".getBytes())
+      val buffer = ByteBuffer.wrap("response".getBytes)
       Some(Message.createBufferMessage(buffer, msg.id))
     })
-    Thread.currentThread.join()  
+    Thread.currentThread.join()
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/network/SenderTest.scala
----------------------------------------------------------------------
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 11c21fc..dcbd183 100644
--- a/core/src/main/scala/org/apache/spark/network/SenderTest.scala
+++ b/core/src/main/scala/org/apache/spark/network/SenderTest.scala
@@ -19,29 +19,29 @@ package org.apache.spark.network
 
 import java.nio.ByteBuffer
 import java.net.InetAddress
+import org.apache.spark.SparkConf
 
 private[spark] object SenderTest {
-  import org.apache.spark.SparkContext.globalConf
   def main(args: Array[String]) {
-    
+
     if (args.length < 2) {
       println("Usage: SenderTest <target host> <target port>")
       System.exit(1)
     }
-   
+
     val targetHost = args(0)
     val targetPort = args(1).toInt
     val targetConnectionManagerId = new ConnectionManagerId(targetHost, targetPort)
 
-    val manager = new ConnectionManager(0, globalConf)
+    val manager = new ConnectionManager(0, new SparkConf)
     println("Started connection manager with id = " + manager.id)
 
-    manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { 
+    manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => {
       println("Received [" + msg + "] from [" + id + "]")
       None
     })
-  
-    val size =  100 * 1024  * 1024 
+
+    val size =  100 * 1024  * 1024
     val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte))
     buffer.flip
 
@@ -50,7 +50,7 @@ private[spark] object SenderTest {
     val count = 100
     (0 until count).foreach(i => {
       val dataMessage = Message.createBufferMessage(buffer.duplicate)
-      val startTime = System.currentTimeMillis  
+      val startTime = System.currentTimeMillis
       /*println("Started timer at " + startTime)*/
       val responseStr = manager.sendMessageReliablySync(targetConnectionManagerId, dataMessage) match {
         case Some(response) =>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala
----------------------------------------------------------------------
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 81b3104..db28ddf 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
@@ -36,7 +36,7 @@ private[spark] class ShuffleCopier(conf: SparkConf) extends Logging {
       resultCollectCallback: (BlockId, Long, ByteBuf) => Unit) {
 
     val handler = new ShuffleCopier.ShuffleClientHandler(resultCollectCallback)
-    val connectTimeout = conf.getOrElse("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 {
@@ -104,10 +104,10 @@ private[spark] object ShuffleCopier extends Logging {
     val threads = if (args.length > 3) args(3).toInt else 10
 
     val copiers = Executors.newFixedThreadPool(80)
-    val tasks = (for (i <- Range(0, threads)) yield { 
+    val tasks = (for (i <- Range(0, threads)) yield {
       Executors.callable(new Runnable() {
         def run() {
-          val copier = new ShuffleCopier(SparkContext.globalConf)
+          val copier = new ShuffleCopier(new SparkConf)
           copier.getBlock(host, port, blockId, echoResultCollectCallBack)
         }
       })

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
----------------------------------------------------------------------
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 9fbe002..2897c4b 100644
--- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
@@ -74,9 +74,6 @@ 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)
   }
@@ -94,7 +91,7 @@ private[spark] object CheckpointRDD extends Logging {
       throw new IOException("Checkpoint failed: temporary path " +
         tempOutputPath + " already exists")
     }
-    val bufferSize = conf.getOrElse("spark.buffer.size",  "65536").toInt
+    val bufferSize = env.conf.getOrElse("spark.buffer.size", "65536").toInt
 
     val fileOutputStream = if (blockSize < 0) {
       fs.create(tempOutputPath, false, bufferSize)
@@ -124,7 +121,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 = conf.getOrElse("spark.buffer.size",  "65536").toInt
+    val bufferSize = env.conf.getOrElse("spark.buffer.size", "65536").toInt
     val fileInputStream = fs.open(path, bufferSize)
     val serializer = env.serializer.newInstance()
     val deserializeStream = serializer.deserializeStream(fileInputStream)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
index 911a002..4ba4696 100644
--- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
@@ -114,7 +114,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part:
       map.changeValue(k, update)
     }
 
-    val ser = SparkEnv.get.serializerManager.get(serializerClass)
+    val ser = SparkEnv.get.serializerManager.get(serializerClass, SparkEnv.get.conf)
     for ((dep, depNum) <- split.deps.zipWithIndex) dep match {
       case NarrowCoGroupSplitDep(rdd, _, itsSplit) => {
         // Read them from the parent

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala
index 3682c84..0ccb309 100644
--- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala
@@ -59,7 +59,7 @@ class ShuffledRDD[K, V, P <: Product2[K, V] : ClassTag](
   override def compute(split: Partition, context: TaskContext): Iterator[P] = {
     val shuffledId = dependencies.head.asInstanceOf[ShuffleDependency[K, V]].shuffleId
     SparkEnv.get.shuffleFetcher.fetch[P](shuffledId, split.index, context,
-      SparkEnv.get.serializerManager.get(serializerClass))
+      SparkEnv.get.serializerManager.get(serializerClass, SparkEnv.get.conf))
   }
 
   override def clearDependencies() {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala
index aab30b1..4f90c7d 100644
--- a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala
@@ -93,7 +93,7 @@ private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag](
 
   override def compute(p: Partition, context: TaskContext): Iterator[(K, V)] = {
     val partition = p.asInstanceOf[CoGroupPartition]
-    val serializer = SparkEnv.get.serializerManager.get(serializerClass)
+    val serializer = SparkEnv.get.serializerManager.get(serializerClass, SparkEnv.get.conf)
     val map = new JHashMap[K, ArrayBuffer[V]]
     def getSeq(k: K): ArrayBuffer[V] = {
       val seq = map.get(k)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
index 963d15b..77aa24e 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -158,7 +158,8 @@ class DAGScheduler(
   val activeJobs = new HashSet[ActiveJob]
   val resultStageToJob = new HashMap[Stage, ActiveJob]
 
-  val metadataCleaner = new MetadataCleaner(MetadataCleanerType.DAG_SCHEDULER, this.cleanup)
+  val metadataCleaner = new MetadataCleaner(
+    MetadataCleanerType.DAG_SCHEDULER, this.cleanup, env.conf)
 
   /**
    * Starts the event processing actor.  The actor has two responsibilities:
@@ -529,7 +530,7 @@ class DAGScheduler(
       case JobSubmitted(jobId, rdd, func, partitions, allowLocal, callSite, listener, properties) =>
         var finalStage: Stage = null
         try {
-          // New stage creation at times and if its not protected, the scheduler thread is killed. 
+          // New stage creation at times and if its not protected, the scheduler thread is killed.
           // e.g. it can fail when jobs are run on HadoopRDD whose underlying hdfs files have been deleted
           finalStage = newStage(rdd, partitions.size, None, jobId, Some(callSite))
         } catch {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala
index 1791ee6..90eb8a7 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala
@@ -32,7 +32,7 @@ import scala.collection.JavaConversions._
 /**
  * Parses and holds information about inputFormat (and files) specified as a parameter.
  */
-class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Class[_], 
+class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Class[_],
                       val path: String) extends Logging {
 
   var mapreduceInputFormat: Boolean = false
@@ -40,7 +40,7 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl
 
   validate()
 
-  override def toString(): String = {
+  override def toString: String = {
     "InputFormatInfo " + super.toString + " .. inputFormatClazz " + inputFormatClazz + ", path : " + path
   }
 
@@ -125,7 +125,7 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl
    }
 
   private def findPreferredLocations(): Set[SplitInfo] = {
-    logDebug("mapreduceInputFormat : " + mapreduceInputFormat + ", mapredInputFormat : " + mapredInputFormat + 
+    logDebug("mapreduceInputFormat : " + mapreduceInputFormat + ", mapredInputFormat : " + mapredInputFormat +
       ", inputFormatClazz : " + inputFormatClazz)
     if (mapreduceInputFormat) {
       return prefLocsFromMapreduceInputFormat()
@@ -143,14 +143,14 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl
 object InputFormatInfo {
   /**
     Computes the preferred locations based on input(s) and returned a location to block map.
-    Typical use of this method for allocation would follow some algo like this 
-    (which is what we currently do in YARN branch) :
+    Typical use of this method for allocation would follow some algo like this:
+
     a) For each host, count number of splits hosted on that host.
     b) Decrement the currently allocated containers on that host.
     c) Compute rack info for each host and update rack -> count map based on (b).
     d) Allocate nodes based on (c)
-    e) On the allocation result, ensure that we dont allocate "too many" jobs on a single node 
-       (even if data locality on that is very high) : this is to prevent fragility of job if a single 
+    e) On the allocation result, ensure that we dont allocate "too many" jobs on a single node
+       (even if data locality on that is very high) : this is to prevent fragility of job if a single
        (or small set of) hosts go down.
 
     go to (a) until required nodes are allocated.

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala
----------------------------------------------------------------------
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 3f55cd5..6092783 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",  "<unknown>"),
+  def this() = this(System.getProperty("user.name", "<unknown>"),
     String.valueOf(System.currentTimeMillis()))
 
   private val logDir =

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
index 310ec62..28f3ba5 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
@@ -32,7 +32,9 @@ private[spark] object ResultTask {
   // expensive on the master node if it needs to launch thousands of tasks.
   val serializedInfoCache = new TimeStampedHashMap[Int, Array[Byte]]
 
-  val metadataCleaner = new MetadataCleaner(MetadataCleanerType.RESULT_TASK, serializedInfoCache.clearOldValues)
+  // TODO: This object shouldn't have global variables
+  val metadataCleaner = new MetadataCleaner(
+    MetadataCleanerType.RESULT_TASK, serializedInfoCache.clearOldValues, new SparkConf)
 
   def serializeInfo(stageId: Int, rdd: RDD[_], func: (TaskContext, Iterator[_]) => _): Array[Byte] = {
     synchronized {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala
----------------------------------------------------------------------
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 9002d33..3cf995e 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala
@@ -52,7 +52,7 @@ private[spark] class FIFOSchedulableBuilder(val rootPool: Pool)
 private[spark] class FairSchedulableBuilder(val rootPool: Pool, conf: SparkConf)
   extends SchedulableBuilder with Logging {
 
-  val schedulerAllocFile = Option(conf.get("spark.scheduler.allocation.file"))
+  val schedulerAllocFile = conf.getOption("spark.scheduler.allocation.file")
   val DEFAULT_SCHEDULER_FILE = "fairscheduler.xml"
   val FAIR_SCHEDULER_PROPERTIES = "spark.scheduler.pool"
   val DEFAULT_POOL_NAME = "default"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
index 0f2deb4..a37ead5 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
@@ -37,7 +37,9 @@ private[spark] object ShuffleMapTask {
   // expensive on the master node if it needs to launch thousands of tasks.
   val serializedInfoCache = new TimeStampedHashMap[Int, Array[Byte]]
 
-  val metadataCleaner = new MetadataCleaner(MetadataCleanerType.SHUFFLE_MAP_TASK, serializedInfoCache.clearOldValues)
+  // TODO: This object shouldn't have global variables
+  val metadataCleaner = new MetadataCleaner(
+    MetadataCleanerType.SHUFFLE_MAP_TASK, serializedInfoCache.clearOldValues, new SparkConf)
 
   def serializeInfo(stageId: Int, rdd: RDD[_], dep: ShuffleDependency[_,_]): Array[Byte] = {
     synchronized {
@@ -152,7 +154,7 @@ private[spark] class ShuffleMapTask(
 
     try {
       // Obtain all the block writers for shuffle blocks.
-      val ser = SparkEnv.get.serializerManager.get(dep.serializerClass)
+      val ser = SparkEnv.get.serializerManager.get(dep.serializerClass, SparkEnv.get.conf)
       shuffle = shuffleBlockManager.forMapTask(dep.shuffleId, partitionId, numOutputSplits, ser)
 
       // Write the map output to its associated buckets.

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala
----------------------------------------------------------------------
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 7e231ec..2707740 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
@@ -51,10 +51,10 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
 {
   val conf = sc.conf
   // How often to check for speculative tasks
-  val SPECULATION_INTERVAL = conf.getOrElse("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 = conf.getOrElse("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.
@@ -91,7 +91,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
   var rootPool: Pool = null
   // default scheduler is FIFO
   val schedulingMode: SchedulingMode = SchedulingMode.withName(
-    conf.getOrElse("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)
@@ -120,7 +120,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
   override def start() {
     backend.start()
 
-    if (conf.getOrElse("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,


[21/33] git commit: Fix some Python docs and make sure to unset SPARK_TESTING in Python tests so we don't get the test spark.conf on the classpath.

Posted by pw...@apache.org.
Fix some Python docs and make sure to unset SPARK_TESTING in Python
tests so we don't get the test spark.conf on the classpath.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/eaa8a68f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/eaa8a68f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/eaa8a68f

Branch: refs/heads/master
Commit: eaa8a68ff08304f713f4f75d39c61c020e0e691d
Parents: 11540b7
Author: Matei Zaharia <ma...@databricks.com>
Authored: Sun Dec 29 20:15:07 2013 -0500
Committer: Matei Zaharia <ma...@databricks.com>
Committed: Sun Dec 29 20:15:07 2013 -0500

----------------------------------------------------------------------
 python/epydoc.conf          |  2 +-
 python/pyspark/__init__.py  | 31 +++++++++++++++++--------------
 python/pyspark/broadcast.py | 11 +++++++++++
 python/pyspark/conf.py      | 10 +++++-----
 python/pyspark/context.py   |  3 ++-
 python/run-tests            |  2 +-
 6 files changed, 37 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/eaa8a68f/python/epydoc.conf
----------------------------------------------------------------------
diff --git a/python/epydoc.conf b/python/epydoc.conf
index 0b42e72..95a6af0 100644
--- a/python/epydoc.conf
+++ b/python/epydoc.conf
@@ -34,4 +34,4 @@ private: no
 
 exclude: pyspark.cloudpickle pyspark.worker pyspark.join
          pyspark.java_gateway pyspark.examples pyspark.shell pyspark.test
-         pyspark.rddsampler pyspark.daemon
+         pyspark.rddsampler pyspark.daemon pyspark.mllib._common

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/eaa8a68f/python/pyspark/__init__.py
----------------------------------------------------------------------
diff --git a/python/pyspark/__init__.py b/python/pyspark/__init__.py
index f1b95ac..2b2c3a0 100644
--- a/python/pyspark/__init__.py
+++ b/python/pyspark/__init__.py
@@ -20,21 +20,24 @@ PySpark is the Python API for Spark.
 
 Public classes:
 
-    - L{SparkContext<pyspark.context.SparkContext>}
-        Main entry point for Spark functionality.
-    - L{RDD<pyspark.rdd.RDD>}
-        A Resilient Distributed Dataset (RDD), the basic abstraction in Spark.
-    - L{Broadcast<pyspark.broadcast.Broadcast>}
-        A broadcast variable that gets reused across tasks.
-    - L{Accumulator<pyspark.accumulators.Accumulator>}
-        An "add-only" shared variable that tasks can only add values to.
-    - L{SparkConf<pyspark.conf.SparkConf}
-        Configuration for a Spark application.
-    - L{SparkFiles<pyspark.files.SparkFiles>}
-        Access files shipped with jobs.
-    - L{StorageLevel<pyspark.storagelevel.StorageLevel>}
-        Finer-grained cache persistence levels.
+  - L{SparkContext<pyspark.context.SparkContext>}
+      Main entry point for Spark functionality.
+  - L{RDD<pyspark.rdd.RDD>}
+      A Resilient Distributed Dataset (RDD), the basic abstraction in Spark.
+  - L{Broadcast<pyspark.broadcast.Broadcast>}
+      A broadcast variable that gets reused across tasks.
+  - L{Accumulator<pyspark.accumulators.Accumulator>}
+      An "add-only" shared variable that tasks can only add values to.
+  - L{SparkConf<pyspark.conf.SparkConf>}
+      For configuring Spark.
+  - L{SparkFiles<pyspark.files.SparkFiles>}
+      Access files shipped with jobs.
+  - L{StorageLevel<pyspark.storagelevel.StorageLevel>}
+      Finer-grained cache persistence levels.
 """
+
+
+
 import sys
 import os
 sys.path.insert(0, os.path.join(os.environ["SPARK_HOME"], "python/lib/py4j0.7.egg"))

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/eaa8a68f/python/pyspark/broadcast.py
----------------------------------------------------------------------
diff --git a/python/pyspark/broadcast.py b/python/pyspark/broadcast.py
index dfdaba2..43f40f8 100644
--- a/python/pyspark/broadcast.py
+++ b/python/pyspark/broadcast.py
@@ -45,7 +45,18 @@ def _from_id(bid):
 
 
 class Broadcast(object):
+    """
+    A broadcast variable created with
+    L{SparkContext.broadcast()<pyspark.context.SparkContext.broadcast>}.
+    Access its value through C{.value}.
+    """
+
     def __init__(self, bid, value, java_broadcast=None, pickle_registry=None):
+        """
+        Should not be called directly by users -- use
+        L{SparkContext.broadcast()<pyspark.context.SparkContext.broadcast>}
+        instead.
+        """
         self.value = value
         self.bid = bid
         self._jbroadcast = java_broadcast

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/eaa8a68f/python/pyspark/conf.py
----------------------------------------------------------------------
diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py
index a79f348..cf98b0e 100644
--- a/python/pyspark/conf.py
+++ b/python/pyspark/conf.py
@@ -55,11 +55,11 @@ class SparkConf(object):
     parameters as key-value pairs.
 
     Most of the time, you would create a SparkConf object with
-    C{SparkConf()}, which will load values from `spark.*` Java system
-    properties and any `spark.conf` on your application's classpath.
-    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.
+    C{SparkConf()}, which will load values from C{spark.*} Java system
+    properties and any C{spark.conf} on your Spark classpath. In this
+    case, system properties take priority over C{spark.conf}, and any
+    parameters you set directly on the C{SparkConf} object take priority
+    over both of those.
 
     For unit tests, you can also call C{SparkConf(false)} to skip
     loading external settings and get the same configuration no matter

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/eaa8a68f/python/pyspark/context.py
----------------------------------------------------------------------
diff --git a/python/pyspark/context.py b/python/pyspark/context.py
index 1244a14..8b02802 100644
--- a/python/pyspark/context.py
+++ b/python/pyspark/context.py
@@ -267,7 +267,8 @@ class SparkContext(object):
 
     def broadcast(self, value):
         """
-        Broadcast a read-only variable to the cluster, returning a C{Broadcast}
+        Broadcast a read-only variable to the cluster, returning a
+        L{Broadcast<pyspark.broadcast.Broadcast>}
         object for reading it in distributed functions. The variable will be
         sent to each cluster only once.
         """

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/eaa8a68f/python/run-tests
----------------------------------------------------------------------
diff --git a/python/run-tests b/python/run-tests
index a0898b3..4b71fff 100755
--- a/python/run-tests
+++ b/python/run-tests
@@ -29,7 +29,7 @@ FAILED=0
 rm -f unit-tests.log
 
 function run_test() {
-    $FWDIR/pyspark $1 2>&1 | tee -a unit-tests.log
+    SPARK_TESTING=0 $FWDIR/pyspark $1 2>&1 | tee -a unit-tests.log
     FAILED=$((PIPESTATUS[0]||$FAILED))
 }
 


[32/33] git commit: Fix Python code after change of getOrElse

Posted by pw...@apache.org.
Fix Python code after change of getOrElse


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/7e8d2e8a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/7e8d2e8a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/7e8d2e8a

Branch: refs/heads/master
Commit: 7e8d2e8a5c88d16c771923504c433491b109ab2a
Parents: 0f60607
Author: Matei Zaharia <ma...@databricks.com>
Authored: Wed Jan 1 23:21:34 2014 -0500
Committer: Matei Zaharia <ma...@databricks.com>
Committed: Wed Jan 1 23:21:34 2014 -0500

----------------------------------------------------------------------
 python/pyspark/conf.py    |  7 ++++++-
 python/pyspark/context.py | 14 ++++++++------
 2 files changed, 14 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/7e8d2e8a/python/pyspark/conf.py
----------------------------------------------------------------------
diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py
index c111e2e..d72aed6 100644
--- a/python/pyspark/conf.py
+++ b/python/pyspark/conf.py
@@ -134,7 +134,12 @@ class SparkConf(object):
 
     def get(self, key, defaultValue=None):
         """Get the configured value for some key, or return a default otherwise."""
-        return self._jconf.get(key, defaultValue)
+        if defaultValue == None:   # Py4J doesn't call the right get() if we pass None
+            if not self._jconf.contains(key):
+                return None
+            return self._jconf.get(key)
+        else:
+            return self._jconf.get(key, defaultValue)
 
     def getAll(self):
         """Get all values as a list of key-value pairs."""

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/7e8d2e8a/python/pyspark/context.py
----------------------------------------------------------------------
diff --git a/python/pyspark/context.py b/python/pyspark/context.py
index d77dd76..f955aad 100644
--- a/python/pyspark/context.py
+++ b/python/pyspark/context.py
@@ -92,11 +92,13 @@ class SparkContext(object):
             self.serializer = BatchedSerializer(self._unbatched_serializer,
                                                 batchSize)
 
-        # Set parameters passed directly to us on the conf; these operations will be
-        # no-ops if the parameters were None
-        self._conf.setMaster(master)
-        self._conf.setAppName(appName)
-        self._conf.setSparkHome(sparkHome)
+        # Set any parameters passed directly to us on the conf
+        if master:
+            self._conf.setMaster(master)
+        if appName:
+            self._conf.setAppName(appName)
+        if sparkHome:
+            self._conf.setSparkHome(sparkHome)
         if environment:
             for key, value in environment.iteritems():
                 self._conf.setExecutorEnv(key, value)
@@ -111,7 +113,7 @@ class SparkContext(object):
         # the classpath or an external config file
         self.master = self._conf.get("spark.master")
         self.appName = self._conf.get("spark.app.name")
-        self.sparkHome = self._conf.getOrElse("spark.home", None)
+        self.sparkHome = self._conf.get("spark.home", None)
         for (k, v) in self._conf.getAll():
             if k.startswith("spark.executorEnv."):
                 varName = k[len("spark.executorEnv."):]


[17/33] git commit: Merge remote-tracking branch 'origin/master' into conf2

Posted by pw...@apache.org.
Merge remote-tracking branch 'origin/master' into conf2

Conflicts:
	core/src/main/scala/org/apache/spark/SparkContext.scala
	core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
	core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
	core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala
	core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala
	core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
	core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala
	core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
	new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
	streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
	streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
	streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
	streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
	streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
	streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
	streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
	streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/b4ceed40
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/b4ceed40
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/b4ceed40

Branch: refs/heads/master
Commit: b4ceed40d6e511a1d475b3f4fbcdd2ad24c02b5a
Parents: 58c6fa2 ad3dfd1
Author: Matei Zaharia <ma...@databricks.com>
Authored: Sun Dec 29 15:08:08 2013 -0500
Committer: Matei Zaharia <ma...@databricks.com>
Committed: Sun Dec 29 15:08:08 2013 -0500

----------------------------------------------------------------------
 .../scala/org/apache/spark/SparkContext.scala   |  45 +-
 .../scala/org/apache/spark/TaskEndReason.scala  |   2 -
 .../spark/deploy/worker/ui/WorkerWebUI.scala    |   6 +-
 .../apache/spark/scheduler/DAGScheduler.scala   |  42 +-
 .../spark/scheduler/ExecutorLossReason.scala    |  38 +
 .../org/apache/spark/scheduler/JobLogger.scala  |   4 -
 .../scala/org/apache/spark/scheduler/Pool.scala |   4 -
 .../apache/spark/scheduler/Schedulable.scala    |   1 -
 .../spark/scheduler/SchedulerBackend.scala      |  34 +
 .../apache/spark/scheduler/SparkListener.scala  |   7 +-
 .../spark/scheduler/SparkListenerBus.scala      |   1 -
 .../spark/scheduler/TaskResultGetter.scala      | 108 +++
 .../apache/spark/scheduler/TaskScheduler.scala  |  11 +-
 .../spark/scheduler/TaskSchedulerImpl.scala     | 469 ++++++++++++
 .../apache/spark/scheduler/TaskSetManager.scala | 700 +++++++++++++++++-
 .../apache/spark/scheduler/WorkerOffer.scala    |  24 +
 .../scheduler/cluster/ClusterScheduler.scala    | 474 ------------
 .../cluster/ClusterTaskSetManager.scala         | 714 -------------------
 .../cluster/CoarseGrainedSchedulerBackend.scala |   6 +-
 .../scheduler/cluster/ExecutorLossReason.scala  |  38 -
 .../scheduler/cluster/SchedulerBackend.scala    |  34 -
 .../cluster/SimrSchedulerBackend.scala          |   4 +-
 .../cluster/SparkDeploySchedulerBackend.scala   |   6 +-
 .../scheduler/cluster/TaskResultGetter.scala    | 109 ---
 .../spark/scheduler/cluster/WorkerOffer.scala   |  24 -
 .../mesos/CoarseMesosSchedulerBackend.scala     |   5 +-
 .../cluster/mesos/MesosSchedulerBackend.scala   |   7 +-
 .../spark/scheduler/local/LocalBackend.scala    | 108 +++
 .../spark/scheduler/local/LocalScheduler.scala  | 224 ------
 .../scheduler/local/LocalTaskSetManager.scala   | 191 -----
 .../spark/storage/ShuffleBlockManager.scala     |  25 +-
 .../org/apache/spark/ui/exec/ExecutorsUI.scala  |  31 +-
 .../apache/spark/ui/jobs/ExecutorSummary.scala  |  27 +
 .../apache/spark/ui/jobs/ExecutorTable.scala    |  90 +++
 .../spark/ui/jobs/JobProgressListener.scala     |  31 +
 .../org/apache/spark/ui/jobs/StagePage.scala    |   5 +-
 .../org/apache/spark/ui/jobs/StageTable.scala   |   2 +-
 .../org/apache/spark/util/MetadataCleaner.scala |   2 +-
 .../apache/spark/util/TimeStampedHashMap.scala  |  15 +-
 .../scala/org/apache/spark/FailureSuite.scala   |   4 +-
 .../SparkContextSchedulerCreationSuite.scala    |  58 +-
 .../spark/scheduler/ClusterSchedulerSuite.scala | 265 +++++++
 .../org/apache/spark/scheduler/FakeTask.scala   |  26 +
 .../spark/scheduler/SparkListenerSuite.scala    |  19 +-
 .../spark/scheduler/TaskResultGetterSuite.scala | 112 +++
 .../spark/scheduler/TaskSetManagerSuite.scala   | 321 +++++++++
 .../cluster/ClusterSchedulerSuite.scala         | 267 -------
 .../cluster/ClusterTaskSetManagerSuite.scala    | 319 ---------
 .../spark/scheduler/cluster/FakeTask.scala      |  27 -
 .../cluster/TaskResultGetterSuite.scala         | 114 ---
 .../scheduler/local/LocalSchedulerSuite.scala   | 227 ------
 .../ui/jobs/JobProgressListenerSuite.scala      |  73 ++
 .../streaming/examples/JavaKafkaWordCount.java  |   2 +-
 .../spark/mllib/api/python/PythonMLLibAPI.scala | 232 ++++++
 .../org/apache/spark/deploy/yarn/Client.scala   |   4 +-
 .../deploy/yarn/YarnAllocationHandler.scala     |  10 +-
 .../cluster/YarnClientClusterScheduler.scala    |   3 +-
 .../cluster/YarnClientSchedulerBackend.scala    |   3 +-
 .../cluster/YarnClusterScheduler.scala          |   3 +-
 project/SparkBuild.scala                        |   5 +-
 python/pyspark/java_gateway.py                  |   1 +
 python/pyspark/mllib/__init__.py                |  20 +
 python/pyspark/mllib/_common.py                 | 227 ++++++
 python/pyspark/mllib/classification.py          |  86 +++
 python/pyspark/mllib/clustering.py              |  79 ++
 python/pyspark/mllib/recommendation.py          |  74 ++
 python/pyspark/mllib/regression.py              | 110 +++
 python/pyspark/serializers.py                   |   2 +-
 python/pyspark/shell.py                         |   2 +-
 spark-class                                     |   6 +-
 spark-class2.cmd                                |   4 +-
 .../org/apache/spark/streaming/Checkpoint.scala |   4 +-
 .../org/apache/spark/streaming/DStream.scala    |  11 +-
 .../apache/spark/streaming/DStreamGraph.scala   |   1 +
 .../scala/org/apache/spark/streaming/Job.scala  |  41 --
 .../org/apache/spark/streaming/JobManager.scala |  88 ---
 .../spark/streaming/NetworkInputTracker.scala   | 174 -----
 .../org/apache/spark/streaming/Scheduler.scala  | 131 ----
 .../spark/streaming/StreamingContext.scala      |  26 +-
 .../api/java/JavaStreamingContext.scala         |   8 +
 .../streaming/dstream/ForEachDStream.scala      |   3 +-
 .../streaming/dstream/NetworkInputDStream.scala |   1 +
 .../spark/streaming/scheduler/BatchInfo.scala   |  55 ++
 .../apache/spark/streaming/scheduler/Job.scala  |  41 ++
 .../streaming/scheduler/JobGenerator.scala      | 132 ++++
 .../streaming/scheduler/JobScheduler.scala      | 108 +++
 .../spark/streaming/scheduler/JobSet.scala      |  68 ++
 .../scheduler/NetworkInputTracker.scala         | 175 +++++
 .../streaming/scheduler/StreamingListener.scala |  75 ++
 .../scheduler/StreamingListenerBus.scala        |  81 +++
 .../spark/streaming/BasicOperationsSuite.scala  |  11 -
 .../spark/streaming/CheckpointSuite.scala       |  26 +-
 .../apache/spark/streaming/FailureSuite.scala   |  13 +-
 .../spark/streaming/InputStreamsSuite.scala     |  10 -
 .../streaming/StreamingListenerSuite.scala      |  71 ++
 .../apache/spark/streaming/TestSuiteBase.scala  |  34 +-
 .../spark/streaming/WindowOperationsSuite.scala |  14 +-
 .../org/apache/spark/deploy/yarn/Client.scala   |   4 +-
 .../deploy/yarn/YarnAllocationHandler.scala     |  10 +-
 .../cluster/YarnClientClusterScheduler.scala    |   3 +-
 .../cluster/YarnClientSchedulerBackend.scala    |   3 +-
 .../cluster/YarnClusterScheduler.scala          |  10 +-
 102 files changed, 4359 insertions(+), 3446 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/core/src/main/scala/org/apache/spark/SparkContext.scala
----------------------------------------------------------------------
diff --cc core/src/main/scala/org/apache/spark/SparkContext.scala
index c109ff9,ad3337d..6f54fa7
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@@ -43,14 -55,14 +43,13 @@@ import org.apache.spark.deploy.{LocalSp
  import org.apache.spark.partial.{ApproximateEvaluator, PartialResult}
  import org.apache.spark.rdd._
  import org.apache.spark.scheduler._
- 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.cluster.{CoarseGrainedSchedulerBackend,
+   SparkDeploySchedulerBackend, SimrSchedulerBackend}
+ import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend}
+ import org.apache.spark.scheduler.local.LocalBackend
  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
@@@ -560,10 -521,8 +559,8 @@@ class SparkContext
      }
      addedFiles(key) = System.currentTimeMillis
  
-     // Fetch the file locally in case a job is executed locally.
-     // Jobs that run through LocalScheduler will already fetch the required dependencies,
-     // but jobs run in DAGScheduler.runLocally() will not so we must fetch the files here.
+     // Fetch the file locally in case a job is executed using DAGScheduler.runLocally().
 -    Utils.fetchFile(path, new File(SparkFiles.getRootDirectory))
 +    Utils.fetchFile(path, new File(SparkFiles.getRootDirectory), conf)
  
      logInfo("Added file " + path + " at " + key + " with timestamp " + addedFiles(key))
    }
@@@ -1090,13 -1071,13 +1099,13 @@@ object SparkContext 
        case LOCAL_CLUSTER_REGEX(numSlaves, coresPerSlave, memoryPerSlave) =>
          // Check to make sure memory requested <= memoryPerSlave. Otherwise Spark will just hang.
          val memoryPerSlaveInt = memoryPerSlave.toInt
 -        if (SparkContext.executorMemoryRequested > memoryPerSlaveInt) {
 +        if (sc.executorMemory > memoryPerSlaveInt) {
            throw new SparkException(
              "Asked to launch cluster with %d MB RAM / worker but requested %d MB/worker".format(
 -              memoryPerSlaveInt, SparkContext.executorMemoryRequested))
 +              memoryPerSlaveInt, sc.executorMemory))
          }
  
-         val scheduler = new ClusterScheduler(sc)
+         val scheduler = new TaskSchedulerImpl(sc)
          val localCluster = new LocalSparkCluster(
            numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt)
          val masterUrls = localCluster.start()
@@@ -1150,8 -1131,8 +1159,8 @@@
  
        case mesosUrl @ MESOS_REGEX(_) =>
          MesosNativeLibrary.load()
-         val scheduler = new ClusterScheduler(sc)
+         val scheduler = new TaskSchedulerImpl(sc)
 -        val coarseGrained = System.getProperty("spark.mesos.coarse", "false").toBoolean
 +        val coarseGrained = sc.conf.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)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala
----------------------------------------------------------------------
diff --cc core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala
index 0000000,89aa098..02bdbba
mode 000000,100644..100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala
@@@ -1,0 -1,37 +1,34 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one or more
+  * contributor license agreements.  See the NOTICE file distributed with
+  * this work for additional information regarding copyright ownership.
+  * The ASF licenses this file to You under the Apache License, Version 2.0
+  * (the "License"); you may not use this file except in compliance with
+  * the License.  You may obtain a copy of the License at
+  *
+  *    http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ 
+ package org.apache.spark.scheduler
+ 
+ import org.apache.spark.SparkContext
+ 
+ /**
+  * A backend interface for scheduling systems that allows plugging in different ones under
+  * ClusterScheduler. We assume a Mesos-like model where the application gets resource offers as
+  * machines become available and can launch tasks on them.
+  */
+ private[spark] trait SchedulerBackend {
+   def start(): Unit
+   def stop(): Unit
+   def reviveOffers(): Unit
+   def defaultParallelism(): Int
+ 
+   def killTask(taskId: Long, executorId: String): Unit = throw new UnsupportedOperationException
 -
 -  // Memory used by each executor (in megabytes)
 -  protected val executorMemory: Int = SparkContext.executorMemoryRequested
+ }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala
----------------------------------------------------------------------
diff --cc core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala
index 0000000,8910272..29b0247
mode 000000,100644..100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala
@@@ -1,0 -1,107 +1,108 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one or more
+  * contributor license agreements.  See the NOTICE file distributed with
+  * this work for additional information regarding copyright ownership.
+  * The ASF licenses this file to You under the Apache License, Version 2.0
+  * (the "License"); you may not use this file except in compliance with
+  * the License.  You may obtain a copy of the License at
+  *
+  *    http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ 
+ package org.apache.spark.scheduler
+ 
+ import java.nio.ByteBuffer
+ import java.util.concurrent.{LinkedBlockingDeque, ThreadFactory, ThreadPoolExecutor, TimeUnit}
+ 
+ import org.apache.spark._
+ import org.apache.spark.TaskState.TaskState
+ import org.apache.spark.serializer.SerializerInstance
+ import org.apache.spark.util.Utils
+ 
+ /**
+  * Runs a thread pool that deserializes and remotely fetches (if necessary) task results.
+  */
+ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedulerImpl)
+   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")
+ 
+   protected val serializer = new ThreadLocal[SerializerInstance] {
+     override def initialValue(): SerializerInstance = {
+       return sparkEnv.closureSerializer.newInstance()
+     }
+   }
+ 
+   def enqueueSuccessfulTask(
+     taskSetManager: TaskSetManager, tid: Long, serializedData: ByteBuffer) {
+     getTaskResultExecutor.execute(new Runnable {
+       override def run() {
+         try {
+           val result = serializer.get().deserialize[TaskResult[_]](serializedData) match {
+             case directResult: DirectTaskResult[_] => directResult
+             case IndirectTaskResult(blockId) =>
+               logDebug("Fetching indirect task result for TID %s".format(tid))
+               scheduler.handleTaskGettingResult(taskSetManager, tid)
+               val serializedTaskResult = sparkEnv.blockManager.getRemoteBytes(blockId)
+               if (!serializedTaskResult.isDefined) {
+                 /* We won't be able to get the task result if the machine that ran the task failed
+                  * between when the task ended and when we tried to fetch the result, or if the
+                  * block manager had to flush the result. */
+                 scheduler.handleFailedTask(
+                   taskSetManager, tid, TaskState.FINISHED, Some(TaskResultLost))
+                 return
+               }
+               val deserializedResult = serializer.get().deserialize[DirectTaskResult[_]](
+                 serializedTaskResult.get)
+               sparkEnv.blockManager.master.removeBlock(blockId)
+               deserializedResult
+           }
+           result.metrics.resultSize = serializedData.limit()
+           scheduler.handleSuccessfulTask(taskSetManager, tid, result)
+         } catch {
+           case cnf: ClassNotFoundException =>
+             val loader = Thread.currentThread.getContextClassLoader
+             taskSetManager.abort("ClassNotFound with classloader: " + loader)
+           case ex: Throwable =>
+             taskSetManager.abort("Exception while deserializing and fetching task: %s".format(ex))
+         }
+       }
+     })
+   }
+ 
+   def enqueueFailedTask(taskSetManager: TaskSetManager, tid: Long, taskState: TaskState,
+     serializedData: ByteBuffer) {
+     var reason: Option[TaskEndReason] = None
+     getTaskResultExecutor.execute(new Runnable {
+       override def run() {
+         try {
+           if (serializedData != null && serializedData.limit() > 0) {
+             reason = Some(serializer.get().deserialize[TaskEndReason](
+               serializedData, getClass.getClassLoader))
+           }
+         } catch {
+           case cnd: ClassNotFoundException =>
+             // Log an error but keep going here -- the task failed, so not catastropic if we can't
+             // deserialize the reason.
+             val loader = Thread.currentThread.getContextClassLoader
+             logError(
+               "Could not deserialize TaskEndReason: ClassNotFound with classloader " + loader)
+           case ex: Throwable => {}
+         }
+         scheduler.handleFailedTask(taskSetManager, tid, taskState, reason)
+       }
+     })
+   }
+ 
+   def stop() {
+     getTaskResultExecutor.shutdownNow()
+   }
+ }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
----------------------------------------------------------------------
diff --cc core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
index 0000000,1b0f82f..56a038d
mode 000000,100644..100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
@@@ -1,0 -1,467 +1,469 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one or more
+  * contributor license agreements.  See the NOTICE file distributed with
+  * this work for additional information regarding copyright ownership.
+  * The ASF licenses this file to You under the Apache License, Version 2.0
+  * (the "License"); you may not use this file except in compliance with
+  * the License.  You may obtain a copy of the License at
+  *
+  *    http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ 
+ package org.apache.spark.scheduler
+ 
+ import java.nio.ByteBuffer
+ import java.util.concurrent.atomic.AtomicLong
+ import java.util.{TimerTask, Timer}
+ 
+ import scala.collection.mutable.ArrayBuffer
+ import scala.collection.mutable.HashMap
+ import scala.collection.mutable.HashSet
+ import scala.concurrent.duration._
+ 
+ import org.apache.spark._
+ import org.apache.spark.TaskState.TaskState
+ import org.apache.spark.scheduler.SchedulingMode.SchedulingMode
+ 
+ /**
+  * Schedules tasks for multiple types of clusters by acting through a SchedulerBackend.
+  * It can also work with a local setup by using a LocalBackend and setting isLocal to true.
+  * It handles common logic, like determining a scheduling order across jobs, waking up to launch
+  * speculative tasks, etc.
 - * 
++ *
+  * Clients should first call initialize() and start(), then submit task sets through the
+  * runTasks method.
+  *
+  * THREADING: SchedulerBackends and task-submitting clients can call this class from multiple
+  * threads, so it needs locks in public API methods to maintain its state. In addition, some
+  * SchedulerBackends sycnchronize on themselves when they want to send events here, and then
+  * acquire a lock on us, so we need to make sure that we don't try to lock the backend while
+  * we are holding a lock on ourselves.
+  */
+ private[spark] class TaskSchedulerImpl(
+     val sc: SparkContext,
 -    val maxTaskFailures : Int = System.getProperty("spark.task.maxFailures", "4").toInt,
++    val maxTaskFailures: Int = System.getProperty("spark.task.maxFailures", "4").toInt,
+     isLocal: Boolean = false)
 -  extends TaskScheduler with Logging {
++  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
+ 
+   // TaskSetManagers are not thread safe, so any access to one should be synchronized
+   // on this class.
+   val activeTaskSets = new HashMap[String, TaskSetManager]
+ 
+   val taskIdToTaskSetId = new HashMap[Long, String]
+   val taskIdToExecutorId = new HashMap[Long, String]
+   val taskSetTaskIds = new HashMap[String, HashSet[Long]]
+ 
+   @volatile private var hasReceivedTask = false
+   @volatile private var hasLaunchedTask = false
+   private val starvationTimer = new Timer(true)
+ 
+   // Incrementing task IDs
+   val nextTaskId = new AtomicLong(0)
+ 
+   // Which executor IDs we have executors on
+   val activeExecutorIds = new HashSet[String]
+ 
+   // The set of executors we have on each host; this is used to compute hostsAlive, which
+   // in turn is used to decide when we can attain data locality on a given host
+   private val executorsByHost = new HashMap[String, HashSet[String]]
+ 
+   private val executorIdToHost = new HashMap[String, String]
+ 
+   // Listener object to pass upcalls into
+   var dagScheduler: DAGScheduler = null
+ 
+   var backend: SchedulerBackend = null
+ 
+   val mapOutputTracker = SparkEnv.get.mapOutputTracker
+ 
+   var schedulableBuilder: SchedulableBuilder = null
+   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)
+ 
+   override def setDAGScheduler(dagScheduler: DAGScheduler) {
+     this.dagScheduler = dagScheduler
+   }
+ 
+   def initialize(backend: SchedulerBackend) {
+     this.backend = backend
+     // temporarily set rootPool name to empty
+     rootPool = new Pool("", schedulingMode, 0, 0)
+     schedulableBuilder = {
+       schedulingMode match {
+         case SchedulingMode.FIFO =>
+           new FIFOSchedulableBuilder(rootPool)
+         case SchedulingMode.FAIR =>
 -          new FairSchedulableBuilder(rootPool)
++          new FairSchedulableBuilder(rootPool, conf)
+       }
+     }
+     schedulableBuilder.buildPools()
+   }
+ 
+   def newTaskId(): Long = nextTaskId.getAndIncrement()
+ 
+   override def start() {
+     backend.start()
+ 
 -    if (!isLocal && System.getProperty("spark.speculation", "false").toBoolean) {
++    if (!isLocal && conf.getOrElse("spark.speculation", "false").toBoolean) {
+       logInfo("Starting speculative execution thread")
+       import sc.env.actorSystem.dispatcher
+       sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL milliseconds,
+             SPECULATION_INTERVAL milliseconds) {
+         checkSpeculatableTasks()
+       }
+     }
+   }
+ 
+   override def submitTasks(taskSet: TaskSet) {
+     val tasks = taskSet.tasks
+     logInfo("Adding task set " + taskSet.id + " with " + tasks.length + " tasks")
+     this.synchronized {
+       val manager = new TaskSetManager(this, taskSet, maxTaskFailures)
+       activeTaskSets(taskSet.id) = manager
+       schedulableBuilder.addTaskSetManager(manager, manager.taskSet.properties)
+       taskSetTaskIds(taskSet.id) = new HashSet[Long]()
+ 
+       if (!isLocal && !hasReceivedTask) {
+         starvationTimer.scheduleAtFixedRate(new TimerTask() {
+           override def run() {
+             if (!hasLaunchedTask) {
+               logWarning("Initial job has not accepted any resources; " +
+                 "check your cluster UI to ensure that workers are registered " +
+                 "and have sufficient memory")
+             } else {
+               this.cancel()
+             }
+           }
+         }, STARVATION_TIMEOUT, STARVATION_TIMEOUT)
+       }
+       hasReceivedTask = true
+     }
+     backend.reviveOffers()
+   }
+ 
+   override def cancelTasks(stageId: Int): Unit = synchronized {
+     logInfo("Cancelling stage " + stageId)
+     activeTaskSets.find(_._2.stageId == stageId).foreach { case (_, tsm) =>
+       // There are two possible cases here:
+       // 1. The task set manager has been created and some tasks have been scheduled.
+       //    In this case, send a kill signal to the executors to kill the task and then abort
+       //    the stage.
+       // 2. The task set manager has been created but no tasks has been scheduled. In this case,
+       //    simply abort the stage.
+       val taskIds = taskSetTaskIds(tsm.taskSet.id)
+       if (taskIds.size > 0) {
+         taskIds.foreach { tid =>
+           val execId = taskIdToExecutorId(tid)
+           backend.killTask(tid, execId)
+         }
+       }
+       logInfo("Stage %d was cancelled".format(stageId))
+       tsm.removeAllRunningTasks()
+       taskSetFinished(tsm)
+     }
+   }
+ 
+   def taskSetFinished(manager: TaskSetManager): Unit = synchronized {
+     // Check to see if the given task set has been removed. This is possible in the case of
+     // multiple unrecoverable task failures (e.g. if the entire task set is killed when it has
+     // more than one running tasks).
+     if (activeTaskSets.contains(manager.taskSet.id)) {
+       activeTaskSets -= manager.taskSet.id
+       manager.parent.removeSchedulable(manager)
+       logInfo("Remove TaskSet %s from pool %s".format(manager.taskSet.id, manager.parent.name))
+       taskIdToTaskSetId --= taskSetTaskIds(manager.taskSet.id)
+       taskIdToExecutorId --= taskSetTaskIds(manager.taskSet.id)
+       taskSetTaskIds.remove(manager.taskSet.id)
+     }
+   }
+ 
+   /**
+    * Called by cluster manager to offer resources on slaves. We respond by asking our active task
+    * sets for tasks in order of priority. We fill each node with tasks in a round-robin manner so
+    * that tasks are balanced across the cluster.
+    */
+   def resourceOffers(offers: Seq[WorkerOffer]): Seq[Seq[TaskDescription]] = synchronized {
+     SparkEnv.set(sc.env)
+ 
+     // Mark each slave as alive and remember its hostname
+     for (o <- offers) {
+       executorIdToHost(o.executorId) = o.host
+       if (!executorsByHost.contains(o.host)) {
+         executorsByHost(o.host) = new HashSet[String]()
+         executorGained(o.executorId, o.host)
+       }
+     }
+ 
+     // Build a list of tasks to assign to each worker
+     val tasks = offers.map(o => new ArrayBuffer[TaskDescription](o.cores))
+     val availableCpus = offers.map(o => o.cores).toArray
+     val sortedTaskSets = rootPool.getSortedTaskSetQueue()
+     for (taskSet <- sortedTaskSets) {
+       logDebug("parentName: %s, name: %s, runningTasks: %s".format(
+         taskSet.parent.name, taskSet.name, taskSet.runningTasks))
+     }
+ 
+     // Take each TaskSet in our scheduling order, and then offer it each node in increasing order
+     // of locality levels so that it gets a chance to launch local tasks on all of them.
+     var launchedTask = false
+     for (taskSet <- sortedTaskSets; maxLocality <- TaskLocality.values) {
+       do {
+         launchedTask = false
+         for (i <- 0 until offers.size) {
+           val execId = offers(i).executorId
+           val host = offers(i).host
+           for (task <- taskSet.resourceOffer(execId, host, availableCpus(i), maxLocality)) {
+             tasks(i) += task
+             val tid = task.taskId
+             taskIdToTaskSetId(tid) = taskSet.taskSet.id
+             taskSetTaskIds(taskSet.taskSet.id) += tid
+             taskIdToExecutorId(tid) = execId
+             activeExecutorIds += execId
+             executorsByHost(host) += execId
+             availableCpus(i) -= 1
+             launchedTask = true
+           }
+         }
+       } while (launchedTask)
+     }
+ 
+     if (tasks.size > 0) {
+       hasLaunchedTask = true
+     }
+     return tasks
+   }
+ 
+   def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) {
+     var failedExecutor: Option[String] = None
+     synchronized {
+       try {
+         if (state == TaskState.LOST && taskIdToExecutorId.contains(tid)) {
+           // We lost this entire executor, so remember that it's gone
+           val execId = taskIdToExecutorId(tid)
+           if (activeExecutorIds.contains(execId)) {
+             removeExecutor(execId)
+             failedExecutor = Some(execId)
+           }
+         }
+         taskIdToTaskSetId.get(tid) match {
+           case Some(taskSetId) =>
+             if (TaskState.isFinished(state)) {
+               taskIdToTaskSetId.remove(tid)
+               if (taskSetTaskIds.contains(taskSetId)) {
+                 taskSetTaskIds(taskSetId) -= tid
+               }
+               taskIdToExecutorId.remove(tid)
+             }
+             activeTaskSets.get(taskSetId).foreach { taskSet =>
+               if (state == TaskState.FINISHED) {
+                 taskSet.removeRunningTask(tid)
+                 taskResultGetter.enqueueSuccessfulTask(taskSet, tid, serializedData)
+               } else if (Set(TaskState.FAILED, TaskState.KILLED, TaskState.LOST).contains(state)) {
+                 taskSet.removeRunningTask(tid)
+                 taskResultGetter.enqueueFailedTask(taskSet, tid, state, serializedData)
+               }
+             }
+           case None =>
+             logInfo("Ignoring update from TID " + tid + " because its task set is gone")
+         }
+       } catch {
+         case e: Exception => logError("Exception in statusUpdate", e)
+       }
+     }
+     // Update the DAGScheduler without holding a lock on this, since that can deadlock
+     if (failedExecutor != None) {
+       dagScheduler.executorLost(failedExecutor.get)
+       backend.reviveOffers()
+     }
+   }
+ 
+   def handleTaskGettingResult(taskSetManager: TaskSetManager, tid: Long) {
+     taskSetManager.handleTaskGettingResult(tid)
+   }
+ 
+   def handleSuccessfulTask(
+     taskSetManager: TaskSetManager,
+     tid: Long,
+     taskResult: DirectTaskResult[_]) = synchronized {
+     taskSetManager.handleSuccessfulTask(tid, taskResult)
+   }
+ 
+   def handleFailedTask(
+     taskSetManager: TaskSetManager,
+     tid: Long,
+     taskState: TaskState,
+     reason: Option[TaskEndReason]) = synchronized {
+     taskSetManager.handleFailedTask(tid, taskState, reason)
+     if (taskState != TaskState.KILLED) {
+       // Need to revive offers again now that the task set manager state has been updated to
+       // reflect failed tasks that need to be re-run.
+       backend.reviveOffers()
+     }
+   }
+ 
+   def error(message: String) {
+     synchronized {
+       if (activeTaskSets.size > 0) {
+         // Have each task set throw a SparkException with the error
+         for ((taskSetId, manager) <- activeTaskSets) {
+           try {
+             manager.error(message)
+           } catch {
+             case e: Exception => logError("Exception in error callback", e)
+           }
+         }
+       } else {
+         // No task sets are active but we still got an error. Just exit since this
+         // must mean the error is during registration.
+         // It might be good to do something smarter here in the future.
+         logError("Exiting due to error from cluster scheduler: " + message)
+         System.exit(1)
+       }
+     }
+   }
+ 
+   override def stop() {
+     if (backend != null) {
+       backend.stop()
+     }
+     if (taskResultGetter != null) {
+       taskResultGetter.stop()
+     }
+ 
+     // sleeping for an arbitrary 5 seconds : to ensure that messages are sent out.
+     // TODO: Do something better !
+     Thread.sleep(5000L)
+   }
+ 
+   override def defaultParallelism() = backend.defaultParallelism()
+ 
+   // Check for speculatable tasks in all our active jobs.
+   def checkSpeculatableTasks() {
+     var shouldRevive = false
+     synchronized {
+       shouldRevive = rootPool.checkSpeculatableTasks()
+     }
+     if (shouldRevive) {
+       backend.reviveOffers()
+     }
+   }
+ 
+   def executorLost(executorId: String, reason: ExecutorLossReason) {
+     var failedExecutor: Option[String] = None
+ 
+     synchronized {
+       if (activeExecutorIds.contains(executorId)) {
+         val hostPort = executorIdToHost(executorId)
+         logError("Lost executor %s on %s: %s".format(executorId, hostPort, reason))
+         removeExecutor(executorId)
+         failedExecutor = Some(executorId)
+       } else {
+          // We may get multiple executorLost() calls with different loss reasons. For example, one
+          // may be triggered by a dropped connection from the slave while another may be a report
+          // of executor termination from Mesos. We produce log messages for both so we eventually
+          // report the termination reason.
+          logError("Lost an executor " + executorId + " (already removed): " + reason)
+       }
+     }
+     // Call dagScheduler.executorLost without holding the lock on this to prevent deadlock
+     if (failedExecutor != None) {
+       dagScheduler.executorLost(failedExecutor.get)
+       backend.reviveOffers()
+     }
+   }
+ 
+   /** Remove an executor from all our data structures and mark it as lost */
+   private def removeExecutor(executorId: String) {
+     activeExecutorIds -= executorId
+     val host = executorIdToHost(executorId)
+     val execs = executorsByHost.getOrElse(host, new HashSet)
+     execs -= executorId
+     if (execs.isEmpty) {
+       executorsByHost -= host
+     }
+     executorIdToHost -= executorId
+     rootPool.executorLost(executorId, host)
+   }
+ 
+   def executorGained(execId: String, host: String) {
+     dagScheduler.executorGained(execId, host)
+   }
+ 
+   def getExecutorsAliveOnHost(host: String): Option[Set[String]] = synchronized {
+     executorsByHost.get(host).map(_.toSet)
+   }
+ 
+   def hasExecutorsAliveOnHost(host: String): Boolean = synchronized {
+     executorsByHost.contains(host)
+   }
+ 
+   def isExecutorAlive(execId: String): Boolean = synchronized {
+     activeExecutorIds.contains(execId)
+   }
+ 
+   // By default, rack is unknown
+   def getRackForHost(value: String): Option[String] = None
+ }
+ 
+ 
+ private[spark] object TaskSchedulerImpl {
+   /**
+    * Used to balance containers across hosts.
+    *
+    * Accepts a map of hosts to resource offers for that host, and returns a prioritized list of
+    * resource offers representing the order in which the offers should be used.  The resource
+    * offers are ordered such that we'll allocate one container on each host before allocating a
+    * second container on any host, and so on, in order to reduce the damage if a host fails.
+    *
+    * For example, given <h1, [o1, o2, o3]>, <h2, [o4]>, <h1, [o5, o6]>, returns
+    * [o1, o5, o4, 02, o6, o3]
+    */
+   def prioritizeContainers[K, T] (map: HashMap[K, ArrayBuffer[T]]): List[T] = {
+     val _keyList = new ArrayBuffer[K](map.size)
+     _keyList ++= map.keys
+ 
+     // order keyList based on population of value in map
+     val keyList = _keyList.sortWith(
+       (left, right) => map(left).size > map(right).size
+     )
+ 
+     val retval = new ArrayBuffer[T](keyList.size * 2)
+     var index = 0
+     var found = true
+ 
+     while (found) {
+       found = false
+       for (key <- keyList) {
+         val containerList: ArrayBuffer[T] = map.get(key).getOrElse(null)
+         assert(containerList != null)
+         // Get the index'th entry for this host - if present
+         if (index < containerList.size){
+           retval += containerList.apply(index)
+           found = true
+         }
+       }
+       index += 1
+     }
+ 
+     retval.toList
+   }
+ }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
----------------------------------------------------------------------
diff --cc core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
index 431d888,fe56960..9ea7fc2
--- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
+++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
@@@ -24,15 -24,10 +24,15 @@@ import org.apache.spark.{SparkConf, Spa
  /**
   * Runs a timer task to periodically clean up metadata (e.g. old files or hashtable entries)
   */
 -class MetadataCleaner(cleanerType: MetadataCleanerType.MetadataCleanerType, cleanupFunc: (Long) => Unit) extends Logging {
 +class MetadataCleaner(
 +    cleanerType: MetadataCleanerType.MetadataCleanerType,
 +    cleanupFunc: (Long) => Unit,
 +    conf: SparkConf)
 +  extends Logging
 +{
    val name = cleanerType.toString
  
-   private val delaySeconds = MetadataCleaner.getDelaySeconds(conf)
 -  private val delaySeconds = MetadataCleaner.getDelaySeconds(cleanerType)
++  private val delaySeconds = MetadataCleaner.getDelaySeconds(conf, cleanerType)
    private val periodSeconds = math.max(10, delaySeconds / 10)
    private val timer = new Timer(name + " cleanup timer", true)
  

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/core/src/test/scala/org/apache/spark/scheduler/ClusterSchedulerSuite.scala
----------------------------------------------------------------------
diff --cc core/src/test/scala/org/apache/spark/scheduler/ClusterSchedulerSuite.scala
index 0000000,702edb8..7bf2020
mode 000000,100644..100644
--- a/core/src/test/scala/org/apache/spark/scheduler/ClusterSchedulerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/ClusterSchedulerSuite.scala
@@@ -1,0 -1,265 +1,265 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one or more
+  * contributor license agreements.  See the NOTICE file distributed with
+  * this work for additional information regarding copyright ownership.
+  * The ASF licenses this file to You under the Apache License, Version 2.0
+  * (the "License"); you may not use this file except in compliance with
+  * the License.  You may obtain a copy of the License at
+  *
+  *    http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ 
+ package org.apache.spark.scheduler
+ 
+ import org.scalatest.FunSuite
+ import org.scalatest.BeforeAndAfter
+ 
+ import org.apache.spark._
+ import scala.collection.mutable.ArrayBuffer
+ 
+ import java.util.Properties
+ 
+ class FakeTaskSetManager(
+     initPriority: Int,
+     initStageId: Int,
+     initNumTasks: Int,
+     clusterScheduler: TaskSchedulerImpl,
+     taskSet: TaskSet)
+   extends TaskSetManager(clusterScheduler, taskSet, 0) {
+ 
+   parent = null
+   weight = 1
+   minShare = 2
+   runningTasks = 0
+   priority = initPriority
+   stageId = initStageId
+   name = "TaskSet_"+stageId
+   override val numTasks = initNumTasks
+   tasksSuccessful = 0
+ 
+   def increaseRunningTasks(taskNum: Int) {
+     runningTasks += taskNum
+     if (parent != null) {
+       parent.increaseRunningTasks(taskNum)
+     }
+   }
+ 
+   def decreaseRunningTasks(taskNum: Int) {
+     runningTasks -= taskNum
+     if (parent != null) {
+       parent.decreaseRunningTasks(taskNum)
+     }
+   }
+ 
+   override def addSchedulable(schedulable: Schedulable) {
+   }
+ 
+   override def removeSchedulable(schedulable: Schedulable) {
+   }
+ 
+   override def getSchedulableByName(name: String): Schedulable = {
+     return null
+   }
+ 
+   override def executorLost(executorId: String, host: String): Unit = {
+   }
+ 
+   override def resourceOffer(
+       execId: String,
+       host: String,
+       availableCpus: Int,
+       maxLocality: TaskLocality.TaskLocality)
+     : Option[TaskDescription] =
+   {
+     if (tasksSuccessful + runningTasks < numTasks) {
+       increaseRunningTasks(1)
+       return Some(new TaskDescription(0, execId, "task 0:0", 0, null))
+     }
+     return None
+   }
+ 
+   override def checkSpeculatableTasks(): Boolean = {
+     return true
+   }
+ 
+   def taskFinished() {
+     decreaseRunningTasks(1)
+     tasksSuccessful +=1
+     if (tasksSuccessful == numTasks) {
+       parent.removeSchedulable(this)
+     }
+   }
+ 
+   def abort() {
+     decreaseRunningTasks(runningTasks)
+     parent.removeSchedulable(this)
+   }
+ }
+ 
+ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging {
+ 
+   def createDummyTaskSetManager(priority: Int, stage: Int, numTasks: Int, cs: TaskSchedulerImpl, taskSet: TaskSet): FakeTaskSetManager = {
+     new FakeTaskSetManager(priority, stage, numTasks, cs , taskSet)
+   }
+ 
+   def resourceOffer(rootPool: Pool): Int = {
+     val taskSetQueue = rootPool.getSortedTaskSetQueue()
+     /* Just for Test*/
+     for (manager <- taskSetQueue) {
+        logInfo("parentName:%s, parent running tasks:%d, name:%s,runningTasks:%d".format(
+          manager.parent.name, manager.parent.runningTasks, manager.name, manager.runningTasks))
+     }
+     for (taskSet <- taskSetQueue) {
+       taskSet.resourceOffer("execId_1", "hostname_1", 1, TaskLocality.ANY) match {
+         case Some(task) =>
+           return taskSet.stageId
+         case None => {}
+       }
+     }
+     -1
+   }
+ 
+   def checkTaskSetId(rootPool: Pool, expectedTaskSetId: Int) {
+     assert(resourceOffer(rootPool) === expectedTaskSetId)
+   }
+ 
+   test("FIFO Scheduler Test") {
+     sc = new SparkContext("local", "ClusterSchedulerSuite")
+     val clusterScheduler = new TaskSchedulerImpl(sc)
+     var tasks = ArrayBuffer[Task[_]]()
+     val task = new FakeTask(0)
+     tasks += task
+     val taskSet = new TaskSet(tasks.toArray,0,0,0,null)
+ 
+     val rootPool = new Pool("", SchedulingMode.FIFO, 0, 0)
+     val schedulableBuilder = new FIFOSchedulableBuilder(rootPool)
+     schedulableBuilder.buildPools()
+ 
+     val taskSetManager0 = createDummyTaskSetManager(0, 0, 2, clusterScheduler, taskSet)
+     val taskSetManager1 = createDummyTaskSetManager(0, 1, 2, clusterScheduler, taskSet)
+     val taskSetManager2 = createDummyTaskSetManager(0, 2, 2, clusterScheduler, taskSet)
+     schedulableBuilder.addTaskSetManager(taskSetManager0, null)
+     schedulableBuilder.addTaskSetManager(taskSetManager1, null)
+     schedulableBuilder.addTaskSetManager(taskSetManager2, null)
+ 
+     checkTaskSetId(rootPool, 0)
+     resourceOffer(rootPool)
+     checkTaskSetId(rootPool, 1)
+     resourceOffer(rootPool)
+     taskSetManager1.abort()
+     checkTaskSetId(rootPool, 2)
+   }
+ 
+   test("Fair Scheduler Test") {
+     sc = new SparkContext("local", "ClusterSchedulerSuite")
+     val clusterScheduler = new TaskSchedulerImpl(sc)
+     var tasks = ArrayBuffer[Task[_]]()
+     val task = new FakeTask(0)
+     tasks += task
+     val taskSet = new TaskSet(tasks.toArray,0,0,0,null)
+ 
+     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)
+     assert(rootPool.getSchedulableByName("1") != null)
+     assert(rootPool.getSchedulableByName("2") != null)
+     assert(rootPool.getSchedulableByName("3") != null)
+     assert(rootPool.getSchedulableByName("1").minShare === 2)
+     assert(rootPool.getSchedulableByName("1").weight === 1)
+     assert(rootPool.getSchedulableByName("2").minShare === 3)
+     assert(rootPool.getSchedulableByName("2").weight === 1)
+     assert(rootPool.getSchedulableByName("3").minShare === 0)
+     assert(rootPool.getSchedulableByName("3").weight === 1)
+ 
+     val properties1 = new Properties()
+     properties1.setProperty("spark.scheduler.pool","1")
+     val properties2 = new Properties()
+     properties2.setProperty("spark.scheduler.pool","2")
+ 
+     val taskSetManager10 = createDummyTaskSetManager(1, 0, 1, clusterScheduler, taskSet)
+     val taskSetManager11 = createDummyTaskSetManager(1, 1, 1, clusterScheduler, taskSet)
+     val taskSetManager12 = createDummyTaskSetManager(1, 2, 2, clusterScheduler, taskSet)
+     schedulableBuilder.addTaskSetManager(taskSetManager10, properties1)
+     schedulableBuilder.addTaskSetManager(taskSetManager11, properties1)
+     schedulableBuilder.addTaskSetManager(taskSetManager12, properties1)
+ 
+     val taskSetManager23 = createDummyTaskSetManager(2, 3, 2, clusterScheduler, taskSet)
+     val taskSetManager24 = createDummyTaskSetManager(2, 4, 2, clusterScheduler, taskSet)
+     schedulableBuilder.addTaskSetManager(taskSetManager23, properties2)
+     schedulableBuilder.addTaskSetManager(taskSetManager24, properties2)
+ 
+     checkTaskSetId(rootPool, 0)
+     checkTaskSetId(rootPool, 3)
+     checkTaskSetId(rootPool, 3)
+     checkTaskSetId(rootPool, 1)
+     checkTaskSetId(rootPool, 4)
+     checkTaskSetId(rootPool, 2)
+     checkTaskSetId(rootPool, 2)
+     checkTaskSetId(rootPool, 4)
+ 
+     taskSetManager12.taskFinished()
+     assert(rootPool.getSchedulableByName("1").runningTasks === 3)
+     taskSetManager24.abort()
+     assert(rootPool.getSchedulableByName("2").runningTasks === 2)
+   }
+ 
+   test("Nested Pool Test") {
+     sc = new SparkContext("local", "ClusterSchedulerSuite")
+     val clusterScheduler = new TaskSchedulerImpl(sc)
+     var tasks = ArrayBuffer[Task[_]]()
+     val task = new FakeTask(0)
+     tasks += task
+     val taskSet = new TaskSet(tasks.toArray,0,0,0,null)
+ 
+     val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0)
+     val pool0 = new Pool("0", SchedulingMode.FAIR, 3, 1)
+     val pool1 = new Pool("1", SchedulingMode.FAIR, 4, 1)
+     rootPool.addSchedulable(pool0)
+     rootPool.addSchedulable(pool1)
+ 
+     val pool00 = new Pool("00", SchedulingMode.FAIR, 2, 2)
+     val pool01 = new Pool("01", SchedulingMode.FAIR, 1, 1)
+     pool0.addSchedulable(pool00)
+     pool0.addSchedulable(pool01)
+ 
+     val pool10 = new Pool("10", SchedulingMode.FAIR, 2, 2)
+     val pool11 = new Pool("11", SchedulingMode.FAIR, 2, 1)
+     pool1.addSchedulable(pool10)
+     pool1.addSchedulable(pool11)
+ 
+     val taskSetManager000 = createDummyTaskSetManager(0, 0, 5, clusterScheduler, taskSet)
+     val taskSetManager001 = createDummyTaskSetManager(0, 1, 5, clusterScheduler, taskSet)
+     pool00.addSchedulable(taskSetManager000)
+     pool00.addSchedulable(taskSetManager001)
+ 
+     val taskSetManager010 = createDummyTaskSetManager(1, 2, 5, clusterScheduler, taskSet)
+     val taskSetManager011 = createDummyTaskSetManager(1, 3, 5, clusterScheduler, taskSet)
+     pool01.addSchedulable(taskSetManager010)
+     pool01.addSchedulable(taskSetManager011)
+ 
+     val taskSetManager100 = createDummyTaskSetManager(2, 4, 5, clusterScheduler, taskSet)
+     val taskSetManager101 = createDummyTaskSetManager(2, 5, 5, clusterScheduler, taskSet)
+     pool10.addSchedulable(taskSetManager100)
+     pool10.addSchedulable(taskSetManager101)
+ 
+     val taskSetManager110 = createDummyTaskSetManager(3, 6, 5, clusterScheduler, taskSet)
+     val taskSetManager111 = createDummyTaskSetManager(3, 7, 5, clusterScheduler, taskSet)
+     pool11.addSchedulable(taskSetManager110)
+     pool11.addSchedulable(taskSetManager111)
+ 
+     checkTaskSetId(rootPool, 0)
+     checkTaskSetId(rootPool, 4)
+     checkTaskSetId(rootPool, 6)
+     checkTaskSetId(rootPool, 2)
+   }
+ }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala
----------------------------------------------------------------------
diff --cc core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala
index 0000000,ca97f7d..4b52d96
mode 000000,100644..100644
--- a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala
@@@ -1,0 -1,112 +1,112 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one or more
+  * contributor license agreements.  See the NOTICE file distributed with
+  * this work for additional information regarding copyright ownership.
+  * The ASF licenses this file to You under the Apache License, Version 2.0
+  * (the "License"); you may not use this file except in compliance with
+  * the License.  You may obtain a copy of the License at
+  *
+  *    http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ 
+ package org.apache.spark.scheduler
+ 
+ import java.nio.ByteBuffer
+ 
+ import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite}
+ 
 -import org.apache.spark.{LocalSparkContext, SparkContext, SparkEnv}
++import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkEnv}
+ import org.apache.spark.storage.TaskResultBlockId
+ 
+ /**
+  * Removes the TaskResult from the BlockManager before delegating to a normal TaskResultGetter.
+  *
+  * Used to test the case where a BlockManager evicts the task result (or dies) before the
+  * TaskResult is retrieved.
+  */
+ class ResultDeletingTaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedulerImpl)
+   extends TaskResultGetter(sparkEnv, scheduler) {
+   var removedResult = false
+ 
+   override def enqueueSuccessfulTask(
+     taskSetManager: TaskSetManager, tid: Long, serializedData: ByteBuffer) {
+     if (!removedResult) {
+       // Only remove the result once, since we'd like to test the case where the task eventually
+       // succeeds.
+       serializer.get().deserialize[TaskResult[_]](serializedData) match {
+         case IndirectTaskResult(blockId) =>
+           sparkEnv.blockManager.master.removeBlock(blockId)
+         case directResult: DirectTaskResult[_] =>
 -          taskSetManager.abort("Internal error: expect only indirect results") 
++          taskSetManager.abort("Internal error: expect only indirect results")
+       }
+       serializedData.rewind()
+       removedResult = true
+     }
+     super.enqueueSuccessfulTask(taskSetManager, tid, serializedData)
 -  } 
++  }
+ }
+ 
+ /**
+  * Tests related to handling task results (both direct and indirect).
+  */
+ class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndAfterAll
+   with LocalSparkContext {
+ 
+   override def beforeAll {
+     // Set the Akka frame size to be as small as possible (it must be an integer, so 1 is as small
+     // as we can make it) so the tests don't take too long.
+     System.setProperty("spark.akka.frameSize", "1")
+   }
+ 
+   override def afterAll {
+     System.clearProperty("spark.akka.frameSize")
+   }
+ 
+   test("handling results smaller than Akka frame size") {
+     sc = new SparkContext("local", "test")
+     val result = sc.parallelize(Seq(1), 1).map(x => 2 * x).reduce((x, y) => x)
+     assert(result === 2)
+   }
+ 
+   test("handling results larger than Akka frame size") {
+     sc = new SparkContext("local", "test")
+     val akkaFrameSize =
+       sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size").toInt
+     val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x, y) => x)
+     assert(result === 1.to(akkaFrameSize).toArray)
+ 
+     val RESULT_BLOCK_ID = TaskResultBlockId(0)
+     assert(sc.env.blockManager.master.getLocations(RESULT_BLOCK_ID).size === 0,
+       "Expect result to be removed from the block manager.")
+   }
+ 
+   test("task retried if result missing from block manager") {
+     // Set the maximum number of task failures to > 0, so that the task set isn't aborted
+     // after the result is missing.
+     sc = new SparkContext("local[1,2]", "test")
+     // If this test hangs, it's probably because no resource offers were made after the task
+     // failed.
+     val scheduler: TaskSchedulerImpl = sc.taskScheduler match {
+       case clusterScheduler: TaskSchedulerImpl =>
+         clusterScheduler
+       case _ =>
+         assert(false, "Expect local cluster to use ClusterScheduler")
+         throw new ClassCastException
+     }
+     scheduler.taskResultGetter = new ResultDeletingTaskResultGetter(sc.env, scheduler)
+     val akkaFrameSize =
+       sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size").toInt
+     val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x, y) => x)
+     assert(result === 1.to(akkaFrameSize).toArray)
+ 
+     // Make sure two tasks were run (one failed one, and a second retried one).
+     assert(scheduler.nextTaskId.get() === 2)
+   }
+ }
+ 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
----------------------------------------------------------------------
diff --cc core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
index 0000000,3dcb01a..5d33e66
mode 000000,100644..100644
--- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
@@@ -1,0 -1,319 +1,321 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one or more
+  * contributor license agreements.  See the NOTICE file distributed with
+  * this work for additional information regarding copyright ownership.
+  * The ASF licenses this file to You under the Apache License, Version 2.0
+  * (the "License"); you may not use this file except in compliance with
+  * the License.  You may obtain a copy of the License at
+  *
+  *    http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ 
+ package org.apache.spark.scheduler
+ 
+ import scala.collection.mutable.ArrayBuffer
+ import scala.collection.mutable
+ 
+ import org.scalatest.FunSuite
+ 
+ import org.apache.spark._
+ import org.apache.spark.executor.TaskMetrics
+ import java.nio.ByteBuffer
+ import org.apache.spark.util.{Utils, FakeClock}
+ 
+ class FakeDAGScheduler(taskScheduler: FakeClusterScheduler) extends DAGScheduler(taskScheduler) {
+   override def taskStarted(task: Task[_], taskInfo: TaskInfo) {
+     taskScheduler.startedTasks += taskInfo.index
+   }
+ 
+   override def taskEnded(
+       task: Task[_],
+       reason: TaskEndReason,
+       result: Any,
+       accumUpdates: mutable.Map[Long, Any],
+       taskInfo: TaskInfo,
+       taskMetrics: TaskMetrics) {
+     taskScheduler.endedTasks(taskInfo.index) = reason
+   }
+ 
+   override def executorGained(execId: String, host: String) {}
+ 
+   override def executorLost(execId: String) {}
+ 
+   override def taskSetFailed(taskSet: TaskSet, reason: String) {
+     taskScheduler.taskSetsFailed += taskSet.id
+   }
+ }
+ 
+ /**
+  * A mock ClusterScheduler implementation that just remembers information about tasks started and
+  * feedback received from the TaskSetManagers. Note that it's important to initialize this with
+  * a list of "live" executors and their hostnames for isExecutorAlive and hasExecutorsAliveOnHost
+  * to work, and these are required for locality in TaskSetManager.
+  */
+ class FakeClusterScheduler(sc: SparkContext, liveExecutors: (String, String)* /* execId, host */)
+   extends TaskSchedulerImpl(sc)
+ {
+   val startedTasks = new ArrayBuffer[Long]
+   val endedTasks = new mutable.HashMap[Long, TaskEndReason]
+   val finishedManagers = new ArrayBuffer[TaskSetManager]
+   val taskSetsFailed = new ArrayBuffer[String]
+ 
+   val executors = new mutable.HashMap[String, String] ++ liveExecutors
+ 
+   dagScheduler = new FakeDAGScheduler(this)
+ 
+   def removeExecutor(execId: String): Unit = executors -= execId
+ 
+   override def taskSetFinished(manager: TaskSetManager): Unit = finishedManagers += manager
+ 
+   override def isExecutorAlive(execId: String): Boolean = executors.contains(execId)
+ 
+   override def hasExecutorsAliveOnHost(host: String): Boolean = executors.values.exists(_ == host)
+ }
+ 
+ class TaskSetManagerSuite 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
+   val MAX_TASK_FAILURES = 4
+ 
+   test("TaskSet with no preferences") {
+     sc = new SparkContext("local", "test")
+     val sched = new FakeClusterScheduler(sc, ("exec1", "host1"))
+     val taskSet = createTaskSet(1)
+     val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES)
+ 
+     // Offer a host with no CPUs
+     assert(manager.resourceOffer("exec1", "host1", 0, ANY) === None)
+ 
+     // Offer a host with process-local as the constraint; this should work because the TaskSet
+     // above won't have any locality preferences
+     val taskOption = manager.resourceOffer("exec1", "host1", 2, TaskLocality.PROCESS_LOCAL)
+     assert(taskOption.isDefined)
+     val task = taskOption.get
+     assert(task.executorId === "exec1")
+     assert(sched.startedTasks.contains(0))
+ 
+     // Re-offer the host -- now we should get no more tasks
+     assert(manager.resourceOffer("exec1", "host1", 2, PROCESS_LOCAL) === None)
+ 
+     // Tell it the task has finished
+     manager.handleSuccessfulTask(0, createTaskResult(0))
+     assert(sched.endedTasks(0) === Success)
+     assert(sched.finishedManagers.contains(manager))
+   }
+ 
+   test("multiple offers with no preferences") {
+     sc = new SparkContext("local", "test")
+     val sched = new FakeClusterScheduler(sc, ("exec1", "host1"))
+     val taskSet = createTaskSet(3)
+     val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES)
+ 
+     // First three offers should all find tasks
+     for (i <- 0 until 3) {
+       val taskOption = manager.resourceOffer("exec1", "host1", 1, PROCESS_LOCAL)
+       assert(taskOption.isDefined)
+       val task = taskOption.get
+       assert(task.executorId === "exec1")
+     }
+     assert(sched.startedTasks.toSet === Set(0, 1, 2))
+ 
+     // Re-offer the host -- now we should get no more tasks
+     assert(manager.resourceOffer("exec1", "host1", 1, PROCESS_LOCAL) === None)
+ 
+     // Finish the first two tasks
+     manager.handleSuccessfulTask(0, createTaskResult(0))
+     manager.handleSuccessfulTask(1, createTaskResult(1))
+     assert(sched.endedTasks(0) === Success)
+     assert(sched.endedTasks(1) === Success)
+     assert(!sched.finishedManagers.contains(manager))
+ 
+     // Finish the last task
+     manager.handleSuccessfulTask(2, createTaskResult(2))
+     assert(sched.endedTasks(2) === Success)
+     assert(sched.finishedManagers.contains(manager))
+   }
+ 
+   test("basic delay scheduling") {
+     sc = new SparkContext("local", "test")
+     val sched = new FakeClusterScheduler(sc, ("exec1", "host1"), ("exec2", "host2"))
+     val taskSet = createTaskSet(4,
+       Seq(TaskLocation("host1", "exec1")),
+       Seq(TaskLocation("host2", "exec2")),
+       Seq(TaskLocation("host1"), TaskLocation("host2", "exec2")),
+       Seq()   // Last task has no locality prefs
+     )
+     val clock = new FakeClock
+     val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
+ 
+     // First offer host1, exec1: first task should be chosen
+     assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0)
+ 
+     // Offer host1, exec1 again: the last task, which has no prefs, should be chosen
+     assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 3)
+ 
+     // Offer host1, exec1 again, at PROCESS_LOCAL level: nothing should get chosen
+     assert(manager.resourceOffer("exec1", "host1", 1, PROCESS_LOCAL) === None)
+ 
+     clock.advance(LOCALITY_WAIT)
+ 
+     // Offer host1, exec1 again, at PROCESS_LOCAL level: nothing should get chosen
+     assert(manager.resourceOffer("exec1", "host1", 1, PROCESS_LOCAL) === None)
+ 
+     // Offer host1, exec1 again, at NODE_LOCAL level: we should choose task 2
+     assert(manager.resourceOffer("exec1", "host1", 1, NODE_LOCAL).get.index == 2)
+ 
+     // Offer host1, exec1 again, at NODE_LOCAL level: nothing should get chosen
+     assert(manager.resourceOffer("exec1", "host1", 1, NODE_LOCAL) === None)
+ 
+     // Offer host1, exec1 again, at ANY level: nothing should get chosen
+     assert(manager.resourceOffer("exec1", "host1", 1, ANY) === None)
+ 
+     clock.advance(LOCALITY_WAIT)
+ 
+     // Offer host1, exec1 again, at ANY level: task 1 should get chosen
+     assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 1)
+ 
+     // Offer host1, exec1 again, at ANY level: nothing should be chosen as we've launched all tasks
+     assert(manager.resourceOffer("exec1", "host1", 1, ANY) === None)
+   }
+ 
+   test("delay scheduling with fallback") {
+     sc = new SparkContext("local", "test")
+     val sched = new FakeClusterScheduler(sc,
+       ("exec1", "host1"), ("exec2", "host2"), ("exec3", "host3"))
+     val taskSet = createTaskSet(5,
+       Seq(TaskLocation("host1")),
+       Seq(TaskLocation("host2")),
+       Seq(TaskLocation("host2")),
+       Seq(TaskLocation("host3")),
+       Seq(TaskLocation("host2"))
+     )
+     val clock = new FakeClock
+     val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
+ 
+     // First offer host1: first task should be chosen
+     assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0)
+ 
+     // Offer host1 again: nothing should get chosen
+     assert(manager.resourceOffer("exec1", "host1", 1, ANY) === None)
+ 
+     clock.advance(LOCALITY_WAIT)
+ 
+     // Offer host1 again: second task (on host2) should get chosen
+     assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 1)
+ 
+     // Offer host1 again: third task (on host2) should get chosen
+     assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 2)
+ 
+     // Offer host2: fifth task (also on host2) should get chosen
+     assert(manager.resourceOffer("exec2", "host2", 1, ANY).get.index === 4)
+ 
+     // Now that we've launched a local task, we should no longer launch the task for host3
+     assert(manager.resourceOffer("exec2", "host2", 1, ANY) === None)
+ 
+     clock.advance(LOCALITY_WAIT)
+ 
+     // After another delay, we can go ahead and launch that task non-locally
+     assert(manager.resourceOffer("exec2", "host2", 1, ANY).get.index === 3)
+   }
+ 
+   test("delay scheduling with failed hosts") {
+     sc = new SparkContext("local", "test")
+     val sched = new FakeClusterScheduler(sc, ("exec1", "host1"), ("exec2", "host2"))
+     val taskSet = createTaskSet(3,
+       Seq(TaskLocation("host1")),
+       Seq(TaskLocation("host2")),
+       Seq(TaskLocation("host3"))
+     )
+     val clock = new FakeClock
+     val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
+ 
+     // First offer host1: first task should be chosen
+     assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0)
+ 
+     // Offer host1 again: third task should be chosen immediately because host3 is not up
+     assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 2)
+ 
+     // After this, nothing should get chosen
+     assert(manager.resourceOffer("exec1", "host1", 1, ANY) === None)
+ 
+     // Now mark host2 as dead
+     sched.removeExecutor("exec2")
+     manager.executorLost("exec2", "host2")
+ 
+     // Task 1 should immediately be launched on host1 because its original host is gone
+     assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 1)
+ 
+     // Now that all tasks have launched, nothing new should be launched anywhere else
+     assert(manager.resourceOffer("exec1", "host1", 1, ANY) === None)
+     assert(manager.resourceOffer("exec2", "host2", 1, ANY) === None)
+   }
+ 
+   test("task result lost") {
+     sc = new SparkContext("local", "test")
+     val sched = new FakeClusterScheduler(sc, ("exec1", "host1"))
+     val taskSet = createTaskSet(1)
+     val clock = new FakeClock
+     val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
+ 
+     assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0)
+ 
+     // Tell it the task has finished but the result was lost.
+     manager.handleFailedTask(0, TaskState.FINISHED, Some(TaskResultLost))
+     assert(sched.endedTasks(0) === TaskResultLost)
+ 
+     // Re-offer the host -- now we should get task 0 again.
+     assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0)
+   }
+ 
+   test("repeated failures lead to task set abortion") {
+     sc = new SparkContext("local", "test")
+     val sched = new FakeClusterScheduler(sc, ("exec1", "host1"))
+     val taskSet = createTaskSet(1)
+     val clock = new FakeClock
+     val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
+ 
+     // Fail the task MAX_TASK_FAILURES times, and check that the task set is aborted
+     // after the last failure.
+     (1 to manager.maxTaskFailures).foreach { index =>
+       val offerResult = manager.resourceOffer("exec1", "host1", 1, ANY)
+       assert(offerResult != None,
+         "Expect resource offer on iteration %s to return a task".format(index))
+       assert(offerResult.get.index === 0)
+       manager.handleFailedTask(offerResult.get.taskId, TaskState.FINISHED, Some(TaskResultLost))
+       if (index < MAX_TASK_FAILURES) {
+         assert(!sched.taskSetsFailed.contains(taskSet.id))
+       } else {
+         assert(sched.taskSetsFailed.contains(taskSet.id))
+       }
+     }
+   }
+ 
+ 
+   /**
+    * Utility method to create a TaskSet, potentially setting a particular sequence of preferred
+    * locations for each task (given as varargs) if this sequence is not empty.
+    */
+   def createTaskSet(numTasks: Int, prefLocs: Seq[TaskLocation]*): TaskSet = {
+     if (prefLocs.size != 0 && prefLocs.size != numTasks) {
+       throw new IllegalArgumentException("Wrong number of task locations")
+     }
+     val tasks = Array.tabulate[Task[_]](numTasks) { i =>
+       new FakeTask(i, if (prefLocs.size != 0) prefLocs(i) else Nil)
+     }
+     new TaskSet(tasks, 0, 0, 0, null)
+   }
+ 
+   def createTaskResult(id: Int): DirectTaskResult[Int] = {
+     val valueSer = SparkEnv.get.serializer.newInstance()
+     new DirectTaskResult[Int](valueSer.serialize(id), mutable.Map.empty, new TaskMetrics)
+   }
+ }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
----------------------------------------------------------------------
diff --cc new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index 963b5b8,9fdee29..1bba6a5
--- 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
@@@ -436,9 -436,11 +436,11 @@@ class Client(conf: Configuration, args
      super.submitApplication(appContext)
    }
  
 -  def monitorApplication(appId: ApplicationId): Boolean = {  
 -    val interval = System.getProperty("spark.yarn.report.interval", "1000").toLong
 +  def monitorApplication(appId: ApplicationId): Boolean = {
++    val interval = new SparkConf().getOrElse("spark.yarn.report.interval", "1000").toLong
+ 
      while (true) {
-       Thread.sleep(1000)
+       Thread.sleep(interval)
        val report = super.getApplicationReport(appId)
  
        logInfo("Application report from ASM: \n" +

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/project/SparkBuild.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/python/pyspark/java_gateway.py
----------------------------------------------------------------------
diff --cc python/pyspark/java_gateway.py
index 128f078,eb79135..d8ca9fc
--- a/python/pyspark/java_gateway.py
+++ b/python/pyspark/java_gateway.py
@@@ -60,8 -60,8 +60,9 @@@ def launch_gateway()
      # Connect to the gateway
      gateway = JavaGateway(GatewayClient(port=port), auto_convert=False)
      # Import the classes used by PySpark
 +    java_import(gateway.jvm, "org.apache.spark.SparkConf")
      java_import(gateway.jvm, "org.apache.spark.api.java.*")
      java_import(gateway.jvm, "org.apache.spark.api.python.*")
+     java_import(gateway.jvm, "org.apache.spark.mllib.api.python.*")
      java_import(gateway.jvm, "scala.Tuple2")
      return gateway

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
----------------------------------------------------------------------
diff --cc streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
index f106bba,7b343d2..35e23c1
--- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
@@@ -39,9 -39,9 +39,9 @@@ class Checkpoint(@transient ssc: Stream
    val graph = ssc.graph
    val checkpointDir = ssc.checkpointDir
    val checkpointDuration = ssc.checkpointDuration
-   val pendingTimes = ssc.scheduler.jobManager.getPendingTimes()
+   val pendingTimes = ssc.scheduler.getPendingTimes()
 -  val delaySeconds = MetadataCleaner.getDelaySeconds
 +  val delaySeconds = MetadataCleaner.getDelaySeconds(ssc.conf)
-   val sparkConf = ssc.sc.conf
++  val sparkConf = ssc.conf
  
    def validate() {
      assert(master != null, "Checkpoint.master is null")

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
----------------------------------------------------------------------
diff --cc streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
index 5842a7c,78d318c..29f673d
--- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
@@@ -39,7 -39,7 +39,8 @@@ import org.apache.spark.api.java.functi
  import org.apache.spark.api.java.{JavaPairRDD, JavaSparkContext, JavaRDD}
  import org.apache.spark.streaming._
  import org.apache.spark.streaming.dstream._
 +import org.apache.spark.SparkConf
+ import org.apache.spark.streaming.scheduler.StreamingListener
  
  /**
   * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
----------------------------------------------------------------------
diff --cc streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
index 0000000,1cd0b9b..dbd0841
mode 000000,100644..100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
@@@ -1,0 -1,131 +1,132 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one or more
+  * contributor license agreements.  See the NOTICE file distributed with
+  * this work for additional information regarding copyright ownership.
+  * The ASF licenses this file to You under the Apache License, Version 2.0
+  * (the "License"); you may not use this file except in compliance with
+  * the License.  You may obtain a copy of the License at
+  *
+  *    http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ 
+ package org.apache.spark.streaming.scheduler
+ 
+ import org.apache.spark.SparkEnv
+ import org.apache.spark.Logging
+ import org.apache.spark.streaming.{Checkpoint, Time, CheckpointWriter}
+ import org.apache.spark.streaming.util.{ManualClock, RecurringTimer, Clock}
+ 
+ /**
+  * This class generates jobs from DStreams as well as drives checkpointing and cleaning
+  * up DStream metadata.
+  */
+ private[streaming]
+ class JobGenerator(jobScheduler: JobScheduler) extends Logging {
+ 
+   initLogging()
++
+   val ssc = jobScheduler.ssc
 -  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,
+     longTime => generateJobs(new Time(longTime)))
+   val graph = ssc.graph
+   lazy val checkpointWriter = if (ssc.checkpointDuration != null && ssc.checkpointDir != null) {
 -    new CheckpointWriter(ssc.checkpointDir)
++    new CheckpointWriter(ssc.conf, ssc.checkpointDir)
+   } else {
+     null
+   }
+ 
+   var latestTime: Time = null
+ 
+   def start() = synchronized {
+     if (ssc.isCheckpointPresent) {
+       restart()
+     } else {
+       startFirstTime()
+     }
+     logInfo("JobGenerator started")
+   }
 -  
++
+   def stop() = synchronized {
+     timer.stop()
+     if (checkpointWriter != null) checkpointWriter.stop()
+     ssc.graph.stop()
+     logInfo("JobGenerator stopped")
+   }
+ 
+   private def startFirstTime() {
+     val startTime = new Time(timer.getStartTime())
+     graph.start(startTime - graph.batchDuration)
+     timer.start(startTime.milliseconds)
+     logInfo("JobGenerator's timer started at " + startTime)
+   }
+ 
+   private def restart() {
+     // If manual clock is being used for testing, then
+     // either set the manual clock to the last checkpointed time,
+     // 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)
+     }
+ 
+     val batchDuration = ssc.graph.batchDuration
+ 
+     // Batches when the master was down, that is,
+     // between the checkpoint and current restart time
+     val checkpointTime = ssc.initialCheckpoint.checkpointTime
+     val restartTime = new Time(timer.getRestartTime(graph.zeroTime.milliseconds))
+     val downTimes = checkpointTime.until(restartTime, batchDuration)
+     logInfo("Batches during down time: " + downTimes.mkString(", "))
+ 
+     // Batches that were unprocessed before failure
+     val pendingTimes = ssc.initialCheckpoint.pendingTimes
+     logInfo("Batches pending processing: " + pendingTimes.mkString(", "))
+     // Reschedule jobs for these times
+     val timesToReschedule = (pendingTimes ++ downTimes).distinct.sorted(Time.ordering)
+     logInfo("Batches to reschedule: " + timesToReschedule.mkString(", "))
+     timesToReschedule.foreach(time =>
+       jobScheduler.runJobs(time, graph.generateJobs(time))
+     )
+ 
+     // Restart the timer
+     timer.start(restartTime.milliseconds)
+     logInfo("JobGenerator's timer restarted at " + restartTime)
+   }
+ 
+   /** Generate jobs and perform checkpoint for the given `time`.  */
+   private def generateJobs(time: Time) {
+     SparkEnv.set(ssc.env)
+     logInfo("\n-----------------------------------------------------\n")
+     jobScheduler.runJobs(time, graph.generateJobs(time))
+     latestTime = time
+     doCheckpoint(time)
+   }
+ 
+   /**
+    * On batch completion, clear old metadata and checkpoint computation.
+    */
+   private[streaming] def onBatchCompletion(time: Time) {
+     ssc.graph.clearOldMetadata(time)
+     doCheckpoint(time)
+   }
+ 
+   /** Perform checkpoint for the give `time`. */
+   private def doCheckpoint(time: Time) = synchronized {
+     if (checkpointWriter != null && (time - graph.zeroTime).isMultipleOf(ssc.checkpointDuration)) {
+       logInfo("Checkpointing graph for time " + time)
+       ssc.graph.updateCheckpointData(time)
+       checkpointWriter.write(new Checkpoint(ssc, time))
+     }
+   }
+ }
+ 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
----------------------------------------------------------------------
diff --cc streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
index 60e986c,b35ca00..ee6b433
--- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
@@@ -23,20 -23,9 +23,9 @@@ import org.apache.spark.rdd.RD
  import org.apache.spark.SparkContext._
  
  import util.ManualClock
 +import org.apache.spark.{SparkContext, SparkConf}
  
  class BasicOperationsSuite extends TestSuiteBase {
--
-   override def framework = "BasicOperationsSuite"
- 
-   conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
- 
-   after {
-     // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
-     System.clearProperty("spark.driver.port")
-     System.clearProperty("spark.hostPort")
-   }
- 
    test("map") {
      val input = Seq(1 to 4, 5 to 8, 9 to 12)
      testOperation(

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
----------------------------------------------------------------------
diff --cc streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
index 3dd6718,e969e91..33464bc
--- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
@@@ -130,12 -133,33 +133,39 @@@ trait TestSuiteBase extends FunSuite wi
    // Whether to actually wait in real time before changing manual clock
    def actuallyWait = false
  
-   // A SparkConf to use in tests. Can be modified before calling setupStreams to configure things.
++  //// A SparkConf to use in tests. Can be modified before calling setupStreams to configure things.
 +  val conf = new SparkConf()
 +    .setMaster(master)
 +    .setAppName(framework)
 +    .set("spark.cleaner.ttl", "3600")
 +
+   // Default before function for any streaming test suite. Override this
+   // if you want to add your stuff to "before" (i.e., don't call before { } )
+   def beforeFunction() {
++    //if (useManualClock) {
++    //  System.setProperty(
++    //    "spark.streaming.clock",
++    //    "org.apache.spark.streaming.util.ManualClock"
++    //  )
++    //} else {
++    //  System.clearProperty("spark.streaming.clock")
++    //}
+     if (useManualClock) {
 -      System.setProperty(
 -        "spark.streaming.clock",
 -        "org.apache.spark.streaming.util.ManualClock"
 -      )
 -    } else {
 -      System.clearProperty("spark.streaming.clock")
++      conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
+     }
 -    // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
 -    System.clearProperty("spark.driver.port")
 -    System.clearProperty("spark.hostPort")
+   }
+ 
+   // Default after function for any streaming test suite. Override this
+   // if you want to add your stuff to "after" (i.e., don't call after { } )
+   def afterFunction() {
+     // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
+     System.clearProperty("spark.driver.port")
+     System.clearProperty("spark.hostPort")
+   }
+ 
+   before(beforeFunction)
+   after(afterFunction)
+ 
    /**
     * Set up required DStreams to test the DStream operation using the two sequences
     * of input collections.

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
----------------------------------------------------------------------
diff --cc yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index cc15088,79dd038..595a7ee
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@@ -421,9 -421,9 +421,11 @@@ class Client(conf: Configuration, args
      super.submitApplication(appContext)
    }
  
 -  def monitorApplication(appId: ApplicationId): Boolean = {  
 +  def monitorApplication(appId: ApplicationId): Boolean = {
++    val interval = new SparkConf().getOrElse("spark.yarn.report.interval", "1000").toLong
++
      while (true) {
--      Thread.sleep(1000)
++      Thread.sleep(interval)
        val report = super.getApplicationReport(appId)
  
        logInfo("Application report from ASM: \n" +

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b4ceed40/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
----------------------------------------------------------------------


[04/33] Various fixes to configuration code

Posted by pw...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala
----------------------------------------------------------------------
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 1d23713..82ed6be 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala
@@ -26,10 +26,10 @@ class Scheduler(ssc: StreamingContext) extends Logging {
 
   initLogging()
 
-  val concurrentJobs = ssc.sc.conf.getOrElse("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)
+    new CheckpointWriter(ssc.conf, ssc.checkpointDir)
   } else {
     null
   }
@@ -50,13 +50,13 @@ class Scheduler(ssc: StreamingContext) extends Logging {
     }
     logInfo("Scheduler started")
   }
-  
+
   def stop() = synchronized {
     timer.stop()
     jobManager.stop()
     if (checkpointWriter != null) checkpointWriter.stop()
     ssc.graph.stop()
-    logInfo("Scheduler stopped")    
+    logInfo("Scheduler stopped")
   }
 
   private def startFirstTime() {
@@ -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 = ssc.sc.conf.getOrElse("spark.streaming.manualClock.jump",  "0").toLong
+      val jumpTime = ssc.sc.conf.getOrElse("spark.streaming.manualClock.jump", "0").toLong
       clock.asInstanceOf[ManualClock].setTime(lastTime + jumpTime)
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
----------------------------------------------------------------------
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 7674422..079841a 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
@@ -87,13 +87,12 @@ class StreamingContext private (
          null, batchDuration)
   }
 
-
   /**
    * Re-create a StreamingContext from a checkpoint file.
    * @param path Path either to the directory that was specified as the checkpoint directory, or
    *             to the checkpoint file 'graph' or 'graph.bk'.
    */
-  def this(path: String) = this(null, CheckpointReader.read(path), null)
+  def this(path: String) = this(null, CheckpointReader.read(new SparkConf(), path), null)
 
   initLogging()
 
@@ -102,11 +101,13 @@ class StreamingContext private (
       "both SparkContext and checkpoint as null")
   }
 
-  if(cp_ != null && cp_.delaySeconds >= 0 && MetadataCleaner.getDelaySeconds < 0) {
-    MetadataCleaner.setDelaySeconds(cp_.delaySeconds)
+  private val conf_ = Option(sc_).map(_.conf).getOrElse(cp_.sparkConf)
+
+  if(cp_ != null && cp_.delaySeconds >= 0 && MetadataCleaner.getDelaySeconds(conf_) < 0) {
+    MetadataCleaner.setDelaySeconds(conf_, cp_.delaySeconds)
   }
 
-  if (MetadataCleaner.getDelaySeconds < 0) {
+  if (MetadataCleaner.getDelaySeconds(conf_) < 0) {
     throw new SparkException("Spark Streaming cannot be used without setting spark.cleaner.ttl; "
       + "set this property before creating a SparkContext (use SPARK_JAVA_OPTS for the shell)")
   }
@@ -115,12 +116,14 @@ class StreamingContext private (
 
   protected[streaming] val sc: SparkContext = {
     if (isCheckpointPresent) {
-      new SparkContext(cp_.sparkConf, cp_.environment)
+      new SparkContext(cp_.sparkConf)
     } else {
       sc_
     }
   }
 
+  protected[streaming] val conf = sc.conf
+
   protected[streaming] val env = SparkEnv.get
 
   protected[streaming] val graph: DStreamGraph = {
@@ -579,13 +582,15 @@ object StreamingContext {
       appName: String,
       sparkHome: String,
       jars: Seq[String],
-      environment: Map[String, String]): SparkContext = {
+      environment: Map[String, String]): SparkContext =
+  {
+    val sc = new SparkContext(master, appName, sparkHome, jars, environment)
     // Set the default cleaner delay to an hour if not already set.
     // This should be sufficient for even 1 second interval.
-    if (MetadataCleaner.getDelaySeconds < 0) {
-      MetadataCleaner.setDelaySeconds(3600)
+    if (MetadataCleaner.getDelaySeconds(sc.conf) < 0) {
+      MetadataCleaner.setDelaySeconds(sc.conf, 3600)
     }
-    new SparkContext(master, appName, sparkHome, jars, environment)
+    sc
   }
 
   protected[streaming] def rddToFileName[T](prefix: String, suffix: String, time: Time): String = {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala
----------------------------------------------------------------------
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 8bf761b..bd607f9 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 = env.conf.getOrElse("spark.driver.host",  "localhost")
-    val port = env.conf.getOrElse("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 = env.conf.getOrElse("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)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala
index fc8655a..6585d49 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala
@@ -20,7 +20,7 @@ package org.apache.spark.streaming.util
 import java.nio.ByteBuffer
 import org.apache.spark.util.{RateLimitedOutputStream, IntParam}
 import java.net.ServerSocket
-import org.apache.spark.{Logging}
+import org.apache.spark.{SparkConf, Logging}
 import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream
 import scala.io.Source
 import java.io.IOException
@@ -42,7 +42,7 @@ object RawTextSender extends Logging {
     // Repeat the input data multiple times to fill in a buffer
     val lines = Source.fromFile(file).getLines().toArray
     val bufferStream = new FastByteArrayOutputStream(blockSize + 1000)
-    val ser = new KryoSerializer().newInstance()
+    val ser = new KryoSerializer(new SparkConf()).newInstance()
     val serStream = ser.serializeStream(bufferStream)
     var i = 0
     while (bufferStream.position < blockSize) {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
----------------------------------------------------------------------
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 da8f135..8c16daa 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
@@ -52,9 +52,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
 
   override def checkpointDir = "checkpoint"
 
-  before {
-    conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
-  }
+  conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
 
   after {
     // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
@@ -70,7 +68,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
     // Set up the streaming context and input streams
     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 outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]]
     val outputStream = new TestOutputStream(networkStream, outputBuffer)
     def output = outputBuffer.flatMap(x => x)
     ssc.registerOutputStream(outputStream)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
----------------------------------------------------------------------
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 d1cab0c..a265284 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
@@ -130,7 +130,11 @@ 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")
+  val conf = new SparkConf()
+    .setMaster(master)
+    .setAppName(framework)
+    .set("spark.cleaner.ttl", "3600")
+
   /**
    * Set up required DStreams to test the DStream operation using the two sequences
    * of input collections.

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
----------------------------------------------------------------------
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 1dd38dd..dc92281 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
@@ -43,7 +43,7 @@ import org.apache.spark.util.Utils
 class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) extends Logging {
 
   def this(args: ApplicationMasterArguments) = this(args, new Configuration())
-  
+
   private var rpc: YarnRPC = YarnRPC.create(conf)
   private var resourceManager: AMRMProtocol = _
   private var appAttemptId: ApplicationAttemptId = _
@@ -68,7 +68,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
 
     // Use priority 30 as its higher then HDFS. Its same priority as MapReduce is using.
     ShutdownHookManager.get().addShutdownHook(new AppMasterShutdownHook(this), 30)
-    
+
     appAttemptId = getApplicationAttemptId()
     isLastAMRetry = appAttemptId.getAttemptId() >= maxAppAttempts
     resourceManager = registerWithResourceManager()
@@ -92,11 +92,11 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     //  }
     //}
     // org.apache.hadoop.io.compress.CompressionCodecFactory.getCodecClasses(conf)
-    
+
     ApplicationMaster.register(this)
     // Start the user's JAR
     userThread = startUserClass()
-    
+
     // This a bit hacky, but we need to wait until the spark.driver.port property has
     // been set by the Thread executing the user class.
     waitForSparkMaster()
@@ -105,11 +105,11 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
 
     // Do this after spark master is up and SparkContext is created so that we can register UI Url
     val appMasterResponse: RegisterApplicationMasterResponse = registerApplicationMaster()
-    
+
     // Allocate all containers
     allocateWorkers()
-    
-    // Wait for the user class to Finish     
+
+    // Wait for the user class to Finish
     userThread.join()
 
     System.exit(0)
@@ -129,7 +129,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     }
     localDirs
   }
-  
+
   private def getApplicationAttemptId(): ApplicationAttemptId = {
     val envs = System.getenv()
     val containerIdString = envs.get(ApplicationConstants.AM_CONTAINER_ID_ENV)
@@ -138,7 +138,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     logInfo("ApplicationAttemptId: " + appAttemptId)
     appAttemptId
   }
-  
+
   private def registerWithResourceManager(): AMRMProtocol = {
     val rmAddress = NetUtils.createSocketAddr(yarnConf.get(
       YarnConfiguration.RM_SCHEDULER_ADDRESS,
@@ -146,26 +146,26 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     logInfo("Connecting to ResourceManager at " + rmAddress)
     rpc.getProxy(classOf[AMRMProtocol], rmAddress, conf).asInstanceOf[AMRMProtocol]
   }
-  
+
   private def registerApplicationMaster(): RegisterApplicationMasterResponse = {
     logInfo("Registering the ApplicationMaster")
     val appMasterRequest = Records.newRecord(classOf[RegisterApplicationMasterRequest])
       .asInstanceOf[RegisterApplicationMasterRequest]
     appMasterRequest.setApplicationAttemptId(appAttemptId)
     // Setting this to master host,port - so that the ApplicationReport at client has some
-    // sensible info. 
+    // sensible info.
     // Users can then monitor stderr/stdout on that node if required.
     appMasterRequest.setHost(Utils.localHostName())
     appMasterRequest.setRpcPort(0)
     appMasterRequest.setTrackingUrl(uiAddress)
     resourceManager.registerApplicationMaster(appMasterRequest)
   }
-  
+
   private def waitForSparkMaster() {
     logInfo("Waiting for spark driver to be reachable.")
     var driverUp = false
     var tries = 0
-    val numTries = conf.getOrElse("spark.yarn.applicationMaster.waitTries",  "10").toInt
+    val numTries = conf.getOrElse("spark.yarn.applicationMaster.waitTries", "10").toInt
     while(!driverUp && tries < numTries) {
       val driverHost = conf.get("spark.driver.host")
       val driverPort = conf.get("spark.driver.port")
@@ -226,7 +226,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
       ApplicationMaster.sparkContextRef.synchronized {
         var count = 0
         val waitTime = 10000L
-        val numTries = conf.getOrElse("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
@@ -241,8 +241,8 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
             yarnConf,
             resourceManager,
             appAttemptId,
-            args, 
-            sparkContext.preferredNodeLocationData) 
+            args,
+            sparkContext.preferredNodeLocationData)
         } else {
           logWarning("Unable to retrieve sparkContext inspite of waiting for %d, numTries = %d".
             format(count * waitTime, numTries))
@@ -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 =
-        conf.getOrElse("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)
@@ -342,11 +342,11 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     for (container <- containers) {
       logInfo("Launching shell command on a new container."
         + ", containerId=" + container.getId()
-        + ", containerNode=" + container.getNodeId().getHost() 
+        + ", containerNode=" + container.getNodeId().getHost()
         + ":" + container.getNodeId().getPort()
         + ", containerNodeURI=" + container.getNodeHttpAddress()
         + ", containerState" + container.getState()
-        + ", containerResourceMemory"  
+        + ", containerResourceMemory"
         + container.getResource().getMemory())
     }
   }
@@ -372,12 +372,12 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
   }
 
   /**
-   * Clean up the staging directory. 
+   * Clean up the staging directory.
    */
-  private def cleanupStagingDir() { 
+  private def cleanupStagingDir() {
     var stagingDirPath: Path = null
     try {
-      val preserveFiles = conf.getOrElse("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) {
@@ -393,7 +393,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     }
   }
 
-  // The shutdown hook that runs when a signal is received AND during normal close of the JVM. 
+  // The shutdown hook that runs when a signal is received AND during normal close of the JVM.
   class AppMasterShutdownHook(appMaster: ApplicationMaster) extends Runnable {
 
     def run() {
@@ -446,18 +446,18 @@ object ApplicationMaster {
     // Note that this will unfortunately not properly clean up the staging files because it gets
     // called too late, after the filesystem is already shutdown.
     if (modified) {
-      Runtime.getRuntime().addShutdownHook(new Thread with Logging { 
+      Runtime.getRuntime().addShutdownHook(new Thread with Logging {
         // This is not only logs, but also ensures that log system is initialized for this instance
         // when we are actually 'run'-ing.
         logInfo("Adding shutdown hook for context " + sc)
-        override def run() { 
-          logInfo("Invoking sc stop from shutdown hook") 
-          sc.stop() 
+        override def run() {
+          logInfo("Invoking sc stop from shutdown hook")
+          sc.stop()
           // Best case ...
           for (master <- applicationMasters) {
             master.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED)
           }
-        } 
+        }
       } )
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
----------------------------------------------------------------------
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 29892e9..cc15088 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
@@ -40,7 +40,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.apache.hadoop.yarn.ipc.YarnRPC
 import org.apache.hadoop.yarn.util.{Apps, Records}
 
-import org.apache.spark.Logging 
+import org.apache.spark.Logging
 import org.apache.spark.util.Utils
 import org.apache.spark.deploy.SparkHadoopUtil
 
@@ -59,7 +59,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
   val STAGING_DIR_PERMISSION: FsPermission = FsPermission.createImmutable(0700:Short)
 
   // App files are world-wide readable and owner writable -> rw-r--r--
-  val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644:Short) 
+  val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644:Short)
 
   // for client user who want to monitor app status by itself.
   def runApp() = {
@@ -103,7 +103,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
         "greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD),
       (args.workerMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: Worker memory size " +
         "must be greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD)
-    ).foreach { case(cond, errStr) => 
+    ).foreach { case(cond, errStr) =>
       if (cond) {
         logError(errStr)
         args.printUsageAndExit(1)
@@ -130,7 +130,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
         queueInfo.getChildQueues.size))
   }
 
-  def verifyClusterResources(app: GetNewApplicationResponse) = { 
+  def verifyClusterResources(app: GetNewApplicationResponse) = {
     val maxMem = app.getMaximumResourceCapability().getMemory()
     logInfo("Max mem capabililty of a single resource in this cluster " + maxMem)
 
@@ -146,7 +146,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     }
 
     // We could add checks to make sure the entire cluster has enough resources but that involves
-    // getting all the node reports and computing ourselves 
+    // getting all the node reports and computing ourselves
   }
 
   def createApplicationSubmissionContext(appId: ApplicationId): ApplicationSubmissionContext = {
@@ -207,7 +207,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
       FileUtil.copy(remoteFs, originalPath, fs, newPath, false, conf)
       fs.setReplication(newPath, replication)
       if (setPerms) fs.setPermission(newPath, new FsPermission(APP_FILE_PERMISSION))
-    } 
+    }
     // Resolve any symlinks in the URI path so using a "current" symlink to point to a specific
     // version shows the specific version in the distributed cache configuration
     val qualPath = fs.makeQualified(newPath)
@@ -230,7 +230,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
       }
     }
     val dst = new Path(fs.getHomeDirectory(), appStagingDir)
-    val replication = conf.getOrElse("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)
@@ -241,7 +241,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
 
     val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
 
-    Map(Client.SPARK_JAR -> System.getenv("SPARK_JAR"), Client.APP_JAR -> args.userJar, 
+    Map(Client.SPARK_JAR -> System.getenv("SPARK_JAR"), Client.APP_JAR -> args.userJar,
       Client.LOG4J_PROP -> System.getenv("SPARK_LOG4J_CONF"))
     .foreach { case(destName, _localPath) =>
       val localPath: String = if (_localPath != null) _localPath.trim() else ""
@@ -253,7 +253,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
         }
         val setPermissions = if (destName.equals(Client.APP_JAR)) true else false
         val destPath = copyRemoteFile(dst, new Path(localURI), replication, setPermissions)
-        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, 
+        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE,
           destName, statCache)
       }
     }
@@ -265,7 +265,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
         val localPath = new Path(localURI)
         val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName())
         val destPath = copyRemoteFile(dst, localPath, replication)
-        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, 
+        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE,
           linkname, statCache, true)
       }
     }
@@ -277,7 +277,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
         val localPath = new Path(localURI)
         val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName())
         val destPath = copyRemoteFile(dst, localPath, replication)
-        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, 
+        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE,
           linkname, statCache)
       }
     }
@@ -289,7 +289,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
         val localPath = new Path(localURI)
         val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName())
         val destPath = copyRemoteFile(dst, localPath, replication)
-        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, 
+        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE,
           linkname, statCache)
       }
     }
@@ -299,7 +299,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
   }
 
   def setupLaunchEnv(
-      localResources: HashMap[String, LocalResource], 
+      localResources: HashMap[String, LocalResource],
       stagingDir: String): HashMap[String, String] = {
     logInfo("Setting up the launch environment")
     val log4jConfLocalRes = localResources.getOrElse(Client.LOG4J_PROP, null)
@@ -354,7 +354,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     // Add Xmx for am memory
     JAVA_OPTS += "-Xmx" + amMemory + "m "
 
-    JAVA_OPTS += " -Djava.io.tmpdir=" + 
+    JAVA_OPTS += " -Djava.io.tmpdir=" +
       new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + " "
 
     // Commenting it out for now - so that people can refer to the properties if required. Remove
@@ -387,11 +387,11 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
       javaCommand = Environment.JAVA_HOME.$() + "/bin/java"
     }
 
-    val commands = List[String](javaCommand + 
+    val commands = List[String](javaCommand +
       " -server " +
       JAVA_OPTS +
       " " + args.amClass +
-      " --class " + args.userClass + 
+      " --class " + args.userClass +
       " --jar " + args.userJar +
       userArgsToString(args) +
       " --worker-memory " + args.workerMemory +
@@ -421,7 +421,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     super.submitApplication(appContext)
   }
 
-  def monitorApplication(appId: ApplicationId): Boolean = {  
+  def monitorApplication(appId: ApplicationId): Boolean = {
     while (true) {
       Thread.sleep(1000)
       val report = super.getApplicationReport(appId)
@@ -443,7 +443,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
 
       val state = report.getYarnApplicationState()
       val dsStatus = report.getFinalApplicationStatus()
-      if (state == YarnApplicationState.FINISHED || 
+      if (state == YarnApplicationState.FINISHED ||
         state == YarnApplicationState.FAILED ||
         state == YarnApplicationState.KILLED) {
         return true
@@ -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
-    conf.set("SPARK_YARN_MODE",  "true")
+    System.setProperty("SPARK_YARN_MODE", "true")
 
     val args = new ClientArguments(argStrings)
 
@@ -479,25 +479,25 @@ object Client {
     Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$())
     // If log4j present, ensure ours overrides all others
     if (addLog4j) {
-      Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + 
+      Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
         Path.SEPARATOR + LOG4J_PROP)
     }
     // Normally the users app.jar is last in case conflicts with spark jars
-    val userClasspathFirst = conf.getOrElse("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.$() + 
+      Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
         Path.SEPARATOR + APP_JAR)
     }
-    Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + 
+    Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
       Path.SEPARATOR + SPARK_JAR)
     Client.populateHadoopClasspath(conf, env)
 
     if (!userClasspathFirst) {
-      Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + 
+      Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
         Path.SEPARATOR + APP_JAR)
     }
-    Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + 
+    Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
       Path.SEPARATOR + "*")
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/642029e7/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
----------------------------------------------------------------------
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 617289f..e9e46a1 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 = conf.getOrElse("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"


[28/33] git commit: Merge remote-tracking branch 'apache/master' into conf2

Posted by pw...@apache.org.
Merge remote-tracking branch 'apache/master' into conf2

Conflicts:
	project/SparkBuild.scala


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/0e5b2adb
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/0e5b2adb
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/0e5b2adb

Branch: refs/heads/master
Commit: 0e5b2adb5c4f2ab1bba275bb3cb2dc2643828f4f
Parents: 42bcfb2 9a0ff72
Author: Matei Zaharia <ma...@databricks.com>
Authored: Wed Jan 1 13:28:54 2014 -0500
Committer: Matei Zaharia <ma...@databricks.com>
Committed: Wed Jan 1 13:28:54 2014 -0500

----------------------------------------------------------------------
 core/pom.xml                                    | 422 ++++++++++---------
 .../org/apache/spark/api/java/JavaPairRDD.scala |  36 ++
 .../org/apache/spark/api/java/JavaRDDLike.scala |  11 +
 .../org/apache/spark/rdd/PairRDDFunctions.scala |  42 ++
 .../main/scala/org/apache/spark/rdd/RDD.scala   |  16 +-
 .../spark/util/SerializableHyperLogLog.scala    |  50 +++
 .../scala/org/apache/spark/JavaAPISuite.java    |  32 ++
 .../spark/rdd/PairRDDFunctionsSuite.scala       |  34 ++
 .../scala/org/apache/spark/rdd/RDDSuite.scala   |  13 +
 .../spark/serializer/KryoSerializerSuite.scala  |   4 +
 pom.xml                                         |   5 +
 project/SparkBuild.scala                        |   3 +-
 12 files changed, 457 insertions(+), 211 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0e5b2adb/core/src/main/scala/org/apache/spark/rdd/RDD.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0e5b2adb/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0e5b2adb/project/SparkBuild.scala
----------------------------------------------------------------------
diff --cc project/SparkBuild.scala
index 204662f,b3b5fc7..2eef2df
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@@ -248,7 -248,7 +248,8 @@@ object SparkBuild extends Build 
          "com.codahale.metrics"     % "metrics-graphite" % "3.0.0",
          "com.twitter"             %% "chill"            % "0.3.1",
          "com.twitter"              % "chill-java"       % "0.3.1",
-         "com.typesafe"             % "config"           % "1.0.2"
++        "com.typesafe"             % "config"           % "1.0.2",
+         "com.clearspring.analytics" % "stream"          % "2.5.1"
        )
    )
  


[30/33] git commit: Miscellaneous fixes from code review.

Posted by pw...@apache.org.
Miscellaneous fixes from code review.

Also replaced SparkConf.getOrElse with just a "get" that takes a default
value, and added getInt, getLong, etc to make code that uses this
simpler later on.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/e2c68642
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/e2c68642
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/e2c68642

Branch: refs/heads/master
Commit: e2c68642c64345434e2034082cf9b299491e9e9f
Parents: 45ff8f4
Author: Matei Zaharia <ma...@databricks.com>
Authored: Wed Jan 1 22:03:39 2014 -0500
Committer: Matei Zaharia <ma...@databricks.com>
Committed: Wed Jan 1 22:03:39 2014 -0500

----------------------------------------------------------------------
 .../main/scala/org/apache/spark/SparkConf.scala | 60 ++++++++++------
 .../scala/org/apache/spark/SparkContext.scala   | 73 ++++++++++++--------
 .../main/scala/org/apache/spark/SparkEnv.scala  | 10 +--
 .../org/apache/spark/api/python/PythonRDD.scala |  4 +-
 .../org/apache/spark/broadcast/Broadcast.scala  |  4 +-
 .../apache/spark/broadcast/HttpBroadcast.scala  |  4 +-
 .../spark/broadcast/TorrentBroadcast.scala      |  2 +-
 .../org/apache/spark/deploy/master/Master.scala | 12 ++--
 .../spark/deploy/master/MasterArguments.scala   |  6 +-
 .../deploy/master/SparkZooKeeperSession.scala   |  2 +-
 .../master/ZooKeeperLeaderElectionAgent.scala   |  2 +-
 .../master/ZooKeeperPersistenceEngine.scala     |  2 +-
 .../org/apache/spark/deploy/worker/Worker.scala |  2 +-
 .../spark/deploy/worker/ui/WorkerWebUI.scala    |  2 +-
 .../org/apache/spark/executor/Executor.scala    |  2 +-
 .../org/apache/spark/io/CompressionCodec.scala  |  4 +-
 .../apache/spark/metrics/MetricsSystem.scala    |  2 +-
 .../spark/network/ConnectionManager.scala       | 18 ++---
 .../spark/network/netty/ShuffleCopier.scala     |  2 +-
 .../org/apache/spark/rdd/CheckpointRDD.scala    |  4 +-
 .../spark/scheduler/TaskResultGetter.scala      |  2 +-
 .../spark/scheduler/TaskSchedulerImpl.scala     | 10 +--
 .../apache/spark/scheduler/TaskSetManager.scala | 16 ++---
 .../cluster/CoarseGrainedSchedulerBackend.scala |  9 +--
 .../cluster/SimrSchedulerBackend.scala          |  2 +-
 .../cluster/SparkDeploySchedulerBackend.scala   |  2 +-
 .../mesos/CoarseMesosSchedulerBackend.scala     |  4 +-
 .../cluster/mesos/MesosSchedulerBackend.scala   |  2 +-
 .../spark/serializer/KryoSerializer.scala       | 13 ++--
 .../spark/storage/BlockFetcherIterator.scala    |  2 +-
 .../org/apache/spark/storage/BlockManager.scala | 24 +++----
 .../spark/storage/BlockManagerMaster.scala      |  4 +-
 .../spark/storage/BlockManagerMasterActor.scala |  4 +-
 .../apache/spark/storage/DiskBlockManager.scala |  2 +-
 .../spark/storage/ShuffleBlockManager.scala     |  6 +-
 .../scala/org/apache/spark/ui/SparkUI.scala     |  4 +-
 .../org/apache/spark/ui/env/EnvironmentUI.scala |  2 +-
 .../spark/ui/jobs/JobProgressListener.scala     |  2 +-
 .../scala/org/apache/spark/util/AkkaUtils.scala | 18 ++---
 .../org/apache/spark/util/MetadataCleaner.scala |  4 +-
 .../scala/org/apache/spark/util/Utils.scala     |  4 +-
 .../scala/org/apache/spark/SparkConfSuite.scala |  2 +-
 .../spark/scheduler/TaskSetManagerSuite.scala   |  2 +-
 .../spark/storage/DiskBlockManagerSuite.scala   | 12 +---
 python/pyspark/conf.py                          | 12 ++--
 .../org/apache/spark/repl/SparkIMain.scala      |  2 +-
 .../streaming/dstream/NetworkInputDStream.scala |  6 +-
 .../streaming/scheduler/JobGenerator.scala      |  4 +-
 .../streaming/scheduler/JobScheduler.scala      |  2 +-
 49 files changed, 206 insertions(+), 189 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/SparkConf.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
index 96239cf..98343e9 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -42,6 +42,12 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable {
 
   /** 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
   }
@@ -51,26 +57,17 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable {
    * run locally with 4 cores, or "spark://master:7077" to run on a Spark standalone cluster.
    */
   def setMaster(master: String): SparkConf = {
-    if (master != null) {
-      settings("spark.master") = master
-    }
-    this
+    set("spark.master", master)
   }
 
   /** Set a name for your application. Shown in the Spark web UI. */
   def setAppName(name: String): SparkConf = {
-    if (name != null) {
-      settings("spark.app.name") = name
-    }
-    this
+    set("spark.app.name", name)
   }
 
   /** Set JAR files to distribute to the cluster. */
   def setJars(jars: Seq[String]): SparkConf = {
-    if (!jars.isEmpty) {
-      settings("spark.jars") = jars.mkString(",")
-    }
-    this
+    set("spark.jars", jars.mkString(","))
   }
 
   /** Set JAR files to distribute to the cluster. (Java-friendly version.) */
@@ -84,8 +81,7 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable {
    * (for example spark.executorEnv.PATH) but this method makes them easier to set.
    */
   def setExecutorEnv(variable: String, value: String): SparkConf = {
-    settings("spark.executorEnv." + variable) = value
-    this
+    set("spark.executorEnv." + variable, value)
   }
 
   /**
@@ -112,10 +108,7 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable {
    * Set the location where Spark is installed on worker nodes.
    */
   def setSparkHome(home: String): SparkConf = {
-    if (home != null) {
-      settings("spark.home") = home
-    }
-    this
+    set("spark.home", home)
   }
 
   /** Set multiple parameters together */
@@ -132,9 +125,20 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable {
     this
   }
 
-  /** Get a parameter; throws an exception if it's not set */
+  /** 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(key)
+    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 */
@@ -145,9 +149,19 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable {
   /** Get all parameters as a list of pairs */
   def getAll: Array[(String, String)] = settings.clone().toArray
 
-  /** Get a parameter, falling back to a default if not set */
-  def getOrElse(k: String, defaultValue: String): String = {
-    settings.getOrElse(k, defaultValue)
+  /** 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 */

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/SparkContext.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 46874c4..84bd0f7 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -22,12 +22,11 @@ import java.net.URI
 import java.util.{UUID, Properties}
 import java.util.concurrent.atomic.AtomicInteger
 
-import scala.collection.{Map, Set, immutable}
+import scala.collection.{Map, Set}
 import scala.collection.generic.Growable
 
 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
@@ -49,7 +48,8 @@ import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, Me
 import org.apache.spark.scheduler.local.LocalBackend
 import org.apache.spark.storage.{BlockManagerSource, RDDInfo, StorageStatus, StorageUtils}
 import org.apache.spark.ui.SparkUI
-import org.apache.spark.util._
+import org.apache.spark.util.{Utils, TimeStampedHashMap, MetadataCleaner, MetadataCleanerType,
+ClosureCleaner}
 
 /**
  * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark
@@ -77,7 +77,7 @@ class SparkContext(
    * @param conf a [[org.apache.spark.SparkConf]] object specifying other Spark parameters
    */
   def this(master: String, appName: String, conf: SparkConf) =
-    this(conf.clone().setMaster(master).setAppName(appName))
+    this(SparkContext.updatedConf(conf, master, appName))
 
   /**
    * Alternative constructor that allows setting common Spark properties directly
@@ -97,13 +97,7 @@ class SparkContext(
       environment: Map[String, String] = Map(),
       preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map()) =
   {
-    this(
-      new SparkConf()
-        .setMaster(master)
-        .setAppName(appName)
-        .setJars(jars)
-        .setExecutorEnv(environment.toSeq)
-        .setSparkHome(sparkHome),
+    this(SparkContext.updatedConf(new SparkConf(), master, appName, sparkHome, jars, environment),
       preferredNodeLocationData)
   }
 
@@ -175,11 +169,9 @@ class SparkContext(
   // Environment variables to pass to our executors
   private[spark] val executorEnvs = HashMap[String, String]()
   // Note: SPARK_MEM is included for Mesos, but overwritten for standalone mode in ExecutorRunner
-  for (key <- Seq("SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS", "SPARK_TESTING")) {
-    val value = System.getenv(key)
-    if (value != null) {
-      executorEnvs(key) = value
-    }
+  for (key <- Seq("SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS", "SPARK_TESTING");
+      value <- Option(System.getenv(key))) {
+    executorEnvs(key) = value
   }
   // Since memory can be set with a system property too, use that
   executorEnvs("SPARK_MEM") = executorMemory + "m"
@@ -220,7 +212,7 @@ class SparkContext(
         hadoopConf.set(key.substring("spark.hadoop.".length), value)
       }
     }
-    val bufferSize = conf.getOrElse("spark.buffer.size", "65536")
+    val bufferSize = conf.get("spark.buffer.size", "65536")
     hadoopConf.set("io.file.buffer.size", bufferSize)
     hadoopConf
   }
@@ -733,13 +725,7 @@ class SparkContext(
    * (in that order of preference). If neither of these is set, return None.
    */
   private[spark] def getSparkHome(): Option[String] = {
-    if (conf.contains("spark.home")) {
-      Some(conf.get("spark.home"))
-    } else if (System.getenv("SPARK_HOME") != null) {
-      Some(System.getenv("SPARK_HOME"))
-    } else {
-      None
-    }
+    conf.getOption("spark.home").orElse(Option(System.getenv("SPARK_HOME")))
   }
 
   /**
@@ -1026,7 +1012,7 @@ object SparkContext {
 
   /**
    * Find the JAR from which a given class was loaded, to make it easy for users to pass
-   * their JARs to SparkContext
+   * their JARs to SparkContext.
    */
   def jarOfClass(cls: Class[_]): Seq[String] = {
     val uri = cls.getResource("/" + cls.getName.replace('.', '/') + ".class")
@@ -1043,10 +1029,41 @@ object SparkContext {
     }
   }
 
-  /** Find the JAR that contains the class of a particular object */
+  /**
+   * Find the JAR that contains the class of a particular object, to make it easy for users
+   * to pass their JARs to SparkContext. In most cases you can call jarOfObject(this) in
+   * your driver program.
+   */
   def jarOfObject(obj: AnyRef): Seq[String] = jarOfClass(obj.getClass)
 
-  // Creates a task scheduler based on a given master URL. Extracted for testing.
+  /**
+   * Creates a modified version of a SparkConf with the parameters that can be passed separately
+   * to SparkContext, to make it easier to write SparkContext's constructors. This ignores
+   * parameters that are passed as the default value of null, instead of throwing an exception
+   * like SparkConf would.
+   */
+  private def updatedConf(
+      conf: SparkConf,
+      master: String,
+      appName: String,
+      sparkHome: String = null,
+      jars: Seq[String] = Nil,
+      environment: Map[String, String] = Map()): SparkConf =
+  {
+    val res = conf.clone()
+    res.setMaster(master)
+    res.setAppName(appName)
+    if (sparkHome != null) {
+      res.setSparkHome(sparkHome)
+    }
+    if (!jars.isEmpty) {
+      res.setJars(jars)
+    }
+    res.setExecutorEnv(environment.toSeq)
+    res
+  }
+
+  /** Creates a task scheduler based on a given master URL. Extracted for testing. */
   private def createTaskScheduler(sc: SparkContext, master: String, appName: String)
       : TaskScheduler =
   {
@@ -1156,7 +1173,7 @@ object SparkContext {
       case mesosUrl @ MESOS_REGEX(_) =>
         MesosNativeLibrary.load()
         val scheduler = new TaskSchedulerImpl(sc)
-        val coarseGrained = sc.conf.getOrElse("spark.mesos.coarse", "false").toBoolean
+        val coarseGrained = sc.conf.get("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)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/SparkEnv.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala
index d06af8e..634a94f 100644
--- a/core/src/main/scala/org/apache/spark/SparkEnv.scala
+++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala
@@ -144,17 +144,17 @@ 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 = conf.getOrElse(propertyName,  defaultClassName)
+      val name = conf.get(propertyName,  defaultClassName)
       Class.forName(name, true, classLoader).newInstance().asInstanceOf[T]
     }
 
     val serializerManager = new SerializerManager
 
     val serializer = serializerManager.setDefault(
-      conf.getOrElse("spark.serializer", "org.apache.spark.serializer.JavaSerializer"), conf)
+      conf.get("spark.serializer", "org.apache.spark.serializer.JavaSerializer"), conf)
 
     val closureSerializer = serializerManager.get(
-      conf.getOrElse("spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer"),
+      conf.get("spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer"),
       conf)
 
     def registerOrLookup(name: String, newActor: => Actor): Either[ActorRef, ActorSelection] = {
@@ -162,8 +162,8 @@ object SparkEnv extends Logging {
         logInfo("Registering " + name)
         Left(actorSystem.actorOf(Props(newActor), name = name))
       } else {
-        val driverHost: String = conf.getOrElse("spark.driver.host", "localhost")
-        val driverPort: Int = conf.getOrElse("spark.driver.port", "7077").toInt
+        val driverHost: String = conf.get("spark.driver.host", "localhost")
+        val driverPort: Int = conf.get("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)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
----------------------------------------------------------------------
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 05fd824..32cc70e 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 = conf.getOrElse("spark.buffer.size", "65536").toInt
+  val bufferSize = conf.get("spark.buffer.size", "65536").toInt
 
   override def getPartitions = parent.partitions
 
@@ -250,7 +250,7 @@ private class PythonAccumulatorParam(@transient serverHost: String, serverPort:
 
   Utils.checkHost(serverHost, "Expected hostname")
 
-  val bufferSize = SparkEnv.get.conf.getOrElse("spark.buffer.size", "65536").toInt
+  val bufferSize = SparkEnv.get.conf.get("spark.buffer.size", "65536").toInt
 
   override def zero(value: JList[Array[Byte]]): JList[Array[Byte]] = new JArrayList
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala
----------------------------------------------------------------------
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 be99d22..0fc478a 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala
@@ -31,7 +31,7 @@ abstract class Broadcast[T](private[spark] val id: Long) extends Serializable {
   override def toString = "Broadcast(" + id + ")"
 }
 
-private[spark] 
+private[spark]
 class BroadcastManager(val _isDriver: Boolean, conf: SparkConf) extends Logging with Serializable {
 
   private var initialized = false
@@ -43,7 +43,7 @@ class BroadcastManager(val _isDriver: Boolean, conf: SparkConf) extends Logging
   private def initialize() {
     synchronized {
       if (!initialized) {
-        val broadcastFactoryClass = conf.getOrElse(
+        val broadcastFactoryClass = conf.get(
           "spark.broadcast.factory", "org.apache.spark.broadcast.HttpBroadcastFactory")
 
         broadcastFactory =

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala
----------------------------------------------------------------------
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 47528bc..db596d5 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala
@@ -92,8 +92,8 @@ private object HttpBroadcast extends Logging {
   def initialize(isDriver: Boolean, conf: SparkConf) {
     synchronized {
       if (!initialized) {
-        bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt
-        compress = conf.getOrElse("spark.broadcast.compress", "true").toBoolean
+        bufferSize = conf.get("spark.buffer.size", "65536").toInt
+        compress = conf.get("spark.broadcast.compress", "true").toBoolean
         if (isDriver) {
           createServer(conf)
           conf.set("spark.httpBroadcast.uri",  serverUri)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
----------------------------------------------------------------------
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 00ec3b9..9530938 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
@@ -180,7 +180,7 @@ extends Logging {
     initialized = false
   }
 
-  lazy val BLOCK_SIZE = conf.getOrElse("spark.broadcast.blockSize", "4096").toInt * 1024
+  lazy val BLOCK_SIZE = conf.get("spark.broadcast.blockSize", "4096").toInt * 1024
 
   def blockifyObject[T](obj: T): TorrentInfo = {
     val byteArray = Utils.serialize[T](obj)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
----------------------------------------------------------------------
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 9c89e36..7b696cf 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
@@ -43,11 +43,11 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
   val conf = new SparkConf
 
   val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss")  // For application IDs
-  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")
+  val WORKER_TIMEOUT = conf.get("spark.worker.timeout", "60").toLong * 1000
+  val RETAINED_APPLICATIONS = conf.get("spark.deploy.retainedApplications", "200").toInt
+  val REAPER_ITERATIONS = conf.get("spark.dead.worker.persistence", "15").toInt
+  val RECOVERY_DIR = conf.get("spark.deploy.recoveryDirectory", "")
+  val RECOVERY_MODE = conf.get("spark.deploy.recoveryMode", "NONE")
 
   var nextAppNumber = 0
   val workers = new HashSet[WorkerInfo]
@@ -88,7 +88,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 = conf.getOrElse("spark.deploy.spreadOut", "true").toBoolean
+  val spreadOutApps = conf.get("spark.deploy.spreadOut", "true").toBoolean
 
   override def preStart() {
     logInfo("Starting Spark master at " + masterUrl)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala
----------------------------------------------------------------------
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 7ce83f9..e7f3224 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
@@ -27,8 +27,8 @@ private[spark] class MasterArguments(args: Array[String], conf: SparkConf) {
   var host = Utils.localHostName()
   var port = 7077
   var webUiPort = 8080
-  
-  // Check for settings in environment variables 
+
+  // Check for settings in environment variables
   if (System.getenv("SPARK_MASTER_HOST") != null) {
     host = System.getenv("SPARK_MASTER_HOST")
   }
@@ -38,7 +38,7 @@ private[spark] class MasterArguments(args: Array[String], conf: SparkConf) {
   if (System.getenv("SPARK_MASTER_WEBUI_PORT") != null) {
     webUiPort = System.getenv("SPARK_MASTER_WEBUI_PORT").toInt
   }
-  if (conf.get("master.ui.port") != null) {
+  if (conf.contains("master.ui.port")) {
     webUiPort = conf.get("master.ui.port").toInt
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala
----------------------------------------------------------------------
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 60c7a7c..999090a 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
@@ -37,7 +37,7 @@ import org.apache.spark.{SparkConf, Logging}
  */
 private[spark] class SparkZooKeeperSession(zkWatcher: SparkZooKeeperWatcher,
     conf: SparkConf) extends Logging {
-  val ZK_URL = conf.getOrElse("spark.deploy.zookeeper.url", "")
+  val ZK_URL = conf.get("spark.deploy.zookeeper.url", "")
 
   val ZK_ACL = ZooDefs.Ids.OPEN_ACL_UNSAFE
   val ZK_TIMEOUT_MILLIS = 30000

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala
----------------------------------------------------------------------
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 a61597b..77c23fb 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
@@ -28,7 +28,7 @@ private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: ActorRef,
     masterUrl: String, conf: SparkConf)
   extends LeaderElectionAgent with SparkZooKeeperWatcher with Logging  {
 
-  val WORKING_DIR = conf.getOrElse("spark.deploy.zookeeper.dir", "/spark") + "/leader_election"
+  val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/leader_election"
 
   private val watcher = new ZooKeeperWatcher()
   private val zk = new SparkZooKeeperSession(this, conf)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
----------------------------------------------------------------------
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 245a558..52000d4 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
@@ -27,7 +27,7 @@ class ZooKeeperPersistenceEngine(serialization: Serialization, conf: SparkConf)
   with SparkZooKeeperWatcher
   with Logging
 {
-  val WORKING_DIR = conf.getOrElse("spark.deploy.zookeeper.dir", "/spark") + "/master_status"
+  val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/master_status"
 
   val zk = new SparkZooKeeperSession(this, conf)
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
----------------------------------------------------------------------
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 f844fcb..fcaf4e9 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
@@ -55,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 = conf.getOrElse("spark.worker.timeout", "60").toLong * 1000 / 4
+  val HEARTBEAT_MILLIS = conf.get("spark.worker.timeout", "60").toLong * 1000 / 4
 
   val REGISTRATION_TIMEOUT = 20.seconds
   val REGISTRATION_RETRIES = 3

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
----------------------------------------------------------------------
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 a801d85..c382034 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
@@ -37,7 +37,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I
   val timeout = AkkaUtils.askTimeout(worker.conf)
   val host = Utils.localHostName()
   val port = requestedPort.getOrElse(
-    worker.conf.getOrElse("worker.ui.port",  WorkerWebUI.DEFAULT_PORT).toInt)
+    worker.conf.get("worker.ui.port",  WorkerWebUI.DEFAULT_PORT).toInt)
 
   var server: Option[Server] = None
   var boundPort: Option[Int] = None

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/executor/Executor.scala
----------------------------------------------------------------------
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 5b70165..3c92c20 100644
--- a/core/src/main/scala/org/apache/spark/executor/Executor.scala
+++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala
@@ -302,7 +302,7 @@ private[spark] class Executor(
    * new classes defined by the REPL as the user types code
    */
   private def addReplClassLoaderIfNeeded(parent: ClassLoader): ClassLoader = {
-    val classUri = conf.getOrElse("spark.repl.class.uri", null)
+    val classUri = conf.get("spark.repl.class.uri", null)
     if (classUri != null) {
       logInfo("Using REPL class URI: " + classUri)
       try {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
----------------------------------------------------------------------
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 075a18b..a1e9884 100644
--- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
+++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
@@ -39,7 +39,7 @@ trait CompressionCodec {
 
 private[spark] object CompressionCodec {
   def createCodec(conf: SparkConf): CompressionCodec = {
-    createCodec(conf, conf.getOrElse(
+    createCodec(conf, conf.get(
       "spark.io.compression.codec", classOf[LZFCompressionCodec].getName))
   }
 
@@ -71,7 +71,7 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec {
 class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec {
 
   override def compressedOutputStream(s: OutputStream): OutputStream = {
-    val blockSize = conf.getOrElse("spark.io.compression.snappy.block.size", "32768").toInt
+    val blockSize = conf.get("spark.io.compression.snappy.block.size", "32768").toInt
     new SnappyOutputStream(s, blockSize)
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
----------------------------------------------------------------------
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 0e41c73..9930537 100644
--- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
+++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
@@ -65,7 +65,7 @@ import org.apache.spark.metrics.source.Source
 private[spark] class MetricsSystem private (val instance: String,
     conf: SparkConf) extends Logging {
 
-  val confFile = conf.getOrElse("spark.metrics.conf", null)
+  val confFile = conf.get("spark.metrics.conf", null)
   val metricsConfig = new MetricsConfig(Option(confFile))
 
   val sinks = new mutable.ArrayBuffer[Sink]

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala
----------------------------------------------------------------------
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 697096f..46c40d0 100644
--- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala
+++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala
@@ -54,22 +54,22 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi
   private val selector = SelectorProvider.provider.openSelector()
 
   private val handleMessageExecutor = new ThreadPoolExecutor(
-    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,
+    conf.get("spark.core.connection.handler.threads.min", "20").toInt,
+    conf.get("spark.core.connection.handler.threads.max", "60").toInt,
+    conf.get("spark.core.connection.handler.threads.keepalive", "60").toInt, TimeUnit.SECONDS,
     new LinkedBlockingDeque[Runnable]())
 
   private val handleReadWriteExecutor = new ThreadPoolExecutor(
-    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,
+    conf.get("spark.core.connection.io.threads.min", "4").toInt,
+    conf.get("spark.core.connection.io.threads.max", "32").toInt,
+    conf.get("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(
-    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,
+    conf.get("spark.core.connection.connect.threads.min", "1").toInt,
+    conf.get("spark.core.connection.connect.threads.max", "8").toInt,
+    conf.get("spark.core.connection.connect.threads.keepalive", "60").toInt, TimeUnit.SECONDS,
     new LinkedBlockingDeque[Runnable]())
 
   private val serverChannel = ServerSocketChannel.open()

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala
----------------------------------------------------------------------
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 db28ddf..b729eb1 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
@@ -36,7 +36,7 @@ private[spark] class ShuffleCopier(conf: SparkConf) extends Logging {
       resultCollectCallback: (BlockId, Long, ByteBuf) => Unit) {
 
     val handler = new ShuffleCopier.ShuffleClientHandler(resultCollectCallback)
-    val connectTimeout = conf.getOrElse("spark.shuffle.netty.connect.timeout", "60000").toInt
+    val connectTimeout = conf.get("spark.shuffle.netty.connect.timeout", "60000").toInt
     val fc = new FileClient(handler, connectTimeout)
 
     try {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
----------------------------------------------------------------------
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 172ba6b..6d4f461 100644
--- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
@@ -97,7 +97,7 @@ private[spark] object CheckpointRDD extends Logging {
       throw new IOException("Checkpoint failed: temporary path " +
         tempOutputPath + " already exists")
     }
-    val bufferSize = env.conf.getOrElse("spark.buffer.size", "65536").toInt
+    val bufferSize = env.conf.get("spark.buffer.size", "65536").toInt
 
     val fileOutputStream = if (blockSize < 0) {
       fs.create(tempOutputPath, false, bufferSize)
@@ -131,7 +131,7 @@ private[spark] object CheckpointRDD extends Logging {
     ): Iterator[T] = {
     val env = SparkEnv.get
     val fs = path.getFileSystem(broadcastedConf.value.value)
-    val bufferSize = env.conf.getOrElse("spark.buffer.size", "65536").toInt
+    val bufferSize = env.conf.get("spark.buffer.size", "65536").toInt
     val fileInputStream = fs.open(path, bufferSize)
     val serializer = env.serializer.newInstance()
     val deserializeStream = serializer.deserializeStream(fileInputStream)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala
index 29b0247..e22b1e5 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala
@@ -31,7 +31,7 @@ import org.apache.spark.util.Utils
 private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedulerImpl)
   extends Logging {
 
-  private val THREADS = sparkEnv.conf.getOrElse("spark.resultGetter.threads", "4").toInt
+  private val THREADS = sparkEnv.conf.get("spark.resultGetter.threads", "4").toInt
   private val getTaskResultExecutor = Utils.newDaemonFixedThreadPool(
     THREADS, "Result resolver thread")
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
index bffd990..d94b706 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
@@ -51,15 +51,15 @@ private[spark] class TaskSchedulerImpl(
     isLocal: Boolean = false)
   extends TaskScheduler with Logging
 {
-  def this(sc: SparkContext) = this(sc, sc.conf.getOrElse("spark.task.maxFailures", "4").toInt)
+  def this(sc: SparkContext) = this(sc, sc.conf.get("spark.task.maxFailures", "4").toInt)
 
   val conf = sc.conf
 
   // How often to check for speculative tasks
-  val SPECULATION_INTERVAL = conf.getOrElse("spark.speculation.interval", "100").toLong
+  val SPECULATION_INTERVAL = conf.get("spark.speculation.interval", "100").toLong
 
   // Threshold above which we warn user initial TaskSet may be starved
-  val STARVATION_TIMEOUT = conf.getOrElse("spark.starvation.timeout", "15000").toLong
+  val STARVATION_TIMEOUT = conf.get("spark.starvation.timeout", "15000").toLong
 
   // TaskSetManagers are not thread safe, so any access to one should be synchronized
   // on this class.
@@ -96,7 +96,7 @@ private[spark] class TaskSchedulerImpl(
   var rootPool: Pool = null
   // default scheduler is FIFO
   val schedulingMode: SchedulingMode = SchedulingMode.withName(
-    conf.getOrElse("spark.scheduler.mode", "FIFO"))
+    conf.get("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)
@@ -125,7 +125,7 @@ private[spark] class TaskSchedulerImpl(
   override def start() {
     backend.start()
 
-    if (!isLocal && conf.getOrElse("spark.speculation", "false").toBoolean) {
+    if (!isLocal && conf.get("spark.speculation", "false").toBoolean) {
       logInfo("Starting speculative execution thread")
       import sc.env.actorSystem.dispatcher
       sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL milliseconds,

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
index b99664a..67ad99a 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
@@ -57,11 +57,11 @@ private[spark] class TaskSetManager(
   val conf = sched.sc.conf
 
   // CPUs to request per task
-  val CPUS_PER_TASK = conf.getOrElse("spark.task.cpus", "1").toInt
+  val CPUS_PER_TASK = conf.get("spark.task.cpus", "1").toInt
 
   // Quantile of tasks at which to start speculation
-  val SPECULATION_QUANTILE = conf.getOrElse("spark.speculation.quantile", "0.75").toDouble
-  val SPECULATION_MULTIPLIER = conf.getOrElse("spark.speculation.multiplier", "1.5").toDouble
+  val SPECULATION_QUANTILE = conf.get("spark.speculation.quantile", "0.75").toDouble
+  val SPECULATION_MULTIPLIER = conf.get("spark.speculation.multiplier", "1.5").toDouble
 
   // Serializer for closures and tasks.
   val env = SparkEnv.get
@@ -116,7 +116,7 @@ private[spark] class TaskSetManager(
 
   // How frequently to reprint duplicate exceptions in full, in milliseconds
   val EXCEPTION_PRINT_INTERVAL =
-    conf.getOrElse("spark.logging.exceptionPrintInterval", "10000").toLong
+    conf.get("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
@@ -678,14 +678,14 @@ private[spark] class TaskSetManager(
   }
 
   private def getLocalityWait(level: TaskLocality.TaskLocality): Long = {
-    val defaultWait = conf.getOrElse("spark.locality.wait", "3000")
+    val defaultWait = conf.get("spark.locality.wait", "3000")
     level match {
       case TaskLocality.PROCESS_LOCAL =>
-        conf.getOrElse("spark.locality.wait.process", defaultWait).toLong
+        conf.get("spark.locality.wait.process", defaultWait).toLong
       case TaskLocality.NODE_LOCAL =>
-        conf.getOrElse("spark.locality.wait.node", defaultWait).toLong
+        conf.get("spark.locality.wait.node", defaultWait).toLong
       case TaskLocality.RACK_LOCAL =>
-        conf.getOrElse("spark.locality.wait.rack", defaultWait).toLong
+        conf.get("spark.locality.wait.rack", defaultWait).toLong
       case TaskLocality.ANY =>
         0L
     }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
----------------------------------------------------------------------
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 b4a3ecc..2f5bcaf 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,7 +22,6 @@ 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
@@ -64,7 +63,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A
       context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
 
       // Periodically revive offers to allow delay scheduling to work
-      val reviveInterval = conf.getOrElse("spark.scheduler.revive.interval", "1000").toLong
+      val reviveInterval = conf.get("spark.scheduler.revive.interval", "1000").toLong
       import context.dispatcher
       context.system.scheduler.schedule(0.millis, reviveInterval.millis, self, ReviveOffers)
     }
@@ -209,8 +208,10 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A
     driverActor ! KillTask(taskId, executorId)
   }
 
-  override def defaultParallelism() = Try(conf.get("spark.default.parallelism")).toOption
-      .map(_.toInt).getOrElse(math.max(totalCoreCount.get(), 2))
+  override def defaultParallelism(): Int = {
+    conf.getOption("spark.default.parallelism").map(_.toInt).getOrElse(
+      math.max(totalCoreCount.get(), 2))
+  }
 
   // Called by subclasses when notified of a lost worker
   def removeExecutor(executorId: String, reason: String) {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala
----------------------------------------------------------------------
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 f41fbbd..b44d1e4 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
@@ -33,7 +33,7 @@ private[spark] class SimrSchedulerBackend(
   val tmpPath = new Path(driverFilePath + "_tmp")
   val filePath = new Path(driverFilePath)
 
-  val maxCores = conf.getOrElse("spark.simr.executor.cores", "1").toInt
+  val maxCores = conf.get("spark.simr.executor.cores", "1").toInt
 
   override def start() {
     super.start()

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
----------------------------------------------------------------------
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 2240775..9858717 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
@@ -38,7 +38,7 @@ private[spark] class SparkDeploySchedulerBackend(
   var stopping = false
   var shutdownCallback : (SparkDeploySchedulerBackend) => Unit = _
 
-  val maxCores = conf.getOrElse("spark.cores.max",  Int.MaxValue.toString).toInt
+  val maxCores = conf.get("spark.cores.max",  Int.MaxValue.toString).toInt
 
   override def start() {
     super.start()

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
----------------------------------------------------------------------
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 9e2cd3f..d247fa4 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
@@ -62,7 +62,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 = conf.getOrElse("spark.cores.max",  Int.MaxValue.toString).toInt
+  val maxCores = conf.get("spark.cores.max",  Int.MaxValue.toString).toInt
 
   // Cores we have acquired with each Mesos task ID
   val coresByTaskId = new HashMap[Int, Int]
@@ -77,7 +77,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 = conf.getOrElse("spark.mesos.extra.cores", "0").toInt
+  val extraCoresPerSlave = conf.get("spark.mesos.extra.cores", "0").toInt
 
   var nextMesosTaskId = 0
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
----------------------------------------------------------------------
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 be96382..c20fc41 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
@@ -340,5 +340,5 @@ private[spark] class MesosSchedulerBackend(
   }
 
   // TODO: query Mesos for number of cores
-  override def defaultParallelism() = sc.conf.getOrElse("spark.default.parallelism", "8").toInt
+  override def defaultParallelism() = sc.conf.get("spark.default.parallelism", "8").toInt
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
----------------------------------------------------------------------
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 2367f3f..a24a3b0 100644
--- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
@@ -29,17 +29,14 @@ import org.apache.spark._
 import org.apache.spark.broadcast.HttpBroadcast
 import org.apache.spark.scheduler.MapStatus
 import org.apache.spark.storage._
-import scala.util.Try
-import org.apache.spark.storage.PutBlock
-import org.apache.spark.storage.GetBlock
-import org.apache.spark.storage.GotBlock
+import org.apache.spark.storage.{GetBlock, GotBlock, PutBlock}
 
 /**
  * A Spark serializer that uses the [[https://code.google.com/p/kryo/ Kryo serialization library]].
  */
 class KryoSerializer(conf: SparkConf) extends org.apache.spark.serializer.Serializer with Logging {
   private val bufferSize = {
-    conf.getOrElse("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024
+    conf.get("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024
   }
 
   def newKryoOutput() = new KryoOutput(bufferSize)
@@ -51,7 +48,7 @@ class KryoSerializer(conf: SparkConf) extends org.apache.spark.serializer.Serial
 
     // 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(conf.getOrElse("spark.kryo.referenceTracking", "true").toBoolean)
+    kryo.setReferences(conf.get("spark.kryo.referenceTracking", "true").toBoolean)
 
     for (cls <- KryoSerializer.toRegister) kryo.register(cls)
 
@@ -61,13 +58,13 @@ class KryoSerializer(conf: SparkConf) extends org.apache.spark.serializer.Serial
 
     // Allow the user to register their own classes by setting spark.kryo.registrator
     try {
-      Try(conf.get("spark.kryo.registrator")).toOption.foreach { regCls =>
+      for (regCls <- conf.getOption("spark.kryo.registrator")) {
         logDebug("Running user registrator: " + regCls)
         val reg = Class.forName(regCls, true, classLoader).newInstance().asInstanceOf[KryoRegistrator]
         reg.registerClasses(kryo)
       }
     } catch {
-      case _: Exception => println("Failed to register spark.kryo.registrator")
+      case e: Exception => logError("Failed to run spark.kryo.registrator", e)
     }
 
     // Register Chill's classes; we do this after our ranges and the user's own classes to let

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala
----------------------------------------------------------------------
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 3b25f68..4747863 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala
@@ -327,7 +327,7 @@ object BlockFetcherIterator {
         fetchRequestsSync.put(request)
       }
 
-      copiers = startCopiers(conf.getOrElse("spark.shuffle.copier.threads", "6").toInt)
+      copiers = startCopiers(conf.get("spark.shuffle.copier.threads", "6").toInt)
       logInfo("Started " + fetchRequestsSync.size + " remote gets in " +
         Utils.getUsedTimeMs(startTime))
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
----------------------------------------------------------------------
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 16ee208..6d2cda9 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
@@ -49,7 +49,7 @@ private[spark] class BlockManager(
 
   val shuffleBlockManager = new ShuffleBlockManager(this)
   val diskBlockManager = new DiskBlockManager(shuffleBlockManager,
-    conf.getOrElse("spark.local.dir",  System.getProperty("java.io.tmpdir")))
+    conf.get("spark.local.dir",  System.getProperty("java.io.tmpdir")))
 
   private val blockInfo = new TimeStampedHashMap[BlockId, BlockInfo]
 
@@ -58,8 +58,8 @@ private[spark] class BlockManager(
 
   // If we use Netty for shuffle, start a new Netty-based shuffle sender service.
   private val nettyPort: Int = {
-    val useNetty = conf.getOrElse("spark.shuffle.use.netty", "false").toBoolean
-    val nettyPortConfig = conf.getOrElse("spark.shuffle.sender.port", "0").toInt
+    val useNetty = conf.get("spark.shuffle.use.netty", "false").toBoolean
+    val nettyPortConfig = conf.get("spark.shuffle.sender.port", "0").toInt
     if (useNetty) diskBlockManager.startShuffleBlockSender(nettyPortConfig) else 0
   }
 
@@ -72,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 =
-    conf.getOrElse("spark.reducer.maxMbInFlight", "48").toLong * 1024 * 1024
+    conf.get("spark.reducer.maxMbInFlight", "48").toLong * 1024 * 1024
 
   // Whether to compress broadcast variables that are stored
-  val compressBroadcast = conf.getOrElse("spark.broadcast.compress", "true").toBoolean
+  val compressBroadcast = conf.get("spark.broadcast.compress", "true").toBoolean
   // Whether to compress shuffle output that are stored
-  val compressShuffle = conf.getOrElse("spark.shuffle.compress", "true").toBoolean
+  val compressShuffle = conf.get("spark.shuffle.compress", "true").toBoolean
   // Whether to compress RDD partitions that are stored serialized
-  val compressRdds = conf.getOrElse("spark.rdd.compress", "false").toBoolean
+  val compressRdds = conf.get("spark.rdd.compress", "false").toBoolean
 
   val heartBeatFrequency = BlockManager.getHeartBeatFrequency(conf)
 
@@ -443,7 +443,7 @@ private[spark] class BlockManager(
       : BlockFetcherIterator = {
 
     val iter =
-      if (conf.getOrElse("spark.shuffle.use.netty", "false").toBoolean) {
+      if (conf.get("spark.shuffle.use.netty", "false").toBoolean) {
         new BlockFetcherIterator.NettyBlockFetcherIterator(this, blocksByAddress, serializer)
       } else {
         new BlockFetcherIterator.BasicBlockFetcherIterator(this, blocksByAddress, serializer)
@@ -469,7 +469,7 @@ private[spark] class BlockManager(
   def getDiskWriter(blockId: BlockId, file: File, serializer: Serializer, bufferSize: Int)
     : BlockObjectWriter = {
     val compressStream: OutputStream => OutputStream = wrapForCompression(blockId, _)
-    val syncWrites = conf.getOrElse("spark.shuffle.sync", "false").toBoolean
+    val syncWrites = conf.get("spark.shuffle.sync", "false").toBoolean
     new DiskBlockObjectWriter(blockId, file, serializer, bufferSize, compressStream, syncWrites)
   }
 
@@ -864,15 +864,15 @@ private[spark] object BlockManager extends Logging {
   val ID_GENERATOR = new IdGenerator
 
   def getMaxMemory(conf: SparkConf): Long = {
-    val memoryFraction = conf.getOrElse("spark.storage.memoryFraction", "0.66").toDouble
+    val memoryFraction = conf.get("spark.storage.memoryFraction", "0.66").toDouble
     (Runtime.getRuntime.maxMemory * memoryFraction).toLong
   }
 
   def getHeartBeatFrequency(conf: SparkConf): Long =
-    conf.getOrElse("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong / 4
+    conf.get("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong / 4
 
   def getDisableHeartBeatsForTesting(conf: SparkConf): Boolean =
-    conf.getOrElse("spark.test.disableBlockManagerHeartBeat", "false").toBoolean
+    conf.get("spark.test.disableBlockManagerHeartBeat", "false").toBoolean
 
   /**
    * Attempt to clean up a ByteBuffer if it is memory-mapped. This uses an *unsafe* Sun API that

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala
----------------------------------------------------------------------
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 8e4a88b..b5afe8c 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala
@@ -31,8 +31,8 @@ private[spark]
 class BlockManagerMaster(var driverActor : Either[ActorRef, ActorSelection],
     conf: SparkConf) extends Logging {
 
-  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 AKKA_RETRY_ATTEMPTS: Int = conf.get("spark.akka.num.retries", "3").toInt
+  val AKKA_RETRY_INTERVAL_MS: Int = conf.get("spark.akka.retry.wait", "3000").toInt
 
   val DRIVER_AKKA_ACTOR_NAME = "BlockManagerMaster"
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
----------------------------------------------------------------------
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 dbbeeb3..58452d9 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
@@ -50,10 +50,10 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act
 
   private val akkaTimeout = AkkaUtils.askTimeout(conf)
 
-  val slaveTimeout = conf.getOrElse("spark.storage.blockManagerSlaveTimeoutMs",
+  val slaveTimeout = conf.get("spark.storage.blockManagerSlaveTimeoutMs",
     "" + (BlockManager.getHeartBeatFrequency(conf) * 3)).toLong
 
-  val checkTimeoutInterval = conf.getOrElse("spark.storage.blockManagerTimeoutIntervalMs",
+  val checkTimeoutInterval = conf.get("spark.storage.blockManagerTimeoutIntervalMs",
     "60000").toLong
 
   var timeoutCheckingTask: Cancellable = null

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
----------------------------------------------------------------------
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 7697092..55dcb37 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 = shuffleManager.conf.getOrElse("spark.diskStore.subDirectories", "64").toInt
+  private val subDirsPerLocalDir = shuffleManager.conf.get("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

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala
----------------------------------------------------------------------
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 151eedb..39dc7bb 100644
--- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala
@@ -27,8 +27,6 @@ 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 {
@@ -66,9 +64,9 @@ class ShuffleBlockManager(blockManager: BlockManager) {
   // 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 =
-    conf.getOrElse("spark.shuffle.consolidateFiles", "false").toBoolean
+    conf.get("spark.shuffle.consolidateFiles", "false").toBoolean
 
-  private val bufferSize = conf.getOrElse("spark.shuffle.file.buffer.kb", "100").toInt * 1024
+  private val bufferSize = conf.get("spark.shuffle.file.buffer.kb", "100").toInt * 1024
 
   /**
    * Contains all the state related to a particular shuffle. This includes a pool of unused

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
----------------------------------------------------------------------
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 0ce8d9c..50dfdbd 100644
--- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
@@ -1,4 +1,4 @@
-/* 
+/*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
  * this work for additional information regarding copyright ownership.
@@ -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 = sc.conf.getOrElse("spark.ui.port", SparkUI.DEFAULT_PORT).toInt
+  val port = sc.conf.get("spark.ui.port", SparkUI.DEFAULT_PORT).toInt
   var boundPort: Option[Int] = None
   var server: Option[Server] = None
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala
----------------------------------------------------------------------
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 6b4602f..88f41be 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
@@ -66,7 +66,7 @@ private[spark] class EnvironmentUI(sc: SparkContext) {
       UIUtils.listingTable(propertyHeaders, propertyRow, otherProperties, fixedWidth = true)
 
     val classPathEntries = classPathProperty._2
-        .split(sc.conf.getOrElse("path.separator", ":"))
+        .split(sc.conf.get("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")}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
----------------------------------------------------------------------
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 315014d..b7b8725 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 = sc.conf.getOrElse("spark.ui.retained_stages", "1000").toInt
+  val RETAINED_STAGES = sc.conf.get("spark.ui.retained_stages", "1000").toInt
   val DEFAULT_POOL_NAME = "default"
 
   val stageIdToPool = new HashMap[Int, String]()

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
----------------------------------------------------------------------
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 58b26f7..362cea5 100644
--- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
+++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
@@ -41,19 +41,19 @@ private[spark] object AkkaUtils {
   def createActorSystem(name: String, host: String, port: Int, indestructible: Boolean = false,
     conf: SparkConf): (ActorSystem, Int) = {
 
-    val akkaThreads   = conf.getOrElse("spark.akka.threads", "4").toInt
-    val akkaBatchSize = conf.getOrElse("spark.akka.batchSize", "15").toInt
+    val akkaThreads   = conf.get("spark.akka.threads", "4").toInt
+    val akkaBatchSize = conf.get("spark.akka.batchSize", "15").toInt
 
-    val akkaTimeout = conf.getOrElse("spark.akka.timeout", "100").toInt
+    val akkaTimeout = conf.get("spark.akka.timeout", "100").toInt
 
-    val akkaFrameSize = conf.getOrElse("spark.akka.frameSize", "10").toInt
+    val akkaFrameSize = conf.get("spark.akka.frameSize", "10").toInt
     val lifecycleEvents =
-      if (conf.getOrElse("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off"
+      if (conf.get("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off"
 
-    val akkaHeartBeatPauses = conf.getOrElse("spark.akka.heartbeat.pauses", "600").toInt
+    val akkaHeartBeatPauses = conf.get("spark.akka.heartbeat.pauses", "600").toInt
     val akkaFailureDetector =
-      conf.getOrElse("spark.akka.failure-detector.threshold", "300.0").toDouble
-    val akkaHeartBeatInterval = conf.getOrElse("spark.akka.heartbeat.interval", "1000").toInt
+      conf.get("spark.akka.failure-detector.threshold", "300.0").toDouble
+    val akkaHeartBeatInterval = conf.get("spark.akka.heartbeat.interval", "1000").toInt
 
     val akkaConf = ConfigFactory.parseString(
       s"""
@@ -89,6 +89,6 @@ private[spark] object AkkaUtils {
 
   /** Returns the default Spark timeout to use for Akka ask operations. */
   def askTimeout(conf: SparkConf): FiniteDuration = {
-    Duration.create(conf.getOrElse("spark.akka.askTimeout", "30").toLong, "seconds")
+    Duration.create(conf.get("spark.akka.askTimeout", "30").toLong, "seconds")
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
----------------------------------------------------------------------
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 9ea7fc2..aa7f52c 100644
--- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
+++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
@@ -74,12 +74,12 @@ object MetadataCleanerType extends Enumeration {
 // initialization of StreamingContext. It's okay for users trying to configure stuff themselves.
 object MetadataCleaner {
   def getDelaySeconds(conf: SparkConf) = {
-    conf.getOrElse("spark.cleaner.ttl", "3500").toInt
+    conf.get("spark.cleaner.ttl", "3500").toInt
   }
 
   def getDelaySeconds(conf: SparkConf, cleanerType: MetadataCleanerType.MetadataCleanerType): Int =
   {
-    conf.getOrElse(MetadataCleanerType.systemProperty(cleanerType), getDelaySeconds(conf).toString)
+    conf.get(MetadataCleanerType.systemProperty(cleanerType), getDelaySeconds(conf).toString)
       .toInt
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/main/scala/org/apache/spark/util/Utils.scala
----------------------------------------------------------------------
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 ca3320b..5f12531 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -311,7 +311,7 @@ private[spark] object Utils extends Logging {
    * multiple paths.
    */
   def getLocalDir(conf: SparkConf): String = {
-    conf.getOrElse("spark.local.dir",  System.getProperty("java.io.tmpdir")).split(',')(0)
+    conf.get("spark.local.dir",  System.getProperty("java.io.tmpdir")).split(',')(0)
   }
 
   /**
@@ -397,7 +397,7 @@ private[spark] object Utils extends Logging {
   }
 
   def localHostPort(conf: SparkConf): String = {
-    val retval = conf.getOrElse("spark.hostPort", null)
+    val retval = conf.get("spark.hostPort", null)
     if (retval == null) {
       logErrorWithStack("spark.hostPort not set but invoking localHostPort")
       return localHostName()

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
index 77c7b82..ef5936d 100644
--- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
@@ -74,7 +74,7 @@ class SparkConfSuite extends FunSuite with LocalSparkContext {
     assert(!conf.contains("k4"), "conf contained k4")
     assert(conf.get("k1") === "v4")
     intercept[Exception] { conf.get("k4") }
-    assert(conf.getOrElse("k4", "not found") === "not found")
+    assert(conf.get("k4", "not found") === "not found")
     assert(conf.getOption("k1") === Some("v4"))
     assert(conf.getOption("k4") === None)
   }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
index 5d33e66..1eec672 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
@@ -83,7 +83,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging {
 
   private val conf = new SparkConf
 
-  val LOCALITY_WAIT = conf.getOrElse("spark.locality.wait", "3000").toLong
+  val LOCALITY_WAIT = conf.get("spark.locality.wait", "3000").toLong
   val MAX_TASK_FAILURES = 4
 
   test("TaskSet with no preferences") {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala
----------------------------------------------------------------------
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 f940448..af4b31d 100644
--- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala
@@ -27,8 +27,8 @@ 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
+class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach {
+  private val testConf = new SparkConf(false)
   val rootDir0 = Files.createTempDir()
   rootDir0.deleteOnExit()
   val rootDir1 = Files.createTempDir()
@@ -38,9 +38,7 @@ 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 = Try(testConf.get(consolidateProp)).toOption
-  testConf.set(consolidateProp, "true")
+  testConf.set("spark.shuffle.consolidateFiles", "true")
 
   val shuffleBlockManager = new ShuffleBlockManager(null) {
     override def conf = testConf.clone
@@ -50,10 +48,6 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with Before
 
   var diskBlockManager: DiskBlockManager = _
 
-  override def afterAll() {
-    oldConsolidate.map(c => System.setProperty(consolidateProp, c))
-  }
-
   override def beforeEach() {
     diskBlockManager = new DiskBlockManager(shuffleBlockManager, rootDirs)
     shuffleBlockManager.idToSegmentMap.clear()

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/python/pyspark/conf.py
----------------------------------------------------------------------
diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py
index 9dcdcfa..c111e2e 100644
--- a/python/pyspark/conf.py
+++ b/python/pyspark/conf.py
@@ -93,7 +93,7 @@ class SparkConf(object):
 
     def set(self, key, value):
         """Set a configuration property."""
-        self._jconf.set(key, value)
+        self._jconf.set(key, unicode(value))
         return self
 
     def setMaster(self, value):
@@ -132,13 +132,9 @@ class SparkConf(object):
             self._jconf.set(k, v)
         return self
 
-    def get(self, key):
-        """Get the configured value for some key, if set."""
-        return self._jconf.get(key)
-
-    def getOrElse(self, key, defaultValue):
-        """Get the value for some key, or return a default otherwise."""
-        return self._jconf.getOrElse(key, defaultValue)
+    def get(self, key, defaultValue=None):
+        """Get the configured value for some key, or return a default otherwise."""
+        return self._jconf.get(key, defaultValue)
 
     def getAll(self):
         """Get all values as a list of key-value pairs."""

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala
----------------------------------------------------------------------
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 a993083..59fdb0b 100644
--- a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala
@@ -89,7 +89,7 @@ import org.apache.spark.util.Utils
       /** Local directory to save .class files too */
       val outputDir = {
         val tmp = System.getProperty("java.io.tmpdir")
-        val rootDir = new SparkConf().getOrElse("spark.repl.classdir",  tmp)
+        val rootDir = new SparkConf().get("spark.repl.classdir",  tmp)
         Utils.createTempDir(rootDir)
       }
       if (SPARK_DEBUG_REPL) {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala
----------------------------------------------------------------------
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 a230845..27d474c 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
@@ -174,8 +174,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 = env.conf.getOrElse("spark.driver.host", "localhost")
-    val port = env.conf.getOrElse("spark.driver.port", "7077").toInt
+    val ip = env.conf.get("spark.driver.host", "localhost")
+    val port = env.conf.get("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
@@ -212,7 +212,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 = env.conf.getOrElse("spark.streaming.blockInterval", "200").toLong
+    val blockInterval = env.conf.get("spark.streaming.blockInterval", "200").toLong
     val blockIntervalTimer = new RecurringTimer(clock, blockInterval, updateCurrentBuffer)
     val blockStorageLevel = storageLevel
     val blocksForPushing = new ArrayBlockingQueue[Block](1000)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
index 844180c..5f8be93 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
@@ -46,7 +46,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging {
     }
   }))
   val clock = {
-    val clockClass = ssc.sc.conf.getOrElse(
+    val clockClass = ssc.sc.conf.get(
       "spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock")
     Class.forName(clockClass).newInstance().asInstanceOf[Clock]
   }
@@ -104,7 +104,7 @@ class JobGenerator(jobScheduler: JobScheduler) 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 = ssc.sc.conf.getOrElse("spark.streaming.manualClock.jump", "0").toLong
+      val jumpTime = ssc.sc.conf.get("spark.streaming.manualClock.jump", "0").toLong
       clock.asInstanceOf[ManualClock].setTime(lastTime + jumpTime)
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2c68642/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala
index 651cdaa..9304fc1 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala
@@ -31,7 +31,7 @@ private[streaming]
 class JobScheduler(val ssc: StreamingContext) extends Logging {
 
   val jobSets = new ConcurrentHashMap[Time, JobSet]
-  val numConcurrentJobs = ssc.conf.getOrElse("spark.streaming.concurrentJobs", "1").toInt
+  val numConcurrentJobs = ssc.conf.get("spark.streaming.concurrentJobs", "1").toInt
   val executor = Executors.newFixedThreadPool(numConcurrentJobs)
   val generator = new JobGenerator(this)
   val listenerBus = new StreamingListenerBus()


[27/33] git commit: Fix two compile errors introduced in merge

Posted by pw...@apache.org.
Fix two compile errors introduced in merge


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/42bcfb2b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/42bcfb2b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/42bcfb2b

Branch: refs/heads/master
Commit: 42bcfb2bb2b532dc12e13d3cfc1b4556bbb2c43c
Parents: ba9338f
Author: Matei Zaharia <ma...@databricks.com>
Authored: Tue Dec 31 18:26:23 2013 -0500
Committer: Matei Zaharia <ma...@databricks.com>
Committed: Tue Dec 31 18:26:23 2013 -0500

----------------------------------------------------------------------
 .../src/main/scala/org/apache/spark/streaming/Checkpoint.scala     | 2 +-
 .../scala/org/apache/spark/streaming/scheduler/JobGenerator.scala  | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/42bcfb2b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
----------------------------------------------------------------------
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 af44327..ca0115f 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
@@ -64,7 +64,7 @@ class CheckpointWriter(conf: SparkConf, checkpointDir: String, hadoopConf: Confi
   val file = new Path(checkpointDir, "graph")
   val MAX_ATTEMPTS = 3
   val executor = Executors.newFixedThreadPool(1)
-  val compressionCodec = CompressionCodec.createCodec()
+  val compressionCodec = CompressionCodec.createCodec(conf)
   // The file to which we actually write - and then "move" to file
   val writeFile = new Path(file.getParent, file.getName + ".next")
   // The file to which existing checkpoint is backed up (i.e. "moved")

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/42bcfb2b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
index e448211..ab60a81 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
@@ -50,7 +50,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging {
   val clock = {
     val clockClass = ssc.sc.conf.getOrElse(
       "spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock")
-    val clock = Class.forName(clockClass).newInstance().asInstanceOf[Clock]
+    Class.forName(clockClass).newInstance().asInstanceOf[Clock]
   }
   val timer = new RecurringTimer(clock, ssc.graph.batchDuration.milliseconds,
     longTime => eventProcessorActor ! GenerateJobs(new Time(longTime)))


[11/33] git commit: Add a StreamingContext constructor that takes a conf object

Posted by pw...@apache.org.
Add a StreamingContext constructor that takes a conf object


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/0900d5c7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/0900d5c7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/0900d5c7

Branch: refs/heads/master
Commit: 0900d5c72aaf6670bb9fcce8e0c0cfa976adcdf7
Parents: a8f3163
Author: Matei Zaharia <ma...@databricks.com>
Authored: Sat Dec 28 21:38:07 2013 -0500
Committer: Matei Zaharia <ma...@databricks.com>
Committed: Sat Dec 28 21:38:07 2013 -0500

----------------------------------------------------------------------
 .../spark/streaming/StreamingContext.scala      | 21 +++++++++++++++++++-
 1 file changed, 20 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0900d5c7/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
----------------------------------------------------------------------
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 079841a..9d2033f 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
@@ -71,6 +71,15 @@ class StreamingContext private (
   }
 
   /**
+   * Create a StreamingContext by providing the configuration necessary for a new SparkContext.
+   * @param conf A standard Spark application configuration
+   * @param batchDuration The time interval at which streaming data will be divided into batches
+   */
+  def this(conf: SparkConf, batchDuration: Duration) = {
+    this(StreamingContext.createNewSparkContext(conf), null, batchDuration)
+  }
+
+  /**
    * Create a StreamingContext by providing the details necessary for creating a new SparkContext.
    * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
    * @param appName A name for your job, to display on the cluster web UI
@@ -577,6 +586,16 @@ object StreamingContext {
     new PairDStreamFunctions[K, V](stream)
   }
 
+  protected[streaming] def createNewSparkContext(conf: SparkConf): SparkContext = {
+    // Set the default cleaner delay to an hour if not already set.
+    // This should be sufficient for even 1 second batch intervals.
+    val sc = new SparkContext(conf)
+    if (MetadataCleaner.getDelaySeconds(sc.conf) < 0) {
+      MetadataCleaner.setDelaySeconds(sc.conf, 3600)
+    }
+    sc
+  }
+
   protected[streaming] def createNewSparkContext(
       master: String,
       appName: String,
@@ -586,7 +605,7 @@ object StreamingContext {
   {
     val sc = new SparkContext(master, appName, sparkHome, jars, environment)
     // Set the default cleaner delay to an hour if not already set.
-    // This should be sufficient for even 1 second interval.
+    // This should be sufficient for even 1 second batch intervals.
     if (MetadataCleaner.getDelaySeconds(sc.conf) < 0) {
       MetadataCleaner.setDelaySeconds(sc.conf, 3600)
     }


[07/33] git commit: Check for SPARK_YARN_MODE through a system property too since it can sometimes be set that way (undoes a change in previous commit)

Posted by pw...@apache.org.
Check for SPARK_YARN_MODE through a system property too since it can
sometimes be set that way (undoes a change in previous commit)


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/a16c52ed
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/a16c52ed
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/a16c52ed

Branch: refs/heads/master
Commit: a16c52ed1bf5b416c9ec8c13dbe84ee203032b1b
Parents: 642029e
Author: Matei Zaharia <ma...@databricks.com>
Authored: Sat Dec 28 17:24:21 2013 -0500
Committer: Matei Zaharia <ma...@databricks.com>
Committed: Sat Dec 28 17:24:21 2013 -0500

----------------------------------------------------------------------
 core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/a16c52ed/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
----------------------------------------------------------------------
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 4f402c1..27dc42b 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
@@ -69,7 +69,8 @@ class SparkHadoopUtil {
 object SparkHadoopUtil {
 
   private val hadoop = {
-    val yarnMode = java.lang.Boolean.valueOf(System.getenv("SPARK_YARN_MODE"))
+    val yarnMode = java.lang.Boolean.valueOf(
+        System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE")))
     if (yarnMode) {
       try {
         Class.forName("org.apache.spark.deploy.yarn.YarnSparkHadoopUtil")


[18/33] git commit: Fix a few settings that were being read as system properties after merge

Posted by pw...@apache.org.
Fix a few settings that were being read as system properties after merge


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/0bd1900c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/0bd1900c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/0bd1900c

Branch: refs/heads/master
Commit: 0bd1900cbce5946999c38293852d8ccd4f838930
Parents: b4ceed4
Author: Matei Zaharia <ma...@databricks.com>
Authored: Sun Dec 29 15:38:46 2013 -0500
Committer: Matei Zaharia <ma...@databricks.com>
Committed: Sun Dec 29 15:38:46 2013 -0500

----------------------------------------------------------------------
 .../spark/scheduler/TaskSchedulerImpl.scala       |  4 +++-
 .../apache/spark/scheduler/TaskSetManager.scala   | 18 ++++++++++--------
 .../spark/streaming/scheduler/JobScheduler.scala  |  4 ++--
 3 files changed, 15 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0bd1900c/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
index 56a038d..bffd990 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
@@ -47,10 +47,12 @@ import org.apache.spark.scheduler.SchedulingMode.SchedulingMode
  */
 private[spark] class TaskSchedulerImpl(
     val sc: SparkContext,
-    val maxTaskFailures: Int = System.getProperty("spark.task.maxFailures", "4").toInt,
+    val maxTaskFailures: Int,
     isLocal: Boolean = false)
   extends TaskScheduler with Logging
 {
+  def this(sc: SparkContext) = this(sc, sc.conf.getOrElse("spark.task.maxFailures", "4").toInt)
+
   val conf = sc.conf
 
   // How often to check for speculative tasks

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0bd1900c/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
index 9b95e41..d752e6f 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
@@ -54,12 +54,14 @@ private[spark] class TaskSetManager(
     clock: Clock = SystemClock)
   extends Schedulable 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
 
   // 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
@@ -118,7 +120,7 @@ private[spark] class TaskSetManager(
 
   // 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
@@ -682,14 +684,14 @@ private[spark] class TaskSetManager(
   }
 
   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
     }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0bd1900c/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala
index 9511ccf..7fd8d41 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala
@@ -25,7 +25,7 @@ import org.apache.spark.streaming._
 
 /**
  * This class schedules jobs to be run on Spark. It uses the JobGenerator to generate
- * the jobs and runs them using a thread pool. Number of threads 
+ * the jobs and runs them using a thread pool. Number of threads
  */
 private[streaming]
 class JobScheduler(val ssc: StreamingContext) extends Logging {
@@ -33,7 +33,7 @@ class JobScheduler(val ssc: StreamingContext) extends Logging {
   initLogging()
 
   val jobSets = new ConcurrentHashMap[Time, JobSet]
-  val numConcurrentJobs = System.getProperty("spark.streaming.concurrentJobs", "1").toInt
+  val numConcurrentJobs = ssc.conf.getOrElse("spark.streaming.concurrentJobs", "1").toInt
   val executor = Executors.newFixedThreadPool(numConcurrentJobs)
   val generator = new JobGenerator(this)
   val listenerBus = new StreamingListenerBus()


[24/33] Updated docs for SparkConf and handled review comments

Posted by pw...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0fa58097/docs/css/bootstrap.min.css
----------------------------------------------------------------------
diff --git a/docs/css/bootstrap.min.css b/docs/css/bootstrap.min.css
index ccb529e..119bb5d 100644
--- a/docs/css/bootstrap.min.css
+++ b/docs/css/bootstrap.min.css
@@ -6,4 +6,4 @@
  * http://www.apache.org/licenses/LICENSE-2.0
  *
  * Designed and built with all the love in the world @twitter by @mdo and @fat.
- */article,aside,details,figcaption,figure,footer,header,hgroup,nav,section{display:block}audio,canvas,video{display:inline-block;*display:inline;*zoom:1}audio:not([controls]){display:none}html{font-size:100%;-webkit-text-size-adjust:100%;-ms-text-size-adjust:100%}a:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}a:hover,a:active{outline:0}sub,sup{position:relative;font-size:75%;line-height:0;vertical-align:baseline}sup{top:-0.5em}sub{bottom:-0.25em}img{height:auto;max-width:100%;vertical-align:middle;border:0;-ms-interpolation-mode:bicubic}#map_canvas img{max-width:none}button,input,select,textarea{margin:0;font-size:100%;vertical-align:middle}button,input{*overflow:visible;line-height:normal}button::-moz-focus-inner,input::-moz-focus-inner{padding:0;border:0}button,input[type="button"],input[type="reset"],input[type="submit"]{cursor:pointer;-webkit-appearance:button}input[type="search"]{-webkit-box-sizing:content-box;-moz-box-sizing:con
 tent-box;box-sizing:content-box;-webkit-appearance:textfield}input[type="search"]::-webkit-search-decoration,input[type="search"]::-webkit-search-cancel-button{-webkit-appearance:none}textarea{overflow:auto;vertical-align:top}.clearfix{*zoom:1}.clearfix:before,.clearfix:after{display:table;line-height:0;content:""}.clearfix:after{clear:both}.hide-text{font:0/0 a;color:transparent;text-shadow:none;background-color:transparent;border:0}.input-block-level{display:block;width:100%;min-height:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}body{margin:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:14px;line-height:20px;color:#333;background-color:#fff}a{color:#08c;text-decoration:none}a:hover{color:#005580;text-decoration:underline}.img-rounded{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.img-polaroid{padding:4px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-box-shadow:0
  1px 3px rgba(0,0,0,0.1);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.1);box-shadow:0 1px 3px rgba(0,0,0,0.1)}.img-circle{-webkit-border-radius:500px;-moz-border-radius:500px;border-radius:500px}.row{margin-left:-20px;*zoom:1}.row:before,.row:after{display:table;line-height:0;content:""}.row:after{clear:both}[class*="span"]{float:left;margin-left:20px}.container,.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.span12{width:940px}.span11{width:860px}.span10{width:780px}.span9{width:700px}.span8{width:620px}.span7{width:540px}.span6{width:460px}.span5{width:380px}.span4{width:300px}.span3{width:220px}.span2{width:140px}.span1{width:60px}.offset12{margin-left:980px}.offset11{margin-left:900px}.offset10{margin-left:820px}.offset9{margin-left:740px}.offset8{margin-left:660px}.offset7{margin-left:580px}.offset6{margin-left:500px}.offset5{margin-left:420px}.offset4{margin-left:340px}.offset3{margin-left:260px}.offset2{margin-left:180px}.offs
 et1{margin-left:100px}.row-fluid{width:100%;*zoom:1}.row-fluid:before,.row-fluid:after{display:table;line-height:0;content:""}.row-fluid:after{clear:both}.row-fluid [class*="span"]{display:block;float:left;width:100%;min-height:30px;margin-left:2.127659574468085%;*margin-left:2.074468085106383%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.row-fluid [class*="span"]:first-child{margin-left:0}.row-fluid .span12{width:100%;*width:99.94680851063829%}.row-fluid .span11{width:91.48936170212765%;*width:91.43617021276594%}.row-fluid .span10{width:82.97872340425532%;*width:82.92553191489361%}.row-fluid .span9{width:74.46808510638297%;*width:74.41489361702126%}.row-fluid .span8{width:65.95744680851064%;*width:65.90425531914893%}.row-fluid .span7{width:57.44680851063829%;*width:57.39361702127659%}.row-fluid .span6{width:48.93617021276595%;*width:48.88297872340425%}.row-fluid .span5{width:40.42553191489362%;*width:40.37234042553192%}.row-fluid .span4{width:31.9
 14893617021278%;*width:31.861702127659576%}.row-fluid .span3{width:23.404255319148934%;*width:23.351063829787233%}.row-fluid .span2{width:14.893617021276595%;*width:14.840425531914894%}.row-fluid .span1{width:6.382978723404255%;*width:6.329787234042553%}.row-fluid .offset12{margin-left:104.25531914893617%;*margin-left:104.14893617021275%}.row-fluid .offset12:first-child{margin-left:102.12765957446808%;*margin-left:102.02127659574467%}.row-fluid .offset11{margin-left:95.74468085106382%;*margin-left:95.6382978723404%}.row-fluid .offset11:first-child{margin-left:93.61702127659574%;*margin-left:93.51063829787232%}.row-fluid .offset10{margin-left:87.23404255319149%;*margin-left:87.12765957446807%}.row-fluid .offset10:first-child{margin-left:85.1063829787234%;*margin-left:84.99999999999999%}.row-fluid .offset9{margin-left:78.72340425531914%;*margin-left:78.61702127659572%}.row-fluid .offset9:first-child{margin-left:76.59574468085106%;*margin-left:76.48936170212764%}.row-fluid .offset8{mar
 gin-left:70.2127659574468%;*margin-left:70.10638297872339%}.row-fluid .offset8:first-child{margin-left:68.08510638297872%;*margin-left:67.9787234042553%}.row-fluid .offset7{margin-left:61.70212765957446%;*margin-left:61.59574468085106%}.row-fluid .offset7:first-child{margin-left:59.574468085106375%;*margin-left:59.46808510638297%}.row-fluid .offset6{margin-left:53.191489361702125%;*margin-left:53.085106382978715%}.row-fluid .offset6:first-child{margin-left:51.063829787234035%;*margin-left:50.95744680851063%}.row-fluid .offset5{margin-left:44.68085106382979%;*margin-left:44.57446808510638%}.row-fluid .offset5:first-child{margin-left:42.5531914893617%;*margin-left:42.4468085106383%}.row-fluid .offset4{margin-left:36.170212765957444%;*margin-left:36.06382978723405%}.row-fluid .offset4:first-child{margin-left:34.04255319148936%;*margin-left:33.93617021276596%}.row-fluid .offset3{margin-left:27.659574468085104%;*margin-left:27.5531914893617%}.row-fluid .offset3:first-child{margin-left:25
 .53191489361702%;*margin-left:25.425531914893618%}.row-fluid .offset2{margin-left:19.148936170212764%;*margin-left:19.04255319148936%}.row-fluid .offset2:first-child{margin-left:17.02127659574468%;*margin-left:16.914893617021278%}.row-fluid .offset1{margin-left:10.638297872340425%;*margin-left:10.53191489361702%}.row-fluid .offset1:first-child{margin-left:8.51063829787234%;*margin-left:8.404255319148938%}[class*="span"].hide,.row-fluid [class*="span"].hide{display:none}[class*="span"].pull-right,.row-fluid [class*="span"].pull-right{float:right}.container{margin-right:auto;margin-left:auto;*zoom:1}.container:before,.container:after{display:table;line-height:0;content:""}.container:after{clear:both}.container-fluid{padding-right:20px;padding-left:20px;*zoom:1}.container-fluid:before,.container-fluid:after{display:table;line-height:0;content:""}.container-fluid:after{clear:both}p{margin:0 0 10px}.lead{margin-bottom:20px;font-size:20px;font-weight:200;line-height:30px}small{font-size:8
 5%}strong{font-weight:bold}em{font-style:italic}cite{font-style:normal}.muted{color:#999}h1,h2,h3,h4,h5,h6{margin:10px 0;font-family:inherit;font-weight:bold;line-height:1;color:inherit;text-rendering:optimizelegibility}h1 small,h2 small,h3 small,h4 small,h5 small,h6 small{font-weight:normal;line-height:1;color:#999}h1{font-size:36px;line-height:40px}h2{font-size:30px;line-height:40px}h3{font-size:24px;line-height:40px}h4{font-size:18px;line-height:20px}h5{font-size:14px;line-height:20px}h6{font-size:12px;line-height:20px}h1 small{font-size:24px}h2 small{font-size:18px}h3 small{font-size:14px}h4 small{font-size:14px}.page-header{padding-bottom:9px;margin:20px 0 30px;border-bottom:1px solid #eee}ul,ol{padding:0;margin:0 0 10px 25px}ul ul,ul ol,ol ol,ol ul{margin-bottom:0}li{line-height:20px}ul.unstyled,ol.unstyled{margin-left:0;list-style:none}dl{margin-bottom:20px}dt,dd{line-height:20px}dt{font-weight:bold}dd{margin-left:10px}.dl-horizontal dt{float:left;width:120px;overflow:hidden;
 clear:left;text-align:right;text-overflow:ellipsis;white-space:nowrap}.dl-horizontal dd{margin-left:130px}hr{margin:20px 0;border:0;border-top:1px solid #eee;border-bottom:1px solid #fff}abbr[title]{cursor:help;border-bottom:1px dotted #999}abbr.initialism{font-size:90%;text-transform:uppercase}blockquote{padding:0 0 0 15px;margin:0 0 20px;border-left:5px solid #eee}blockquote p{margin-bottom:0;font-size:16px;font-weight:300;line-height:25px}blockquote small{display:block;line-height:20px;color:#999}blockquote small:before{content:'\2014 \00A0'}blockquote.pull-right{float:right;padding-right:15px;padding-left:0;border-right:5px solid #eee;border-left:0}blockquote.pull-right p,blockquote.pull-right small{text-align:right}blockquote.pull-right small:before{content:''}blockquote.pull-right small:after{content:'\00A0 \2014'}q:before,q:after,blockquote:before,blockquote:after{content:""}address{display:block;margin-bottom:20px;font-style:normal;line-height:20px}code,pre{padding:0 3px 2px
 ;font-family:Monaco,Menlo,Consolas,"Courier New",monospace;font-size:12px;color:#333;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}code{padding:2px 4px;color:#d14;background-color:#f7f7f9;border:1px solid #e1e1e8}pre{display:block;padding:9.5px;margin:0 0 10px;font-size:13px;line-height:20px;word-break:break-all;word-wrap:break-word;white-space:pre;white-space:pre-wrap;background-color:#f5f5f5;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.15);-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}pre.prettyprint{margin-bottom:20px}pre code{padding:0;color:inherit;background-color:transparent;border:0}.pre-scrollable{max-height:340px;overflow-y:scroll}form{margin:0 0 20px}fieldset{padding:0;margin:0;border:0}legend{display:block;width:100%;padding:0;margin-bottom:20px;font-size:21px;line-height:40px;color:#333;border:0;border-bottom:1px solid #e5e5e5}legend small{font-size:15px;color:#999}label,input,button,select,textarea{font-size:14px;font-we
 ight:normal;line-height:20px}input,button,select,textarea{font-family:"Helvetica Neue",Helvetica,Arial,sans-serif}label{display:block;margin-bottom:5px}select,textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{display:inline-block;height:20px;padding:4px 6px;margin-bottom:9px;font-size:14px;line-height:20px;color:#555;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}input,textarea{width:210px}textarea{height:auto}textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"]
 ,.uneditable-input{background-color:#fff;border:1px solid #ccc;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-webkit-transition:border linear .2s,box-shadow linear .2s;-moz-transition:border linear .2s,box-shadow linear .2s;-o-transition:border linear .2s,box-shadow linear .2s;transition:border linear .2s,box-shadow linear .2s}textarea:focus,input[type="text"]:focus,input[type="password"]:focus,input[type="datetime"]:focus,input[type="datetime-local"]:focus,input[type="date"]:focus,input[type="month"]:focus,input[type="time"]:focus,input[type="week"]:focus,input[type="number"]:focus,input[type="email"]:focus,input[type="url"]:focus,input[type="search"]:focus,input[type="tel"]:focus,input[type="color"]:focus,.uneditable-input:focus{border-color:rgba(82,168,236,0.8);outline:0;outline:thin dotted \9;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6);-moz-
 box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6)}input[type="radio"],input[type="checkbox"]{margin:4px 0 0;margin-top:1px \9;*margin-top:0;line-height:normal;cursor:pointer}input[type="file"],input[type="image"],input[type="submit"],input[type="reset"],input[type="button"],input[type="radio"],input[type="checkbox"]{width:auto}select,input[type="file"]{height:30px;*margin-top:4px;line-height:30px}select{width:220px;background-color:#fff;border:1px solid #bbb}select[multiple],select[size]{height:auto}select:focus,input[type="file"]:focus,input[type="radio"]:focus,input[type="checkbox"]:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.uneditable-input,.uneditable-textarea{color:#999;cursor:not-allowed;background-color:#fcfcfc;border-color:#ccc;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.025);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.025)
 ;box-shadow:inset 0 1px 2px rgba(0,0,0,0.025)}.uneditable-input{overflow:hidden;white-space:nowrap}.uneditable-textarea{width:auto;height:auto}input:-moz-placeholder,textarea:-moz-placeholder{color:#999}input:-ms-input-placeholder,textarea:-ms-input-placeholder{color:#999}input::-webkit-input-placeholder,textarea::-webkit-input-placeholder{color:#999}.radio,.checkbox{min-height:18px;padding-left:18px}.radio input[type="radio"],.checkbox input[type="checkbox"]{float:left;margin-left:-18px}.controls>.radio:first-child,.controls>.checkbox:first-child{padding-top:5px}.radio.inline,.checkbox.inline{display:inline-block;padding-top:5px;margin-bottom:0;vertical-align:middle}.radio.inline+.radio.inline,.checkbox.inline+.checkbox.inline{margin-left:10px}.input-mini{width:60px}.input-small{width:90px}.input-medium{width:150px}.input-large{width:210px}.input-xlarge{width:270px}.input-xxlarge{width:530px}input[class*="span"],select[class*="span"],textarea[class*="span"],.uneditable-input[class*
 ="span"],.row-fluid input[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"]{float:none;margin-left:0}.input-append input[class*="span"],.input-append .uneditable-input[class*="span"],.input-prepend input[class*="span"],.input-prepend .uneditable-input[class*="span"],.row-fluid input[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"],.row-fluid .input-prepend [class*="span"],.row-fluid .input-append [class*="span"]{display:inline-block}input,textarea,.uneditable-input{margin-left:0}.controls-row [class*="span"]+[class*="span"]{margin-left:20px}input.span12,textarea.span12,.uneditable-input.span12{width:926px}input.span11,textarea.span11,.uneditable-input.span11{width:846px}input.span10,textarea.span10,.uneditable-input.span10{width:766px}input.span9,textarea.span9,.uneditable-input.span9{width:686px}input.span8,textarea.span8,.unedit
 able-input.span8{width:606px}input.span7,textarea.span7,.uneditable-input.span7{width:526px}input.span6,textarea.span6,.uneditable-input.span6{width:446px}input.span5,textarea.span5,.uneditable-input.span5{width:366px}input.span4,textarea.span4,.uneditable-input.span4{width:286px}input.span3,textarea.span3,.uneditable-input.span3{width:206px}input.span2,textarea.span2,.uneditable-input.span2{width:126px}input.span1,textarea.span1,.uneditable-input.span1{width:46px}.controls-row{*zoom:1}.controls-row:before,.controls-row:after{display:table;line-height:0;content:""}.controls-row:after{clear:both}.controls-row [class*="span"]{float:left}input[disabled],select[disabled],textarea[disabled],input[readonly],select[readonly],textarea[readonly]{cursor:not-allowed;background-color:#eee}input[type="radio"][disabled],input[type="checkbox"][disabled],input[type="radio"][readonly],input[type="checkbox"][readonly]{background-color:transparent}.control-group.warning>label,.control-group.warning .h
 elp-block,.control-group.warning .help-inline{color:#c09853}.control-group.warning .checkbox,.control-group.warning .radio,.control-group.warning input,.control-group.warning select,.control-group.warning textarea{color:#c09853;border-color:#c09853;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.warning .checkbox:focus,.control-group.warning .radio:focus,.control-group.warning input:focus,.control-group.warning select:focus,.control-group.warning textarea:focus{border-color:#a47e3c;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e}.control-group.warning .input-prepend .add-on,.control-group.warning .input-append .add-on{color:#c09853;background-color:#fcf8e3;border-color:#c09853}.control-group.error>label,.control-group.error 
 .help-block,.control-group.error .help-inline{color:#b94a48}.control-group.error .checkbox,.control-group.error .radio,.control-group.error input,.control-group.error select,.control-group.error textarea{color:#b94a48;border-color:#b94a48;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.error .checkbox:focus,.control-group.error .radio:focus,.control-group.error input:focus,.control-group.error select:focus,.control-group.error textarea:focus{border-color:#953b39;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392}.control-group.error .input-prepend .add-on,.control-group.error .input-append .add-on{color:#b94a48;background-color:#f2dede;border-color:#b94a48}.control-group.success>label,.control-group.success .help-block,.control
 -group.success .help-inline{color:#468847}.control-group.success .checkbox,.control-group.success .radio,.control-group.success input,.control-group.success select,.control-group.success textarea{color:#468847;border-color:#468847;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.success .checkbox:focus,.control-group.success .radio:focus,.control-group.success input:focus,.control-group.success select:focus,.control-group.success textarea:focus{border-color:#356635;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b}.control-group.success .input-prepend .add-on,.control-group.success .input-append .add-on{color:#468847;background-color:#dff0d8;border-color:#468847}input:focus:required:invalid,textarea:focus:required:invalid,selec
 t:focus:required:invalid{color:#b94a48;border-color:#ee5f5b}input:focus:required:invalid:focus,textarea:focus:required:invalid:focus,select:focus:required:invalid:focus{border-color:#e9322d;-webkit-box-shadow:0 0 6px #f8b9b7;-moz-box-shadow:0 0 6px #f8b9b7;box-shadow:0 0 6px #f8b9b7}.form-actions{padding:19px 20px 20px;margin-top:20px;margin-bottom:20px;background-color:#f5f5f5;border-top:1px solid #e5e5e5;*zoom:1}.form-actions:before,.form-actions:after{display:table;line-height:0;content:""}.form-actions:after{clear:both}.help-block,.help-inline{color:#595959}.help-block{display:block;margin-bottom:10px}.help-inline{display:inline-block;*display:inline;padding-left:5px;vertical-align:middle;*zoom:1}.input-append,.input-prepend{margin-bottom:5px;font-size:0;white-space:nowrap}.input-append input,.input-prepend input,.input-append select,.input-prepend select,.input-append .uneditable-input,.input-prepend .uneditable-input{position:relative;margin-bottom:0;*margin-left:0;font-size:1
 4px;vertical-align:top;-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.input-append input:focus,.input-prepend input:focus,.input-append select:focus,.input-prepend select:focus,.input-append .uneditable-input:focus,.input-prepend .uneditable-input:focus{z-index:2}.input-append .add-on,.input-prepend .add-on{display:inline-block;width:auto;height:20px;min-width:16px;padding:4px 5px;font-size:14px;font-weight:normal;line-height:20px;text-align:center;text-shadow:0 1px 0 #fff;background-color:#eee;border:1px solid #ccc}.input-append .add-on,.input-prepend .add-on,.input-append .btn,.input-prepend .btn{margin-left:-1px;vertical-align:top;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-append .active,.input-prepend .active{background-color:#a9dba9;border-color:#46a546}.input-prepend .add-on,.input-prepend .btn{margin-right:-1px}.input-prepend .add-on:first-child,.input-prepend .btn:first-child{-webkit-border-radius:3px 0 0 
 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-append input,.input-append select,.input-append .uneditable-input{-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-append .add-on:last-child,.input-append .btn:last-child{-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.input-prepend.input-append input,.input-prepend.input-append select,.input-prepend.input-append .uneditable-input{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-prepend.input-append .add-on:first-child,.input-prepend.input-append .btn:first-child{margin-right:-1px;-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-prepend.input-append .add-on:last-child,.input-prepend.input-append .btn:last-child{margin-left:-1px;-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}input.search-query{padding-right:14px;pad
 ding-right:4px \9;padding-left:14px;padding-left:4px \9;margin-bottom:0;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.form-search .input-append .search-query,.form-search .input-prepend .search-query{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.form-search .input-append .search-query{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search .input-append .btn{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .search-query{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .btn{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search input,.form-inline input,.form-horizontal input,.form-search textarea,.form-inline textarea,.form-horizontal textarea,.form-search select,.form-inline select,.for
 m-horizontal select,.form-search .help-inline,.form-inline .help-inline,.form-horizontal .help-inline,.form-search .uneditable-input,.form-inline .uneditable-input,.form-horizontal .uneditable-input,.form-search .input-prepend,.form-inline .input-prepend,.form-horizontal .input-prepend,.form-search .input-append,.form-inline .input-append,.form-horizontal .input-append{display:inline-block;*display:inline;margin-bottom:0;vertical-align:middle;*zoom:1}.form-search .hide,.form-inline .hide,.form-horizontal .hide{display:none}.form-search label,.form-inline label,.form-search .btn-group,.form-inline .btn-group{display:inline-block}.form-search .input-append,.form-inline .input-append,.form-search .input-prepend,.form-inline .input-prepend{margin-bottom:0}.form-search .radio,.form-search .checkbox,.form-inline .radio,.form-inline .checkbox{padding-left:0;margin-bottom:0;vertical-align:middle}.form-search .radio input[type="radio"],.form-search .checkbox input[type="checkbox"],.form-inli
 ne .radio input[type="radio"],.form-inline .checkbox input[type="checkbox"]{float:left;margin-right:3px;margin-left:0}.control-group{margin-bottom:10px}legend+.control-group{margin-top:20px;-webkit-margin-top-collapse:separate}.form-horizontal .control-group{margin-bottom:20px;*zoom:1}.form-horizontal .control-group:before,.form-horizontal .control-group:after{display:table;line-height:0;content:""}.form-horizontal .control-group:after{clear:both}.form-horizontal .control-label{float:left;width:140px;padding-top:5px;text-align:right}.form-horizontal .controls{*display:inline-block;*padding-left:20px;margin-left:160px;*margin-left:0}.form-horizontal .controls:first-child{*padding-left:160px}.form-horizontal .help-block{margin-top:10px;margin-bottom:0}.form-horizontal .form-actions{padding-left:160px}table{max-width:100%;background-color:transparent;border-collapse:collapse;border-spacing:0}.table{width:100%;margin-bottom:20px}.table th,.table td{padding:8px;line-height:20px;text-alig
 n:left;vertical-align:top;border-top:1px solid #ddd}.table th{font-weight:bold}.table thead th{vertical-align:bottom}.table caption+thead tr:first-child th,.table caption+thead tr:first-child td,.table colgroup+thead tr:first-child th,.table colgroup+thead tr:first-child td,.table thead:first-child tr:first-child th,.table thead:first-child tr:first-child td{border-top:0}.table tbody+tbody{border-top:2px solid #ddd}.table-condensed th,.table-condensed td{padding:4px 5px}.table-bordered{border:1px solid #ddd;border-collapse:separate;*border-collapse:collapse;border-left:0;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.table-bordered th,.table-bordered td{border-left:1px solid #ddd}.table-bordered caption+thead tr:first-child th,.table-bordered caption+tbody tr:first-child th,.table-bordered caption+tbody tr:first-child td,.table-bordered colgroup+thead tr:first-child th,.table-bordered colgroup+tbody tr:first-child th,.table-bordered colgroup+tbody tr:first-child
  td,.table-bordered thead:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child td{border-top:0}.table-bordered thead:first-child tr:first-child th:first-child,.table-bordered tbody:first-child tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered thead:first-child tr:first-child th:last-child,.table-bordered tbody:first-child tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-radius-topright:4px}.table-bordered thead:last-child tr:last-child th:first-child,.table-bordered tbody:last-child tr:last-child td:first-child,.table-bordered tfoot:last-child tr:last-child td:first-child{-webkit-border-radius:0 0 0 4px;-moz-border-radius:0 0 0 4px;border-radius:0 0 0 4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomleft:4px}.table-b
 ordered thead:last-child tr:last-child th:last-child,.table-bordered tbody:last-child tr:last-child td:last-child,.table-bordered tfoot:last-child tr:last-child td:last-child{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-bottomright:4px}.table-bordered caption+thead tr:first-child th:first-child,.table-bordered caption+tbody tr:first-child td:first-child,.table-bordered colgroup+thead tr:first-child th:first-child,.table-bordered colgroup+tbody tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered caption+thead tr:first-child th:last-child,.table-bordered caption+tbody tr:first-child td:last-child,.table-bordered colgroup+thead tr:first-child th:last-child,.table-bordered colgroup+tbody tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-right-topleft:4px}.table-striped tbody tr:nth-child(odd) td,.table-s
 triped tbody tr:nth-child(odd) th{background-color:#f9f9f9}.table-hover tbody tr:hover td,.table-hover tbody tr:hover th{background-color:#f5f5f5}table [class*=span],.row-fluid table [class*=span]{display:table-cell;float:none;margin-left:0}table .span1{float:none;width:44px;margin-left:0}table .span2{float:none;width:124px;margin-left:0}table .span3{float:none;width:204px;margin-left:0}table .span4{float:none;width:284px;margin-left:0}table .span5{float:none;width:364px;margin-left:0}table .span6{float:none;width:444px;margin-left:0}table .span7{float:none;width:524px;margin-left:0}table .span8{float:none;width:604px;margin-left:0}table .span9{float:none;width:684px;margin-left:0}table .span10{float:none;width:764px;margin-left:0}table .span11{float:none;width:844px;margin-left:0}table .span12{float:none;width:924px;margin-left:0}table .span13{float:none;width:1004px;margin-left:0}table .span14{float:none;width:1084px;margin-left:0}table .span15{float:none;width:1164px;margin-left:
 0}table .span16{float:none;width:1244px;margin-left:0}table .span17{float:none;width:1324px;margin-left:0}table .span18{float:none;width:1404px;margin-left:0}table .span19{float:none;width:1484px;margin-left:0}table .span20{float:none;width:1564px;margin-left:0}table .span21{float:none;width:1644px;margin-left:0}table .span22{float:none;width:1724px;margin-left:0}table .span23{float:none;width:1804px;margin-left:0}table .span24{float:none;width:1884px;margin-left:0}.table tbody tr.success td{background-color:#dff0d8}.table tbody tr.error td{background-color:#f2dede}.table tbody tr.info td{background-color:#d9edf7}[class^="icon-"],[class*=" icon-"]{display:inline-block;width:14px;height:14px;margin-top:1px;*margin-right:.3em;line-height:14px;vertical-align:text-top;background-image:url("../img/glyphicons-halflings.png");background-position:14px 14px;background-repeat:no-repeat}.icon-white,.nav>.active>a>[class^="icon-"],.nav>.active>a>[class*=" icon-"],.dropdown-menu>li>a:hover>[clas
 s^="icon-"],.dropdown-menu>li>a:hover>[class*=" icon-"],.dropdown-menu>.active>a>[class^="icon-"],.dropdown-menu>.active>a>[class*=" icon-"]{background-image:url("../img/glyphicons-halflings-white.png")}.icon-glass{background-position:0 0}.icon-music{background-position:-24px 0}.icon-search{background-position:-48px 0}.icon-envelope{background-position:-72px 0}.icon-heart{background-position:-96px 0}.icon-star{background-position:-120px 0}.icon-star-empty{background-position:-144px 0}.icon-user{background-position:-168px 0}.icon-film{background-position:-192px 0}.icon-th-large{background-position:-216px 0}.icon-th{background-position:-240px 0}.icon-th-list{background-position:-264px 0}.icon-ok{background-position:-288px 0}.icon-remove{background-position:-312px 0}.icon-zoom-in{background-position:-336px 0}.icon-zoom-out{background-position:-360px 0}.icon-off{background-position:-384px 0}.icon-signal{background-position:-408px 0}.icon-cog{background-position:-432px 0}.icon-trash{back
 ground-position:-456px 0}.icon-home{background-position:0 -24px}.icon-file{background-position:-24px -24px}.icon-time{background-position:-48px -24px}.icon-road{background-position:-72px -24px}.icon-download-alt{background-position:-96px -24px}.icon-download{background-position:-120px -24px}.icon-upload{background-position:-144px -24px}.icon-inbox{background-position:-168px -24px}.icon-play-circle{background-position:-192px -24px}.icon-repeat{background-position:-216px -24px}.icon-refresh{background-position:-240px -24px}.icon-list-alt{background-position:-264px -24px}.icon-lock{background-position:-287px -24px}.icon-flag{background-position:-312px -24px}.icon-headphones{background-position:-336px -24px}.icon-volume-off{background-position:-360px -24px}.icon-volume-down{background-position:-384px -24px}.icon-volume-up{background-position:-408px -24px}.icon-qrcode{background-position:-432px -24px}.icon-barcode{background-position:-456px -24px}.icon-tag{background-position:0 -48px}.ic
 on-tags{background-position:-25px -48px}.icon-book{background-position:-48px -48px}.icon-bookmark{background-position:-72px -48px}.icon-print{background-position:-96px -48px}.icon-camera{background-position:-120px -48px}.icon-font{background-position:-144px -48px}.icon-bold{background-position:-167px -48px}.icon-italic{background-position:-192px -48px}.icon-text-height{background-position:-216px -48px}.icon-text-width{background-position:-240px -48px}.icon-align-left{background-position:-264px -48px}.icon-align-center{background-position:-288px -48px}.icon-align-right{background-position:-312px -48px}.icon-align-justify{background-position:-336px -48px}.icon-list{background-position:-360px -48px}.icon-indent-left{background-position:-384px -48px}.icon-indent-right{background-position:-408px -48px}.icon-facetime-video{background-position:-432px -48px}.icon-picture{background-position:-456px -48px}.icon-pencil{background-position:0 -72px}.icon-map-marker{background-position:-24px -72p
 x}.icon-adjust{background-position:-48px -72px}.icon-tint{background-position:-72px -72px}.icon-edit{background-position:-96px -72px}.icon-share{background-position:-120px -72px}.icon-check{background-position:-144px -72px}.icon-move{background-position:-168px -72px}.icon-step-backward{background-position:-192px -72px}.icon-fast-backward{background-position:-216px -72px}.icon-backward{background-position:-240px -72px}.icon-play{background-position:-264px -72px}.icon-pause{background-position:-288px -72px}.icon-stop{background-position:-312px -72px}.icon-forward{background-position:-336px -72px}.icon-fast-forward{background-position:-360px -72px}.icon-step-forward{background-position:-384px -72px}.icon-eject{background-position:-408px -72px}.icon-chevron-left{background-position:-432px -72px}.icon-chevron-right{background-position:-456px -72px}.icon-plus-sign{background-position:0 -96px}.icon-minus-sign{background-position:-24px -96px}.icon-remove-sign{background-position:-48px -96px
 }.icon-ok-sign{background-position:-72px -96px}.icon-question-sign{background-position:-96px -96px}.icon-info-sign{background-position:-120px -96px}.icon-screenshot{background-position:-144px -96px}.icon-remove-circle{background-position:-168px -96px}.icon-ok-circle{background-position:-192px -96px}.icon-ban-circle{background-position:-216px -96px}.icon-arrow-left{background-position:-240px -96px}.icon-arrow-right{background-position:-264px -96px}.icon-arrow-up{background-position:-289px -96px}.icon-arrow-down{background-position:-312px -96px}.icon-share-alt{background-position:-336px -96px}.icon-resize-full{background-position:-360px -96px}.icon-resize-small{background-position:-384px -96px}.icon-plus{background-position:-408px -96px}.icon-minus{background-position:-433px -96px}.icon-asterisk{background-position:-456px -96px}.icon-exclamation-sign{background-position:0 -120px}.icon-gift{background-position:-24px -120px}.icon-leaf{background-position:-48px -120px}.icon-fire{backgrou
 nd-position:-72px -120px}.icon-eye-open{background-position:-96px -120px}.icon-eye-close{background-position:-120px -120px}.icon-warning-sign{background-position:-144px -120px}.icon-plane{background-position:-168px -120px}.icon-calendar{background-position:-192px -120px}.icon-random{width:16px;background-position:-216px -120px}.icon-comment{background-position:-240px -120px}.icon-magnet{background-position:-264px -120px}.icon-chevron-up{background-position:-288px -120px}.icon-chevron-down{background-position:-313px -119px}.icon-retweet{background-position:-336px -120px}.icon-shopping-cart{background-position:-360px -120px}.icon-folder-close{background-position:-384px -120px}.icon-folder-open{width:16px;background-position:-408px -120px}.icon-resize-vertical{background-position:-432px -119px}.icon-resize-horizontal{background-position:-456px -118px}.icon-hdd{background-position:0 -144px}.icon-bullhorn{background-position:-24px -144px}.icon-bell{background-position:-48px -144px}.icon-
 certificate{background-position:-72px -144px}.icon-thumbs-up{background-position:-96px -144px}.icon-thumbs-down{background-position:-120px -144px}.icon-hand-right{background-position:-144px -144px}.icon-hand-left{background-position:-168px -144px}.icon-hand-up{background-position:-192px -144px}.icon-hand-down{background-position:-216px -144px}.icon-circle-arrow-right{background-position:-240px -144px}.icon-circle-arrow-left{background-position:-264px -144px}.icon-circle-arrow-up{background-position:-288px -144px}.icon-circle-arrow-down{background-position:-312px -144px}.icon-globe{background-position:-336px -144px}.icon-wrench{background-position:-360px -144px}.icon-tasks{background-position:-384px -144px}.icon-filter{background-position:-408px -144px}.icon-briefcase{background-position:-432px -144px}.icon-fullscreen{background-position:-456px -144px}.dropup,.dropdown{position:relative}.dropdown-toggle{*margin-bottom:-3px}.dropdown-toggle:active,.open .dropdown-toggle{outline:0}.car
 et{display:inline-block;width:0;height:0;vertical-align:top;border-top:4px solid #000;border-right:4px solid transparent;border-left:4px solid transparent;content:""}.dropdown .caret{margin-top:8px;margin-left:2px}.dropdown-menu{position:absolute;top:100%;left:0;z-index:1000;display:none;float:left;min-width:160px;padding:5px 0;margin:2px 0 0;list-style:none;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);*border-right-width:2px;*border-bottom-width:2px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0,0,0,0.2);-moz-box-shadow:0 5px 10px rgba(0,0,0,0.2);box-shadow:0 5px 10px rgba(0,0,0,0.2);-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box}.dropdown-menu.pull-right{right:0;left:auto}.dropdown-menu .divider{*width:100%;height:1px;margin:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.dropdown-menu a{display:blo
 ck;padding:3px 20px;clear:both;font-weight:normal;line-height:20px;color:#333;white-space:nowrap}.dropdown-menu li>a:hover,.dropdown-menu li>a:focus,.dropdown-submenu:hover>a{color:#fff;text-decoration:none;background-color:#0088cc;background-color:#0088cc;background-image:-moz-linear-gradient(top,#0088cc,#0087b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#0088cc),to(#0087b3));background-image:-webkit-linear-gradient(top,#0088cc,#0087b3);background-image:-o-linear-gradient(top,#0088cc,#0087b3);background-image:linear-gradient(to bottom,#0088cc,#0087b3);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0087b3',GradientType=0)}.dropdown-menu .active>a,.dropdown-menu .active>a:hover{color:#fff;text-decoration:none;background-color:#0088cc;background-color:#0081c2;background-image:linear-gradient(to bottom,#0088cc,#0087b3);background-image:-moz-linear-gradient(top,#0088cc,#0087b3);background-image:-webki
 t-gradient(linear,0 0,0 100%,from(#0088cc),to(#0087b3));background-image:-webkit-linear-gradient(top,#0088cc,#0087b3);background-image:-o-linear-gradient(top,#0088cc,#0087b3);background-repeat:repeat-x;outline:0;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0087b3',GradientType=0)}.dropdown-menu .disabled>a,.dropdown-menu .disabled>a:hover{color:#999}.dropdown-menu .disabled>a:hover{text-decoration:none;cursor:default;background-color:transparent}.open{*z-index:1000}.open>.dropdown-menu{display:block}.pull-right>.dropdown-menu{right:0;left:auto}.dropup .caret,.navbar-fixed-bottom .dropdown .caret{border-top:0;border-bottom:4px solid #000;content:"\2191"}.dropup .dropdown-menu,.navbar-fixed-bottom .dropdown .dropdown-menu{top:auto;bottom:100%;margin-bottom:1px}.dropdown-submenu{position:relative}.dropdown-submenu>.dropdown-menu{top:0;left:100%;margin-top:-6px;margin-left:-1px;-webkit-border-radius:0 6px 6px 6px;-moz-border-radius:0 6px 6p
 x 6px;border-radius:0 6px 6px 6px}.dropdown-submenu:hover .dropdown-menu{display:block}.dropdown-submenu>a:after{display:block;float:right;width:0;height:0;margin-top:5px;margin-right:-10px;border-color:transparent;border-left-color:#ccc;border-style:solid;border-width:5px 0 5px 5px;content:" "}.dropdown-submenu:hover>a:after{border-left-color:#fff}.dropdown .dropdown-menu .nav-header{padding-right:20px;padding-left:20px}.typeahead{margin-top:2px;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.well{min-height:20px;padding:19px;margin-bottom:20px;background-color:#f5f5f5;border:1px solid #e3e3e3;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);box-shadow:inset 0 1px 1px rgba(0,0,0,0.05)}.well blockquote{border-color:#ddd;border-color:rgba(0,0,0,0.15)}.well-large{padding:24px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.well-
 small{padding:9px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.fade{opacity:0;-webkit-transition:opacity .15s linear;-moz-transition:opacity .15s linear;-o-transition:opacity .15s linear;transition:opacity .15s linear}.fade.in{opacity:1}.collapse{position:relative;height:0;overflow:hidden;overflow:visible \9;-webkit-transition:height .35s ease;-moz-transition:height .35s ease;-o-transition:height .35s ease;transition:height .35s ease}.collapse.in{height:auto}.close{float:right;font-size:20px;font-weight:bold;line-height:20px;color:#000;text-shadow:0 1px 0 #fff;opacity:.2;filter:alpha(opacity=20)}.close:hover{color:#000;text-decoration:none;cursor:pointer;opacity:.4;filter:alpha(opacity=40)}button.close{padding:0;cursor:pointer;background:transparent;border:0;-webkit-appearance:none}.btn{display:inline-block;*display:inline;padding:4px 14px;margin-bottom:0;*margin-left:.3em;font-size:14px;line-height:20px;*line-height:20px;color:#333;text-align:center;text-shad
 ow:0 1px 1px rgba(255,255,255,0.75);vertical-align:middle;cursor:pointer;background-color:#f5f5f5;*background-color:#e6e6e6;background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#e6e6e6));background-image:-webkit-linear-gradient(top,#fff,#e6e6e6);background-image:-o-linear-gradient(top,#fff,#e6e6e6);background-image:linear-gradient(to bottom,#fff,#e6e6e6);background-image:-moz-linear-gradient(top,#fff,#e6e6e6);background-repeat:repeat-x;border:1px solid #bbb;*border:0;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);border-color:#e6e6e6 #e6e6e6 #bfbfbf;border-bottom-color:#a2a2a2;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffffff',endColorstr='#ffe6e6e6',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false);*zoom:1;-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,
 0.2),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05)}.btn:hover,.btn:active,.btn.active,.btn.disabled,.btn[disabled]{color:#333;background-color:#e6e6e6;*background-color:#d9d9d9}.btn:active,.btn.active{background-color:#ccc \9}.btn:first-child{*margin-left:0}.btn:hover{color:#333;text-decoration:none;background-color:#e6e6e6;*background-color:#d9d9d9;background-position:0 -15px;-webkit-transition:background-position .1s linear;-moz-transition:background-position .1s linear;-o-transition:background-position .1s linear;transition:background-position .1s linear}.btn:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.btn.active,.btn:active{background-color:#e6e6e6;background-color:#d9d9d9 \9;background-image:none;outline:0;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset
  0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05)}.btn.disabled,.btn[disabled]{cursor:default;background-color:#e6e6e6;background-image:none;opacity:.65;filter:alpha(opacity=65);-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-large{padding:9px 14px;font-size:16px;line-height:normal;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.btn-large [class^="icon-"]{margin-top:2px}.btn-small{padding:3px 9px;font-size:12px;line-height:18px}.btn-small [class^="icon-"]{margin-top:0}.btn-mini{padding:2px 6px;font-size:11px;line-height:16px}.btn-block{display:block;width:100%;padding-right:0;padding-left:0;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.btn-block+.btn-block{margin-top:5px}.btn-primary.active,.btn-warning.active,.btn-danger.active,.btn-success.active,.btn-info.active,.btn-inverse.active{color:rgba(255,255,255,0.75)}.btn{border-color:#c5c5c5;border-color:rgba(0,0,0,0.15) rgba(0,0,0,0.15) rgba(0,0,0,0.25)}.btn-
 primary{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#006dcc;*background-color:#04c;background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#04c));background-image:-webkit-linear-gradient(top,#08c,#04c);background-image:-o-linear-gradient(top,#08c,#04c);background-image:linear-gradient(to bottom,#08c,#04c);background-image:-moz-linear-gradient(top,#08c,#04c);background-repeat:repeat-x;border-color:#04c #04c #002a80;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0044cc',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-primary:hover,.btn-primary:active,.btn-primary.active,.btn-primary.disabled,.btn-primary[disabled]{color:#fff;background-color:#04c;*background-color:#003bb3}.btn-primary:active,.btn-primary.active{background-color:#039 \9}.btn-warning{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-col
 or:#faa732;*background-color:#f89406;background-image:-webkit-gradient(linear,0 0,0 100%,from(#fbb450),to(#f89406));background-image:-webkit-linear-gradient(top,#fbb450,#f89406);background-image:-o-linear-gradient(top,#fbb450,#f89406);background-image:linear-gradient(to bottom,#fbb450,#f89406);background-image:-moz-linear-gradient(top,#fbb450,#f89406);background-repeat:repeat-x;border-color:#f89406 #f89406 #ad6704;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fffbb450',endColorstr='#fff89406',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-warning:hover,.btn-warning:active,.btn-warning.active,.btn-warning.disabled,.btn-warning[disabled]{color:#fff;background-color:#f89406;*background-color:#df8505}.btn-warning:active,.btn-warning.active{background-color:#c67605 \9}.btn-danger{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#da4f49;*background-colo
 r:#bd362f;background-image:-webkit-gradient(linear,0 0,0 100%,from(#ee5f5b),to(#bd362f));background-image:-webkit-linear-gradient(top,#ee5f5b,#bd362f);background-image:-o-linear-gradient(top,#ee5f5b,#bd362f);background-image:linear-gradient(to bottom,#ee5f5b,#bd362f);background-image:-moz-linear-gradient(top,#ee5f5b,#bd362f);background-repeat:repeat-x;border-color:#bd362f #bd362f #802420;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffee5f5b',endColorstr='#ffbd362f',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-danger:hover,.btn-danger:active,.btn-danger.active,.btn-danger.disabled,.btn-danger[disabled]{color:#fff;background-color:#bd362f;*background-color:#a9302a}.btn-danger:active,.btn-danger.active{background-color:#942a25 \9}.btn-success{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#5bb75b;*background-color:#51a351;background-image:-webki
 t-gradient(linear,0 0,0 100%,from(#62c462),to(#51a351));background-image:-webkit-linear-gradient(top,#62c462,#51a351);background-image:-o-linear-gradient(top,#62c462,#51a351);background-image:linear-gradient(to bottom,#62c462,#51a351);background-image:-moz-linear-gradient(top,#62c462,#51a351);background-repeat:repeat-x;border-color:#51a351 #51a351 #387038;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff62c462',endColorstr='#ff51a351',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-success:hover,.btn-success:active,.btn-success.active,.btn-success.disabled,.btn-success[disabled]{color:#fff;background-color:#51a351;*background-color:#499249}.btn-success:active,.btn-success.active{background-color:#408140 \9}.btn-info{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#49afcd;*background-color:#2f96b4;background-image:-webkit-gradient(linear,0 0,0 100%,
 from(#5bc0de),to(#2f96b4));background-image:-webkit-linear-gradient(top,#5bc0de,#2f96b4);background-image:-o-linear-gradient(top,#5bc0de,#2f96b4);background-image:linear-gradient(to bottom,#5bc0de,#2f96b4);background-image:-moz-linear-gradient(top,#5bc0de,#2f96b4);background-repeat:repeat-x;border-color:#2f96b4 #2f96b4 #1f6377;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff2f96b4',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-info:hover,.btn-info:active,.btn-info.active,.btn-info.disabled,.btn-info[disabled]{color:#fff;background-color:#2f96b4;*background-color:#2a85a0}.btn-info:active,.btn-info.active{background-color:#24748c \9}.btn-inverse{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#363636;*background-color:#222;background-image:-webkit-gradient(linear,0 0,0 100%,from(#444),to(#222));background-image:-webkit-line
 ar-gradient(top,#444,#222);background-image:-o-linear-gradient(top,#444,#222);background-image:linear-gradient(to bottom,#444,#222);background-image:-moz-linear-gradient(top,#444,#222);background-repeat:repeat-x;border-color:#222 #222 #000;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff444444',endColorstr='#ff222222',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-inverse:hover,.btn-inverse:active,.btn-inverse.active,.btn-inverse.disabled,.btn-inverse[disabled]{color:#fff;background-color:#222;*background-color:#151515}.btn-inverse:active,.btn-inverse.active{background-color:#080808 \9}button.btn,input[type="submit"].btn{*padding-top:3px;*padding-bottom:3px}button.btn::-moz-focus-inner,input[type="submit"].btn::-moz-focus-inner{padding:0;border:0}button.btn.btn-large,input[type="submit"].btn.btn-large{*padding-top:7px;*padding-bottom:7px}button.btn.btn-small,input
 [type="submit"].btn.btn-small{*padding-top:3px;*padding-bottom:3px}button.btn.btn-mini,input[type="submit"].btn.btn-mini{*padding-top:1px;*padding-bottom:1px}.btn-link,.btn-link:active{background-color:transparent;background-image:none;-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-link{color:#08c;cursor:pointer;border-color:transparent;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-link:hover{color:#005580;text-decoration:underline;background-color:transparent}.btn-group{position:relative;*margin-left:.3em;font-size:0;white-space:nowrap}.btn-group:first-child{*margin-left:0}.btn-group+.btn-group{margin-left:5px}.btn-toolbar{margin-top:10px;margin-bottom:10px;font-size:0}.btn-toolbar .btn-group{display:inline-block;*display:inline;*zoom:1}.btn-toolbar .btn+.btn,.btn-toolbar .btn-group+.btn,.btn-toolbar .btn+.btn-group{margin-left:5px}.btn-group>.btn{position:relative;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group>.btn+.b
 tn{margin-left:-1px}.btn-group>.btn,.btn-group>.dropdown-menu{font-size:14px}.btn-group>.btn-mini{font-size:11px}.btn-group>.btn-small{font-size:12px}.btn-group>.btn-large{font-size:16px}.btn-group>.btn:first-child{margin-left:0;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-bottomleft:4px;-moz-border-radius-topleft:4px}.btn-group>.btn:last-child,.btn-group>.dropdown-toggle{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-bottomright:4px}.btn-group>.btn.large:first-child{margin-left:0;-webkit-border-bottom-left-radius:6px;border-bottom-left-radius:6px;-webkit-border-top-left-radius:6px;border-top-left-radius:6px;-moz-border-radius-bottomleft:6px;-moz-border-radius-topleft:6px}.btn-group>.btn.large:last-child,.btn-group>.large.dropdown-toggle{-webkit
 -border-top-right-radius:6px;border-top-right-radius:6px;-webkit-border-bottom-right-radius:6px;border-bottom-right-radius:6px;-moz-border-radius-topright:6px;-moz-border-radius-bottomright:6px}.btn-group>.btn:hover,.btn-group>.btn:focus,.btn-group>.btn:active,.btn-group>.btn.active{z-index:2}.btn-group .dropdown-toggle:active,.btn-group.open .dropdown-toggle{outline:0}.btn-group>.btn+.dropdown-toggle{*padding-top:5px;padding-right:8px;*padding-bottom:5px;padding-left:8px;-webkit-box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05)}.btn-group>.btn-mini+.dropdown-toggle{*padding-top:2px;padding-right:5px;*padding-bottom:2px;padding-left:5px}.btn-group>.btn-small+.dropdown-toggle{*padding-top:5px;*padd
 ing-bottom:4px}.btn-group>.btn-large+.dropdown-toggle{*padding-top:7px;padding-right:12px;*padding-bottom:7px;padding-left:12px}.btn-group.open .dropdown-toggle{background-image:none;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05)}.btn-group.open .btn.dropdown-toggle{background-color:#e6e6e6}.btn-group.open .btn-primary.dropdown-toggle{background-color:#04c}.btn-group.open .btn-warning.dropdown-toggle{background-color:#f89406}.btn-group.open .btn-danger.dropdown-toggle{background-color:#bd362f}.btn-group.open .btn-success.dropdown-toggle{background-color:#51a351}.btn-group.open .btn-info.dropdown-toggle{background-color:#2f96b4}.btn-group.open .btn-inverse.dropdown-toggle{background-color:#222}.btn .caret{margin-top:8px;margin-left:0}.btn-mini .caret,.btn-small .caret,.btn-large .caret{margin-top:6px}.btn-la
 rge .caret{border-top-width:5px;border-right-width:5px;border-left-width:5px}.dropup .btn-large .caret{border-top:0;border-bottom:5px solid #000}.btn-primary .caret,.btn-warning .caret,.btn-danger .caret,.btn-info .caret,.btn-success .caret,.btn-inverse .caret{border-top-color:#fff;border-bottom-color:#fff}.btn-group-vertical{display:inline-block;*display:inline;*zoom:1}.btn-group-vertical .btn{display:block;float:none;width:100%;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group-vertical .btn+.btn{margin-top:-1px;margin-left:0}.btn-group-vertical .btn:first-child{-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.btn-group-vertical .btn:last-child{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.btn-group-vertical .btn-large:first-child{-webkit-border-radius:6px 6px 0 0;-moz-border-radius:6px 6px 0 0;border-radius:6px 6px 0 0}.btn-group-vertical .btn-large:last-child{-webkit-bor
 der-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.alert{padding:8px 35px 8px 14px;margin-bottom:20px;color:#c09853;text-shadow:0 1px 0 rgba(255,255,255,0.5);background-color:#fcf8e3;border:1px solid #fbeed5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.alert h4{margin:0}.alert .close{position:relative;top:-2px;right:-21px;line-height:20px}.alert-success{color:#468847;background-color:#dff0d8;border-color:#d6e9c6}.alert-danger,.alert-error{color:#b94a48;background-color:#f2dede;border-color:#eed3d7}.alert-info{color:#3a87ad;background-color:#d9edf7;border-color:#bce8f1}.alert-block{padding-top:14px;padding-bottom:14px}.alert-block>p,.alert-block>ul{margin-bottom:0}.alert-block p+p{margin-top:5px}.nav{margin-bottom:20px;margin-left:0;list-style:none}.nav>li>a{display:block}.nav>li>a:hover{text-decoration:none;background-color:#eee}.nav>.pull-right{float:right}.nav-header{display:block;padding:3px 15px;font-size:11px;font-weight:bold
 ;line-height:20px;color:#999;text-shadow:0 1px 0 rgba(255,255,255,0.5);text-transform:uppercase}.nav li+.nav-header{margin-top:9px}.nav-list{padding-right:15px;padding-left:15px;margin-bottom:0}.nav-list>li>a,.nav-list .nav-header{margin-right:-15px;margin-left:-15px;text-shadow:0 1px 0 rgba(255,255,255,0.5)}.nav-list>li>a{padding:3px 15px}.nav-list>.active>a,.nav-list>.active>a:hover{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.2);background-color:#08c}.nav-list [class^="icon-"]{margin-right:2px}.nav-list .divider{*width:100%;height:1px;margin:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.nav-tabs,.nav-pills{*zoom:1}.nav-tabs:before,.nav-pills:before,.nav-tabs:after,.nav-pills:after{display:table;line-height:0;content:""}.nav-tabs:after,.nav-pills:after{clear:both}.nav-tabs>li,.nav-pills>li{float:left}.nav-tabs>li>a,.nav-pills>li>a{padding-right:12px;padding-left:12px;margin-right:2px;line-height:14px}.nav-tabs{border-bottom:1px s
 olid #ddd}.nav-tabs>li{margin-bottom:-1px}.nav-tabs>li>a{padding-top:8px;padding-bottom:8px;line-height:20px;border:1px solid transparent;-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.nav-tabs>li>a:hover{border-color:#eee #eee #ddd}.nav-tabs>.active>a,.nav-tabs>.active>a:hover{color:#555;cursor:default;background-color:#fff;border:1px solid #ddd;border-bottom-color:transparent}.nav-pills>li>a{padding-top:8px;padding-bottom:8px;margin-top:2px;margin-bottom:2px;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.nav-pills>.active>a,.nav-pills>.active>a:hover{color:#fff;background-color:#08c}.nav-stacked>li{float:none}.nav-stacked>li>a{margin-right:0}.nav-tabs.nav-stacked{border-bottom:0}.nav-tabs.nav-stacked>li>a{border:1px solid #ddd;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.nav-tabs.nav-stacked>li:first-child>a{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-top-left-radi
 us:4px;border-top-left-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-topleft:4px}.nav-tabs.nav-stacked>li:last-child>a{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomright:4px;-moz-border-radius-bottomleft:4px}.nav-tabs.nav-stacked>li>a:hover{z-index:2;border-color:#ddd}.nav-pills.nav-stacked>li>a{margin-bottom:3px}.nav-pills.nav-stacked>li:last-child>a{margin-bottom:1px}.nav-tabs .dropdown-menu{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.nav-pills .dropdown-menu{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.nav .dropdown-toggle .caret{margin-top:6px;border-top-color:#08c;border-bottom-color:#08c}.nav .dropdown-toggle:hover .caret{border-top-color:#005580;border-bottom-color:#005580}.nav-tabs .dropdown-toggle .caret{margin-top:8px}.nav .active .dropdown-toggle .caret{border-top-color:
 #fff;border-bottom-color:#fff}.nav-tabs .active .dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.nav>.dropdown.active>a:hover{cursor:pointer}.nav-tabs .open .dropdown-toggle,.nav-pills .open .dropdown-toggle,.nav>li.dropdown.open.active>a:hover{color:#fff;background-color:#999;border-color:#999}.nav li.dropdown.open .caret,.nav li.dropdown.open.active .caret,.nav li.dropdown.open a:hover .caret{border-top-color:#fff;border-bottom-color:#fff;opacity:1;filter:alpha(opacity=100)}.tabs-stacked .open>a:hover{border-color:#999}.tabbable{*zoom:1}.tabbable:before,.tabbable:after{display:table;line-height:0;content:""}.tabbable:after{clear:both}.tab-content{overflow:auto}.tabs-below>.nav-tabs,.tabs-right>.nav-tabs,.tabs-left>.nav-tabs{border-bottom:0}.tab-content>.tab-pane,.pill-content>.pill-pane{display:none}.tab-content>.active,.pill-content>.active{display:block}.tabs-below>.nav-tabs{border-top:1px solid #ddd}.tabs-below>.nav-tabs>li{margin-top:-1px;margin-bottom:0
 }.tabs-below>.nav-tabs>li>a{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.tabs-below>.nav-tabs>li>a:hover{border-top-color:#ddd;border-bottom-color:transparent}.tabs-below>.nav-tabs>.active>a,.tabs-below>.nav-tabs>.active>a:hover{border-color:transparent #ddd #ddd #ddd}.tabs-left>.nav-tabs>li,.tabs-right>.nav-tabs>li{float:none}.tabs-left>.nav-tabs>li>a,.tabs-right>.nav-tabs>li>a{min-width:74px;margin-right:0;margin-bottom:3px}.tabs-left>.nav-tabs{float:left;margin-right:19px;border-right:1px solid #ddd}.tabs-left>.nav-tabs>li>a{margin-right:-1px;-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px}.tabs-left>.nav-tabs>li>a:hover{border-color:#eee #ddd #eee #eee}.tabs-left>.nav-tabs .active>a,.tabs-left>.nav-tabs .active>a:hover{border-color:#ddd transparent #ddd #ddd;*border-right-color:#fff}.tabs-right>.nav-tabs{float:right;margin-left:19px;border-left:1px solid #ddd}.tabs-right>.nav-tabs>li>a{marg
 in-left:-1px;-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.tabs-right>.nav-tabs>li>a:hover{border-color:#eee #eee #eee #ddd}.tabs-right>.nav-tabs .active>a,.tabs-right>.nav-tabs .active>a:hover{border-color:#ddd #ddd #ddd transparent;*border-left-color:#fff}.nav>.disabled>a{color:#999}.nav>.disabled>a:hover{text-decoration:none;cursor:default;background-color:transparent}.navbar{*position:relative;*z-index:2;margin-bottom:20px;overflow:visible;color:#555}.navbar-inner{min-height:40px;padding-right:20px;padding-left:20px;background-color:#fafafa;background-image:-moz-linear-gradient(top,#fff,#d4e4f4);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#d4e4f4));background-image:-webkit-linear-gradient(top,#fff,#d4e4f4);background-image:-o-linear-gradient(top,#fff,#d4e4f4);background-image:linear-gradient(to bottom,#fff,#d4e4f4);background-repeat:repeat-x;border:1px solid #d4d4d4;-webkit-border-radius:4px;-moz-border-radius:4
 px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffffff',endColorstr='#ffd4e4f4',GradientType=0);-webkit-box-shadow:0 1px 4px rgba(0,0,0,0.065);-moz-box-shadow:0 1px 4px rgba(0,0,0,0.065);box-shadow:0 1px 4px rgba(0,0,0,0.065)}.navbar .container{width:auto}.nav-collapse.collapse{height:auto}.navbar .brand{display:block;float:left;padding:10px 20px 10px;margin-left:-20px;font-size:20px;font-weight:200;color:#555;text-shadow:0 1px 0 #fff}.navbar .brand:hover{text-decoration:none}.navbar-text{margin-bottom:0;line-height:40px}.navbar-link{color:#555}.navbar-link:hover{color:#333}.navbar .divider-vertical{height:40px;margin:0 9px;border-right:1px solid #fff;border-left:1px solid #f2f2f2}.navbar .btn,.navbar .btn-group{margin-top:6px}.navbar .btn-group .btn{margin:0}.navbar-form{margin-bottom:0;*zoom:1}.navbar-form:before,.navbar-form:after{display:table;line-height:0;content:""}.navbar-form:after{clear:both}.navbar-form input,.navbar-form select,.n
 avbar-form .radio,.navbar-form .checkbox{margin-top:5px}.navbar-form input,.navbar-form select,.navbar-form .btn{display:inline-block;margin-bottom:0}.navbar-form input[type="image"],.navbar-form input[type="checkbox"],.navbar-form input[type="radio"]{margin-top:3px}.navbar-form .input-append,.navbar-form .input-prepend{margin-top:6px;white-space:nowrap}.navbar-form .input-append input,.navbar-form .input-prepend input{margin-top:0}.navbar-search{position:relative;float:left;margin-top:5px;margin-bottom:0}.navbar-search .search-query{padding:4px 14px;margin-bottom:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:13px;font-weight:normal;line-height:1;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.navbar-static-top{position:static;width:100%;margin-bottom:0}.navbar-static-top .navbar-inner{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.navbar-fixed-top,.navbar-fixed-bottom{position:fixed;right:0;left:0;z-index:1030;margin-botto
 m:0}.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner,.navbar-static-top .navbar-inner{border:0}.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner{padding-right:0;padding-left:0;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.navbar-fixed-top{top:0}.navbar-fixed-top .navbar-inner,.navbar-static-top .navbar-inner{-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1)}.navbar-fixed-bottom{bottom:0}.navbar-fixed-bottom .navbar-inner{-webkit-box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1);box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1)}.navbar .nav{position:relati
 ve;left:0;display:block;float:left;margin:0 10px 0 0}.navbar .nav.pull-right{float:right}.navbar .nav>li{float:left}.navbar .nav>li>a{float:none;padding:10px 15px 10px;color:#555;text-decoration:none;text-shadow:0 1px 0 #fff}.navbar .nav .dropdown-toggle .caret{margin-top:8px}.navbar .nav>li>a:focus,.navbar .nav>li>a:hover{color:#333;text-decoration:none;background-color:transparent}.navbar .nav>.active>a,.navbar .nav>.active>a:hover,.navbar .nav>.active>a:focus{color:#555;text-decoration:none;background-color:#e5e5e5;-webkit-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);-moz-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);box-shadow:inset 0 3px 8px rgba(0,0,0,0.125)}.navbar .btn-navbar{display:none;float:right;padding:7px 10px;margin-right:5px;margin-left:5px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#ededed;*background-color:#e5e5e5;background-image:-webkit-gradient(linear,0 0,0 100%,from(#f2f2f2),to(#e5e5e5));background-image:-webkit-linear-gradient(top,#f2f2f
 2,#e5e5e5);background-image:-o-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:linear-gradient(to bottom,#f2f2f2,#e5e5e5);background-image:-moz-linear-gradient(top,#f2f2f2,#e5e5e5);background-repeat:repeat-x;border-color:#e5e5e5 #e5e5e5 #bfbfbf;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fff2f2f2',endColorstr='#ffe5e5e5',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false);-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075);box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075)}.navbar .btn-navbar:hover,.navbar .btn-navbar:active,.navbar .btn-navbar.active,.navbar .btn-navbar.disabled,.navbar .btn-navbar[disabled]{color:#fff;background-color:#e5e5e5;*background-color:#d9d9d9}.navbar .btn-navbar:active,.navbar .btn-navbar.activ
 e{background-color:#ccc \9}.navbar .btn-navbar .icon-bar{display:block;width:18px;height:2px;background-color:#f5f5f5;-webkit-border-radius:1px;-moz-border-radius:1px;border-radius:1px;-webkit-box-shadow:0 1px 0 rgba(0,0,0,0.25);-moz-box-shadow:0 1px 0 rgba(0,0,0,0.25);box-shadow:0 1px 0 rgba(0,0,0,0.25)}.btn-navbar .icon-bar+.icon-bar{margin-top:3px}.navbar .nav>li>.dropdown-menu:before{position:absolute;top:-7px;left:9px;display:inline-block;border-right:7px solid transparent;border-bottom:7px solid #ccc;border-left:7px solid transparent;border-bottom-color:rgba(0,0,0,0.2);content:''}.navbar .nav>li>.dropdown-menu:after{position:absolute;top:-6px;left:10px;display:inline-block;border-right:6px solid transparent;border-bottom:6px solid #fff;border-left:6px solid transparent;content:''}.navbar-fixed-bottom .nav>li>.dropdown-menu:before{top:auto;bottom:-7px;border-top:7px solid #ccc;border-bottom:0;border-top-color:rgba(0,0,0,0.2)}.navbar-fixed-bottom .nav>li>.dropdown-menu:after{top
 :auto;bottom:-6px;border-top:6px solid #fff;border-bottom:0}.navbar .nav li.dropdown.open>.dropdown-toggle,.navbar .nav li.dropdown.active>.dropdown-toggle,.navbar .nav li.dropdown.open.active>.dropdown-toggle{color:#555;background-color:#e5e5e5}.navbar .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .nav li.dropdown.open>.dropdown-toggle .caret,.navbar .nav li.dropdown.active>.dropdown-toggle .caret,.navbar .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .pull-right>li>.dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right{right:0;left:auto}.navbar .pull-right>li>.dropdown-menu:before,.navbar .nav>li>.dropdown-menu.pull-right:before{right:12px;left:auto}.navbar .pull-right>li>.dropdown-menu:after,.navbar .nav>li>.dropdown-menu.pull-right:after{right:13px;left:auto}.navbar .pull-right>li>.dropdown-menu .dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right .dropdown-menu{r
 ight:100%;left:auto;margin-right:-1px;margin-left:0;-webkit-border-radius:6px 0 6px 6px;-moz-border-radius:6px 0 6px 6px;border-radius:6px 0 6px 6px}.navbar-inverse{color:#999}.navbar-inverse .navbar-inner{background-color:#1b1b1b;background-image:-moz-linear-gradient(top,#222,#111);background-image:-webkit-gradient(linear,0 0,0 100%,from(#222),to(#111));background-image:-webkit-linear-gradient(top,#222,#111);background-image:-o-linear-gradient(top,#222,#111);background-image:linear-gradient(to bottom,#222,#111);background-repeat:repeat-x;border-color:#252525;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff222222',endColorstr='#ff111111',GradientType=0)}.navbar-inverse .brand,.navbar-inverse .nav>li>a{color:#999;text-shadow:0 -1px 0 rgba(0,0,0,0.25)}.navbar-inverse .brand:hover,.navbar-inverse .nav>li>a:hover{color:#fff}.navbar-inverse .nav>li>a:focus,.navbar-inverse .nav>li>a:hover{color:#fff;background-color:transparent}.navbar-inverse .nav .active>a,.navbar-in
 verse .nav .active>a:hover,.navbar-inverse .nav .active>a:focus{color:#fff;background-color:#111}.navbar-inverse .navbar-link{color:#999}.navbar-inverse .navbar-link:hover{color:#fff}.navbar-inverse .divider-vertical{border-right-color:#222;border-left-color:#111}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle{color:#fff;background-color:#111}.navbar-inverse .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#999;border-bottom-color:#999}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.navbar-inverse .navbar-search .search-query{color:#fff;background-color:#515151;border-color:#111;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15)
 ;-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);-webkit-transition:none;-moz-transition:none;-o-transition:none;transition:none}.navbar-inverse .navbar-search .search-query:-moz-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:-ms-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query::-webkit-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:focus,.navbar-inverse .navbar-search .search-query.focused{padding:5px 15px;color:#333;text-shadow:0 1px 0 #fff;background-color:#fff;border:0;outline:0;-webkit-box-shadow:0 0 3px rgba(0,0,0,0.15);-moz-box-shadow:0 0 3px rgba(0,0,0,0.15);box-shadow:0 0 3px rgba(0,0,0,0.15)}.navbar-inverse .btn-navbar{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e0e0e;*background-color:#040404;background-image:-webkit-gradient(linear,0 0,0 100%,from(#151515),to(#040404));bac
 kground-image:-webkit-linear-gradient(top,#151515,#040404);background-image:-o-linear-gradient(top,#151515,#040404);background-image:linear-gradient(to bottom,#151515,#040404);background-image:-moz-linear-gradient(top,#151515,#040404);background-repeat:repeat-x;border-color:#040404 #040404 #000;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff151515',endColorstr='#ff040404',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.navbar-inverse .btn-navbar:hover,.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active,.navbar-inverse .btn-navbar.disabled,.navbar-inverse .btn-navbar[disabled]{color:#fff;background-color:#040404;*background-color:#000}.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active{background-color:#000 \9}.breadcrumb{padding:8px 15px;margin:0 0 20px;list-style:none;background-color:#f5f5f5;-webkit-border-radius:4px;-moz-border-r
 adius:4px;border-radius:4px}.breadcrumb li{display:inline-block;*display:inline;text-shadow:0 1px 0 #fff;*zoom:1}.breadcrumb .divider{padding:0 5px;color:#ccc}.breadcrumb .active{color:#999}.pagination{height:40px;margin:20px 0}.pagination ul{display:inline-block;*display:inline;margin-bottom:0;margin-left:0;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px;*zoom:1;-webkit-box-shadow:0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:0 1px 2px rgba(0,0,0,0.05);box-shadow:0 1px 2px rgba(0,0,0,0.05)}.pagination li{display:inline}.pagination a,.pagination span{float:left;padding:0 14px;line-height:38px;text-decoration:none;background-color:#fff;border:1px solid #ddd;border-left-width:0}.pagination a:hover,.pagination .active a,.pagination .active span{background-color:#f5f5f5}.pagination .active a,.pagination .active span{color:#999;cursor:default}.pagination .disabled span,.pagination .disabled a,.pagination .disabled a:hover{color:#999;cursor:default;background-color:transpa
 rent}.pagination li:first-child a,.pagination li:first-child span{border-left-width:1px;-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.pagination li:last-child a,.pagination li:last-child span{-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.pagination-centered{text-align:center}.pagination-right{text-align:right}.pager{margin:20px 0;text-align:center;list-style:none;*zoom:1}.pager:before,.pager:after{display:table;line-height:0;content:""}.pager:after{clear:both}.pager li{display:inline}.pager a{display:inline-block;padding:5px 14px;background-color:#fff;border:1px solid #ddd;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.pager a:hover{text-decoration:none;background-color:#f5f5f5}.pager .next a{float:right}.pager .previous a{float:left}.pager .disabled a,.pager .disabled a:hover{color:#999;cursor:default;background-color:#fff}.modal-open .dropdown-menu{z-index:2050}.moda
 l-open .dropdown.open{*z-index:2050}.modal-open .popover{z-index:2060}.modal-open .tooltip{z-index:2080}.modal-backdrop{position:fixed;top:0;right:0;bottom:0;left:0;z-index:1040;background-color:#000}.modal-backdrop.fade{opacity:0}.modal-backdrop,.modal-backdrop.fade.in{opacity:.8;filter:alpha(opacity=80)}.modal{position:fixed;top:50%;left:50%;z-index:1050;width:560px;margin:-250px 0 0 -280px;overflow:auto;background-color:#fff;border:1px solid #999;border:1px solid rgba(0,0,0,0.3);*border:1px solid #999;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 3px 7px rgba(0,0,0,0.3);-moz-box-shadow:0 3px 7px rgba(0,0,0,0.3);box-shadow:0 3px 7px rgba(0,0,0,0.3);-webkit-background-clip:padding-box;-moz-background-clip:padding-box;background-clip:padding-box}.modal.fade{top:-25%;-webkit-transition:opacity .3s linear,top .3s ease-out;-moz-transition:opacity .3s linear,top .3s ease-out;-o-transition:opacity .3s linear,top .3s ease-out;transition:opacity .3
 s linear,top .3s ease-out}.modal.fade.in{top:50%}.modal-header{padding:9px 15px;border-bottom:1px solid #eee}.modal-header .close{margin-top:2px}.modal-header h3{margin:0;line-height:30px}.modal-body{max-height:400px;padding:15px;overflow-y:auto}.modal-form{margin-bottom:0}.modal-footer{padding:14px 15px 15px;margin-bottom:0;text-align:right;background-color:#f5f5f5;border-top:1px solid #ddd;-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px;*zoom:1;-webkit-box-shadow:inset 0 1px 0 #fff;-moz-box-shadow:inset 0 1px 0 #fff;box-shadow:inset 0 1px 0 #fff}.modal-footer:before,.modal-footer:after{display:table;line-height:0;content:""}.modal-footer:after{clear:both}.modal-footer .btn+.btn{margin-bottom:0;margin-left:5px}.modal-footer .btn-group .btn+.btn{margin-left:-1px}.tooltip{position:absolute;z-index:1030;display:block;padding:5px;font-size:11px;opacity:0;filter:alpha(opacity=0);visibility:visible}.tooltip.in{opacity:.8;filter:alpha(opacity=80)
 }.tooltip.top{margin-top:-3px}.tooltip.right{margin-left:3px}.tooltip.bottom{margin-top:3px}.tooltip.left{margin-left:-3px}.tooltip-inner{max-width:200px;padding:3px 8px;color:#fff;text-align:center;text-decoration:none;background-color:#000;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.tooltip-arrow{position:absolute;width:0;height:0;border-color:transparent;border-style:solid}.tooltip.top .tooltip-arrow{bottom:0;left:50%;margin-left:-5px;border-top-color:#000;border-width:5px 5px 0}.tooltip.right .tooltip-arrow{top:50%;left:0;margin-top:-5px;border-right-color:#000;border-width:5px 5px 5px 0}.tooltip.left .tooltip-arrow{top:50%;right:0;margin-top:-5px;border-left-color:#000;border-width:5px 0 5px 5px}.tooltip.bottom .tooltip-arrow{top:0;left:50%;margin-left:-5px;border-bottom-color:#000;border-width:0 5px 5px}.popover{position:absolute;top:0;left:0;z-index:1010;display:none;width:236px;padding:1px;background-color:#fff;border:1px solid #ccc;border:1px solid r
 gba(0,0,0,0.2);-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0,0,0,0.2);-moz-box-shadow:0 5px 10px rgba(0,0,0,0.2);box-shadow:0 5px 10px rgba(0,0,0,0.2);-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box}.popover.top{margin-bottom:10px}.popover.right{margin-left:10px}.popover.bottom{margin-top:10px}.popover.left{margin-right:10px}.popover-title{padding:8px 14px;margin:0;font-size:14px;font-weight:normal;line-height:18px;background-color:#f7f7f7;border-bottom:1px solid #ebebeb;-webkit-border-radius:5px 5px 0 0;-moz-border-radius:5px 5px 0 0;border-radius:5px 5px 0 0}.popover-content{padding:9px 14px}.popover-content p,.popover-content ul,.popover-content ol{margin-bottom:0}.popover .arrow,.popover .arrow:after{position:absolute;display:inline-block;width:0;height:0;border-color:transparent;border-style:solid}.popover .arrow:after{z-index:-1;content:""}.popover.top .arrow{bottom:-10px;lef
 t:50%;margin-left:-10px;border-top-color:#fff;border-width:10px 10px 0}.popover.top .arrow:after{bottom:-1px;left:-11px;border-top-color:rgba(0,0,0,0.25);border-width:11px 11px 0}.popover.right .arrow{top:50%;left:-10px;margin-top:-10px;border-right-color:#fff;border-width:10px 10px 10px 0}.popover.right .arrow:after{bottom:-11px;left:-1px;border-right-color:rgba(0,0,0,0.25);border-width:11px 11px 11px 0}.popover.bottom .arrow{top:-10px;left:50%;margin-left:-10px;border-bottom-color:#fff;border-width:0 10px 10px}.popover.bottom .arrow:after{top:-1px;left:-11px;border-bottom-color:rgba(0,0,0,0.25);border-width:0 11px 11px}.popover.left .arrow{top:50%;right:-10px;margin-top:-10px;border-left-color:#fff;border-width:10px 0 10px 10px}.popover.left .arrow:after{right:-1px;bottom:-11px;border-left-color:rgba(0,0,0,0.25);border-width:11px 0 11px 11px}.thumbnails{margin-left:-20px;list-style:none;*zoom:1}.thumbnails:before,.thumbnails:after{display:table;line-height:0;content:""}.thumbnails
 :after{clear:both}.row-fluid .thumbnails{margin-left:0}.thumbnails>li{float:left;margin-bottom:20px;margin-left:20px}.thumbnail{display:block;padding:4px;line-height:20px;border:1px solid #ddd;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.055);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.055);box-shadow:0 1px 3px rgba(0,0,0,0.055);-webkit-transition:all .2s ease-in-out;-moz-transition:all .2s ease-in-out;-o-transition:all .2s ease-in-out;transition:all .2s ease-in-out}a.thumbnail:hover{border-color:#08c;-webkit-box-shadow:0 1px 4px rgba(0,105,214,0.25);-moz-box-shadow:0 1px 4px rgba(0,105,214,0.25);box-shadow:0 1px 4px rgba(0,105,214,0.25)}.thumbnail>img{display:block;max-width:100%;margin-right:auto;margin-left:auto}.thumbnail .caption{padding:9px;color:#555}.label,.badge{font-size:11.844px;font-weight:bold;line-height:14px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);white-space:nowrap;vertical-align:baseline;background-
 color:#999}.label{padding:1px 4px 2px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.badge{padding:1px 9px 2px;-webkit-border-radius:9px;-moz-border-radius:9px;border-radius:9px}a.label:hover,a.badge:hover{color:#fff;text-decoration:none;cursor:pointer}.label-important,.badge-important{background-color:#b94a48}.label-important[href],.badge-important[href]{background-color:#953b39}.label-warning,.badge-warning{background-color:#f89406}.label-warning[href],.badge-warning[href]{background-color:#c67605}.label-success,.badge-success{background-color:#468847}.label-success[href],.badge-success[href]{background-color:#356635}.label-info,.badge-info{background-color:#3a87ad}.label-info[href],.badge-info[href]{background-color:#2d6987}.label-inverse,.badge-inverse{background-color:#333}.label-inverse[href],.badge-inverse[href]{background-color:#1a1a1a}.btn .label,.btn .badge{position:relative;top:-1px}.btn-mini .label,.btn-mini .badge{top:0}@-webkit-keyframes progress-b
 ar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-moz-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-ms-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-o-keyframes progress-bar-stripes{from{background-position:0 0}to{background-position:40px 0}}@keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}.progress{height:20px;margin-bottom:20px;overflow:hidden;background-color:#f7f7f7;background-image:-moz-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-webkit-gradient(linear,0 0,0 100%,from(#f5f5f5),to(#f9f9f9));background-image:-webkit-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-o-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:linear-gradient(to bottom,#f5f5f5,#f9f9f9);background-repeat:repeat-x;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(sta
 rtColorstr='#fff5f5f5',endColorstr='#fff9f9f9',GradientType=0);-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1)}.progress .bar{float:left;width:0;height:100%;font-size:12px;color:#fff;text-align:center;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e90d2;background-image:-moz-linear-gradient(top,#149bdf,#0480be);background-image:-webkit-gradient(linear,0 0,0 100%,from(#149bdf),to(#0480be));background-image:-webkit-linear-gradient(top,#149bdf,#0480be);background-image:-o-linear-gradient(top,#149bdf,#0480be);background-image:linear-gradient(to bottom,#149bdf,#0480be);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff149bdf',endColorstr='#ff0480be',GradientType=0);-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);-webkit-box-sizing:border-box;-moz-bo
 x-sizing:border-box;box-sizing:border-box;-webkit-transition:width .6s ease;-moz-transition:width .6s ease;-o-transition:width .6s ease;transition:width .6s ease}.progress .bar+.bar{-webkit-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15)}.progress-striped .bar{background-color:#149bdf;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15)
  25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);-webkit-background-size:40px 40px;-moz-background-size:40px 40px;-o-background-size:40px 40px;background-size:40px 40px}.progress.active .bar{-webkit-animation:progress-bar-stripes 2s linear infinite;-moz-animation:progress-bar-stripes 2s linear infinite;-ms-animation:progress-bar-stripes 2s linear infinite;-o-animation:progress-bar-stripes 2s linear infinite;animation:progress-bar-stripes 2s linear infinite}.progress-danger .bar,.progress .bar-danger{background-color:#dd514c;background-image:-moz-linear-gr
 adient(top,#ee5f5b,#c43c35);background-image:-webkit-gradient(linear,0 0,0 100%,from(#ee5f5b),to(#c43c35));background-image:-webkit-linear-gradient(top,#ee5f5b,#c43c35);background-image:-o-linear-gradient(top,#ee5f5b,#c43c35);background-image:linear-gradient(to bottom,#ee5f5b,#c43c35);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffee5f5b',endColorstr='#ffc43c35',GradientType=0)}.progress-danger.progress-striped .bar,.progress-striped .bar-danger{background-color:#ee5f5b;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);backgrou
 nd-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-success .bar,.progress .bar-success{background-color:#5eb95e;background-image:-moz-linear-gradient(top,#62c462,#57a957);background-image:-webkit-gradient(linear,0 0,0 100%,from(#62c462),to(#57a957));background-image:-webkit-linear-gradient(top,#62c462,#57a957);background-image:-o-linear-gradient(top,#62c462,#57a957);background-image:linear-gradient(to bottom,#62c462,#57a957);background-repeat:repeat-x;filter:progid:dximagetransform.microsof
 t.gradient(startColorstr='#ff62c462',endColorstr='#ff57a957',GradientType=0)}.progress-success.progress-striped .bar,.progress-striped .bar-success{background-color:#62c462;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-

<TRUNCATED>

[02/33] spark-544, introducing SparkConf and related configuration overhaul.

Posted by pw...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
----------------------------------------------------------------------
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 7e22c84..4055590 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

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala
----------------------------------------------------------------------
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 e8fecec..d01329b 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()

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
----------------------------------------------------------------------
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 7127a72..d6b8ac2 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()
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala
----------------------------------------------------------------------
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 e68c527..ff6cc37 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")
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
----------------------------------------------------------------------
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 84fe309..2a3b0e1 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(

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
----------------------------------------------------------------------
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 50cbc2c..9bb92b4 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
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala
----------------------------------------------------------------------
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 01e9516..6069c1d 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()

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
----------------------------------------------------------------------
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 e748c22..17cec81 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)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala
----------------------------------------------------------------------
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 e51c5b3..ee2ae47 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))
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
----------------------------------------------------------------------
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 19a025a..ffd166e 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

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala
----------------------------------------------------------------------
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 e1d68ef..fde7d63 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) {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
----------------------------------------------------------------------
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 21022e1..05502e4 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

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala
----------------------------------------------------------------------
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 b4451fc..61e63c6 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

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
----------------------------------------------------------------------
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 fcd2e97..8f528ba 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

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala
----------------------------------------------------------------------
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 e828e1d..850d317 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

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala
----------------------------------------------------------------------
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 a8db37d..b3b3893 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(
-      "<driver>", actorSystem, blockManagerMaster, serializer, 1024 * 1024)
+      "<driver>", 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)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
----------------------------------------------------------------------
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 f1d86c0..0ce8d9c 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
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
----------------------------------------------------------------------
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 fcd1b51..14751e8 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)
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala
----------------------------------------------------------------------
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 c5bf2ac..b637d37 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")}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
----------------------------------------------------------------------
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 6b85474..f01a138 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]()

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
----------------------------------------------------------------------
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 1c8b51b..76febd5 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")
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
----------------------------------------------------------------------
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 7b41ef8..bf71d17 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))

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala
----------------------------------------------------------------------
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 a25b37a..1407c39 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 {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/main/scala/org/apache/spark/util/Utils.scala
----------------------------------------------------------------------
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 3f7858d..fd5888e 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()

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala
index 271dc90..10b8b44 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"))
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala
----------------------------------------------------------------------
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 7181333..4ecdde0 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
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
----------------------------------------------------------------------
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 706d84a..2aa259d 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

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala
----------------------------------------------------------------------
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 002368f..dd12261 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)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala
----------------------------------------------------------------------
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 95d3553..34d2e4c 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)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala
----------------------------------------------------------------------
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 bb28a31..2bb827c 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")

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala
----------------------------------------------------------------------
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 27c2d53..618fae7 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
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
----------------------------------------------------------------------
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 5b4d63b..4ef5538 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("<driver>", actorSystem, master, serializer, 2000)
+    store = new BlockManager("<driver>", 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("<driver>", actorSystem, master, serializer, 2000)
+    store = new BlockManager("<driver>", 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("<driver>", actorSystem, master, serializer, 2000)
+    store = new BlockManager("<driver>", 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("<driver>", actorSystem, master, serializer, 2000)
+    store = new BlockManager("<driver>", 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("<driver>", actorSystem, master, serializer, 2000)
+    store = new BlockManager("<driver>", 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("<driver>", actorSystem, master, serializer, 2000)
+    store = new BlockManager("<driver>", 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("<driver>", actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", 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("<driver>", actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", 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("<driver>", actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", 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("<driver>", actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", 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("<driver>", actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", 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("<driver>", actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", 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("<driver>", actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", 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("<driver>", actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", 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("<driver>", actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", 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("<driver>", actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", 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("<driver>", actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", 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("<driver>", actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", 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("<driver>", actorSystem, master, serializer, 500)
+    store = new BlockManager("<driver>", 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("<driver>", actorSystem, master, new JavaSerializer, 1200)
+    store = new BlockManager("<driver>", actorSystem, master, new JavaSerializer, 1200, conf)
 
     // The put should fail since a1 is not serializable.
     class UnserializableClass

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala
----------------------------------------------------------------------
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 070982e..f940448 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)
   }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala
----------------------------------------------------------------------
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 5aff26f..a5facd5 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()
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala
----------------------------------------------------------------------
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 72b5c7b..12c430b 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 <inputFile> <threshold> <numPartitions> <host> <usePartitioner>")
       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)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala
----------------------------------------------------------------------
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 ddf6855..5bf0b7a 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 <inputFile> <threshold> <numIterations> <host> <usePartitioner>")
       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)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala
----------------------------------------------------------------------
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 50e3f96..2402409 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)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala
----------------------------------------------------------------------
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 36853ac..2f2d106 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)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
----------------------------------------------------------------------
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 eeeca3e..433268a 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) {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2573add9/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
----------------------------------------------------------------------
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 9467881..a322f60 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.$() +