diff options
author | Matei Zaharia <matei@eecs.berkeley.edu> | 2013-10-22 23:15:33 -0700 |
---|---|---|
committer | Matei Zaharia <matei@eecs.berkeley.edu> | 2013-10-22 23:15:33 -0700 |
commit | 452aa36d671d3fdf53058e80cbd86787c8870cd7 (patch) | |
tree | ae37bf1fac9fb438f2acd0cddc6f1fec4e93d07b /docs | |
parent | 9dfcf53a08645227b8722f07068466c7b7909721 (diff) | |
parent | c8748c25eb2812d82a1b6b8433a1531bb00fa335 (diff) | |
download | spark-452aa36d671d3fdf53058e80cbd86787c8870cd7.tar.gz spark-452aa36d671d3fdf53058e80cbd86787c8870cd7.tar.bz2 spark-452aa36d671d3fdf53058e80cbd86787c8870cd7.zip |
Merge pull request #97 from ewencp/pyspark-system-properties
Add classmethod to SparkContext to set system properties.
Add a new classmethod to SparkContext to set system properties like is
possible in Scala/Java. Unlike the Java/Scala implementations, there's
no access to System until the JVM bridge is created. Since
SparkContext handles that, move the initialization of the JVM
connection to a separate classmethod that can safely be called
repeatedly as long as the same instance (or no instance) is provided.
Diffstat (limited to 'docs')
-rw-r--r-- | docs/python-programming-guide.md | 11 |
1 files changed, 11 insertions, 0 deletions
diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 6c2336ad0c..55e39b1de1 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -131,6 +131,17 @@ 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: + +{% highlight python %} +from pyspark import SparkContext +SparkContext.setSystemProperty('spark.executor.memory', '2g') +sc = SparkContext("local", "App Name") +{% endhighlight %} + # API Docs [API documentation](api/pyspark/index.html) for PySpark is available as Epydoc. |