aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--core/src/main/scala/org/apache/spark/SparkConf.scala18
-rw-r--r--core/src/test/resources/spark.conf8
-rw-r--r--core/src/test/scala/org/apache/spark/SparkConfSuite.scala47
-rw-r--r--docs/configuration.md28
-rw-r--r--project/SparkBuild.scala1
-rw-r--r--python/pyspark/conf.py10
6 files changed, 41 insertions, 71 deletions
diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
index 369c6ce78f..951bfd79d0 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -20,19 +20,17 @@ package org.apache.spark
import scala.collection.JavaConverters._
import scala.collection.mutable.HashMap
-import com.typesafe.config.ConfigFactory
import java.io.{ObjectInputStream, ObjectOutputStream, IOException}
/**
* 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.
+ * values from any `spark.*` Java system properties set in your application as well. In this case,
+ * parameters you set directly on the `SparkConf` object take priority over system properties.
*
* 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.
+ * get the same configuration no matter what the system properties are.
*
* All setter methods in this class support chaining. For example, you can write
* `new SparkConf().setMaster("local").setAppName("My app")`.
@@ -40,7 +38,7 @@ import java.io.{ObjectInputStream, ObjectOutputStream, IOException}
* 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
+ * @param loadDefaults whether to also load values from Java system properties
*/
class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging {
@@ -50,11 +48,9 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging {
private val settings = new HashMap[String, String]()
if (loadDefaults) {
- ConfigFactory.invalidateCaches()
- val typesafeConfig = ConfigFactory.systemProperties()
- .withFallback(ConfigFactory.parseResources("spark.conf"))
- for (e <- typesafeConfig.entrySet().asScala if e.getKey.startsWith("spark.")) {
- settings(e.getKey) = e.getValue.unwrapped.toString
+ // Load any spark.* system properties
+ for ((k, v) <- System.getProperties.asScala if k.startsWith("spark.")) {
+ settings(k) = v
}
}
diff --git a/core/src/test/resources/spark.conf b/core/src/test/resources/spark.conf
deleted file mode 100644
index aa4e751235..0000000000
--- a/core/src/test/resources/spark.conf
+++ /dev/null
@@ -1,8 +0,0 @@
-# A simple spark.conf file used only in our unit tests
-
-spark.test.intTestProperty = 1
-
-spark.test {
- stringTestProperty = "hi"
- listTestProperty = ["a", "b"]
-}
diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
index fa49974db4..87e9012622 100644
--- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
@@ -20,35 +20,23 @@ 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") {
+ test("loading from system properties") {
try {
- System.setProperty("spark.test.intTestProperty", "2")
+ System.setProperty("spark.test.testProperty", "2")
val conf = new SparkConf()
- assert(conf.get("spark.test.intTestProperty") === "2")
- assert(conf.get("spark.test.stringTestProperty") === "hi")
+ assert(conf.get("spark.test.testProperty") === "2")
} finally {
- System.clearProperty("spark.test.intTestProperty")
+ System.clearProperty("spark.test.testProperty")
}
}
test("initializing without loading defaults") {
try {
- System.setProperty("spark.test.intTestProperty", "2")
+ System.setProperty("spark.test.testProperty", "2")
val conf = new SparkConf(false)
- assert(!conf.contains("spark.test.intTestProperty"))
- assert(!conf.contains("spark.test.stringTestProperty"))
+ assert(!conf.contains("spark.test.testProperty"))
} finally {
- System.clearProperty("spark.test.intTestProperty")
+ System.clearProperty("spark.test.testProperty")
}
}
@@ -124,4 +112,25 @@ class SparkConfSuite extends FunSuite with LocalSparkContext {
assert(sc.master === "local[2]")
assert(sc.appName === "My other app")
}
+
+ test("nested property names") {
+ // This wasn't supported by some external conf parsing libraries
+ try {
+ System.setProperty("spark.test.a", "a")
+ System.setProperty("spark.test.a.b", "a.b")
+ System.setProperty("spark.test.a.b.c", "a.b.c")
+ val conf = new SparkConf()
+ assert(conf.get("spark.test.a") === "a")
+ assert(conf.get("spark.test.a.b") === "a.b")
+ assert(conf.get("spark.test.a.b.c") === "a.b.c")
+ conf.set("spark.test.a.b", "A.B")
+ assert(conf.get("spark.test.a") === "a")
+ assert(conf.get("spark.test.a.b") === "A.B")
+ assert(conf.get("spark.test.a.b.c") === "a.b.c")
+ } finally {
+ System.clearProperty("spark.test.a")
+ System.clearProperty("spark.test.a.b")
+ System.clearProperty("spark.test.a.b.c")
+ }
+ }
}
diff --git a/docs/configuration.md b/docs/configuration.md
index da70cabba2..00864906b3 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -18,8 +18,8 @@ Spark provides three locations to configure the system:
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.
+Alternatively, Spark will also load them from Java system properties, for compatibility with old versions
+of Spark.
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
@@ -468,30 +468,6 @@ Apart from these, the following properties are also available, and may be useful
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 be configured through environment variables, which are read from the `conf/spark-env.sh`
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index a9f9937cb1..151b1e7c79 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -277,7 +277,6 @@ 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.clearspring.analytics" % "stream" % "2.5.1"
)
)
diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py
index d72aed6a30..3870cd8f2b 100644
--- a/python/pyspark/conf.py
+++ b/python/pyspark/conf.py
@@ -61,14 +61,12 @@ class SparkConf(object):
Most of the time, you would create a SparkConf object with
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.
+ properties as well. In this case, any parameters you set directly on
+ the C{SparkConf} object take priority over system properties.
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.
+ what the system properties are.
All setter methods in this class support chaining. For example,
you can write C{conf.setMaster("local").setAppName("My app")}.
@@ -82,7 +80,7 @@ class SparkConf(object):
Create a new Spark configuration.
@param loadDefaults: whether to load values from Java system
- properties and classpath (True by default)
+ properties (True by default)
@param _jvm: internal parameter used to pass a handle to the
Java VM; does not need to be set by users
"""