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:30:01 UTC

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

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