spark-instrumented-optimizer/python
Bryan Cutler 5e79ae3b40 [SPARK-23961][SPARK-27548][PYTHON] Fix error when toLocalIterator goes out of scope and properly raise errors from worker
## What changes were proposed in this pull request?

This fixes an error when a PySpark local iterator, for both RDD and DataFrames, goes out of scope and the connection is closed before fully consuming the iterator. The error occurs on the JVM in the serving thread, when Python closes the local socket while the JVM is writing to it. This usually happens when there is enough data to fill the socket read buffer, causing the write call to block.

Additionally, this fixes a problem when an error occurs in the Python worker and the collect job is cancelled with an exception. Previously, the Python driver was never notified of the error so the user could get a partial result (iteration until the error) and the application will continue. With this change, an error in the worker is sent to the Python iterator and is then raised.

The change here introduces a protocol for PySpark local iterators that work as follows:

1) The local socket connection is made when the iterator is created
2) When iterating, Python first sends a request for partition data as a non-zero integer
3) While the JVM local iterator over partitions has next, it triggers a job to collect the next partition
4) The JVM sends a nonzero response to indicate it has the next partition to send
5) The next partition is sent to Python and read by the PySpark deserializer
6) After sending the entire partition, an `END_OF_DATA_SECTION` is sent to Python which stops the deserializer and allows to make another request
7) When the JVM gets a request from Python but has already consumed it's local iterator, it will send a zero response to Python and both will close the socket cleanly
8) If an error occurs in the worker, a negative response is sent to Python followed by the error message. Python will then raise a RuntimeError with the message, stopping iteration.
9) When the PySpark local iterator is garbage-collected, it will read any remaining data from the current partition (this is data that has already been collected) and send a request of zero to tell the JVM to stop collection jobs and close the connection.

Steps 1, 3, 5, 6 are the same as before. Step 8 was completely missing before because errors in the worker were never communicated back to Python. The other steps add synchronization to allow for a clean closing of the socket, with a small trade-off in performance for each partition. This is mainly because the JVM does not start collecting partition data until it receives a request to do so, where before it would eagerly write all data until the socket receive buffer is full.

## How was this patch tested?

Added new unit tests for DataFrame and RDD `toLocalIterator` and tested not fully consuming the iterator. Manual tests with Python 2.7  and 3.6.

Closes #24070 from BryanCutler/pyspark-toLocalIterator-clean-stop-SPARK-23961.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-05-07 14:47:39 -07:00
..
docs [SPARK-27557][DOC] Add copy button to Python API docs for easier copying of code-blocks 2019-05-01 11:26:18 -05:00
lib [SPARK-25891][PYTHON] Upgrade to Py4J 0.10.8.1 2018-10-31 09:55:03 -07:00
pyspark [SPARK-23961][SPARK-27548][PYTHON] Fix error when toLocalIterator goes out of scope and properly raise errors from worker 2019-05-07 14:47:39 -07:00
test_coverage [SPARK-7721][PYTHON][TESTS] Adds PySpark coverage generation script 2018-01-22 22:12:50 +09:00
test_support [SPARK-23094][SPARK-23723][SPARK-23724][SQL] Support custom encoding for json files 2018-04-29 11:25:31 +08:00
.coveragerc [SPARK-7721][PYTHON][TESTS] Adds PySpark coverage generation script 2018-01-22 22:12:50 +09:00
.gitignore [SPARK-3946] gitignore in /python includes wrong directory 2014-10-14 14:09:39 -07:00
MANIFEST.in [SPARK-26803][PYTHON] Add sbin subdirectory to pyspark 2019-02-27 08:39:55 -06:00
pylintrc [SPARK-13596][BUILD] Move misc top-level build files into appropriate subdirs 2016-03-07 14:48:02 -08:00
README.md [SPARK-25891][PYTHON] Upgrade to Py4J 0.10.8.1 2018-10-31 09:55:03 -07:00
run-tests [SPARK-8583] [SPARK-5482] [BUILD] Refactor python/run-tests to integrate with dev/run-tests module system 2015-06-27 20:24:34 -07:00
run-tests-with-coverage [SPARK-26252][PYTHON] Add support to run specific unittests and/or doctests in python/run-tests script 2018-12-05 15:22:08 +08:00
run-tests.py [SPARK-25079][PYTHON] update python3 executable to 3.6.x 2019-04-19 10:03:50 +09:00
setup.cfg [SPARK-1267][SPARK-18129] Allow PySpark to be pip installed 2016-11-16 14:22:15 -08:00
setup.py [SPARK-27276][PYTHON][SQL] Increase minimum version of pyarrow to 0.12.1 and remove prior workarounds 2019-04-22 19:30:31 +09:00

Apache Spark

Spark is a fast and general cluster computing system for Big Data. It provides high-level APIs in Scala, Java, Python, and R, and an optimized engine that supports general computation graphs for data analysis. It also supports a rich set of higher-level tools including Spark SQL for SQL and DataFrames, MLlib for machine learning, GraphX for graph processing, and Spark Streaming for stream processing.

http://spark.apache.org/

Online Documentation

You can find the latest Spark documentation, including a programming guide, on the project web page

Python Packaging

This README file only contains basic information related to pip installed PySpark. This packaging is currently experimental and may change in future versions (although we will do our best to keep compatibility). Using PySpark requires the Spark JARs, and if you are building this from source please see the builder instructions at "Building Spark".

The Python packaging for Spark is not intended to replace all of the other use cases. This Python packaged version of Spark is suitable for interacting with an existing cluster (be it Spark standalone, YARN, or Mesos) - but does not contain the tools required to set up your own standalone Spark cluster. You can download the full version of Spark from the Apache Spark downloads page.

NOTE: If you are using this with a Spark standalone cluster you must ensure that the version (including minor version) matches or you may experience odd errors.

Python Requirements

At its core PySpark depends on Py4J (currently version 0.10.8.1), but some additional sub-packages have their own extra requirements for some features (including numpy, pandas, and pyarrow).