Commit graph

48 commits

Author SHA1 Message Date
zero323 5574734093 [SPARK-32138][FOLLOW-UP] Drop obsolete StringIO import branching
### What changes were proposed in this pull request?

Removal of branched `StringIO` import.

### Why are the changes needed?

Top level `StringIO` is no longer present in Python 3.x.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Existing tests.

Closes #29590 from zero323/SPARK-32138-FOLLOW-UP.

Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-31 16:56:50 +09:00
Fokko Driesprong a1e459ed9f [SPARK-32719][PYTHON] Add Flake8 check missing imports
https://issues.apache.org/jira/browse/SPARK-32719

### What changes were proposed in this pull request?

Add a check to detect missing imports. This makes sure that if we use a specific class, it should be explicitly imported (not using a wildcard).

### Why are the changes needed?

To make sure that the quality of the Python code is up to standard.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Existing unit-tests and Flake8 static analysis

Closes #29563 from Fokko/fd-add-check-missing-imports.

Authored-by: Fokko Driesprong <fokko@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-31 11:23:31 +09:00
Fokko Driesprong 9fcf0ea718 [SPARK-32319][PYSPARK] Disallow the use of unused imports
Disallow the use of unused imports:

- Unnecessary increases the memory footprint of the application
- Removes the imports that are required for the examples in the docstring from the file-scope to the example itself. This keeps the files itself clean, and gives a more complete example as it also includes the imports :)

