spark-instrumented-optimizer/python
Gabor Somogyi 0bb911d979 [SPARK-33143][PYTHON] Add configurable timeout to python server and client
### What changes were proposed in this pull request?
Spark creates local server to serialize several type of data for python. The python code tries to connect to the server, immediately after it's created but there are several system calls in between (this may change in each Spark version):
* getaddrinfo
* socket
* settimeout
* connect

Under some circumstances in heavy user environments these calls can be super slow (more than 15 seconds). These issues must be analyzed one-by-one but since these are system calls the underlying OS and/or DNS servers must be debugged and fixed. This is not trivial task and at the same time data processing must work somehow. In this PR I'm only intended to add a configuration possibility to increase the mentioned timeouts in order to be able to provide temporary workaround. The rootcause analysis is ongoing but I think this can vary in each case.

Because the server part doesn't contain huge amount of log entries to with one can measure time, I've added some.

### Why are the changes needed?
Provide workaround when localhost python server connection timeout appears.

### Does this PR introduce _any_ user-facing change?
Yes, new configuration added.

### How was this patch tested?
Existing unit tests + manual test.
```
#Compile Spark

echo "spark.io.encryption.enabled true" >> conf/spark-defaults.conf
echo "spark.python.authenticate.socketTimeout 10" >> conf/spark-defaults.conf

$ ./bin/pyspark
Python 3.8.5 (default, Jul 21 2020, 10:48:26)
[Clang 11.0.3 (clang-1103.0.32.62)] on darwin
Type "help", "copyright", "credits" or "license" for more information.
20/11/20 10:17:03 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
20/11/20 10:17:03 WARN SparkEnv: I/O encryption enabled without RPC encryption: keys will be visible on the wire.
Welcome to
      ____              __
     / __/__  ___ _____/ /__
    _\ \/ _ \/ _ `/ __/  '_/
   /__ / .__/\_,_/_/ /_/\_\   version 3.1.0-SNAPSHOT
      /_/

Using Python version 3.8.5 (default, Jul 21 2020 10:48:26)
Spark context Web UI available at http://192.168.0.189:4040
Spark context available as 'sc' (master = local[*], app id = local-1605863824276).
SparkSession available as 'spark'.
>>> sc.setLogLevel("TRACE")
>>> sc.parallelize([0, 2, 3, 4, 6], 5).glom().collect()
20/11/20 10:17:09 TRACE PythonParallelizeServer: Creating listening socket
20/11/20 10:17:09 TRACE PythonParallelizeServer: Setting timeout to 10 sec
20/11/20 10:17:09 TRACE PythonParallelizeServer: Waiting for connection on port 59726
20/11/20 10:17:09 TRACE PythonParallelizeServer: Connection accepted from address /127.0.0.1:59727
20/11/20 10:17:09 TRACE PythonParallelizeServer: Client authenticated
20/11/20 10:17:09 TRACE PythonParallelizeServer: Closing server
...
20/11/20 10:17:10 TRACE SocketFuncServer: Creating listening socket
20/11/20 10:17:10 TRACE SocketFuncServer: Setting timeout to 10 sec
20/11/20 10:17:10 TRACE SocketFuncServer: Waiting for connection on port 59735
20/11/20 10:17:10 TRACE SocketFuncServer: Connection accepted from address /127.0.0.1:59736
20/11/20 10:17:10 TRACE SocketFuncServer: Client authenticated
20/11/20 10:17:10 TRACE SocketFuncServer: Closing server
[[0], [2], [3], [4], [6]]
>>>
```

Closes #30389 from gaborgsomogyi/SPARK-33143.

Lead-authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Co-authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-23 15:19:34 +09:00
..
docs [MINOR][DOCS] Document 'without' value for HADOOP_VERSION in pip installation 2020-11-20 13:14:20 +09:00
lib [SPARK-30884][PYSPARK] Upgrade to Py4J 0.10.9 2020-02-20 09:09:30 -08:00
pyspark [SPARK-33143][PYTHON] Add configurable timeout to python server and client 2020-11-23 15:19:34 +09: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-32714][PYTHON] Initial pyspark-stubs port 2020-09-24 14:15:36 +09:00
mypy.ini [SPARK-33002][PYTHON] Remove non-API annotations 2020-10-07 19:53:59 +09:00
pylintrc [SPARK-32435][PYTHON] Remove heapq3 port from Python 3 2020-07-27 20:10:13 +09:00
README.md [SPARK-30884][PYSPARK] Upgrade to Py4J 0.10.9 2020-02-20 09:09:30 -08:00
run-tests [SPARK-29672][PYSPARK] update spark testing framework to use python3 2019-11-14 10:18:55 -08: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-33189][PYTHON][TESTS] Add env var to tests for legacy nested timestamps in pyarrow 2020-10-21 09:13:33 +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-33371][PYTHON] Update setup.py and tests for Python 3.9 2020-11-06 15:05:37 -08:00

Apache Spark

Spark is a unified analytics engine for large-scale data processing. 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 Structured Streaming for stream processing.

https://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, but some additional sub-packages have their own extra requirements for some features (including numpy, pandas, and pyarrow).