aboutsummaryrefslogtreecommitdiff
path: root/docs/configuration.md
diff options
context:
space:
mode:
authorPatrick Wendell <pwendell@gmail.com>2014-01-01 21:29:12 -0800
committerPatrick Wendell <pwendell@gmail.com>2014-01-01 21:29:12 -0800
commit3713f8129a618a633a7aca8c944960c3e7ac9d3b (patch)
treeff3aa8fa3460078007259a6a6479dc4aec27b50a /docs/configuration.md
parentc1d928a897f8daed5d7e74f4af476b67046f348d (diff)
parent7e8d2e8a5c88d16c771923504c433491b109ab2a (diff)
downloadspark-3713f8129a618a633a7aca8c944960c3e7ac9d3b.tar.gz
spark-3713f8129a618a633a7aca8c944960c3e7ac9d3b.tar.bz2
spark-3713f8129a618a633a7aca8c944960c3e7ac9d3b.zip
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.
Diffstat (limited to 'docs/configuration.md')
-rw-r--r--docs/configuration.md71
1 files changed, 56 insertions, 15 deletions
diff --git a/docs/configuration.md b/docs/configuration.md
index 677d182e50..567aba07f0 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.