```
fokkodriesprongFan spark % flake8 python | grep -i "imported but unused"
python/pyspark/cloudpickle.py:46:1: F401 'functools.partial' imported but unused
python/pyspark/cloudpickle.py:55:1: F401 'traceback' imported but unused
python/pyspark/heapq3.py:868:5: F401 '_heapq.*' imported but unused
python/pyspark/__init__.py:61:1: F401 'pyspark.version.__version__' imported but unused
python/pyspark/__init__.py:62:1: F401 'pyspark._globals._NoValue' imported but unused
python/pyspark/__init__.py:115:1: F401 'pyspark.sql.SQLContext' imported but unused
python/pyspark/__init__.py:115:1: F401 'pyspark.sql.HiveContext' imported but unused
python/pyspark/__init__.py:115:1: F401 'pyspark.sql.Row' imported but unused
python/pyspark/rdd.py:21:1: F401 're' imported but unused
python/pyspark/rdd.py:29:1: F401 'tempfile.NamedTemporaryFile' imported but unused
python/pyspark/mllib/regression.py:26:1: F401 'pyspark.mllib.linalg.SparseVector' imported but unused
python/pyspark/mllib/clustering.py:28:1: F401 'pyspark.mllib.linalg.SparseVector' imported but unused
python/pyspark/mllib/clustering.py:28:1: F401 'pyspark.mllib.linalg.DenseVector' imported but unused
python/pyspark/mllib/classification.py:26:1: F401 'pyspark.mllib.linalg.SparseVector' imported but unused
python/pyspark/mllib/feature.py:28:1: F401 'pyspark.mllib.linalg.DenseVector' imported but unused
python/pyspark/mllib/feature.py:28:1: F401 'pyspark.mllib.linalg.SparseVector' imported but unused
python/pyspark/mllib/feature.py:30:1: F401 'pyspark.mllib.regression.LabeledPoint' imported but unused
python/pyspark/mllib/tests/test_linalg.py:18:1: F401 'sys' imported but unused
python/pyspark/mllib/tests/test_linalg.py:642:5: F401 'pyspark.mllib.tests.test_linalg.*' imported but unused
python/pyspark/mllib/tests/test_feature.py:21:1: F401 'numpy.random' imported but unused
python/pyspark/mllib/tests/test_feature.py:21:1: F401 'numpy.exp' imported but unused
python/pyspark/mllib/tests/test_feature.py:23:1: F401 'pyspark.mllib.linalg.Vector' imported but unused
python/pyspark/mllib/tests/test_feature.py:23:1: F401 'pyspark.mllib.linalg.VectorUDT' imported but unused
python/pyspark/mllib/tests/test_feature.py:185:5: F401 'pyspark.mllib.tests.test_feature.*' imported but unused
python/pyspark/mllib/tests/test_util.py:97:5: F401 'pyspark.mllib.tests.test_util.*' imported but unused
python/pyspark/mllib/tests/test_stat.py:23:1: F401 'pyspark.mllib.linalg.Vector' imported but unused
python/pyspark/mllib/tests/test_stat.py:23:1: F401 'pyspark.mllib.linalg.SparseVector' imported but unused
python/pyspark/mllib/tests/test_stat.py:23:1: F401 'pyspark.mllib.linalg.DenseVector' imported but unused
python/pyspark/mllib/tests/test_stat.py:23:1: F401 'pyspark.mllib.linalg.VectorUDT' imported but unused
python/pyspark/mllib/tests/test_stat.py:23:1: F401 'pyspark.mllib.linalg._convert_to_vector' imported but unused
python/pyspark/mllib/tests/test_stat.py:23:1: F401 'pyspark.mllib.linalg.DenseMatrix' imported but unused
python/pyspark/mllib/tests/test_stat.py:23:1: F401 'pyspark.mllib.linalg.SparseMatrix' imported but unused
python/pyspark/mllib/tests/test_stat.py:23:1: F401 'pyspark.mllib.linalg.MatrixUDT' imported but unused
python/pyspark/mllib/tests/test_stat.py:181:5: F401 'pyspark.mllib.tests.test_stat.*' imported but unused
python/pyspark/mllib/tests/test_streaming_algorithms.py:18:1: F401 'time.time' imported but unused
python/pyspark/mllib/tests/test_streaming_algorithms.py:18:1: F401 'time.sleep' imported but unused
python/pyspark/mllib/tests/test_streaming_algorithms.py:470:5: F401 'pyspark.mllib.tests.test_streaming_algorithms.*' imported but unused
python/pyspark/mllib/tests/test_algorithms.py:295:5: F401 'pyspark.mllib.tests.test_algorithms.*' imported but unused
python/pyspark/tests/test_serializers.py:90:13: F401 'xmlrunner' imported but unused
python/pyspark/tests/test_rdd.py:21:1: F401 'sys' imported but unused
python/pyspark/tests/test_rdd.py:29:1: F401 'pyspark.resource.ResourceProfile' imported but unused
python/pyspark/tests/test_rdd.py:885:5: F401 'pyspark.tests.test_rdd.*' imported but unused
python/pyspark/tests/test_readwrite.py:19:1: F401 'sys' imported but unused
python/pyspark/tests/test_readwrite.py:22:1: F401 'array.array' imported but unused
python/pyspark/tests/test_readwrite.py:309:5: F401 'pyspark.tests.test_readwrite.*' imported but unused
python/pyspark/tests/test_join.py:62:5: F401 'pyspark.tests.test_join.*' imported but unused
python/pyspark/tests/test_taskcontext.py:19:1: F401 'shutil' imported but unused
python/pyspark/tests/test_taskcontext.py:325:5: F401 'pyspark.tests.test_taskcontext.*' imported but unused
python/pyspark/tests/test_conf.py:36:5: F401 'pyspark.tests.test_conf.*' imported but unused
python/pyspark/tests/test_broadcast.py:148:5: F401 'pyspark.tests.test_broadcast.*' imported but unused
python/pyspark/tests/test_daemon.py:76:5: F401 'pyspark.tests.test_daemon.*' imported but unused
python/pyspark/tests/test_util.py:77:5: F401 'pyspark.tests.test_util.*' imported but unused
python/pyspark/tests/test_pin_thread.py:19:1: F401 'random' imported but unused
python/pyspark/tests/test_pin_thread.py:149:5: F401 'pyspark.tests.test_pin_thread.*' imported but unused
python/pyspark/tests/test_worker.py:19:1: F401 'sys' imported but unused
python/pyspark/tests/test_worker.py:26:5: F401 'resource' imported but unused
python/pyspark/tests/test_worker.py:203:5: F401 'pyspark.tests.test_worker.*' imported but unused
python/pyspark/tests/test_profiler.py:101:5: F401 'pyspark.tests.test_profiler.*' imported but unused
python/pyspark/tests/test_shuffle.py:18:1: F401 'sys' imported but unused
python/pyspark/tests/test_shuffle.py:171:5: F401 'pyspark.tests.test_shuffle.*' imported but unused
python/pyspark/tests/test_rddbarrier.py:43:5: F401 'pyspark.tests.test_rddbarrier.*' imported but unused
python/pyspark/tests/test_context.py:129:13: F401 'userlibrary.UserClass' imported but unused
python/pyspark/tests/test_context.py:140:13: F401 'userlib.UserClass' imported but unused
python/pyspark/tests/test_context.py:310:5: F401 'pyspark.tests.test_context.*' imported but unused
python/pyspark/tests/test_appsubmit.py:241:5: F401 'pyspark.tests.test_appsubmit.*' imported but unused
python/pyspark/streaming/dstream.py:18:1: F401 'sys' imported but unused
python/pyspark/streaming/tests/test_dstream.py:27:1: F401 'pyspark.RDD' imported but unused
python/pyspark/streaming/tests/test_dstream.py:647:5: F401 'pyspark.streaming.tests.test_dstream.*' imported but unused
python/pyspark/streaming/tests/test_kinesis.py:83:5: F401 'pyspark.streaming.tests.test_kinesis.*' imported but unused
python/pyspark/streaming/tests/test_listener.py:152:5: F401 'pyspark.streaming.tests.test_listener.*' imported but unused
python/pyspark/streaming/tests/test_context.py:178:5: F401 'pyspark.streaming.tests.test_context.*' imported but unused
python/pyspark/testing/utils.py:30:5: F401 'scipy.sparse' imported but unused
python/pyspark/testing/utils.py:36:5: F401 'numpy as np' imported but unused
python/pyspark/ml/regression.py:25:1: F401 'pyspark.ml.tree._TreeEnsembleParams' imported but unused
python/pyspark/ml/regression.py:25:1: F401 'pyspark.ml.tree._HasVarianceImpurity' imported but unused
python/pyspark/ml/regression.py:29:1: F401 'pyspark.ml.wrapper.JavaParams' imported but unused
python/pyspark/ml/util.py:19:1: F401 'sys' imported but unused
python/pyspark/ml/__init__.py:25:1: F401 'pyspark.ml.pipeline' imported but unused
python/pyspark/ml/pipeline.py:18:1: F401 'sys' imported but unused
python/pyspark/ml/stat.py:22:1: F401 'pyspark.ml.linalg.DenseMatrix' imported but unused
python/pyspark/ml/stat.py:22:1: F401 'pyspark.ml.linalg.Vectors' imported but unused
python/pyspark/ml/tests/test_training_summary.py:18:1: F401 'sys' imported but unused
python/pyspark/ml/tests/test_training_summary.py:364:5: F401 'pyspark.ml.tests.test_training_summary.*' imported but unused
python/pyspark/ml/tests/test_linalg.py:381:5: F401 'pyspark.ml.tests.test_linalg.*' imported but unused
python/pyspark/ml/tests/test_tuning.py:427:9: F401 'pyspark.sql.functions as F' imported but unused
python/pyspark/ml/tests/test_tuning.py:757:5: F401 'pyspark.ml.tests.test_tuning.*' imported but unused
python/pyspark/ml/tests/test_wrapper.py:120:5: F401 'pyspark.ml.tests.test_wrapper.*' imported but unused
python/pyspark/ml/tests/test_feature.py:19:1: F401 'sys' imported but unused
python/pyspark/ml/tests/test_feature.py:304:5: F401 'pyspark.ml.tests.test_feature.*' imported but unused
python/pyspark/ml/tests/test_image.py:19:1: F401 'py4j' imported but unused
python/pyspark/ml/tests/test_image.py:22:1: F401 'pyspark.testing.mlutils.PySparkTestCase' imported but unused
python/pyspark/ml/tests/test_image.py:71:5: F401 'pyspark.ml.tests.test_image.*' imported but unused
python/pyspark/ml/tests/test_persistence.py:456:5: F401 'pyspark.ml.tests.test_persistence.*' imported but unused
python/pyspark/ml/tests/test_evaluation.py:56:5: F401 'pyspark.ml.tests.test_evaluation.*' imported but unused
python/pyspark/ml/tests/test_stat.py:43:5: F401 'pyspark.ml.tests.test_stat.*' imported but unused
python/pyspark/ml/tests/test_base.py:70:5: F401 'pyspark.ml.tests.test_base.*' imported but unused
python/pyspark/ml/tests/test_param.py:20:1: F401 'sys' imported but unused
python/pyspark/ml/tests/test_param.py:375:5: F401 'pyspark.ml.tests.test_param.*' imported but unused
python/pyspark/ml/tests/test_pipeline.py:62:5: F401 'pyspark.ml.tests.test_pipeline.*' imported but unused
python/pyspark/ml/tests/test_algorithms.py:333:5: F401 'pyspark.ml.tests.test_algorithms.*' imported but unused
python/pyspark/ml/param/__init__.py:18:1: F401 'sys' imported but unused
python/pyspark/resource/tests/test_resources.py:17:1: F401 'random' imported but unused
python/pyspark/resource/tests/test_resources.py:20:1: F401 'pyspark.resource.ResourceProfile' imported but unused
python/pyspark/resource/tests/test_resources.py:75:5: F401 'pyspark.resource.tests.test_resources.*' imported but unused
python/pyspark/sql/functions.py:32:1: F401 'pyspark.sql.udf.UserDefinedFunction' imported but unused
python/pyspark/sql/functions.py:34:1: F401 'pyspark.sql.pandas.functions.pandas_udf' imported but unused
python/pyspark/sql/session.py:30:1: F401 'pyspark.sql.types.Row' imported but unused
python/pyspark/sql/session.py:30:1: F401 'pyspark.sql.types.StringType' imported but unused
python/pyspark/sql/readwriter.py:1084:5: F401 'pyspark.sql.Row' imported but unused
python/pyspark/sql/context.py:26:1: F401 'pyspark.sql.types.IntegerType' imported but unused
python/pyspark/sql/context.py:26:1: F401 'pyspark.sql.types.Row' imported but unused
python/pyspark/sql/context.py:26:1: F401 'pyspark.sql.types.StringType' imported but unused
python/pyspark/sql/context.py:27:1: F401 'pyspark.sql.udf.UDFRegistration' imported but unused
python/pyspark/sql/streaming.py:1212:5: F401 'pyspark.sql.Row' imported but unused
python/pyspark/sql/tests/test_utils.py:55:5: F401 'pyspark.sql.tests.test_utils.*' imported but unused
python/pyspark/sql/tests/test_pandas_map.py:18:1: F401 'sys' imported but unused
python/pyspark/sql/tests/test_pandas_map.py:22:1: F401 'pyspark.sql.functions.pandas_udf' imported but unused
python/pyspark/sql/tests/test_pandas_map.py:22:1: F401 'pyspark.sql.functions.PandasUDFType' imported but unused
python/pyspark/sql/tests/test_pandas_map.py:119:5: F401 'pyspark.sql.tests.test_pandas_map.*' imported but unused
python/pyspark/sql/tests/test_catalog.py:193:5: F401 'pyspark.sql.tests.test_catalog.*' imported but unused
python/pyspark/sql/tests/test_group.py:39:5: F401 'pyspark.sql.tests.test_group.*' imported but unused
python/pyspark/sql/tests/test_session.py:361:5: F401 'pyspark.sql.tests.test_session.*' imported but unused
python/pyspark/sql/tests/test_conf.py:49:5: F401 'pyspark.sql.tests.test_conf.*' imported but unused
python/pyspark/sql/tests/test_pandas_cogrouped_map.py:19:1: F401 'sys' imported but unused
python/pyspark/sql/tests/test_pandas_cogrouped_map.py:21:1: F401 'pyspark.sql.functions.sum' imported but unused
python/pyspark/sql/tests/test_pandas_cogrouped_map.py:21:1: F401 'pyspark.sql.functions.PandasUDFType' imported but unused
python/pyspark/sql/tests/test_pandas_cogrouped_map.py:29:5: F401 'pandas.util.testing.assert_series_equal' imported but unused
python/pyspark/sql/tests/test_pandas_cogrouped_map.py:32:5: F401 'pyarrow as pa' imported but unused
python/pyspark/sql/tests/test_pandas_cogrouped_map.py:248:5: F401 'pyspark.sql.tests.test_pandas_cogrouped_map.*' imported but unused
python/pyspark/sql/tests/test_udf.py:24:1: F401 'py4j' imported but unused
python/pyspark/sql/tests/test_pandas_udf_typehints.py:246:5: F401 'pyspark.sql.tests.test_pandas_udf_typehints.*' imported but unused
python/pyspark/sql/tests/test_functions.py:19:1: F401 'sys' imported but unused
python/pyspark/sql/tests/test_functions.py:362:9: F401 'pyspark.sql.functions.exists' imported but unused
python/pyspark/sql/tests/test_functions.py:387:5: F401 'pyspark.sql.tests.test_functions.*' imported but unused
python/pyspark/sql/tests/test_pandas_udf_scalar.py:21:1: F401 'sys' imported but unused
python/pyspark/sql/tests/test_pandas_udf_scalar.py:45:5: F401 'pyarrow as pa' imported but unused
python/pyspark/sql/tests/test_pandas_udf_window.py:355:5: F401 'pyspark.sql.tests.test_pandas_udf_window.*' imported but unused
python/pyspark/sql/tests/test_arrow.py:38:5: F401 'pyarrow as pa' imported but unused
python/pyspark/sql/tests/test_pandas_grouped_map.py:20:1: F401 'sys' imported but unused
python/pyspark/sql/tests/test_pandas_grouped_map.py:38:5: F401 'pyarrow as pa' imported but unused
python/pyspark/sql/tests/test_dataframe.py:382:9: F401 'pyspark.sql.DataFrame' imported but unused
python/pyspark/sql/avro/functions.py:125:5: F401 'pyspark.sql.Row' imported but unused
python/pyspark/sql/pandas/functions.py:19:1: F401 'sys' imported but unused
```

After:
```
fokkodriesprongFan spark % flake8 python | grep -i "imported but unused"
fokkodriesprongFan spark %
```

### What changes were proposed in this pull request?

Removing unused imports from the Python files to keep everything nice and tidy.

### Why are the changes needed?

Cleaning up of the imports that aren't used, and suppressing the imports that are used as references to other modules, preserving backward compatibility.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Adding the rule to the existing Flake8 checks.

Closes #29121 from Fokko/SPARK-32319.

Authored-by: Fokko Driesprong <fokko@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-08 08:51:57 -07:00
Takuya UESHIN 7deb67c28f [SPARK-32160][CORE][PYSPARK][FOLLOWUP] Change the config name to switch allow/disallow SparkContext in executors
### What changes were proposed in this pull request?

This is a follow-up of #29278.
This PR changes the config name to switch allow/disallow `SparkContext` in executors as per the comment https://github.com/apache/spark/pull/29278#pullrequestreview-460256338.

### Why are the changes needed?

The config name `spark.executor.allowSparkContext` is more reasonable.

### Does this PR introduce _any_ user-facing change?

Yes, the config name is changed.

### How was this patch tested?

Updated tests.

Closes #29340 from ueshin/issues/SPARK-32160/change_config_name.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-04 12:45:06 +09:00
Takuya UESHIN 8014b0b5d6 [SPARK-32160][CORE][PYSPARK] Add a config to switch allow/disallow to create SparkContext in executors
### What changes were proposed in this pull request?

This is a follow-up of #28986.
This PR adds a config to switch allow/disallow to create `SparkContext` in executors.

- `spark.driver.allowSparkContextInExecutors`

### Why are the changes needed?

Some users or libraries actually create `SparkContext` in executors.
We shouldn't break their workloads.

### Does this PR introduce _any_ user-facing change?

Yes, users will be able to create `SparkContext` in executors with the config enabled.

### How was this patch tested?

More tests are added.

Closes #29278 from ueshin/issues/SPARK-32160/add_configs.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-31 17:28:35 +09:00
HyukjinKwon 89d9b7cc64 [SPARK-32010][PYTHON][CORE] Add InheritableThread for local properties and fixing a thread leak issue in pinned thread mode
### What changes were proposed in this pull request?

This PR proposes:

1. To introduce `InheritableThread` class, that works identically with `threading.Thread` but it can inherit the inheritable attributes of a JVM thread such as `InheritableThreadLocal`.

    This was a problem from the pinned thread mode, see also https://github.com/apache/spark/pull/24898. Now it works as below:

    ```python
    import pyspark

    spark.sparkContext.setLocalProperty("a", "hi")
    def print_prop():
        print(spark.sparkContext.getLocalProperty("a"))

    pyspark.InheritableThread(target=print_prop).start()
    ```

    ```
    hi
    ```

2. Also, it adds the resource leak fix into `InheritableThread`. Py4J leaks the thread and does not close the connection from Python to JVM. In `InheritableThread`, it manually closes the connections when PVM garbage collection happens. So, JVM threads finish safely. I manually verified by profiling but there's also another easy way to verify:

    ```bash
    PYSPARK_PIN_THREAD=true ./bin/pyspark
    ```

    ```python
    >>> from threading import Thread
    >>> Thread(target=lambda: spark.range(1000).collect()).start()
    >>> Thread(target=lambda: spark.range(1000).collect()).start()
    >>> Thread(target=lambda: spark.range(1000).collect()).start()
    >>> spark._jvm._gateway_client.deque
    deque([<py4j.clientserver.ClientServerConnection object at 0x119f7aba8>, <py4j.clientserver.ClientServerConnection object at 0x119fc9b70>, <py4j.clientserver.ClientServerConnection object at 0x119fc9e10>, <py4j.clientserver.ClientServerConnection object at 0x11a015358>, <py4j.clientserver.ClientServerConnection object at 0x119fc00f0>])
    >>> Thread(target=lambda: spark.range(1000).collect()).start()
    >>> spark._jvm._gateway_client.deque
    deque([<py4j.clientserver.ClientServerConnection object at 0x119f7aba8>, <py4j.clientserver.ClientServerConnection object at 0x119fc9b70>, <py4j.clientserver.ClientServerConnection object at 0x119fc9e10>, <py4j.clientserver.ClientServerConnection object at 0x11a015358>, <py4j.clientserver.ClientServerConnection object at 0x119fc08d0>, <py4j.clientserver.ClientServerConnection object at 0x119fc00f0>])
    ```

    This issue is fixed now.

3. Because now we have a fix for the issue here, it also proposes to deprecate `collectWithJobGroup` which was a temporary workaround added to avoid this leak issue.

### Why are the changes needed?

To support pinned thread mode properly without a resource leak, and a proper inheritable local properties.

### Does this PR introduce _any_ user-facing change?

Yes, it adds an API `InheritableThread` class for pinned thread mode.

### How was this patch tested?

Manually tested as described above, and unit test was added as well.

Closes #28968 from HyukjinKwon/SPARK-32010.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-30 10:15:25 +09:00
HyukjinKwon 4ad9bfd53b [SPARK-32138] Drop Python 2.7, 3.4 and 3.5
### What changes were proposed in this pull request?

This PR aims to drop Python 2.7, 3.4 and 3.5.

Roughly speaking, it removes all the widely known Python 2 compatibility workarounds such as `sys.version` comparison, `__future__`. Also, it removes the Python 2 dedicated codes such as `ArrayConstructor` in Spark.

### Why are the changes needed?

 1. Unsupport EOL Python versions
 2. Reduce maintenance overhead and remove a bit of legacy codes and hacks for Python 2.
 3. PyPy2 has a critical bug that causes a flaky test, SPARK-28358 given my testing and investigation.
 4. Users can use Python type hints with Pandas UDFs without thinking about Python version
 5. Users can leverage one latest cloudpickle, https://github.com/apache/spark/pull/28950. With Python 3.8+ it can also leverage C pickle.

### Does this PR introduce _any_ user-facing change?

Yes, users cannot use Python 2.7, 3.4 and 3.5 in the upcoming Spark version.

### How was this patch tested?

Manually tested and also tested in Jenkins.

Closes #28957 from HyukjinKwon/SPARK-32138.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-14 11:22:44 +09:00
Takuya UESHIN cfecc2030d [SPARK-32160][CORE][PYSPARK] Disallow to create SparkContext in executors
### What changes were proposed in this pull request?

This PR proposes to disallow to create `SparkContext` in executors, e.g., in UDFs.

### Why are the changes needed?

Currently executors can create SparkContext, but shouldn't be able to create it.

```scala
sc.range(0, 1).foreach { _ =>
  new SparkContext(new SparkConf().setAppName("test").setMaster("local"))
}
```

### Does this PR introduce _any_ user-facing change?

Yes, users won't be able to create `SparkContext` in executors.

### How was this patch tested?

Addes tests.

Closes #28986 from ueshin/issues/SPARK-32160/disallow_spark_context_in_executors.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-09 15:51:56 +09:00
HyukjinKwon 29c51d682b [SPARK-31788][CORE][DSTREAM][PYTHON] Recover the support of union for different types of RDD and DStreams
### What changes were proposed in this pull request?

This PR manually specifies the class for the input array being used in `(SparkContext|StreamingContext).union`. It fixes a regression introduced from SPARK-25737.

```python
rdd1 = sc.parallelize([1,2,3,4,5])
rdd2 = sc.parallelize([6,7,8,9,10])
pairRDD1 = rdd1.zip(rdd2)
sc.union([pairRDD1, pairRDD1]).collect()
```

in the current master and `branch-3.0`:

```
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/.../spark/python/pyspark/context.py", line 870, in union
    jrdds[i] = rdds[i]._jrdd
  File "/.../spark/python/lib/py4j-0.10.9-src.zip/py4j/java_collections.py", line 238, in __setitem__
  File "/.../spark/python/lib/py4j-0.10.9-src.zip/py4j/java_collections.py", line 221, in __set_item
  File "/.../spark/python/lib/py4j-0.10.9-src.zip/py4j/protocol.py", line 332, in get_return_value
py4j.protocol.Py4JError: An error occurred while calling None.None. Trace:
py4j.Py4JException: Cannot convert org.apache.spark.api.java.JavaPairRDD to org.apache.spark.api.java.JavaRDD
	at py4j.commands.ArrayCommand.convertArgument(ArrayCommand.java:166)
	at py4j.commands.ArrayCommand.setArray(ArrayCommand.java:144)
	at py4j.commands.ArrayCommand.execute(ArrayCommand.java:97)
	at py4j.GatewayConnection.run(GatewayConnection.java:238)
	at java.lang.Thread.run(Thread.java:748)
```

which works in Spark 2.4.5:

```
[(1, 6), (2, 7), (3, 8), (4, 9), (5, 10), (1, 6), (2, 7), (3, 8), (4, 9), (5, 10)]
```

It assumed the class of the input array is the same `JavaRDD` or `JavaDStream`; however, that can be different such as `JavaPairRDD`.

This fix is based on redsanket's initial approach, and will be co-authored.

### Why are the changes needed?

To fix a regression from Spark 2.4.5.

### Does this PR introduce _any_ user-facing change?

No, it's only in unreleased branches. This is to fix a regression.

### How was this patch tested?

Manually tested, and a unittest was added.

Closes #28648 from HyukjinKwon/SPARK-31788.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-06-01 09:43:03 +09:00
HyukjinKwon 7fb2275f00 Revert "[SPARK-31788][CORE][PYTHON] Fix UnionRDD of PairRDDs"
This reverts commit a61911c50c.
2020-05-27 10:15:33 +09:00
schintap a61911c50c [SPARK-31788][CORE][PYTHON] Fix UnionRDD of PairRDDs
### What changes were proposed in this pull request?
UnionRDD of PairRDDs causing a bug. The fix is to check for instance type before proceeding

### Why are the changes needed?
Changes are needed to avoid users running into issues with union rdd operation with any other type other than JavaRDD.

### Does this PR introduce _any_ user-facing change?
Yes

Before:
SparkSession available as 'spark'.
>>> rdd1 = sc.parallelize([1,2,3,4,5])
>>> rdd2 = sc.parallelize([6,7,8,9,10])
>>> pairRDD1 = rdd1.zip(rdd2)
>>> unionRDD1 = sc.union([pairRDD1, pairRDD1])
Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/home/gs/spark/latest/python/pyspark/context.py", line 870,
in union jrdds[i] = rdds[i]._jrdd
File "/home/gs/spark/latest/python/lib/py4j-0.10.9-src.zip/py4j/java_collections.py", line 238, in setitem File "/home/gs/spark/latest/python/lib/py4j-0.10.9-src.zip/py4j/java_collections.py", line 221,
in __set_item File "/home/gs/spark/latest/python/lib/py4j-0.10.9-src.zip/py4j/protocol.py", line 332, in get_return_value py4j.protocol.Py4JError: An error occurred while calling None.None. Trace: py4j.Py4JException: Cannot convert org.apache.spark.api.java.JavaPairRDD to org.apache.spark.api.java.JavaRDD at py4j.commands.ArrayCommand.convertArgument(ArrayCommand.java:166) at py4j.commands.ArrayCommand.setArray(ArrayCommand.java:144) at py4j.commands.ArrayCommand.execute(ArrayCommand.java:97) at py4j.GatewayConnection.run(GatewayConnection.java:238) at java.lang.Thread.run(Thread.java:748)

After:
>>> rdd2 = sc.parallelize([6,7,8,9,10])
>>> pairRDD1 = rdd1.zip(rdd2)
>>> unionRDD1 = sc.union([pairRDD1, pairRDD1])
>>> unionRDD1.collect()
[(1, 6), (2, 7), (3, 8), (4, 9), (5, 10), (1, 6), (2, 7), (3, 8), (4, 9), (5, 10)]

### How was this patch tested?
Tested with the reproduced piece of code above manually

Closes #28603 from redsanket/SPARK-31788.

Authored-by: schintap <schintap@verizonmedia.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-05-25 10:29:08 +09:00
Weichen Xu ee1de66fe4 [SPARK-31549][PYSPARK] Add a develop API invoking collect on Python RDD with user-specified job group
### What changes were proposed in this pull request?
I add a new API in pyspark RDD class:

def collectWithJobGroup(self, groupId, description, interruptOnCancel=False)

This API do the same thing with `rdd.collect`, but it can specify the job group when do collect.
The purpose of adding this API is, if we use:

```
sc.setJobGroup("group-id...")
rdd.collect()
```
The `setJobGroup` API in pyspark won't work correctly. This related to a bug discussed in
https://issues.apache.org/jira/browse/SPARK-31549

Note:

This PR is a rather temporary workaround for `PYSPARK_PIN_THREAD`, and as a step to migrate to  `PYSPARK_PIN_THREAD` smoothly. It targets Spark 3.0.

- `PYSPARK_PIN_THREAD` is unstable at this moment that affects whole PySpark applications.
- It is impossible to make it runtime configuration as it has to be set before JVM is launched.
- There is a thread leak issue between Python and JVM. We should address but it's not a release blocker for Spark 3.0 since the feature is experimental. I plan to handle this after Spark 3.0 due to stability.

Once `PYSPARK_PIN_THREAD` is enabled by default, we should remove this API out ideally. I will target to deprecate this API in Spark 3.1.

### Why are the changes needed?
Fix bug.

### Does this PR introduce any user-facing change?
A develop API in pyspark: `pyspark.RDD. collectWithJobGroup`

### How was this patch tested?
Unit test.

Closes #28395 from WeichenXu123/collect_with_job_group.

Authored-by: Weichen Xu <weichen.xu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-05-01 10:08:16 +09:00
Thomas Graves 95aec091e4 [SPARK-29641][PYTHON][CORE] Stage Level Sched: Add python api's and tests
### What changes were proposed in this pull request?

As part of the Stage level scheduling features, add the Python api's to set resource profiles.
This also adds the functionality to properly apply the pyspark memory configuration when specified in the ResourceProfile. The pyspark memory configuration is being passed in the task local properties. This was an easy way to get it to the PythonRunner that needs it. I modeled this off how the barrier task scheduling is passing the addresses. As part of this I added in the JavaRDD api's because those are needed by python.

### Why are the changes needed?

python api for this feature

### Does this PR introduce any user-facing change?

Yes adds the java and python apis for user to specify a ResourceProfile to use stage level scheduling.

### How was this patch tested?

unit tests and manually tested on yarn. Tests also run to verify it errors properly on standalone and local mode where its not yet supported.

Closes #28085 from tgravescs/SPARK-29641-pr-base.

Lead-authored-by: Thomas Graves <tgraves@nvidia.com>
Co-authored-by: Thomas Graves <tgraves@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-04-23 10:20:39 +09:00
yi.wu b517f991fe [SPARK-30969][CORE] Remove resource coordination support from Standalone
### What changes were proposed in this pull request?

Remove automatically resource coordination support from Standalone.

### Why are the changes needed?

Resource coordination is mainly designed for the scenario where multiple workers launched on the same host. However, it's, actually, a non-existed  scenario for today's Spark. Because, Spark now can start multiple executors in a single Worker, while it only allow one executor per Worker at very beginning. So, now, it really help nothing for user to launch multiple workers on the same host. Thus, it's not worth for us to bring over complicated implementation and potential high maintain cost for such an impossible scenario.

### Does this PR introduce any user-facing change?

No, it's Spark 3.0 feature.

### How was this patch tested?

Pass Jenkins.

Closes #27722 from Ngone51/abandon_coordination.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2020-03-02 11:23:07 -08:00
sarthfrey-db 274b328f57 [SPARK-30667][CORE] Add all gather method to BarrierTaskContext
Fix for #27395

### What changes were proposed in this pull request?

The `allGather` method is added to the `BarrierTaskContext`. This method contains the same functionality as the `BarrierTaskContext.barrier` method; it blocks the task until all tasks make the call, at which time they may continue execution. In addition, the `allGather` method takes an input message. Upon returning from the `allGather` the task receives a list of all the messages sent by all the tasks that made the `allGather` call.

### Why are the changes needed?

There are many situations where having the tasks communicate in a synchronized way is useful. One simple example is if each task needs to start a server to serve requests from one another; first the tasks must find a free port (the result of which is undetermined beforehand) and then start making requests, but to do so they each must know the port chosen by the other task. An `allGather` method would allow them to inform each other of the port they will run on.

### Does this PR introduce any user-facing change?

Yes, an `BarrierTaskContext.allGather` method will be available through the Scala, Java, and Python APIs.

### How was this patch tested?

Most of the code path is already covered by tests to the `barrier` method, since this PR includes a refactor so that much code is shared by the `barrier` and `allGather` methods. However, a test is added to assert that an all gather on each tasks partition ID will return a list of every partition ID.

An example through the Python API:
```python
>>> from pyspark import BarrierTaskContext
>>>
>>> def f(iterator):
...     context = BarrierTaskContext.get()
...     return [context.allGather('{}'.format(context.partitionId()))]
...
>>> sc.parallelize(range(4), 4).barrier().mapPartitions(f).collect()[0]
[u'3', u'1', u'0', u'2']
```

Closes #27640 from sarthfrey/master.

Lead-authored-by: sarthfrey-db <sarth.frey@databricks.com>
Co-authored-by: sarthfrey <sarth.frey@gmail.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2020-02-21 11:40:28 -08:00
Xingbo Jiang e32411eb07 Revert "[SPARK-30667][CORE] Add allGather method to BarrierTaskContext"
This reverts commit af63971cb7.
2020-02-19 17:04:47 -08:00
sarthfrey-db af63971cb7 [SPARK-30667][CORE] Add allGather method to BarrierTaskContext
### What changes were proposed in this pull request?

The `allGather` method is added to the `BarrierTaskContext`. This method contains the same functionality as the `BarrierTaskContext.barrier` method; it blocks the task until all tasks make the call, at which time they may continue execution. In addition, the `allGather` method takes an input message. Upon returning from the `allGather` the task receives a list of all the messages sent by all the tasks that made the `allGather` call.

### Why are the changes needed?

There are many situations where having the tasks communicate in a synchronized way is useful. One simple example is if each task needs to start a server to serve requests from one another; first the tasks must find a free port (the result of which is undetermined beforehand) and then start making requests, but to do so they each must know the port chosen by the other task. An `allGather` method would allow them to inform each other of the port they will run on.

### Does this PR introduce any user-facing change?

Yes, an `BarrierTaskContext.allGather` method will be available through the Scala, Java, and Python APIs.

### How was this patch tested?

Most of the code path is already covered by tests to the `barrier` method, since this PR includes a refactor so that much code is shared by the `barrier` and `allGather` methods. However, a test is added to assert that an all gather on each tasks partition ID will return a list of every partition ID.

An example through the Python API:
```python
>>> from pyspark import BarrierTaskContext
>>>
>>> def f(iterator):
...     context = BarrierTaskContext.get()
...     return [context.allGather('{}'.format(context.partitionId()))]
...
>>> sc.parallelize(range(4), 4).barrier().mapPartitions(f).collect()[0]
[u'3', u'1', u'0', u'2']
```

Closes #27395 from sarthfrey/master.

Lead-authored-by: sarthfrey-db <sarth.frey@databricks.com>
Co-authored-by: sarthfrey <sarth.frey@gmail.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
(cherry picked from commit 57254c9719)
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2020-02-19 12:10:51 -08:00
Xingbo Jiang fa3517cdb1 Revert "[SPARK-30667][CORE] Add allGather method to BarrierTaskContext"
This reverts commit 57254c9719.
2020-02-13 17:43:55 -08:00
sarthfrey-db 57254c9719 [SPARK-30667][CORE] Add allGather method to BarrierTaskContext
### What changes were proposed in this pull request?

The `allGather` method is added to the `BarrierTaskContext`. This method contains the same functionality as the `BarrierTaskContext.barrier` method; it blocks the task until all tasks make the call, at which time they may continue execution. In addition, the `allGather` method takes an input message. Upon returning from the `allGather` the task receives a list of all the messages sent by all the tasks that made the `allGather` call.

### Why are the changes needed?

There are many situations where having the tasks communicate in a synchronized way is useful. One simple example is if each task needs to start a server to serve requests from one another; first the tasks must find a free port (the result of which is undetermined beforehand) and then start making requests, but to do so they each must know the port chosen by the other task. An `allGather` method would allow them to inform each other of the port they will run on.

### Does this PR introduce any user-facing change?

Yes, an `BarrierTaskContext.allGather` method will be available through the Scala, Java, and Python APIs.

### How was this patch tested?

Most of the code path is already covered by tests to the `barrier` method, since this PR includes a refactor so that much code is shared by the `barrier` and `allGather` methods. However, a test is added to assert that an all gather on each tasks partition ID will return a list of every partition ID.

An example through the Python API:
```python
>>> from pyspark import BarrierTaskContext
>>>
>>> def f(iterator):
...     context = BarrierTaskContext.get()
...     return [context.allGather('{}'.format(context.partitionId()))]
...
>>> sc.parallelize(range(4), 4).barrier().mapPartitions(f).collect()[0]
[u'3', u'1', u'0', u'2']
```

Closes #27395 from sarthfrey/master.

Lead-authored-by: sarthfrey-db <sarth.frey@databricks.com>
Co-authored-by: sarthfrey <sarth.frey@gmail.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2020-02-13 16:15:00 -08:00
Thomas Graves 496f6ac860 [SPARK-29148][CORE] Add stage level scheduling dynamic allocation and scheduler backend changes
### What changes were proposed in this pull request?

This is another PR for stage level scheduling. In particular this adds changes to the dynamic allocation manager and the scheduler backend to be able to track what executors are needed per ResourceProfile.  Note the api is still private to Spark until the entire feature gets in, so this functionality will be there but only usable by tests for profiles other then the DefaultProfile.

The main changes here are simply tracking things on a ResourceProfile basis as well as sending the executor requests to the scheduler backend for all ResourceProfiles.

I introduce a ResourceProfileManager in this PR that will track all the actual ResourceProfile objects so that we can keep them all in a single place and just pass around and use in datastructures the resource profile id. The resource profile id can be used with the ResourceProfileManager to get the actual ResourceProfile contents.

There are various places in the code that use executor "slots" for things.  The ResourceProfile adds functionality to keep that calculation in it.   This logic is more complex then it should due to standalone mode and mesos coarse grained not setting the executor cores config. They default to all cores on the worker, so calculating slots is harder there.
This PR keeps the functionality to make the cores the limiting resource because the scheduler still uses that for "slots" for a few things.

This PR does also add the resource profile id to the Stage and stage info classes to be able to test things easier.   That full set of changes will come with the scheduler PR that will be after this one.

The PR stops at the scheduler backend pieces for the cluster manager and the real YARN support hasn't been added in this PR, that again will be in a separate PR, so this has a few of the API changes up to the cluster manager and then just uses the default profile requests to continue.

The code for the entire feature is here for reference: https://github.com/apache/spark/pull/27053/files although it needs to be upmerged again as well.

### Why are the changes needed?

Needed for stage level scheduling feature.

### Does this PR introduce any user-facing change?

No user facing api changes added here.

### How was this patch tested?

Lots of unit tests and manually testing. I tested on yarn, k8s, standalone, local modes. Ran both failure and success cases.

Closes #27313 from tgravescs/SPARK-29148.

Authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-02-12 16:45:42 -06:00
HyukjinKwon 0823aec463 [SPARK-30480][PYTHON][TESTS] Increases the memory limit being tested in 'WorkerMemoryTest.test_memory_limit'
### What changes were proposed in this pull request?

This PR proposes to increase the memory in `WorkerMemoryTest.test_memory_limit` in order to make the test pass with PyPy.

The test is currently failed only in PyPy as below in some PRs unexpectedly:

```
Current mem limits: 18446744073709551615 of max 18446744073709551615

Setting mem limits to 1048576 of max 1048576

RPython traceback:
  File "pypy_module_pypyjit_interp_jit.c", line 289, in portal_5
  File "pypy_interpreter_pyopcode.c", line 3468, in handle_bytecode__AccessDirect_None
  File "pypy_interpreter_pyopcode.c", line 5558, in dispatch_bytecode__AccessDirect_None
out of memory: couldn't allocate the next arena
ERROR
```

It seems related to how PyPy allocates the memory and GC works PyPy-specifically. There seems nothing wrong in this configuration implementation itself in PySpark side.

I roughly tested in higher PyPy versions on Ubuntu (PyPy v7.3.0) and this test seems passing fine so I suspect this might be an issue in old PyPy behaviours.

The change only increases the limit so it would not affect actual memory allocations. It just needs to test if the limit is properly set in worker sides. For clarification, the memory is unlimited in the machine if not set.

### Why are the changes needed?

To make the tests pass and unblock other PRs.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Manually and Jenkins should test it out.

Closes #27186 from HyukjinKwon/SPARK-30480.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-13 18:47:15 +09:00
HyukjinKwon d0983af38f Revert "[SPARK-30480][PYSPARK][TESTS] Fix 'test_memory_limit' on pyspark test"
This reverts commit afd70a0f6f.
2020-01-10 22:35:54 +09:00
Jungtaek Lim (HeartSaVioR) afd70a0f6f [SPARK-30480][PYSPARK][TESTS] Fix 'test_memory_limit' on pyspark test
### What changes were proposed in this pull request?

This patch increases the memory limit in the test 'test_memory_limit' from 1m to 8m.
Credit to srowen and HyukjinKwon to provide the idea of suspicion and guide how to fix.

### Why are the changes needed?

We observed consistent Pyspark test failures on multiple PRs (#26955, #26201, #27064) which block the PR builds whenever the test is included.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Jenkins builds passed in WIP PR (#27159)

Closes #27162 from HeartSaVioR/SPARK-30480.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-10 15:30:54 +09:00
HyukjinKwon 4ec04e5ef3 [SPARK-22340][PYTHON] Add a mode to pin Python thread into JVM's
## What changes were proposed in this pull request?

This PR proposes to add **Single threading model design (pinned thread model)** mode which is an experimental mode to sync threads on PVM and JVM. See https://www.py4j.org/advanced_topics.html#using-single-threading-model-pinned-thread

### Multi threading model

Currently, PySpark uses this model. Threads on PVM and JVM are independent. For instance, in a different Python thread, callbacks are received and relevant Python codes are executed. JVM threads are reused when possible.

Py4J will create a new thread every time a command is received and there is no thread available. See the current model we're using - https://www.py4j.org/advanced_topics.html#the-multi-threading-model

One problem in this model is that we can't sync threads on PVM and JVM out of the box. This leads to some problems in particular at some codes related to threading in JVM side. See:
7056e004ee/core/src/main/scala/org/apache/spark/SparkContext.scala (L334)
Due to reusing JVM threads, seems the job groups in Python threads cannot be set in each thread as described in the JIRA.

### Single threading model design (pinned thread model)

This mode pins and syncs the threads on PVM and JVM to work around the problem above. For instance, in the same Python thread, callbacks are received and relevant Python codes are executed. See https://www.py4j.org/advanced_topics.html#the-single-threading-model

Even though this mode can sync threads on PVM and JVM for other thread related code paths,
 this might cause another problem: seems unable to inherit properties as below (assuming multi-thread mode still creates new threads when existing threads are busy, I suspect this issue already exists when multiple jobs are submitted in multi-thread mode; however, it can be always seen in single threading mode):

```bash
$ PYSPARK_PIN_THREAD=true ./bin/pyspark
```

```python
import threading

spark.sparkContext.setLocalProperty("a", "hi")
def print_prop():
    print(spark.sparkContext.getLocalProperty("a"))

threading.Thread(target=print_prop).start()
```

```
None
```

Unlike Scala side:

```scala
spark.sparkContext.setLocalProperty("a", "hi")
new Thread(new Runnable {
  def run() = println(spark.sparkContext.getLocalProperty("a"))
}).start()
```

```
hi
```

This behaviour potentially could cause weird issues but this PR currently does not target this fix this for now since this mode is experimental.

### How does this PR fix?

Basically there are two types of Py4J servers `GatewayServer` and `ClientServer`.  The former is for multi threading and the latter is for single threading. This PR adds a switch to use the latter.

In Scala side:
The logic to select a server is encapsulated in `Py4JServer` and use `Py4JServer` at `PythonRunner` for Spark summit and `PythonGatewayServer` for Spark shell. Each uses `ClientServer` when `PYSPARK_PIN_THREAD` is `true` and `GatewayServer` otherwise.

In Python side:
Simply do an if-else to switch the server to talk. It uses `ClientServer` when `PYSPARK_PIN_THREAD` is `true` and `GatewayServer` otherwise.

This is disabled by default for now.

## How was this patch tested?

Manually tested. This can be tested via:

```python
PYSPARK_PIN_THREAD=true ./bin/pyspark
```

and/or

```bash
cd python
./run-tests --python-executables=python --testnames "pyspark.tests.test_pin_thread"
```

Also, ran the Jenkins tests with `PYSPARK_PIN_THREAD` enabled.

Closes #24898 from HyukjinKwon/pinned-thread.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-11-08 06:44:58 +09:00
Dongjoon Hyun c55265cd2d [SPARK-29739][PYSPARK][TESTS] Use java instead of cc in test_pipe_functions
### What changes were proposed in this pull request?

This PR aims to replace `cc` with `java` in `test_pipe_functions` of `test_rdd.py`.

### Why are the changes needed?

Currently, `test_rdd.py` assumes `cc` installation during `rdd.pipe` tests.
This requires us to install `gcc` for python testing. If we use `java`, we can have the same test coverage and we don't need to install it because it's already installed in `PySpark` test environment.

This will be helpful when we build a dockerized parallel testing environment.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Pass the existing PySpark tests.

Closes #26383 from dongjoon-hyun/SPARK-29739.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-03 23:03:38 -08:00
Xianyang Liu 1e599e5005 [SPARK-29582][PYSPARK] Support TaskContext.get() in a barrier task from Python side
### What changes were proposed in this pull request?

Add support of `TaskContext.get()` in a barrier task from Python side, this makes it easier to migrate legacy user code to barrier execution mode.

### Why are the changes needed?

In Spark Core, there is a `TaskContext` object which is a singleton. We set a task context instance which can be TaskContext or BarrierTaskContext before the task function startup, and unset it to none after the function end. So we can both get TaskContext and BarrierTaskContext with the object. However we can only get the BarrierTaskContext with `BarrierTaskContext`, we will get `None` if we get it by `TaskContext.get` in a barrier stage.

This is useful when people switch from normal code to barrier code, and only need a little update.

### Does this PR introduce any user-facing change?

Yes.
Previously:
```python
def func(iterator):
    task_context = TaskContext.get() . # this could be None.
    barrier_task_context = BarrierTaskContext.get() # get the BarrierTaskContext instance
    ...

rdd.barrier().mapPartitions(func)
```

Proposed:
```python
def func(iterator):
    task_context = TaskContext.get() . # this could also get the BarrierTaskContext instance which is same as barrier_task_context
    barrier_task_context = BarrierTaskContext.get() # get the BarrierTaskContext instance
    ...

rdd.barrier().mapPartitions(func)
```

### How was this patch tested?

New UT tests.

Closes #26239 from ConeyLiu/barrier_task_context.

Authored-by: Xianyang Liu <xianyang.liu@intel.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-10-31 13:10:44 +09:00
Xianyang Liu 0a7095156b [SPARK-29499][CORE][PYSPARK] Add mapPartitionsWithIndex for RDDBarrier
### What changes were proposed in this pull request?

Add mapPartitionsWithIndex for RDDBarrier.

### Why are the changes needed?

There is only one method in `RDDBarrier`. We often use the partition index as a label for the current partition. We need to get the index from `TaskContext`  index in the method of `mapPartitions` which is not convenient.
### Does this PR introduce any user-facing change?

No

### How was this patch tested?

New UT.

Closes #26148 from ConeyLiu/barrier-index.

Authored-by: Xianyang Liu <xianyang.liu@intel.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2019-10-23 13:46:09 +02:00
Xianjin YE 8c8016a152 [SPARK-21045][PYTHON] Allow non-ascii string as an exception message from python execution in Python 2
### What changes were proposed in this pull request?

This PR allows non-ascii string as an exception message in Python 2 by explicitly en/decoding in case of `str` in Python 2.

### Why are the changes needed?

Previously PySpark will hang when the `UnicodeDecodeError` occurs and the real exception cannot be passed to the JVM side.

See the reproducer as below:

```python
def f():
    raise Exception("中")
spark = SparkSession.builder.master('local').getOrCreate()
spark.sparkContext.parallelize([1]).map(lambda x: f()).count()
```

### Does this PR introduce any user-facing change?

User may not observe hanging for the similar cases.

### How was this patch tested?

Added a new test and manually checking.

This pr is based on #18324, credits should also go to dataknocker.
To make lint-python happy for python3, it also includes a followup fix for #25814

Closes #25847 from advancedxy/python_exception_19926_and_21045.

Authored-by: Xianjin YE <advancedxy@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-09-21 08:09:19 +09:00
Holden Karau 42050c3f4f [SPARK-27659][PYTHON] Allow PySpark to prefetch during toLocalIterator
### What changes were proposed in this pull request?

This PR allows Python toLocalIterator to prefetch the next partition while the first partition is being collected. The PR also adds a demo micro bench mark in the examples directory, we may wish to keep this or not.

### Why are the changes needed?

In https://issues.apache.org/jira/browse/SPARK-23961 / 5e79ae3b40 we changed PySpark to only pull one partition at a time. This is memory efficient, but if partitions take time to compute this can mean we're spending more time blocking.

### Does this PR introduce any user-facing change?

A new param is added to toLocalIterator

### How was this patch tested?

New unit test inside of `test_rdd.py` checks the time that the elements are evaluated at. Another test that the results remain the same are added to `test_dataframe.py`.

I also ran a micro benchmark in the examples directory `prefetch.py` which shows an improvement of ~40% in this specific use case.

>
> 19/08/16 17:11:36 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
> Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties
> Setting default log level to "WARN".
> To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
> Running timers:
>
> [Stage 32:>                                                         (0 + 1) / 1]
> Results:
>
> Prefetch time:
>
> 100.228110831
>
>
> Regular time:
>
> 188.341721614
>
>
>

Closes #25515 from holdenk/SPARK-27659-allow-pyspark-tolocalitr-to-prefetch.

Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
2019-09-20 09:59:31 -07:00
wuyi cbad616d4c [SPARK-27371][CORE] Support GPU-aware resources scheduling in Standalone
## What changes were proposed in this pull request?

In this PR, we implements a complete process of GPU-aware resources scheduling
in Standalone. The whole process looks like: Worker sets up isolated resources
when it starts up and registers to master along with its resources. And, Master
picks up usable workers according to driver/executor's resource requirements to
launch driver/executor on them. Then, Worker launches the driver/executor after
preparing resources file, which is created under driver/executor's working directory,
with specified resource addresses(told by master). When driver/executor finished,
their resources could be recycled to worker. Finally, if a worker stops, it
should always release its resources firstly.

For the case of Workers and Drivers in **client** mode run on the same host, we introduce
a config option named `spark.resources.coordinate.enable`(default true) to indicate
whether Spark should coordinate resources for user. If `spark.resources.coordinate.enable=false`, user should be responsible for configuring different resources for Workers and Drivers when use resourcesFile or discovery script. If true, Spark would help user to assign different  resources for Workers and Drivers.

The solution for Spark to coordinate resources among Workers and Drivers is:

Generally, use a shared file named *____allocated_resources____.json* to sync allocated
resources info among Workers and Drivers on the same host.

After a Worker or Driver found all resources using the configured resourcesFile and/or
discovery script during launching, it should filter out available resources by excluding resources already allocated in *____allocated_resources____.json* and acquire resources from available resources according to its own requirement. After that, it should write its allocated resources along with its process id (pid) into *____allocated_resources____.json*.  Pid (proposed by tgravescs) here used to check whether the allocated resources are still valid in case of Worker or Driver crashes and doesn't release resources properly. And when a Worker or Driver finished, normally, it would always clean up its own allocated resources in *____allocated_resources____.json*.

Note that we'll always get a file lock before any access to file *____allocated_resources____.json*
and release the lock finally.

Futhermore, we appended resources info in `WorkerSchedulerStateResponse` to work
around master change behaviour in HA mode.

## How was this patch tested?

Added unit tests in WorkerSuite, MasterSuite, SparkContextSuite.

Manually tested with client/cluster mode (e.g. multiple workers) in a single node Standalone.

Closes #25047 from Ngone51/SPARK-27371.

Authored-by: wuyi <ngone_5451@163.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2019-08-09 07:49:03 -05:00
wuyi 94499af6f0 [SPARK-28486][CORE][PYTHON] Map PythonBroadcast's data file to a BroadcastBlock to avoid delete by GC
## What changes were proposed in this pull request?

Currently, PythonBroadcast may delete its data file while a python worker still needs it. This happens because PythonBroadcast overrides the `finalize()` method to delete its data file. So, when GC happens and no  references on broadcast variable, it may trigger `finalize()` to delete
data file. That's also means, data under python Broadcast variable couldn't be deleted when `unpersist()`/`destroy()` called but relys on GC.

In this PR, we removed the `finalize()` method, and map the PythonBroadcast data file to a BroadcastBlock(which has the same broadcast id with the broadcast variable who wrapped this PythonBroadcast) when PythonBroadcast is deserializing. As a result, the data file could be deleted just like other pieces of the Broadcast variable when `unpersist()`/`destroy()` called and do not rely on GC any more.

## How was this patch tested?

Added a Python test, and tested manually(verified create/delete the broadcast block).

Closes #25262 from Ngone51/SPARK-28486.

Authored-by: wuyi <ngone_5451@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-05 20:18:53 +09:00
WeichenXu b3394db193 [SPARK-28582][PYTHON] Fix flaky test DaemonTests.do_termination_test which fail on Python 3.7
## What changes were proposed in this pull request?

This PR picks up https://github.com/apache/spark/pull/25315 back after removing `Popen.wait` usage which exists in Python 3 only. I saw the last test results wrongly and thought it was passed.

Fix flaky test DaemonTests.do_termination_test which fail on Python 3.7. I add a sleep after the test connection to daemon.

## How was this patch tested?

Run test
```
python/run-tests --python-executables=python3.7 --testname "pyspark.tests.test_daemon DaemonTests"
```
**Before**
Fail on test "test_termination_sigterm". And we can see daemon process do not exit.
**After**
Test passed

Closes #25343 from HyukjinKwon/SPARK-28582.

Authored-by: WeichenXu <weichen.xu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-03 10:31:15 +09:00
Dongjoon Hyun 8ae032d78d Revert "[SPARK-28582][PYSPARK] Fix flaky test DaemonTests.do_termination_test which fail on Python 3.7"
This reverts commit fbeee0c5bc.
2019-08-02 10:14:20 -07:00
WeichenXu fbeee0c5bc [SPARK-28582][PYSPARK] Fix flaky test DaemonTests.do_termination_test which fail on Python 3.7
## What changes were proposed in this pull request?

Fix flaky test DaemonTests.do_termination_test which fail on Python 3.7. I add a sleep after the test connection to daemon.

## How was this patch tested?

Run test
```
python/run-tests --python-executables=python3.7 --testname "pyspark.tests.test_daemon DaemonTests"
```
**Before**
Fail on test "test_termination_sigterm". And we can see daemon process do not exit.
**After**
Test passed

Closes #25315 from WeichenXu123/fix_py37_daemon.

Authored-by: WeichenXu <weichen.xu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-02 22:07:06 +09:00
Thomas Graves f84cca2d84 [SPARK-28234][CORE][PYTHON] Add python and JavaSparkContext support to get resources
## What changes were proposed in this pull request?

Add python api support and JavaSparkContext support for resources().  I needed the JavaSparkContext support for it to properly translate into python with the py4j stuff.

## How was this patch tested?

Unit tests added and manually tested in local cluster mode and on yarn.

Closes #25087 from tgravescs/SPARK-28234-python.

Authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-11 09:32:58 +09:00
HyukjinKwon 7c05f61514 [SPARK-28130][PYTHON] Print pretty messages for skipped tests when xmlrunner is available in PySpark
## What changes were proposed in this pull request?

Currently, pretty skipped message added by f7435bec6a mechanism seems not working when xmlrunner is installed apparently.

This PR fixes two things:

1. When `xmlrunner` is installed, seems `xmlrunner` does not respect `vervosity` level in unittests (default is level 1).

    So the output looks as below

    ```
    Running tests...
     ----------------------------------------------------------------------
    SSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSS
    ----------------------------------------------------------------------
    ```

    So it is not caught by our message detection mechanism.

2. If we manually set the `vervocity` level to `xmlrunner`, it prints messages as below:

    ```
    test_mixed_udf (pyspark.sql.tests.test_pandas_udf_scalar.ScalarPandasUDFTests) ... SKIP (0.000s)
    test_mixed_udf_and_sql (pyspark.sql.tests.test_pandas_udf_scalar.ScalarPandasUDFTests) ... SKIP (0.000s)
    ...
    ```

    This is different in our Jenkins machine:

    ```
    test_createDataFrame_column_name_encoding (pyspark.sql.tests.test_arrow.ArrowTests) ... skipped 'Pandas >= 0.23.2 must be installed; however, it was not found.'
    test_createDataFrame_does_not_modify_input (pyspark.sql.tests.test_arrow.ArrowTests) ... skipped 'Pandas >= 0.23.2 must be installed; however, it was not found.'
    ...
    ```

    Note that last `SKIP` is different. This PR fixes the regular expression to catch `SKIP` case as well.

## How was this patch tested?

Manually tested.

**Before:**

```
Starting test(python2.7): pyspark....
Finished test(python2.7): pyspark.... (0s)
...
Tests passed in 562 seconds

========================================================================
...
```

**After:**

```
Starting test(python2.7): pyspark....
Finished test(python2.7): pyspark.... (48s) ... 93 tests were skipped
...
Tests passed in 560 seconds

Skipped tests pyspark.... with python2.7:
      pyspark...(...) ... SKIP (0.000s)
...

========================================================================
...
```

Closes #24927 from HyukjinKwon/SPARK-28130.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-24 09:58:17 +09:00
HyukjinKwon f984f6acfe Revert "[SPARK-27870][SQL][PYSPARK] Flush batch timely for pandas UDF (for improving pandas UDFs pipeline)"
## What changes were proposed in this pull request?

This PR reverts 9c4eb99c52 for the reasons below:

1. An alternative was not considered properly, https://github.com/apache/spark/pull/24734#issuecomment-500101639 https://github.com/apache/spark/pull/24734#issuecomment-500102340 https://github.com/apache/spark/pull/24734#issuecomment-499202982 - I opened a PR https://github.com/apache/spark/pull/24826

2. 9c4eb99c52 fixed timely flushing which behaviour is somewhat hacky and the timing isn't also guaranteed (in case each batch takes longer to process).

3. For pipelining for smaller batches, looks it's better to allow to configure buffer size rather than having another factor to flush

## How was this patch tested?

N/A

Closes #24827 from HyukjinKwon/revert-flush.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-09 08:28:31 -07:00
WeichenXu 9c4eb99c52 [SPARK-27870][SQL][PYSPARK] Flush batch timely for pandas UDF (for improving pandas UDFs pipeline)
## What changes were proposed in this pull request?

Flush batch timely for pandas UDF.

This could improve performance when multiple pandas UDF plans are pipelined.

When batch being flushed in time, downstream pandas UDFs will get pipelined as soon as possible, and pipeline will help hide the donwstream UDFs computation time. For example:

When the first UDF start computing on batch-3, the second pipelined UDF can start computing on batch-2, and the third pipelined UDF can start computing on batch-1.

If we do not flush each batch in time, the donwstream UDF's pipeline will lag behind too much, which may increase the total processing time.

I add flush at two places:
* JVM process feed data into python worker. In jvm side, when write one batch, flush it
* VM process read data from python worker output, In python worker side, when write one batch, flush it

If no flush, the default buffer size for them are both 65536. Especially in the ML case, in order to make realtime prediction, we will make batch size very small. The buffer size is too large for the case, which cause downstream pandas UDF pipeline lag behind too much.

### Note
* This is only applied to pandas scalar UDF.
* Do not flush for each batch. The minimum interval between two flush is 0.1 second. This avoid too frequent flushing when batch size is small. It works like:
```
        last_flush_time = time.time()
        for batch in iterator:
                writer.write_batch(batch)
                flush_time = time.time()
                if self.flush_timely and (flush_time - last_flush_time > 0.1):
                      stream.flush()
                      last_flush_time = flush_time
```

## How was this patch tested?

### Benchmark to make sure the flush do not cause performance regression
#### Test code:
```
numRows = ...
batchSize = ...

spark.conf.set('spark.sql.execution.arrow.maxRecordsPerBatch', str(batchSize))
df = spark.range(1, numRows + 1, numPartitions=1).select(col('id').alias('a'))

pandas_udf("int", PandasUDFType.SCALAR)
def fp1(x):
    return x + 10

beg_time = time.time()
result = df.select(sum(fp1('a'))).head()
print("result: " + str(result[0]))
print("consume time: " + str(time.time() - beg_time))
```
#### Test Result:

 params        | Consume time (Before) | Consume time (After)
------------ | ----------------------- | ----------------------
numRows=100000000, batchSize=10000 | 23.43s | 24.64s
numRows=100000000, batchSize=1000 | 36.73s | 34.50s
numRows=10000000, batchSize=100 | 35.67s | 32.64s
numRows=1000000, batchSize=10 | 33.60s | 32.11s
numRows=100000, batchSize=1 | 33.36s | 31.82s

### Benchmark pipelined pandas UDF
#### Test code:
```
spark.conf.set('spark.sql.execution.arrow.maxRecordsPerBatch', '1')
df = spark.range(1, 31, numPartitions=1).select(col('id').alias('a'))

pandas_udf("int", PandasUDFType.SCALAR)
def fp1(x):
    print("run fp1")
    time.sleep(1)
    return x + 100

pandas_udf("int", PandasUDFType.SCALAR)
def fp2(x, y):
    print("run fp2")
    time.sleep(1)
    return x + y

beg_time = time.time()
result = df.select(sum(fp2(fp1('a'), col('a')))).head()
print("result: " + str(result[0]))
print("consume time: " + str(time.time() - beg_time))

```
#### Test Result:

**Before**: consume time: 63.57s
**After**: consume time: 32.43s
**So the PR improve performance by make downstream UDF get pipelined early.**

Please review https://spark.apache.org/contributing.html before opening a pull request.

Closes #24734 from WeichenXu123/improve_pandas_udf_pipeline.

Lead-authored-by: WeichenXu <weichen.xu@databricks.com>
Co-authored-by: Xiangrui Meng <meng@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-07 14:02:43 -07:00
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
Hyukjin Kwon a67e8426e3 [SPARK-27000][PYTHON] Upgrades cloudpickle to v0.8.0
## What changes were proposed in this pull request?

After upgrading cloudpickle to 0.6.1 at https://github.com/apache/spark/pull/20691, one regression was found. Cloudpickle had a critical https://github.com/cloudpipe/cloudpickle/pull/240 for that.

Basically, it currently looks existing globals would override globals shipped in a function's, meaning:

**Before:**

```python
>>> def hey():
...     return "Hi"
...
>>> spark.range(1).rdd.map(lambda _: hey()).collect()
['Hi']
>>> def hey():
...     return "Yeah"
...
>>> spark.range(1).rdd.map(lambda _: hey()).collect()
['Hi']
```

**After:**

```python
>>> def hey():
...     return "Hi"
...
>>> spark.range(1).rdd.map(lambda _: hey()).collect()
['Hi']
>>>
>>> def hey():
...     return "Yeah"
...
>>> spark.range(1).rdd.map(lambda _: hey()).collect()
['Yeah']
```

Therefore, this PR upgrades cloudpickle to 0.8.0.

Note that cloudpickle's release cycle is quite short.

Between 0.6.1 and 0.7.0, it contains minor bug fixes. I don't see notable changes to double check and/or avoid.

There is virtually only this fix between 0.7.0 and 0.8.1 - other fixes are about testing.

## How was this patch tested?

Manually tested, tests were added. Verified unit tests were added in cloudpickle.

Closes #23904 from HyukjinKwon/SPARK-27000.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-02-28 02:33:10 +09:00
Boris Shminke 75ea89ad94 [SPARK-18161][PYTHON] Update cloudpickle to v0.6.1
## What changes were proposed in this pull request?

In this PR we've done two things:
1) updated the Spark's copy of cloudpickle to 0.6.1 (current stable)
The main reason Spark stayed with cloudpickle 0.4.x was that the default pickle protocol was changed in later versions.

2) started using pickle.HIGHEST_PROTOCOL for both Python 2 and Python 3 for serializers and broadcast
[Pyrolite](https://github.com/irmen/Pyrolite) has such Pickle protocol version support: reading: 0,1,2,3,4; writing: 2.

## How was this patch tested?

Jenkins tests.

Authors: Sloane Simmons, Boris Shminke

This contribution is original work of Sloane Simmons and Boris Shminke and they licensed it to the project under the project's open source license.

Closes #20691 from inpefess/pickle_protocol_4.

Lead-authored-by: Boris Shminke <boris@shminke.me>
Co-authored-by: singularperturbation <sloanes.k@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-02-02 10:49:45 +08:00
Sean Owen 8171b156eb [SPARK-26771][CORE][GRAPHX] Make .unpersist(), .destroy() consistently non-blocking by default
## What changes were proposed in this pull request?

Make .unpersist(), .destroy() non-blocking by default and adjust callers to request blocking only where important.

This also adds an optional blocking argument to Pyspark's RDD.unpersist(), which never had one.

## How was this patch tested?

Existing tests.

Closes #23685 from srowen/SPARK-26771.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-02-01 18:29:55 -06:00
Hyukjin Kwon 3a17c6a06b [SPARK-26743][PYTHON] Adds a test to check the actual resource limit set via 'spark.executor.pyspark.memory'
## What changes were proposed in this pull request?

https://github.com/apache/spark/pull/21977 added a feature to limit Python worker resource limit.
This PR is kind of a followup of it. It proposes to add a test that checks the actual resource limit set by 'spark.executor.pyspark.memory'.

## How was this patch tested?

Unit tests were added.

Closes #23663 from HyukjinKwon/test_rlimit.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-28 10:02:27 +08:00
Yuanjian Li 98e831d321 [SPARK-25921][FOLLOW UP][PYSPARK] Fix barrier task run without BarrierTaskContext while python worker reuse
## What changes were proposed in this pull request?

It's the follow-up PR for #22962, contains the following works:
- Remove `__init__` in TaskContext and BarrierTaskContext.
- Add more comments to explain the fix.
- Rewrite UT in a new class.

## How was this patch tested?

New UT in test_taskcontext.py

Closes #23435 from xuanyuanking/SPARK-25921-follow.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-11 14:28:37 +08:00
Yuanjian Li dbbba80b3c [SPARK-26549][PYSPARK] Fix for python worker reuse take no effect for parallelize lazy iterable range
## What changes were proposed in this pull request?

During the follow-up work(#23435) for PySpark worker reuse scenario, we found that the worker reuse takes no effect for `sc.parallelize(xrange(...))`. It happened because of the specialize rdd.parallelize logic for xrange(introduced in #3264) generated data by lazy iterable range, which don't need to use the passed-in iterator. But this will break the end of stream checking in python worker and finally cause worker reuse takes no effect. See more details in [SPARK-26549](https://issues.apache.org/jira/browse/SPARK-26549) description.

We fix this by force using the passed-in iterator.

## How was this patch tested?
New UT in test_worker.py.

Closes #23470 from xuanyuanking/SPARK-26549.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-09 11:55:12 +08:00
Imran Rashid 32515d205a [SPARK-26349][PYSPARK] Forbid insecure py4j gateways
Spark always creates secure py4j connections between java and python,
but it also allows users to pass in their own connection. This ensures
that even passed in connections are secure.

Added test cases verifying the failure with a (mocked) insecure gateway.

This is closely related to SPARK-26019, but this entirely forbids the
insecure connection, rather than creating the "escape-hatch".

Closes #23441 from squito/SPARK-26349.

Authored-by: Imran Rashid <irashid@cloudera.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-01-08 11:26:36 -08:00
schintap 9b23be2e95 [SPARK-26201] Fix python broadcast with encryption
## What changes were proposed in this pull request?
Python with rpc and disk encryption enabled along with a python broadcast variable and just read the value back on the driver side the job failed with:

Traceback (most recent call last): File "broadcast.py", line 37, in <module> words_new.value File "/pyspark.zip/pyspark/broadcast.py", line 137, in value File "pyspark.zip/pyspark/broadcast.py", line 122, in load_from_path File "pyspark.zip/pyspark/broadcast.py", line 128, in load EOFError: Ran out of input

To reproduce use configs: --conf spark.network.crypto.enabled=true --conf spark.io.encryption.enabled=true

Code:

words_new = sc.broadcast(["scala", "java", "hadoop", "spark", "akka"])
words_new.value
print(words_new.value)

## How was this patch tested?
words_new = sc.broadcast([“scala”, “java”, “hadoop”, “spark”, “akka”])
textFile = sc.textFile(“README.md”)
wordCounts = textFile.flatMap(lambda line: line.split()).map(lambda word: (word + words_new.value[1], 1)).reduceByKey(lambda a, b: a+b)
 count = wordCounts.count()
 print(count)
 words_new.value
 print(words_new.value)

Closes #23166 from redsanket/SPARK-26201.

Authored-by: schintap <schintap@oath.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2018-11-30 12:48:56 -06:00
hyukjinkwon 03306a6df3 [SPARK-26036][PYTHON] Break large tests.py files into smaller files
## What changes were proposed in this pull request?

This PR continues to break down a big large file into smaller files. See https://github.com/apache/spark/pull/23021. It targets to follow https://github.com/numpy/numpy/tree/master/numpy.

Basically this PR proposes to break down `pyspark/tests.py` into ...:

```
pyspark
...
├── testing
...
│   └── utils.py
├── tests
│   ├── __init__.py
│   ├── test_appsubmit.py
│   ├── test_broadcast.py
│   ├── test_conf.py
│   ├── test_context.py
│   ├── test_daemon.py
│   ├── test_join.py
│   ├── test_profiler.py
│   ├── test_rdd.py
│   ├── test_readwrite.py
│   ├── test_serializers.py
│   ├── test_shuffle.py
│   ├── test_taskcontext.py
│   ├── test_util.py
│   └── test_worker.py
...
```

## How was this patch tested?

Existing tests should cover.

`cd python` and .`/run-tests-with-coverage`. Manually checked they are actually being ran.

Each test (not officially) can be ran via:

```bash
SPARK_TESTING=1 ./bin/pyspark pyspark.tests.test_context
```

Note that if you're using Mac and Python 3, you might have to `OBJC_DISABLE_INITIALIZE_FORK_SAFETY=YES`.

Closes #23033 from HyukjinKwon/SPARK-26036.

Authored-by: hyukjinkwon <gurwls223@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-11-15 12:30:52 +08:00