diff options
author | Matei Zaharia <matei@databricks.com> | 2013-12-30 22:17:28 -0500 |
---|---|---|
committer | Matei Zaharia <matei@databricks.com> | 2013-12-30 22:17:28 -0500 |
commit | 0fa5809768cf60ec62b4277f04e23a44dc1582e2 (patch) | |
tree | fee16620755769a70975c41d894db43633b18098 /python/pyspark | |
parent | 994f080f8ae3372366e6004600ba791c8a372ff0 (diff) | |
download | spark-0fa5809768cf60ec62b4277f04e23a44dc1582e2.tar.gz spark-0fa5809768cf60ec62b4277f04e23a44dc1582e2.tar.bz2 spark-0fa5809768cf60ec62b4277f04e23a44dc1582e2.zip |
Updated docs for SparkConf and handled review comments
Diffstat (limited to 'python/pyspark')
-rw-r--r-- | python/pyspark/conf.py | 24 | ||||
-rw-r--r-- | python/pyspark/context.py | 24 |
2 files changed, 31 insertions, 17 deletions
diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py index c07dd88307..9dcdcfaa67 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 diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 12ac0299e2..ee2f5cbe34 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 |