Skip to content
Snippets Groups Projects
Commit 58c6fa20 authored by Matei Zaharia's avatar Matei Zaharia
Browse files

Add Python docs about SparkConf

parent 615fb649
No related branches found
No related tags found
No related merge requests found
...@@ -50,29 +50,62 @@ u'value1' ...@@ -50,29 +50,62 @@ u'value1'
class SparkConf(object): 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): 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 from pyspark.context import SparkContext
SparkContext._ensure_initialized() SparkContext._ensure_initialized()
_jvm = _jvm or SparkContext._jvm _jvm = _jvm or SparkContext._jvm
self._jconf = _jvm.SparkConf(loadDefaults) self._jconf = _jvm.SparkConf(loadDefaults)
def set(self, key, value): def set(self, key, value):
"""Set a configuration property."""
self._jconf.set(key, value) self._jconf.set(key, value)
return self return self
def setMaster(self, value): def setMaster(self, value):
"""Set master URL to connect to."""
self._jconf.setMaster(value) self._jconf.setMaster(value)
return self return self
def setAppName(self, value): def setAppName(self, value):
"""Set application name."""
self._jconf.setAppName(value) self._jconf.setAppName(value)
return self return self
def setSparkHome(self, value): def setSparkHome(self, value):
"""
Set path where Spark is installed on worker nodes (needed for some
deployment modes).
"""
self._jconf.setSparkHome(value) self._jconf.setSparkHome(value)
return self return self
def setExecutorEnv(self, key=None, value=None, pairs=None): 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): 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") raise Exception("Either pass one key-value pair or a list of pairs")
elif key != None: elif key != None:
...@@ -83,23 +116,32 @@ class SparkConf(object): ...@@ -83,23 +116,32 @@ class SparkConf(object):
return self return self
def setAll(self, pairs): 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: for (k, v) in pairs:
self._jconf.set(k, v) self._jconf.set(k, v)
return self return self
def get(self, key): def get(self, key):
"""Get the configured value for some key, if set."""
return self._jconf.get(key) return self._jconf.get(key)
def getOrElse(self, key, defaultValue): def getOrElse(self, key, defaultValue):
"""Get the value for some key, or return a default otherwise."""
return self._jconf.getOrElse(key, defaultValue) return self._jconf.getOrElse(key, defaultValue)
def getAll(self): def getAll(self):
"""Get all values as a list of key-value pairs."""
pairs = [] pairs = []
for elem in self._jconf.getAll(): for elem in self._jconf.getAll():
pairs.append((elem._1(), elem._2())) pairs.append((elem._1(), elem._2()))
return pairs return pairs
def contains(self, key): def contains(self, key):
"""Does this configuration contain a given key?"""
return self._jconf.contains(key) return self._jconf.contains(key)
......
...@@ -53,7 +53,8 @@ class SparkContext(object): ...@@ -53,7 +53,8 @@ class SparkContext(object):
def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None,
environment=None, batchSize=1024, serializer=PickleSerializer(), conf=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 @param master: Cluster URL to connect to
(e.g. mesos://host:port, spark://host:port, local[4]). (e.g. mesos://host:port, spark://host:port, local[4]).
......
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment