aboutsummaryrefslogtreecommitdiff
path: root/docs/python-programming-guide.md
diff options
context:
space:
mode:
authorMatei Zaharia <matei@databricks.com>2014-05-06 15:12:35 -0700
committerMatei Zaharia <matei@databricks.com>2014-05-06 15:12:35 -0700
commit951a5d939863b42da83ac2569d5e9d7ed680e119 (patch)
tree6ff0c545f577b05a86ce33d339cd0d487e935a38 /docs/python-programming-guide.md
parentec09acdd4a72333e1c9c2e9d8e12e9c4c07770c8 (diff)
downloadspark-951a5d939863b42da83ac2569d5e9d7ed680e119.tar.gz
spark-951a5d939863b42da83ac2569d5e9d7ed680e119.tar.bz2
spark-951a5d939863b42da83ac2569d5e9d7ed680e119.zip
[SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN. This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging. In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit. In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0. Author: Matei Zaharia <matei@databricks.com> Closes #664 from mateiz/py-submit and squashes the following commits: 15e9669 [Matei Zaharia] Fix some uses of path.separator property 051278c [Matei Zaharia] Small style fixes 0afe886 [Matei Zaharia] Add license headers 4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests 15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside 47c0655 [Matei Zaharia] More work to make spark-submit work with Python: d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
Diffstat (limited to 'docs/python-programming-guide.md')
-rw-r--r--docs/python-programming-guide.md28
1 files changed, 14 insertions, 14 deletions
diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md
index 98c456228a..8ea22e15a4 100644
--- a/docs/python-programming-guide.md
+++ b/docs/python-programming-guide.md
@@ -60,12 +60,9 @@ By default, PySpark requires `python` to be available on the system `PATH` and u
All of PySpark's library dependencies, including [Py4J](http://py4j.sourceforge.net/), are bundled with PySpark and automatically imported.
-Standalone PySpark applications should be run using the `bin/pyspark` script, which automatically configures the Java and Python environment using the settings in `conf/spark-env.sh` or `.cmd`.
-The script automatically adds the `bin/pyspark` package to the `PYTHONPATH`.
+Standalone PySpark applications should be run using the `bin/spark-submit` script, which automatically
+configures the Java and Python environment for running Spark.
-# Running PySpark on YARN
-
-To run PySpark against a YARN cluster, simply set the MASTER environment variable to "yarn-client".
# Interactive Use
@@ -103,7 +100,7 @@ $ MASTER=local[4] ./bin/pyspark
## IPython
-It is also possible to launch PySpark in [IPython](http://ipython.org), the
+It is also possible to launch the PySpark shell in [IPython](http://ipython.org), the
enhanced Python interpreter. PySpark works with IPython 1.0.0 and later. To
use IPython, set the `IPYTHON` variable to `1` when running `bin/pyspark`:
@@ -123,18 +120,17 @@ IPython also works on a cluster or on multiple cores if you set the `MASTER` env
# Standalone Programs
-PySpark can also be used from standalone Python scripts by creating a SparkContext in your script and running the script using `bin/pyspark`.
+PySpark can also be used from standalone Python scripts by creating a SparkContext in your script and running the script using `bin/spark-submit`.
The Quick Start guide includes a [complete example](quick-start.html#a-standalone-app-in-python) of a standalone Python application.
-Code dependencies can be deployed by listing them in the `pyFiles` option in the SparkContext constructor:
+Code dependencies can be deployed by passing .zip or .egg files in the `--py-files` option of `spark-submit`:
-{% highlight python %}
-from pyspark import SparkContext
-sc = SparkContext("local", "App Name", pyFiles=['MyFile.py', 'lib.zip', 'app.egg'])
+{% highlight bash %}
+./bin/spark-submit --py-files lib1.zip,lib2.zip my_script.py
{% endhighlight %}
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.
+Code dependencies can also be added to an existing SparkContext at runtime using its `addPyFile()` method.
You can set [configuration properties](configuration.html#spark-properties) by passing a
[SparkConf](api/python/pyspark.conf.SparkConf-class.html) object to SparkContext:
@@ -142,12 +138,16 @@ You can set [configuration properties](configuration.html#spark-properties) by p
{% highlight python %}
from pyspark import SparkConf, SparkContext
conf = (SparkConf()
- .setMaster("local")
.setAppName("My app")
.set("spark.executor.memory", "1g"))
sc = SparkContext(conf = conf)
{% endhighlight %}
+`spark-submit` supports launching Python applications on standalone, Mesos or YARN clusters, through
+its `--master` argument. However, it currently requires the Python driver program to run on the local
+machine, not the cluster (i.e. the `--deploy-mode` parameter cannot be `cluster`).
+
+
# API Docs
[API documentation](api/python/index.html) for PySpark is available as Epydoc.
@@ -164,6 +164,6 @@ some example applications.
PySpark also includes several sample programs in the [`python/examples` folder](https://github.com/apache/spark/tree/master/python/examples).
You can run them by passing the files to `pyspark`; e.g.:
- ./bin/pyspark python/examples/wordcount.py
+ ./bin/spark-submit python/examples/wordcount.py
Each program prints usage help when run without arguments.