You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by gu...@apache.org on 2020/12/18 01:10:19 UTC
[spark] branch branch-3.1 updated: [SPARK-33824][PYTHON][DOCS]
Restructure and improve Python package management page
This is an automated email from the ASF dual-hosted git repository.
gurwls223 pushed a commit to branch branch-3.1
in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/branch-3.1 by this push:
new 16739f3 [SPARK-33824][PYTHON][DOCS] Restructure and improve Python package management page
16739f3 is described below
commit 16739f3cece54adaae27757c90f0003f417757f0
Author: HyukjinKwon <gu...@apache.org>
AuthorDate: Fri Dec 18 10:03:07 2020 +0900
[SPARK-33824][PYTHON][DOCS] Restructure and improve Python package management page
### What changes were proposed in this pull request?
This PR proposes to restructure and refine the Python dependency management page.
I lately wrote a blog post which will be published soon, and decided contribute some of the contents back to PySpark documentation.
FWIW, it has been reviewed by some tech writers and engineers.
I built the site for making the review easier: https://hyukjin-spark.readthedocs.io/en/stable/user_guide/python_packaging.html
### Why are the changes needed?
For better documentation.
### Does this PR introduce _any_ user-facing change?
It's doc change but only in unreleased bracnhs for now.
### How was this patch tested?
I manually built the docs as:
```bash
cd python/docs
make clean html
open
```
Closes #30822 from HyukjinKwon/SPARK-33824.
Authored-by: HyukjinKwon <gu...@apache.org>
Signed-off-by: HyukjinKwon <gu...@apache.org>
(cherry picked from commit 6315118676c99ccef2566c50ab9873de8876e468)
Signed-off-by: HyukjinKwon <gu...@apache.org>
---
python/docs/source/user_guide/python_packaging.rst | 200 +++++++++++++--------
1 file changed, 125 insertions(+), 75 deletions(-)
diff --git a/python/docs/source/user_guide/python_packaging.rst b/python/docs/source/user_guide/python_packaging.rst
index 0aff6dc..71d8e53 100644
--- a/python/docs/source/user_guide/python_packaging.rst
+++ b/python/docs/source/user_guide/python_packaging.rst
@@ -17,7 +17,7 @@
=========================
-3rd Party Python Packages
+Python Package Management
=========================
When you want to run your PySpark application on a cluster such as YARN, Kubernetes, Mesos, etc., you need to make
@@ -51,10 +51,11 @@ Here is the script ``app.py`` from the previous example that will be executed on
main(SparkSession.builder.getOrCreate())
-There are multiple ways to ship the dependencies to the cluster:
+There are multiple ways to manage Python dependencies in the cluster:
- Using PySpark Native Features
-- Using Zipped Virtual Environment
+- Using Conda
+- Using Virtualenv
- Using PEX
@@ -62,54 +63,51 @@ Using PySpark Native Features
-----------------------------
PySpark allows to upload Python files (``.py``), zipped Python packages (``.zip``), and Egg files (``.egg``)
-to the executors by setting the configuration setting ``spark.submit.pyFiles`` or by directly
-calling :meth:`pyspark.SparkContext.addPyFile`.
+to the executors by:
-This is an easy way to ship additional custom Python code to the cluster. You can just add individual files or zip whole
-packages and upload them. Using :meth:`pyspark.SparkContext.addPyFile` allows to upload code
-even after having started your job.
+- Setting the configuration setting ``spark.submit.pyFiles``
+- Setting ``--py-files`` option in Spark scripts
+- Directly calling :meth:`pyspark.SparkContext.addPyFile` in applications
-Note that it doesn't allow to add packages built as `Wheels <https://www.python.org/dev/peps/pep-0427/>`_ and therefore doesn't
-allow to include dependencies with native code.
+This is a straightforward method to ship additional custom Python code to the cluster. You can just add individual files or zip whole
+packages and upload them. Using :meth:`pyspark.SparkContext.addPyFile` allows to upload code even after having started your job.
+However, it does not allow to add packages built as `Wheels <https://www.python.org/dev/peps/pep-0427/>`_ and therefore
+does not allow to include dependencies with native code.
-Using Zipped Virtual Environment
---------------------------------
-The idea of zipped environments is to zip your whole `virtual environment <https://docs.python.org/3/tutorial/venv.html>`_,
-ship it to the cluster, unzip it remotely and target the Python interpreter from inside this zipped environment.
+Using Conda
+-----------
-Zip Virtual Environment
-~~~~~~~~~~~~~~~~~~~~~~~
+`Conda <https://docs.conda.io/en/latest/>`_ is one of the most widely-used Python package management systems. PySpark users can directly
+use a Conda environment to ship their third-party Python packages by leveraging
+`conda-pack <https://conda.github.io/conda-pack/spark.html>`_ which is a command line tool creating
+relocatable Conda environments.
-You can zip the virtual environment on your own or use tools for doing this:
-
-* `conda-pack <https://conda.github.io/conda-pack/spark.html>`_ for conda environments
-* `venv-pack <https://jcristharif.com/venv-pack/spark.html>`_ for virtual environments
-
-Example with `conda-pack`:
+The example below creates a Conda environment to use on both the driver and executor and packs
+it into an archive file. This archive file captures the Conda environment for Python and stores
+both Python interpreter and all its relevant dependencies.
.. code-block:: bash
- conda create -y -n pyspark_env -c conda-forge pyarrow==2.0.0 pandas==1.1.4 conda-pack==0.5.0
- conda activate pyspark_env
- conda pack -f -o pyspark_env.tar.gz
-
-Upload to Spark Executors
-~~~~~~~~~~~~~~~~~~~~~~~~~
+ conda create -y -n pyspark_conda_env -c conda-forge pyarrow pandas conda-pack
+ conda activate pyspark_conda_env
+ conda pack -f -o pyspark_conda_env.tar.gz
-Unzipping will be done by Spark when using target ``--archives`` option in spark-submit
-or setting ``spark.archives`` configuration.
+After that, you can ship it together with scripts or in the code by using the ``--archives`` option
+or ``spark.archives`` configuration (``spark.yarn.dist.archives`` in YARN). It automatically unpacks the archive on executors.
-Example with ``spark-submit``:
+In the case of a ``spark-submit`` script, you can use it as follows:
.. code-block:: bash
export PYSPARK_DRIVER_PYTHON=python
export PYSPARK_PYTHON=./environment/bin/python
- spark-submit --master=... --archives pyspark_env.tar.gz#environment app.py
+ spark-submit --archives pyspark_conda_env.tar.gz#environment app.py
-Example using ``SparkSession.builder``:
+Note that ``PYSPARK_DRIVER_PYTHON`` above is not required for cluster modes in YARN or Kubernetes.
+
+If you’re on a regular Python shell or notebook, you can try it as shown below:
.. code-block:: python
@@ -118,67 +116,117 @@ Example using ``SparkSession.builder``:
from app import main
os.environ['PYSPARK_PYTHON'] = "./environment/bin/python"
- spark = SparkSession.builder.master("...").config("spark.archives", "pyspark_env.tar.gz#environment").getOrCreate()
+ spark = SparkSession.builder.config(
+ "spark.archives", # 'spark.yarn.dist.archives' in YARN.
+ "pyspark_conda_env.tar.gz#environment").getOrCreate()
main(spark)
-Example with ``pyspark`` shell:
+For a pyspark shell:
.. code-block:: bash
export PYSPARK_DRIVER_PYTHON=python
export PYSPARK_PYTHON=./environment/bin/python
- pyspark --master=... --archives pyspark_env.tar.gz#environment
+ pyspark --archives pyspark_conda_env.tar.gz#environment
-Using PEX
----------
+Using Virtualenv
+----------------
-`PEX <https://github.com/pantsbuild/pex>`_ is a library for generating ``.pex`` (Python EXecutable) files.
-A PEX file is a self-contained executable Python environment. It can be seen as the Python equivalent of Java uber-JARs (a.k.a. fat JARs).
+`Virtualenv <https://virtualenv.pypa.io/en/latest/>`_ is a Python tool to create isolated Python environments.
+Since Python 3.3, a subset of its features has been integrated into Python as a standard library under
+the `venv <https://docs.python.org/3/library/venv.html>`_ module. PySpark users can use virtualenv to manage
+Python dependencies in their clusters by using `venv-pack <https://jcristharif.com/venv-pack/index.html>`_
+in a similar way as conda-pack.
-You need to build the PEX file somewhere with all your requirements and then upload it to each Spark executor.
+A virtual environment to use on both driver and executor can be created as demonstrated below.
+It packs the current virtual environment to an archive file, and It self-contains both Python interpreter
+and the dependencies.
-Using CLI to Build PEX file
-~~~~~~~~~~~~~~~~~~~~~~~~~~~
.. code-block:: bash
- pex pyspark==3.0.1 pyarrow==0.15.1 pandas==0.25.3 -o myarchive.pex
+ python -m venv pyspark_venv
+ source pyspark_venv/bin/activate
+ pip install pyarrow pandas venv-pack
+ venv-pack -o pyspark_venv.tar.gz
+You can directly pass/unpack the archive file and enable the environment on executors by leveraging
+the ``--archives`` option or ``spark.archives`` configuration (``spark.yarn.dist.archives`` in YARN).
-Invoking the PEX file will by default invoke the Python interpreter. pyarrow, pandas and pyspark will be included in the PEX file.
+For ``spark-submit``, you can use it by running the command as follows. Also, notice that
+``PYSPARK_DRIVER_PYTHON`` is not necessary in Kubernetes or YARN cluster modes.
.. code-block:: bash
- ./myarchive.pex
- Python 3.6.6 (default, Jan 26 2019, 16:53:05)
- (InteractiveConsole)
- >>> import pyarrow
- >>> import pandas
- >>> import pyspark
- >>>
+ export PYSPARK_DRIVER_PYTHON=python
+ export PYSPARK_PYTHON=./environment/bin/python
+ spark-submit --archives pyspark_venv.tar.gz#environment app.py
-This can also be done directly with the Python API. For more information on how to build PEX files,
-please refer to `Building .pex files <https://pex.readthedocs.io/en/stable/buildingpex.html>`_
+For regular Python shells or notebooks:
-Upload to Spark Executors
-~~~~~~~~~~~~~~~~~~~~~~~~~
+.. code-block:: bash
-The upload can be done by setting ``--files`` option in spark-submit or setting ``spark.files`` configuration (``spark.yarn.dist.files`` on YARN)
-and changing the ``PYSPARK_PYTHON`` environment variable to change the Python interpreter to the PEX executable on each executor.
+ import os
+ from pyspark.sql import SparkSession
+ from app import main
-..
- TODO: we should also document the way on other cluster modes.
+ os.environ['PYSPARK_PYTHON'] = "./environment/bin/python"
+ spark = SparkSession.builder.config(
+ "spark.archives", # 'spark.yarn.dist.archives' in YARN.
+ "pyspark_venv.tar.gz#environment").getOrCreate()
+ main(spark)
-Example with ``spark-submit`` on YARN:
+In the case of a pyspark shell:
.. code-block:: bash
export PYSPARK_DRIVER_PYTHON=python
- export PYSPARK_PYTHON=./myarchive.pex
- spark-submit --master=yarn --deploy-mode client --files myarchive.pex app.py
+ export PYSPARK_PYTHON=./environment/bin/python
+ pyspark --archives pyspark_venv.tar.gz#environment
+
+
+Using PEX
+---------
-Example using ``SparkSession.builder`` on YARN:
+PySpark can also use `PEX <https://github.com/pantsbuild/pex>`_ to ship the Python packages
+together. PEX is a tool that creates a self-contained Python environment. This is similar
+to Conda or virtualenv, but a ``.pex`` file is executable by itself.
+
+The following example creates a ``.pex`` file for the driver and executor to use.
+The file contains the Python dependencies specified with the ``pex`` command.
+
+.. code-block:: bash
+
+ pip install pyarrow pandas pex
+ pex pyspark pyarrow pandas -o pyspark_pex_env.pex
+
+This file behaves similarly with a regular Python interpreter.
+
+.. code-block:: bash
+
+ ./pyspark_pex_env.pex -c "import pandas; print(pandas.__version__)"
+ 1.1.5
+
+However, ``.pex`` file does not include a Python interpreter itself under the hood so all
+nodes in a cluster should have the same Python interpreter installed.
+
+In order to transfer and use the ``.pex`` file in a cluster, you should ship it via the
+``spark.files`` configuration (``spark.yarn.dist.files`` in YARN) or ``--files`` option because they are regular files instead
+of directories or archive files.
+
+For application submission, you run the commands as shown below.
+Note that ``PYSPARK_DRIVER_PYTHON`` is not needed for cluster modes in YARN or Kubernetes,
+and you may also need to set ``PYSPARK_PYTHON`` environment variable on
+the AppMaster ``--conf spark.yarn.appMasterEnv.PYSPARK_PYTHON=./myarchive.pex`` in YARN cluster mode.
+
+.. code-block:: bash
+
+ export PYSPARK_DRIVER_PYTHON=python
+ export PYSPARK_PYTHON=./pyspark_pex_env.pex
+ spark-submit --files pyspark_pex_env.pex app.py
+
+For regular Python shells or notebooks:
.. code-block:: python
@@ -186,19 +234,21 @@ Example using ``SparkSession.builder`` on YARN:
from pyspark.sql import SparkSession
from app import main
- os.environ['PYSPARK_PYTHON']="./myarchive.pex"
- builder = SparkSession.builder
- builder.master("yarn") \
- .config("spark.submit.deployMode", "client") \
- .config("spark.yarn.dist.files", "myarchive.pex")
- spark = builder.getOrCreate()
+ os.environ['PYSPARK_PYTHON'] = "./pyspark_pex_env.pex"
+ spark = SparkSession.builder.config(
+ "spark.files", # 'spark.yarn.dist.files' in YARN.
+ "pyspark_pex_env.pex").getOrCreate()
main(spark)
-Notes
-~~~~~
+For the interactive pyspark shell, the commands are almost the same:
-* The Python interpreter that has been used to generate the PEX file must be available on each executor. PEX doesn't include the Python interpreter.
+.. code-block:: bash
-* In YARN cluster mode you may also need to set ``PYSPARK_PYTHON`` environment variable on the AppMaster ``--conf spark.yarn.appMasterEnv.PYSPARK_PYTHON=./myarchive.pex``.
+ export PYSPARK_DRIVER_PYTHON=python
+ export PYSPARK_PYTHON=./pyspark_pex_env.pex
+ pyspark --files pyspark_pex_env.pex
-* An end-to-end Docker example for deploying a standalone PySpark with ``SparkSession.builder`` and PEX can be found `here <https://github.com/criteo/cluster-pack/blob/master/examples/spark-with-S3/README.md>`_ - it uses cluster-pack, a library on top of PEX that automatizes the the intermediate step of having to create & upload the PEX manually.
+An end-to-end Docker example for deploying a standalone PySpark with ``SparkSession.builder`` and PEX
+can be found `here <https://github.com/criteo/cluster-pack/blob/master/examples/spark-with-S3/README.md>`_
+- it uses cluster-pack, a library on top of PEX that automatizes the the intermediate step of having
+to create & upload the PEX manually.
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org