2017-04-11 15:18:31 -04:00
|
|
|
# -*- coding: utf-8 -*-
|
|
|
|
#
|
|
|
|
# Licensed to the Apache Software Foundation (ASF) under one or more
|
|
|
|
# contributor license agreements. See the NOTICE file distributed with
|
|
|
|
# this work for additional information regarding copyright ownership.
|
|
|
|
# The ASF licenses this file to You under the Apache License, Version 2.0
|
|
|
|
# (the "License"); you may not use this file except in compliance with
|
|
|
|
# the License. You may obtain a copy of the License at
|
|
|
|
#
|
|
|
|
# http://www.apache.org/licenses/LICENSE-2.0
|
|
|
|
#
|
|
|
|
# Unless required by applicable law or agreed to in writing, software
|
|
|
|
# distributed under the License is distributed on an "AS IS" BASIS,
|
|
|
|
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
|
|
# See the License for the specific language governing permissions and
|
|
|
|
# limitations under the License.
|
|
|
|
#
|
2018-03-08 06:29:07 -05:00
|
|
|
|
2021-05-24 20:50:22 -04:00
|
|
|
import functools
|
[SPARK-33407][PYTHON] Simplify the exception message from Python UDFs (disabled by default)
### What changes were proposed in this pull request?
This PR proposes to simplify the exception messages from Python UDFS.
Currently, the exception message from Python UDFs is as below:
```python
from pyspark.sql.functions import udf; spark.range(10).select(udf(lambda x: x/0)("id")).collect()
```
```python
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../python/pyspark/sql/dataframe.py", line 427, in show
print(self._jdf.showString(n, 20, vertical))
File "/.../python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", line 1305, in __call__
File "/.../python/pyspark/sql/utils.py", line 127, in deco
raise_from(converted)
File "<string>", line 3, in raise_from
pyspark.sql.utils.PythonException:
An exception was thrown from Python worker in the executor:
Traceback (most recent call last):
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 605, in main
process()
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 597, in process
serializer.dump_stream(out_iter, outfile)
File "/.../python/lib/pyspark.zip/pyspark/serializers.py", line 223, in dump_stream
self.serializer.dump_stream(self._batched(iterator), stream)
File "/.../python/lib/pyspark.zip/pyspark/serializers.py", line 141, in dump_stream
for obj in iterator:
File "/.../python/lib/pyspark.zip/pyspark/serializers.py", line 212, in _batched
for item in iterator:
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 450, in mapper
result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in udfs)
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 450, in <genexpr>
result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in udfs)
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 90, in <lambda>
return lambda *a: f(*a)
File "/.../python/lib/pyspark.zip/pyspark/util.py", line 107, in wrapper
return f(*args, **kwargs)
File "<stdin>", line 1, in <lambda>
ZeroDivisionError: division by zero
```
Actually, almost all cases, users only care about `ZeroDivisionError: division by zero`. We don't really have to show the internal stuff in 99% cases.
This PR adds a configuration `spark.sql.execution.pyspark.udf.simplifiedException.enabled` (disabled by default) that hides the internal tracebacks related to Python worker, (de)serialization, etc.
```python
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../python/pyspark/sql/dataframe.py", line 427, in show
print(self._jdf.showString(n, 20, vertical))
File "/.../python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", line 1305, in __call__
File "/.../python/pyspark/sql/utils.py", line 127, in deco
raise_from(converted)
File "<string>", line 3, in raise_from
pyspark.sql.utils.PythonException:
An exception was thrown from Python worker in the executor:
Traceback (most recent call last):
File "<stdin>", line 1, in <lambda>
ZeroDivisionError: division by zero
```
The trackback will be shown from the point when any non-PySpark file is seen in the traceback.
### Why are the changes needed?
Without this configuration. such internal tracebacks are exposed to users directly especially for shall or notebook users in PySpark. 99% cases people don't care about the internal Python worker, (de)serialization and related tracebacks. It just makes the exception more difficult to read. For example, one statement of `x/0` above shows a very long traceback and most of them are unnecessary.
This configuration enables the ability to show simplified tracebacks which users will likely be most interested in.
### Does this PR introduce _any_ user-facing change?
By default, no. It adds one configuration that simplifies the exception message. See the example above.
### How was this patch tested?
Manually tested:
```bash
$ pyspark --conf spark.sql.execution.pyspark.udf.simplifiedException.enabled=true
```
```python
from pyspark.sql.functions import udf; spark.sparkContext.setLogLevel("FATAL"); spark.range(10).select(udf(lambda x: x/0)("id")).collect()
```
and unittests were also added.
Closes #30309 from HyukjinKwon/SPARK-33407.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-17 00:15:31 -05:00
|
|
|
import itertools
|
|
|
|
import os
|
|
|
|
import platform
|
2018-05-01 22:55:01 -04:00
|
|
|
import re
|
2018-03-08 06:29:07 -05:00
|
|
|
import sys
|
[SPARK-33407][PYTHON] Simplify the exception message from Python UDFs (disabled by default)
### What changes were proposed in this pull request?
This PR proposes to simplify the exception messages from Python UDFS.
Currently, the exception message from Python UDFs is as below:
```python
from pyspark.sql.functions import udf; spark.range(10).select(udf(lambda x: x/0)("id")).collect()
```
```python
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../python/pyspark/sql/dataframe.py", line 427, in show
print(self._jdf.showString(n, 20, vertical))
File "/.../python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", line 1305, in __call__
File "/.../python/pyspark/sql/utils.py", line 127, in deco
raise_from(converted)
File "<string>", line 3, in raise_from
pyspark.sql.utils.PythonException:
An exception was thrown from Python worker in the executor:
Traceback (most recent call last):
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 605, in main
process()
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 597, in process
serializer.dump_stream(out_iter, outfile)
File "/.../python/lib/pyspark.zip/pyspark/serializers.py", line 223, in dump_stream
self.serializer.dump_stream(self._batched(iterator), stream)
File "/.../python/lib/pyspark.zip/pyspark/serializers.py", line 141, in dump_stream
for obj in iterator:
File "/.../python/lib/pyspark.zip/pyspark/serializers.py", line 212, in _batched
for item in iterator:
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 450, in mapper
result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in udfs)
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 450, in <genexpr>
result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in udfs)
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 90, in <lambda>
return lambda *a: f(*a)
File "/.../python/lib/pyspark.zip/pyspark/util.py", line 107, in wrapper
return f(*args, **kwargs)
File "<stdin>", line 1, in <lambda>
ZeroDivisionError: division by zero
```
Actually, almost all cases, users only care about `ZeroDivisionError: division by zero`. We don't really have to show the internal stuff in 99% cases.
This PR adds a configuration `spark.sql.execution.pyspark.udf.simplifiedException.enabled` (disabled by default) that hides the internal tracebacks related to Python worker, (de)serialization, etc.
```python
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../python/pyspark/sql/dataframe.py", line 427, in show
print(self._jdf.showString(n, 20, vertical))
File "/.../python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", line 1305, in __call__
File "/.../python/pyspark/sql/utils.py", line 127, in deco
raise_from(converted)
File "<string>", line 3, in raise_from
pyspark.sql.utils.PythonException:
An exception was thrown from Python worker in the executor:
Traceback (most recent call last):
File "<stdin>", line 1, in <lambda>
ZeroDivisionError: division by zero
```
The trackback will be shown from the point when any non-PySpark file is seen in the traceback.
### Why are the changes needed?
Without this configuration. such internal tracebacks are exposed to users directly especially for shall or notebook users in PySpark. 99% cases people don't care about the internal Python worker, (de)serialization and related tracebacks. It just makes the exception more difficult to read. For example, one statement of `x/0` above shows a very long traceback and most of them are unnecessary.
This configuration enables the ability to show simplified tracebacks which users will likely be most interested in.
### Does this PR introduce _any_ user-facing change?
By default, no. It adds one configuration that simplifies the exception message. See the example above.
### How was this patch tested?
Manually tested:
```bash
$ pyspark --conf spark.sql.execution.pyspark.udf.simplifiedException.enabled=true
```
```python
from pyspark.sql.functions import udf; spark.sparkContext.setLogLevel("FATAL"); spark.range(10).select(udf(lambda x: x/0)("id")).collect()
```
and unittests were also added.
Closes #30309 from HyukjinKwon/SPARK-33407.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-17 00:15:31 -05:00
|
|
|
import threading
|
[SPARK-29341][PYTHON] Upgrade cloudpickle to 1.0.0
### What changes were proposed in this pull request?
This patch upgrades cloudpickle to 1.0.0 version.
Main changes:
1. cleanup unused functions: https://github.com/cloudpipe/cloudpickle/commit/936f16fac89986453c4bb3a4af9f04da16d30a9a
2. Fix relative imports inside function body: https://github.com/cloudpipe/cloudpickle/commit/31ecdd6f57c6013a1affb21f69e86e638f463710
3. Write kw only arguments to pickle: https://github.com/cloudpipe/cloudpickle/commit/6cb47185284548d5706beccd69f172586d127502
### Why are the changes needed?
We should include new bug fix like https://github.com/cloudpipe/cloudpickle/commit/6cb47185284548d5706beccd69f172586d127502, because users might use such python function in PySpark.
```python
>>> def f(a, *, b=1):
... return a + b
...
>>> rdd = sc.parallelize([1, 2, 3])
>>> rdd.map(f).collect()
[Stage 0:> (0 + 12) / 12]19/10/03 00:42:24 ERROR Executor: Exception in task 3.0 in stage 0.0 (TID 3)
org.apache.spark.api.python.PythonException: Traceback (most recent call last):
File "/spark/python/lib/pyspark.zip/pyspark/worker.py", line 598, in main
process()
File "/spark/python/lib/pyspark.zip/pyspark/worker.py", line 590, in process
serializer.dump_stream(out_iter, outfile)
File "/spark/python/lib/pyspark.zip/pyspark/serializers.py", line 513, in dump_stream
vs = list(itertools.islice(iterator, batch))
File "/spark/python/lib/pyspark.zip/pyspark/util.py", line 99, in wrapper
return f(*args, **kwargs)
TypeError: f() missing 1 required keyword-only argument: 'b'
```
After:
```python
>>> def f(a, *, b=1):
... return a + b
...
>>> rdd = sc.parallelize([1, 2, 3])
>>> rdd.map(f).collect()
[2, 3, 4]
```
### Does this PR introduce any user-facing change?
Yes. This fixes two bugs when pickling Python functions.
### How was this patch tested?
Existing tests.
Closes #26009 from viirya/upgrade-cloudpickle.
Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-10-03 06:20:51 -04:00
|
|
|
import traceback
|
[SPARK-33407][PYTHON] Simplify the exception message from Python UDFs (disabled by default)
### What changes were proposed in this pull request?
This PR proposes to simplify the exception messages from Python UDFS.
Currently, the exception message from Python UDFs is as below:
```python
from pyspark.sql.functions import udf; spark.range(10).select(udf(lambda x: x/0)("id")).collect()
```
```python
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../python/pyspark/sql/dataframe.py", line 427, in show
print(self._jdf.showString(n, 20, vertical))
File "/.../python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", line 1305, in __call__
File "/.../python/pyspark/sql/utils.py", line 127, in deco
raise_from(converted)
File "<string>", line 3, in raise_from
pyspark.sql.utils.PythonException:
An exception was thrown from Python worker in the executor:
Traceback (most recent call last):
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 605, in main
process()
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 597, in process
serializer.dump_stream(out_iter, outfile)
File "/.../python/lib/pyspark.zip/pyspark/serializers.py", line 223, in dump_stream
self.serializer.dump_stream(self._batched(iterator), stream)
File "/.../python/lib/pyspark.zip/pyspark/serializers.py", line 141, in dump_stream
for obj in iterator:
File "/.../python/lib/pyspark.zip/pyspark/serializers.py", line 212, in _batched
for item in iterator:
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 450, in mapper
result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in udfs)
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 450, in <genexpr>
result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in udfs)
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 90, in <lambda>
return lambda *a: f(*a)
File "/.../python/lib/pyspark.zip/pyspark/util.py", line 107, in wrapper
return f(*args, **kwargs)
File "<stdin>", line 1, in <lambda>
ZeroDivisionError: division by zero
```
Actually, almost all cases, users only care about `ZeroDivisionError: division by zero`. We don't really have to show the internal stuff in 99% cases.
This PR adds a configuration `spark.sql.execution.pyspark.udf.simplifiedException.enabled` (disabled by default) that hides the internal tracebacks related to Python worker, (de)serialization, etc.
```python
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../python/pyspark/sql/dataframe.py", line 427, in show
print(self._jdf.showString(n, 20, vertical))
File "/.../python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", line 1305, in __call__
File "/.../python/pyspark/sql/utils.py", line 127, in deco
raise_from(converted)
File "<string>", line 3, in raise_from
pyspark.sql.utils.PythonException:
An exception was thrown from Python worker in the executor:
Traceback (most recent call last):
File "<stdin>", line 1, in <lambda>
ZeroDivisionError: division by zero
```
The trackback will be shown from the point when any non-PySpark file is seen in the traceback.
### Why are the changes needed?
Without this configuration. such internal tracebacks are exposed to users directly especially for shall or notebook users in PySpark. 99% cases people don't care about the internal Python worker, (de)serialization and related tracebacks. It just makes the exception more difficult to read. For example, one statement of `x/0` above shows a very long traceback and most of them are unnecessary.
This configuration enables the ability to show simplified tracebacks which users will likely be most interested in.
### Does this PR introduce _any_ user-facing change?
By default, no. It adds one configuration that simplifies the exception message. See the example above.
### How was this patch tested?
Manually tested:
```bash
$ pyspark --conf spark.sql.execution.pyspark.udf.simplifiedException.enabled=true
```
```python
from pyspark.sql.functions import udf; spark.sparkContext.setLogLevel("FATAL"); spark.range(10).select(udf(lambda x: x/0)("id")).collect()
```
and unittests were also added.
Closes #30309 from HyukjinKwon/SPARK-33407.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-17 00:15:31 -05:00
|
|
|
import types
|
2017-04-11 15:18:31 -04:00
|
|
|
|
[SPARK-35946][PYTHON] Respect Py4J server in InheritableThread API
### What changes were proposed in this pull request?
Currently ,we sets the environment variable `PYSPARK_PIN_THREAD` at the client side of `InhertiableThread` API for Py4J (`python/pyspark/util.py`). If the Py4J gateway is created somewhere else (e.g., Zeppelin, etc), it could introduce a breakage at:
```python
from pyspark import SparkContext
jvm = SparkContext._jvm
thread_connection = jvm._gateway_client.get_thread_connection()
# `AttributeError: 'GatewayClient' object has no attribute 'get_thread_connection'` (non-pinned thread mode)
# `get_thread_connection` is only in 'ClientServer' (pinned thread mode)
```
This PR proposes to check the given gateway created, and do the pinned thread mode behaviour accordingly so we can avoid any breakage when Py4J server/gateway is created separately from somewhere else without a pinned thread mode.
### Why are the changes needed?
To avoid any potential breakage.
### Does this PR introduce _any_ user-facing change?
No, the change happened only in the master (https://github.com/apache/spark/commit/fdd7ca5f4e35a906090f3c6b160bdba9ac9fd0ca).
### How was this patch tested?
This is actually a partial revert of https://github.com/apache/spark/commit/fdd7ca5f4e35a906090f3c6b160bdba9ac9fd0ca. As long as the existing tests pass, I guess we're all good.
I also manually tested to make doubly sure:
**Before**:
```python
>>> from pyspark import InheritableThread, inheritable_thread_target
>>> InheritableThread(lambda: 1).start()
>>> inheritable_thread_target(lambda: 1)()
Traceback (most recent call last):
File "/.../python3.8/lib/python3.8/threading.py", line 932, in _bootstrap_inner
self.run()
File "/.../python3.8/lib/python3.8/threading.py", line 870, in run
self._target(*self._args, **self._kwargs)
File "/.../spark/python/pyspark/util.py", line 361, in copy_local_properties
InheritableThread._clean_py4j_conn_for_current_thread()
File "/.../spark/python/pyspark/util.py", line 381, in _clean_py4j_conn_for_current_thread
thread_connection = jvm._gateway_client.get_thread_connection()
AttributeError: 'GatewayClient' object has no attribute 'get_thread_connection'
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../spark/python/pyspark/util.py", line 324, in wrapped
InheritableThread._clean_py4j_conn_for_current_thread()
File "/.../spark/python/pyspark/util.py", line 381, in _clean_py4j_conn_for_current_thread
thread_connection = jvm._gateway_client.get_thread_connection()
AttributeError: 'GatewayClient' object has no attribute 'get_thread_connection'
```
**After**:
```python
>>> from pyspark import InheritableThread, inheritable_thread_target
>>> InheritableThread(lambda: 1).start()
>>> inheritable_thread_target(lambda: 1)()
1
```
Closes #33147 from HyukjinKwon/SPARK-35946.
Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-30 01:18:54 -04:00
|
|
|
from py4j.clientserver import ClientServer
|
|
|
|
|
2020-10-07 06:53:59 -04:00
|
|
|
__all__ = [] # type: ignore
|
2017-04-11 15:18:31 -04:00
|
|
|
|
|
|
|
|
[SPARK-29341][PYTHON] Upgrade cloudpickle to 1.0.0
### What changes were proposed in this pull request?
This patch upgrades cloudpickle to 1.0.0 version.
Main changes:
1. cleanup unused functions: https://github.com/cloudpipe/cloudpickle/commit/936f16fac89986453c4bb3a4af9f04da16d30a9a
2. Fix relative imports inside function body: https://github.com/cloudpipe/cloudpickle/commit/31ecdd6f57c6013a1affb21f69e86e638f463710
3. Write kw only arguments to pickle: https://github.com/cloudpipe/cloudpickle/commit/6cb47185284548d5706beccd69f172586d127502
### Why are the changes needed?
We should include new bug fix like https://github.com/cloudpipe/cloudpickle/commit/6cb47185284548d5706beccd69f172586d127502, because users might use such python function in PySpark.
```python
>>> def f(a, *, b=1):
... return a + b
...
>>> rdd = sc.parallelize([1, 2, 3])
>>> rdd.map(f).collect()
[Stage 0:> (0 + 12) / 12]19/10/03 00:42:24 ERROR Executor: Exception in task 3.0 in stage 0.0 (TID 3)
org.apache.spark.api.python.PythonException: Traceback (most recent call last):
File "/spark/python/lib/pyspark.zip/pyspark/worker.py", line 598, in main
process()
File "/spark/python/lib/pyspark.zip/pyspark/worker.py", line 590, in process
serializer.dump_stream(out_iter, outfile)
File "/spark/python/lib/pyspark.zip/pyspark/serializers.py", line 513, in dump_stream
vs = list(itertools.islice(iterator, batch))
File "/spark/python/lib/pyspark.zip/pyspark/util.py", line 99, in wrapper
return f(*args, **kwargs)
TypeError: f() missing 1 required keyword-only argument: 'b'
```
After:
```python
>>> def f(a, *, b=1):
... return a + b
...
>>> rdd = sc.parallelize([1, 2, 3])
>>> rdd.map(f).collect()
[2, 3, 4]
```
### Does this PR introduce any user-facing change?
Yes. This fixes two bugs when pickling Python functions.
### How was this patch tested?
Existing tests.
Closes #26009 from viirya/upgrade-cloudpickle.
Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-10-03 06:20:51 -04:00
|
|
|
def print_exec(stream):
|
|
|
|
ei = sys.exc_info()
|
|
|
|
traceback.print_exception(ei[0], ei[1], ei[2], None, stream)
|
|
|
|
|
|
|
|
|
2018-05-08 09:22:54 -04:00
|
|
|
class VersionUtils(object):
|
2018-05-01 22:55:01 -04:00
|
|
|
"""
|
2018-05-08 09:22:54 -04:00
|
|
|
Provides utility method to determine Spark versions with given input string.
|
|
|
|
"""
|
|
|
|
@staticmethod
|
|
|
|
def majorMinorVersion(sparkVersion):
|
|
|
|
"""
|
|
|
|
Given a Spark version string, return the (major version number, minor version number).
|
|
|
|
E.g., for 2.0.1-SNAPSHOT, return (2, 0).
|
2018-05-01 22:55:01 -04:00
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2018-05-08 09:22:54 -04:00
|
|
|
>>> sparkVersion = "2.4.0"
|
|
|
|
>>> VersionUtils.majorMinorVersion(sparkVersion)
|
|
|
|
(2, 4)
|
|
|
|
>>> sparkVersion = "2.3.0-SNAPSHOT"
|
|
|
|
>>> VersionUtils.majorMinorVersion(sparkVersion)
|
|
|
|
(2, 3)
|
|
|
|
"""
|
2018-09-12 23:19:43 -04:00
|
|
|
m = re.search(r'^(\d+)\.(\d+)(\..*)?$', sparkVersion)
|
2018-05-08 09:22:54 -04:00
|
|
|
if m is not None:
|
|
|
|
return (int(m.group(1)), int(m.group(2)))
|
|
|
|
else:
|
|
|
|
raise ValueError("Spark tried to parse '%s' as a Spark" % sparkVersion +
|
|
|
|
" version string, but it could not find the major and minor" +
|
|
|
|
" version numbers.")
|
2018-05-01 22:55:01 -04:00
|
|
|
|
|
|
|
|
2018-05-30 06:11:33 -04:00
|
|
|
def fail_on_stopiteration(f):
|
|
|
|
"""
|
|
|
|
Wraps the input function to fail on 'StopIteration' by raising a 'RuntimeError'
|
[SPARK-23754][PYTHON][FOLLOWUP] Move UDF stop iteration wrapping from driver to executor
## What changes were proposed in this pull request?
SPARK-23754 was fixed in #21383 by changing the UDF code to wrap the user function, but this required a hack to save its argspec. This PR reverts this change and fixes the `StopIteration` bug in the worker
## How does this work?
The root of the problem is that when an user-supplied function raises a `StopIteration`, pyspark might stop processing data, if this function is used in a for-loop. The solution is to catch `StopIteration`s exceptions and re-raise them as `RuntimeError`s, so that the execution fails and the error is reported to the user. This is done using the `fail_on_stopiteration` wrapper, in different ways depending on where the function is used:
- In RDDs, the user function is wrapped in the driver, because this function is also called in the driver itself.
- In SQL UDFs, the function is wrapped in the worker, since all processing happens there. Moreover, the worker needs the signature of the user function, which is lost when wrapping it, but passing this signature to the worker requires a not so nice hack.
## How was this patch tested?
Same tests, plus tests for pandas UDFs
Author: edorigatti <emilio.dorigatti@gmail.com>
Closes #21467 from e-dorigatti/fix_udf_hack.
2018-06-10 22:15:42 -04:00
|
|
|
prevents silent loss of data when 'f' is used in a for loop in Spark code
|
2018-05-30 06:11:33 -04:00
|
|
|
"""
|
|
|
|
def wrapper(*args, **kwargs):
|
|
|
|
try:
|
|
|
|
return f(*args, **kwargs)
|
|
|
|
except StopIteration as exc:
|
|
|
|
raise RuntimeError(
|
|
|
|
"Caught StopIteration thrown from user's code; failing the task",
|
|
|
|
exc
|
|
|
|
)
|
|
|
|
|
|
|
|
return wrapper
|
|
|
|
|
|
|
|
|
[SPARK-33407][PYTHON] Simplify the exception message from Python UDFs (disabled by default)
### What changes were proposed in this pull request?
This PR proposes to simplify the exception messages from Python UDFS.
Currently, the exception message from Python UDFs is as below:
```python
from pyspark.sql.functions import udf; spark.range(10).select(udf(lambda x: x/0)("id")).collect()
```
```python
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../python/pyspark/sql/dataframe.py", line 427, in show
print(self._jdf.showString(n, 20, vertical))
File "/.../python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", line 1305, in __call__
File "/.../python/pyspark/sql/utils.py", line 127, in deco
raise_from(converted)
File "<string>", line 3, in raise_from
pyspark.sql.utils.PythonException:
An exception was thrown from Python worker in the executor:
Traceback (most recent call last):
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 605, in main
process()
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 597, in process
serializer.dump_stream(out_iter, outfile)
File "/.../python/lib/pyspark.zip/pyspark/serializers.py", line 223, in dump_stream
self.serializer.dump_stream(self._batched(iterator), stream)
File "/.../python/lib/pyspark.zip/pyspark/serializers.py", line 141, in dump_stream
for obj in iterator:
File "/.../python/lib/pyspark.zip/pyspark/serializers.py", line 212, in _batched
for item in iterator:
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 450, in mapper
result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in udfs)
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 450, in <genexpr>
result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in udfs)
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 90, in <lambda>
return lambda *a: f(*a)
File "/.../python/lib/pyspark.zip/pyspark/util.py", line 107, in wrapper
return f(*args, **kwargs)
File "<stdin>", line 1, in <lambda>
ZeroDivisionError: division by zero
```
Actually, almost all cases, users only care about `ZeroDivisionError: division by zero`. We don't really have to show the internal stuff in 99% cases.
This PR adds a configuration `spark.sql.execution.pyspark.udf.simplifiedException.enabled` (disabled by default) that hides the internal tracebacks related to Python worker, (de)serialization, etc.
```python
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../python/pyspark/sql/dataframe.py", line 427, in show
print(self._jdf.showString(n, 20, vertical))
File "/.../python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", line 1305, in __call__
File "/.../python/pyspark/sql/utils.py", line 127, in deco
raise_from(converted)
File "<string>", line 3, in raise_from
pyspark.sql.utils.PythonException:
An exception was thrown from Python worker in the executor:
Traceback (most recent call last):
File "<stdin>", line 1, in <lambda>
ZeroDivisionError: division by zero
```
The trackback will be shown from the point when any non-PySpark file is seen in the traceback.
### Why are the changes needed?
Without this configuration. such internal tracebacks are exposed to users directly especially for shall or notebook users in PySpark. 99% cases people don't care about the internal Python worker, (de)serialization and related tracebacks. It just makes the exception more difficult to read. For example, one statement of `x/0` above shows a very long traceback and most of them are unnecessary.
This configuration enables the ability to show simplified tracebacks which users will likely be most interested in.
### Does this PR introduce _any_ user-facing change?
By default, no. It adds one configuration that simplifies the exception message. See the example above.
### How was this patch tested?
Manually tested:
```bash
$ pyspark --conf spark.sql.execution.pyspark.udf.simplifiedException.enabled=true
```
```python
from pyspark.sql.functions import udf; spark.sparkContext.setLogLevel("FATAL"); spark.range(10).select(udf(lambda x: x/0)("id")).collect()
```
and unittests were also added.
Closes #30309 from HyukjinKwon/SPARK-33407.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-17 00:15:31 -05:00
|
|
|
def walk_tb(tb):
|
|
|
|
while tb is not None:
|
|
|
|
yield tb
|
|
|
|
tb = tb.tb_next
|
|
|
|
|
|
|
|
|
|
|
|
def try_simplify_traceback(tb):
|
|
|
|
"""
|
|
|
|
Simplify the traceback. It removes the tracebacks in the current package, and only
|
|
|
|
shows the traceback that is related to the thirdparty and user-specified codes.
|
|
|
|
|
|
|
|
Returns
|
|
|
|
-------
|
|
|
|
TracebackType or None
|
|
|
|
Simplified traceback instance. It returns None if it fails to simplify.
|
|
|
|
|
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This keeps the tracebacks once it sees they are from a different file even
|
|
|
|
though the following tracebacks are from the current package.
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
|
|
|
>>> import importlib
|
|
|
|
>>> import sys
|
|
|
|
>>> import traceback
|
|
|
|
>>> import tempfile
|
|
|
|
>>> with tempfile.TemporaryDirectory() as tmp_dir:
|
|
|
|
... with open("%s/dummy_module.py" % tmp_dir, "w") as f:
|
|
|
|
... _ = f.write(
|
|
|
|
... 'def raise_stop_iteration():\\n'
|
|
|
|
... ' raise StopIteration()\\n\\n'
|
|
|
|
... 'def simple_wrapper(f):\\n'
|
|
|
|
... ' def wrapper(*a, **k):\\n'
|
|
|
|
... ' return f(*a, **k)\\n'
|
|
|
|
... ' return wrapper\\n')
|
|
|
|
... f.flush()
|
|
|
|
... spec = importlib.util.spec_from_file_location(
|
|
|
|
... "dummy_module", "%s/dummy_module.py" % tmp_dir)
|
|
|
|
... dummy_module = importlib.util.module_from_spec(spec)
|
|
|
|
... spec.loader.exec_module(dummy_module)
|
|
|
|
>>> def skip_doctest_traceback(tb):
|
|
|
|
... import pyspark
|
|
|
|
... root = os.path.dirname(pyspark.__file__)
|
|
|
|
... pairs = zip(walk_tb(tb), traceback.extract_tb(tb))
|
|
|
|
... for cur_tb, cur_frame in pairs:
|
|
|
|
... if cur_frame.filename.startswith(root):
|
|
|
|
... return cur_tb
|
|
|
|
|
|
|
|
Regular exceptions should show the file name of the current package as below.
|
|
|
|
|
|
|
|
>>> exc_info = None
|
|
|
|
>>> try:
|
|
|
|
... fail_on_stopiteration(dummy_module.raise_stop_iteration)()
|
|
|
|
... except Exception as e:
|
|
|
|
... tb = sys.exc_info()[-1]
|
|
|
|
... e.__cause__ = None
|
|
|
|
... exc_info = "".join(
|
|
|
|
... traceback.format_exception(type(e), e, tb))
|
|
|
|
>>> print(exc_info) # doctest: +NORMALIZE_WHITESPACE, +ELLIPSIS
|
|
|
|
Traceback (most recent call last):
|
|
|
|
File ...
|
|
|
|
...
|
|
|
|
File "/.../pyspark/util.py", line ...
|
|
|
|
...
|
|
|
|
RuntimeError: ...
|
|
|
|
>>> "pyspark/util.py" in exc_info
|
|
|
|
True
|
|
|
|
|
|
|
|
If the traceback is simplified with this method, it hides the current package file name:
|
|
|
|
|
|
|
|
>>> exc_info = None
|
|
|
|
>>> try:
|
|
|
|
... fail_on_stopiteration(dummy_module.raise_stop_iteration)()
|
|
|
|
... except Exception as e:
|
|
|
|
... tb = try_simplify_traceback(sys.exc_info()[-1])
|
|
|
|
... e.__cause__ = None
|
|
|
|
... exc_info = "".join(
|
|
|
|
... traceback.format_exception(
|
|
|
|
... type(e), e, try_simplify_traceback(skip_doctest_traceback(tb))))
|
|
|
|
>>> print(exc_info) # doctest: +NORMALIZE_WHITESPACE, +ELLIPSIS
|
|
|
|
RuntimeError: ...
|
|
|
|
>>> "pyspark/util.py" in exc_info
|
|
|
|
False
|
|
|
|
|
|
|
|
In the case below, the traceback contains the current package in the middle.
|
|
|
|
In this case, it just hides the top occurrence only.
|
|
|
|
|
|
|
|
>>> exc_info = None
|
|
|
|
>>> try:
|
|
|
|
... fail_on_stopiteration(dummy_module.simple_wrapper(
|
|
|
|
... fail_on_stopiteration(dummy_module.raise_stop_iteration)))()
|
|
|
|
... except Exception as e:
|
|
|
|
... tb = sys.exc_info()[-1]
|
|
|
|
... e.__cause__ = None
|
|
|
|
... exc_info_a = "".join(
|
|
|
|
... traceback.format_exception(type(e), e, tb))
|
|
|
|
... exc_info_b = "".join(
|
|
|
|
... traceback.format_exception(
|
|
|
|
... type(e), e, try_simplify_traceback(skip_doctest_traceback(tb))))
|
|
|
|
>>> exc_info_a.count("pyspark/util.py")
|
|
|
|
2
|
|
|
|
>>> exc_info_b.count("pyspark/util.py")
|
|
|
|
1
|
|
|
|
"""
|
|
|
|
if "pypy" in platform.python_implementation().lower():
|
|
|
|
# Traceback modification is not supported with PyPy in PySpark.
|
|
|
|
return None
|
|
|
|
if sys.version_info[:2] < (3, 7):
|
|
|
|
# Traceback creation is not supported Python < 3.7.
|
|
|
|
# See https://bugs.python.org/issue30579.
|
|
|
|
return None
|
|
|
|
|
|
|
|
import pyspark
|
|
|
|
|
|
|
|
root = os.path.dirname(pyspark.__file__)
|
|
|
|
tb_next = None
|
|
|
|
new_tb = None
|
|
|
|
pairs = zip(walk_tb(tb), traceback.extract_tb(tb))
|
|
|
|
last_seen = []
|
|
|
|
|
|
|
|
for cur_tb, cur_frame in pairs:
|
|
|
|
if not cur_frame.filename.startswith(root):
|
|
|
|
# Filter the stacktrace from the PySpark source itself.
|
|
|
|
last_seen = [(cur_tb, cur_frame)]
|
|
|
|
break
|
|
|
|
|
|
|
|
for cur_tb, cur_frame in reversed(list(itertools.chain(last_seen, pairs))):
|
|
|
|
# Once we have seen the file names outside, don't skip.
|
|
|
|
new_tb = types.TracebackType(
|
|
|
|
tb_next=tb_next,
|
|
|
|
tb_frame=cur_tb.tb_frame,
|
|
|
|
tb_lasti=cur_tb.tb_frame.f_lasti,
|
|
|
|
tb_lineno=cur_tb.tb_frame.f_lineno)
|
|
|
|
tb_next = new_tb
|
|
|
|
return new_tb
|
|
|
|
|
|
|
|
|
2019-03-10 21:15:07 -04:00
|
|
|
def _print_missing_jar(lib_name, pkg_name, jar_name, spark_version):
|
|
|
|
print("""
|
|
|
|
________________________________________________________________________________________________
|
|
|
|
|
|
|
|
Spark %(lib_name)s libraries not found in class path. Try one of the following.
|
|
|
|
|
|
|
|
1. Include the %(lib_name)s library and its dependencies with in the
|
|
|
|
spark-submit command as
|
|
|
|
|
|
|
|
$ bin/spark-submit --packages org.apache.spark:spark-%(pkg_name)s:%(spark_version)s ...
|
|
|
|
|
|
|
|
2. Download the JAR of the artifact from Maven Central http://search.maven.org/,
|
|
|
|
Group Id = org.apache.spark, Artifact Id = spark-%(jar_name)s, Version = %(spark_version)s.
|
|
|
|
Then, include the jar in the spark-submit command as
|
|
|
|
|
|
|
|
$ bin/spark-submit --jars <spark-%(jar_name)s.jar> ...
|
|
|
|
|
|
|
|
________________________________________________________________________________________________
|
|
|
|
|
|
|
|
""" % {
|
|
|
|
"lib_name": lib_name,
|
|
|
|
"pkg_name": pkg_name,
|
|
|
|
"jar_name": jar_name,
|
|
|
|
"spark_version": spark_version
|
|
|
|
})
|
|
|
|
|
|
|
|
|
2020-04-22 21:20:39 -04:00
|
|
|
def _parse_memory(s):
|
|
|
|
"""
|
|
|
|
Parse a memory string in the format supported by Java (e.g. 1g, 200m) and
|
|
|
|
return the value in MiB
|
|
|
|
|
2020-11-15 20:21:50 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2020-04-22 21:20:39 -04:00
|
|
|
>>> _parse_memory("256m")
|
|
|
|
256
|
|
|
|
>>> _parse_memory("2g")
|
|
|
|
2048
|
|
|
|
"""
|
|
|
|
units = {'g': 1024, 'm': 1, 't': 1 << 20, 'k': 1.0 / 1024}
|
|
|
|
if s[-1].lower() not in units:
|
|
|
|
raise ValueError("invalid format: " + s)
|
|
|
|
return int(float(s[:-1]) * units[s[-1].lower()])
|
|
|
|
|
[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-29 21:15:25 -04:00
|
|
|
|
2021-05-24 20:50:22 -04:00
|
|
|
def inheritable_thread_target(f):
|
|
|
|
"""
|
|
|
|
Return thread target wrapper which is recommended to be used in PySpark when the
|
|
|
|
pinned thread mode is enabled. The wrapper function, before calling original
|
|
|
|
thread target, it inherits the inheritable properties specific
|
|
|
|
to JVM thread such as ``InheritableThreadLocal``.
|
|
|
|
|
|
|
|
Also, note that pinned thread mode does not close the connection from Python
|
|
|
|
to JVM when the thread is finished in the Python side. With this wrapper, Python
|
|
|
|
garbage-collects the Python thread instance and also closes the connection
|
|
|
|
which finishes JVM thread correctly.
|
|
|
|
|
|
|
|
When the pinned thread mode is off, it return the original ``f``.
|
|
|
|
|
2021-06-17 23:02:29 -04:00
|
|
|
.. versionadded:: 3.2.0
|
|
|
|
|
2021-05-24 20:50:22 -04:00
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
f : function
|
|
|
|
the original thread target.
|
|
|
|
|
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This API is experimental.
|
|
|
|
|
2021-06-19 22:48:38 -04:00
|
|
|
It is important to know that it captures the local properties when you decorate it
|
|
|
|
whereas :class:`InheritableThread` captures when the thread is started.
|
|
|
|
Therefore, it is encouraged to decorate it when you want to capture the local
|
|
|
|
properties.
|
2021-05-24 20:50:22 -04:00
|
|
|
|
|
|
|
For example, the local properties from the current Spark context is captured
|
2021-06-19 22:48:38 -04:00
|
|
|
when you define a function here instead of the invocation:
|
2021-05-24 20:50:22 -04:00
|
|
|
|
|
|
|
>>> @inheritable_thread_target
|
|
|
|
... def target_func():
|
|
|
|
... pass # your codes.
|
|
|
|
|
|
|
|
If you have any updates on local properties afterwards, it would not be reflected to
|
|
|
|
the Spark context in ``target_func()``.
|
|
|
|
|
|
|
|
The example below mimics the behavior of JVM threads as close as possible:
|
|
|
|
|
|
|
|
>>> Thread(target=inheritable_thread_target(target_func)).start() # doctest: +SKIP
|
|
|
|
"""
|
|
|
|
from pyspark import SparkContext
|
|
|
|
|
[SPARK-35946][PYTHON] Respect Py4J server in InheritableThread API
### What changes were proposed in this pull request?
Currently ,we sets the environment variable `PYSPARK_PIN_THREAD` at the client side of `InhertiableThread` API for Py4J (`python/pyspark/util.py`). If the Py4J gateway is created somewhere else (e.g., Zeppelin, etc), it could introduce a breakage at:
```python
from pyspark import SparkContext
jvm = SparkContext._jvm
thread_connection = jvm._gateway_client.get_thread_connection()
# `AttributeError: 'GatewayClient' object has no attribute 'get_thread_connection'` (non-pinned thread mode)
# `get_thread_connection` is only in 'ClientServer' (pinned thread mode)
```
This PR proposes to check the given gateway created, and do the pinned thread mode behaviour accordingly so we can avoid any breakage when Py4J server/gateway is created separately from somewhere else without a pinned thread mode.
### Why are the changes needed?
To avoid any potential breakage.
### Does this PR introduce _any_ user-facing change?
No, the change happened only in the master (https://github.com/apache/spark/commit/fdd7ca5f4e35a906090f3c6b160bdba9ac9fd0ca).
### How was this patch tested?
This is actually a partial revert of https://github.com/apache/spark/commit/fdd7ca5f4e35a906090f3c6b160bdba9ac9fd0ca. As long as the existing tests pass, I guess we're all good.
I also manually tested to make doubly sure:
**Before**:
```python
>>> from pyspark import InheritableThread, inheritable_thread_target
>>> InheritableThread(lambda: 1).start()
>>> inheritable_thread_target(lambda: 1)()
Traceback (most recent call last):
File "/.../python3.8/lib/python3.8/threading.py", line 932, in _bootstrap_inner
self.run()
File "/.../python3.8/lib/python3.8/threading.py", line 870, in run
self._target(*self._args, **self._kwargs)
File "/.../spark/python/pyspark/util.py", line 361, in copy_local_properties
InheritableThread._clean_py4j_conn_for_current_thread()
File "/.../spark/python/pyspark/util.py", line 381, in _clean_py4j_conn_for_current_thread
thread_connection = jvm._gateway_client.get_thread_connection()
AttributeError: 'GatewayClient' object has no attribute 'get_thread_connection'
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../spark/python/pyspark/util.py", line 324, in wrapped
InheritableThread._clean_py4j_conn_for_current_thread()
File "/.../spark/python/pyspark/util.py", line 381, in _clean_py4j_conn_for_current_thread
thread_connection = jvm._gateway_client.get_thread_connection()
AttributeError: 'GatewayClient' object has no attribute 'get_thread_connection'
```
**After**:
```python
>>> from pyspark import InheritableThread, inheritable_thread_target
>>> InheritableThread(lambda: 1).start()
>>> inheritable_thread_target(lambda: 1)()
1
```
Closes #33147 from HyukjinKwon/SPARK-35946.
Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-30 01:18:54 -04:00
|
|
|
if isinstance(SparkContext._gateway, ClientServer):
|
|
|
|
# Here's when the pinned-thread mode (PYSPARK_PIN_THREAD) is on.
|
|
|
|
|
2021-06-19 22:48:38 -04:00
|
|
|
# NOTICE the internal difference vs `InheritableThread`. `InheritableThread`
|
|
|
|
# copies local properties when the thread starts but `inheritable_thread_target`
|
|
|
|
# copies when the function is wrapped.
|
|
|
|
properties = SparkContext._active_spark_context._jsc.sc().getLocalProperties().clone()
|
2021-05-24 20:50:22 -04:00
|
|
|
|
|
|
|
@functools.wraps(f)
|
2021-06-19 22:48:38 -04:00
|
|
|
def wrapped(*args, **kwargs):
|
2021-05-24 20:50:22 -04:00
|
|
|
try:
|
|
|
|
# Set local properties in child thread.
|
2021-06-19 22:48:38 -04:00
|
|
|
SparkContext._active_spark_context._jsc.sc().setLocalProperties(properties)
|
2021-05-24 20:50:22 -04:00
|
|
|
return f(*args, **kwargs)
|
|
|
|
finally:
|
2021-06-19 22:48:38 -04:00
|
|
|
InheritableThread._clean_py4j_conn_for_current_thread()
|
|
|
|
return wrapped
|
2021-05-24 20:50:22 -04:00
|
|
|
else:
|
|
|
|
return f
|
|
|
|
|
|
|
|
|
[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-29 21:15:25 -04:00
|
|
|
class InheritableThread(threading.Thread):
|
|
|
|
"""
|
|
|
|
Thread that is recommended to be used in PySpark instead of :class:`threading.Thread`
|
|
|
|
when the pinned thread mode is enabled. The usage of this class is exactly same as
|
|
|
|
:class:`threading.Thread` but correctly inherits the inheritable properties specific
|
|
|
|
to JVM thread such as ``InheritableThreadLocal``.
|
|
|
|
|
|
|
|
Also, note that pinned thread mode does not close the connection from Python
|
|
|
|
to JVM when the thread is finished in the Python side. With this class, Python
|
|
|
|
garbage-collects the Python thread instance and also closes the connection
|
|
|
|
which finishes JVM thread correctly.
|
|
|
|
|
|
|
|
When the pinned thread mode is off, this works as :class:`threading.Thread`.
|
|
|
|
|
|
|
|
.. versionadded:: 3.1.0
|
2020-11-15 20:21:50 -05:00
|
|
|
|
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This API is experimental.
|
[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-29 21:15:25 -04:00
|
|
|
"""
|
|
|
|
def __init__(self, target, *args, **kwargs):
|
2021-06-19 22:48:38 -04:00
|
|
|
from pyspark import SparkContext
|
|
|
|
|
[SPARK-35946][PYTHON] Respect Py4J server in InheritableThread API
### What changes were proposed in this pull request?
Currently ,we sets the environment variable `PYSPARK_PIN_THREAD` at the client side of `InhertiableThread` API for Py4J (`python/pyspark/util.py`). If the Py4J gateway is created somewhere else (e.g., Zeppelin, etc), it could introduce a breakage at:
```python
from pyspark import SparkContext
jvm = SparkContext._jvm
thread_connection = jvm._gateway_client.get_thread_connection()
# `AttributeError: 'GatewayClient' object has no attribute 'get_thread_connection'` (non-pinned thread mode)
# `get_thread_connection` is only in 'ClientServer' (pinned thread mode)
```
This PR proposes to check the given gateway created, and do the pinned thread mode behaviour accordingly so we can avoid any breakage when Py4J server/gateway is created separately from somewhere else without a pinned thread mode.
### Why are the changes needed?
To avoid any potential breakage.
### Does this PR introduce _any_ user-facing change?
No, the change happened only in the master (https://github.com/apache/spark/commit/fdd7ca5f4e35a906090f3c6b160bdba9ac9fd0ca).
### How was this patch tested?
This is actually a partial revert of https://github.com/apache/spark/commit/fdd7ca5f4e35a906090f3c6b160bdba9ac9fd0ca. As long as the existing tests pass, I guess we're all good.
I also manually tested to make doubly sure:
**Before**:
```python
>>> from pyspark import InheritableThread, inheritable_thread_target
>>> InheritableThread(lambda: 1).start()
>>> inheritable_thread_target(lambda: 1)()
Traceback (most recent call last):
File "/.../python3.8/lib/python3.8/threading.py", line 932, in _bootstrap_inner
self.run()
File "/.../python3.8/lib/python3.8/threading.py", line 870, in run
self._target(*self._args, **self._kwargs)
File "/.../spark/python/pyspark/util.py", line 361, in copy_local_properties
InheritableThread._clean_py4j_conn_for_current_thread()
File "/.../spark/python/pyspark/util.py", line 381, in _clean_py4j_conn_for_current_thread
thread_connection = jvm._gateway_client.get_thread_connection()
AttributeError: 'GatewayClient' object has no attribute 'get_thread_connection'
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../spark/python/pyspark/util.py", line 324, in wrapped
InheritableThread._clean_py4j_conn_for_current_thread()
File "/.../spark/python/pyspark/util.py", line 381, in _clean_py4j_conn_for_current_thread
thread_connection = jvm._gateway_client.get_thread_connection()
AttributeError: 'GatewayClient' object has no attribute 'get_thread_connection'
```
**After**:
```python
>>> from pyspark import InheritableThread, inheritable_thread_target
>>> InheritableThread(lambda: 1).start()
>>> inheritable_thread_target(lambda: 1)()
1
```
Closes #33147 from HyukjinKwon/SPARK-35946.
Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-30 01:18:54 -04:00
|
|
|
if isinstance(SparkContext._gateway, ClientServer):
|
|
|
|
# Here's when the pinned-thread mode (PYSPARK_PIN_THREAD) is on.
|
2021-06-19 22:48:38 -04:00
|
|
|
def copy_local_properties(*a, **k):
|
|
|
|
# self._props is set before starting the thread to match the behavior with JVM.
|
|
|
|
assert hasattr(self, "_props")
|
|
|
|
SparkContext._active_spark_context._jsc.sc().setLocalProperties(self._props)
|
|
|
|
try:
|
|
|
|
return target(*a, **k)
|
|
|
|
finally:
|
|
|
|
InheritableThread._clean_py4j_conn_for_current_thread()
|
|
|
|
|
|
|
|
super(InheritableThread, self).__init__(
|
|
|
|
target=copy_local_properties, *args, **kwargs)
|
|
|
|
else:
|
|
|
|
super(InheritableThread, self).__init__(target=target, *args, **kwargs)
|
[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-29 21:15:25 -04:00
|
|
|
|
2021-06-19 22:48:38 -04:00
|
|
|
def start(self, *args, **kwargs):
|
|
|
|
from pyspark import SparkContext
|
[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-29 21:15:25 -04:00
|
|
|
|
[SPARK-35946][PYTHON] Respect Py4J server in InheritableThread API
### What changes were proposed in this pull request?
Currently ,we sets the environment variable `PYSPARK_PIN_THREAD` at the client side of `InhertiableThread` API for Py4J (`python/pyspark/util.py`). If the Py4J gateway is created somewhere else (e.g., Zeppelin, etc), it could introduce a breakage at:
```python
from pyspark import SparkContext
jvm = SparkContext._jvm
thread_connection = jvm._gateway_client.get_thread_connection()
# `AttributeError: 'GatewayClient' object has no attribute 'get_thread_connection'` (non-pinned thread mode)
# `get_thread_connection` is only in 'ClientServer' (pinned thread mode)
```
This PR proposes to check the given gateway created, and do the pinned thread mode behaviour accordingly so we can avoid any breakage when Py4J server/gateway is created separately from somewhere else without a pinned thread mode.
### Why are the changes needed?
To avoid any potential breakage.
### Does this PR introduce _any_ user-facing change?
No, the change happened only in the master (https://github.com/apache/spark/commit/fdd7ca5f4e35a906090f3c6b160bdba9ac9fd0ca).
### How was this patch tested?
This is actually a partial revert of https://github.com/apache/spark/commit/fdd7ca5f4e35a906090f3c6b160bdba9ac9fd0ca. As long as the existing tests pass, I guess we're all good.
I also manually tested to make doubly sure:
**Before**:
```python
>>> from pyspark import InheritableThread, inheritable_thread_target
>>> InheritableThread(lambda: 1).start()
>>> inheritable_thread_target(lambda: 1)()
Traceback (most recent call last):
File "/.../python3.8/lib/python3.8/threading.py", line 932, in _bootstrap_inner
self.run()
File "/.../python3.8/lib/python3.8/threading.py", line 870, in run
self._target(*self._args, **self._kwargs)
File "/.../spark/python/pyspark/util.py", line 361, in copy_local_properties
InheritableThread._clean_py4j_conn_for_current_thread()
File "/.../spark/python/pyspark/util.py", line 381, in _clean_py4j_conn_for_current_thread
thread_connection = jvm._gateway_client.get_thread_connection()
AttributeError: 'GatewayClient' object has no attribute 'get_thread_connection'
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../spark/python/pyspark/util.py", line 324, in wrapped
InheritableThread._clean_py4j_conn_for_current_thread()
File "/.../spark/python/pyspark/util.py", line 381, in _clean_py4j_conn_for_current_thread
thread_connection = jvm._gateway_client.get_thread_connection()
AttributeError: 'GatewayClient' object has no attribute 'get_thread_connection'
```
**After**:
```python
>>> from pyspark import InheritableThread, inheritable_thread_target
>>> InheritableThread(lambda: 1).start()
>>> inheritable_thread_target(lambda: 1)()
1
```
Closes #33147 from HyukjinKwon/SPARK-35946.
Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-30 01:18:54 -04:00
|
|
|
if isinstance(SparkContext._gateway, ClientServer):
|
|
|
|
# Here's when the pinned-thread mode (PYSPARK_PIN_THREAD) is on.
|
|
|
|
|
2021-06-19 22:48:38 -04:00
|
|
|
# Local property copy should happen in Thread.start to mimic JVM's behavior.
|
|
|
|
self._props = SparkContext._active_spark_context._jsc.sc().getLocalProperties().clone()
|
|
|
|
return super(InheritableThread, self).start(*args, **kwargs)
|
|
|
|
|
|
|
|
@staticmethod
|
|
|
|
def _clean_py4j_conn_for_current_thread():
|
|
|
|
from pyspark import SparkContext
|
|
|
|
|
|
|
|
jvm = SparkContext._jvm
|
[SPARK-35834][PYTHON] Use the same cleanup logic as Py4J in inheritable thread API
### What changes were proposed in this pull request?
This PR fixes the cleanup logic in inheritable thread API by following Py4J cleanup logic at https://github.com/bartdag/py4j/blob/master/py4j-python/src/py4j/clientserver.py#L269-L278.
Currently the tests that use `inheritable_thread_target` are flaky (https://github.com/apache/spark/runs/2870944288):
```
======================================================================
ERROR [71.813s]: test_save_load_pipeline_estimator (pyspark.ml.tests.test_tuning.CrossValidatorTests)
----------------------------------------------------------------------
Traceback (most recent call last):
File "/__w/spark/spark/python/pyspark/ml/tests/test_tuning.py", line 589, in test_save_load_pipeline_estimator
self._run_test_save_load_pipeline_estimator(DummyLogisticRegression)
File "/__w/spark/spark/python/pyspark/ml/tests/test_tuning.py", line 572, in _run_test_save_load_pipeline_estimator
cvModel2 = crossval2.fit(training)
File "/__w/spark/spark/python/pyspark/ml/base.py", line 161, in fit
return self._fit(dataset)
File "/__w/spark/spark/python/pyspark/ml/tuning.py", line 747, in _fit
bestModel = est.fit(dataset, epm[bestIndex])
File "/__w/spark/spark/python/pyspark/ml/base.py", line 159, in fit
return self.copy(params)._fit(dataset)
File "/__w/spark/spark/python/pyspark/ml/pipeline.py", line 114, in _fit
model = stage.fit(dataset)
File "/__w/spark/spark/python/pyspark/ml/base.py", line 161, in fit
return self._fit(dataset)
File "/__w/spark/spark/python/pyspark/ml/pipeline.py", line 114, in _fit
model = stage.fit(dataset)
File "/__w/spark/spark/python/pyspark/ml/base.py", line 161, in fit
return self._fit(dataset)
File "/__w/spark/spark/python/pyspark/ml/classification.py", line 2924, in _fit
models = pool.map(inheritable_thread_target(trainSingleClass), range(numClasses))
File "/__t/Python/3.6.13/x64/lib/python3.6/multiprocessing/pool.py", line 266, in map
return self._map_async(func, iterable, mapstar, chunksize).get()
File "/__t/Python/3.6.13/x64/lib/python3.6/multiprocessing/pool.py", line 644, in get
raise self._value
File "/__t/Python/3.6.13/x64/lib/python3.6/multiprocessing/pool.py", line 119, in worker
result = (True, func(*args, **kwds))
File "/__t/Python/3.6.13/x64/lib/python3.6/multiprocessing/pool.py", line 44, in mapstar
return list(map(*args))
File "/__w/spark/spark/python/pyspark/util.py", line 324, in wrapped
InheritableThread._clean_py4j_conn_for_current_thread()
File "/__w/spark/spark/python/pyspark/util.py", line 389, in _clean_py4j_conn_for_current_thread
del connections[i]
IndexError: deque index out of range
----------------------------------------------------------------------
```
This seems to be because the connection deque `jvm._gateway_client.deque` is accessed, and modified by other threads. Therefore, the number of threads could be changed in the middle. Using `SparkContext._lock` doesn't protect because the deque can be updated for every Java instance access in Py4J.
This PR proposes to use the atomic `deque.remove` in the problematic dequeue alone with try-catch on `ValueError` in case it's [deleted by Py4J](https://github.com/bartdag/py4j/blob/master/py4j-python/src/py4j/clientserver.py#L269-L278).
### Why are the changes needed?
To fix the flakiness in the tests, and avoid possible breakage in user application by using this API.
### Does this PR introduce _any_ user-facing change?
If users were dependent on InheritableThread with pinned thread mode on, they might have faced such issues intermittently. This PR fixes it.
### How was this patch tested?
Manually tested. CI should test it out too.
Closes #32989 from HyukjinKwon/SPARK-35834.
Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-06-20 23:00:16 -04:00
|
|
|
thread_connection = jvm._gateway_client.get_thread_connection()
|
2021-06-19 22:48:38 -04:00
|
|
|
if thread_connection is not None:
|
[SPARK-35834][PYTHON] Use the same cleanup logic as Py4J in inheritable thread API
### What changes were proposed in this pull request?
This PR fixes the cleanup logic in inheritable thread API by following Py4J cleanup logic at https://github.com/bartdag/py4j/blob/master/py4j-python/src/py4j/clientserver.py#L269-L278.
Currently the tests that use `inheritable_thread_target` are flaky (https://github.com/apache/spark/runs/2870944288):
```
======================================================================
ERROR [71.813s]: test_save_load_pipeline_estimator (pyspark.ml.tests.test_tuning.CrossValidatorTests)
----------------------------------------------------------------------
Traceback (most recent call last):
File "/__w/spark/spark/python/pyspark/ml/tests/test_tuning.py", line 589, in test_save_load_pipeline_estimator
self._run_test_save_load_pipeline_estimator(DummyLogisticRegression)
File "/__w/spark/spark/python/pyspark/ml/tests/test_tuning.py", line 572, in _run_test_save_load_pipeline_estimator
cvModel2 = crossval2.fit(training)
File "/__w/spark/spark/python/pyspark/ml/base.py", line 161, in fit
return self._fit(dataset)
File "/__w/spark/spark/python/pyspark/ml/tuning.py", line 747, in _fit
bestModel = est.fit(dataset, epm[bestIndex])
File "/__w/spark/spark/python/pyspark/ml/base.py", line 159, in fit
return self.copy(params)._fit(dataset)
File "/__w/spark/spark/python/pyspark/ml/pipeline.py", line 114, in _fit
model = stage.fit(dataset)
File "/__w/spark/spark/python/pyspark/ml/base.py", line 161, in fit
return self._fit(dataset)
File "/__w/spark/spark/python/pyspark/ml/pipeline.py", line 114, in _fit
model = stage.fit(dataset)
File "/__w/spark/spark/python/pyspark/ml/base.py", line 161, in fit
return self._fit(dataset)
File "/__w/spark/spark/python/pyspark/ml/classification.py", line 2924, in _fit
models = pool.map(inheritable_thread_target(trainSingleClass), range(numClasses))
File "/__t/Python/3.6.13/x64/lib/python3.6/multiprocessing/pool.py", line 266, in map
return self._map_async(func, iterable, mapstar, chunksize).get()
File "/__t/Python/3.6.13/x64/lib/python3.6/multiprocessing/pool.py", line 644, in get
raise self._value
File "/__t/Python/3.6.13/x64/lib/python3.6/multiprocessing/pool.py", line 119, in worker
result = (True, func(*args, **kwds))
File "/__t/Python/3.6.13/x64/lib/python3.6/multiprocessing/pool.py", line 44, in mapstar
return list(map(*args))
File "/__w/spark/spark/python/pyspark/util.py", line 324, in wrapped
InheritableThread._clean_py4j_conn_for_current_thread()
File "/__w/spark/spark/python/pyspark/util.py", line 389, in _clean_py4j_conn_for_current_thread
del connections[i]
IndexError: deque index out of range
----------------------------------------------------------------------
```
This seems to be because the connection deque `jvm._gateway_client.deque` is accessed, and modified by other threads. Therefore, the number of threads could be changed in the middle. Using `SparkContext._lock` doesn't protect because the deque can be updated for every Java instance access in Py4J.
This PR proposes to use the atomic `deque.remove` in the problematic dequeue alone with try-catch on `ValueError` in case it's [deleted by Py4J](https://github.com/bartdag/py4j/blob/master/py4j-python/src/py4j/clientserver.py#L269-L278).
### Why are the changes needed?
To fix the flakiness in the tests, and avoid possible breakage in user application by using this API.
### Does this PR introduce _any_ user-facing change?
If users were dependent on InheritableThread with pinned thread mode on, they might have faced such issues intermittently. This PR fixes it.
### How was this patch tested?
Manually tested. CI should test it out too.
Closes #32989 from HyukjinKwon/SPARK-35834.
Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-06-20 23:00:16 -04:00
|
|
|
try:
|
|
|
|
# Dequeue is shared across other threads but it's thread-safe.
|
|
|
|
# If this function has to be invoked one more time in the same thead
|
|
|
|
# Py4J will create a new connection automatically.
|
|
|
|
jvm._gateway_client.deque.remove(thread_connection)
|
|
|
|
except ValueError:
|
|
|
|
# Should never reach this point
|
|
|
|
return
|
|
|
|
finally:
|
|
|
|
thread_connection.close()
|
[SPARK-33407][PYTHON] Simplify the exception message from Python UDFs (disabled by default)
### What changes were proposed in this pull request?
This PR proposes to simplify the exception messages from Python UDFS.
Currently, the exception message from Python UDFs is as below:
```python
from pyspark.sql.functions import udf; spark.range(10).select(udf(lambda x: x/0)("id")).collect()
```
```python
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../python/pyspark/sql/dataframe.py", line 427, in show
print(self._jdf.showString(n, 20, vertical))
File "/.../python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", line 1305, in __call__
File "/.../python/pyspark/sql/utils.py", line 127, in deco
raise_from(converted)
File "<string>", line 3, in raise_from
pyspark.sql.utils.PythonException:
An exception was thrown from Python worker in the executor:
Traceback (most recent call last):
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 605, in main
process()
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 597, in process
serializer.dump_stream(out_iter, outfile)
File "/.../python/lib/pyspark.zip/pyspark/serializers.py", line 223, in dump_stream
self.serializer.dump_stream(self._batched(iterator), stream)
File "/.../python/lib/pyspark.zip/pyspark/serializers.py", line 141, in dump_stream
for obj in iterator:
File "/.../python/lib/pyspark.zip/pyspark/serializers.py", line 212, in _batched
for item in iterator:
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 450, in mapper
result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in udfs)
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 450, in <genexpr>
result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in udfs)
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 90, in <lambda>
return lambda *a: f(*a)
File "/.../python/lib/pyspark.zip/pyspark/util.py", line 107, in wrapper
return f(*args, **kwargs)
File "<stdin>", line 1, in <lambda>
ZeroDivisionError: division by zero
```
Actually, almost all cases, users only care about `ZeroDivisionError: division by zero`. We don't really have to show the internal stuff in 99% cases.
This PR adds a configuration `spark.sql.execution.pyspark.udf.simplifiedException.enabled` (disabled by default) that hides the internal tracebacks related to Python worker, (de)serialization, etc.
```python
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../python/pyspark/sql/dataframe.py", line 427, in show
print(self._jdf.showString(n, 20, vertical))
File "/.../python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", line 1305, in __call__
File "/.../python/pyspark/sql/utils.py", line 127, in deco
raise_from(converted)
File "<string>", line 3, in raise_from
pyspark.sql.utils.PythonException:
An exception was thrown from Python worker in the executor:
Traceback (most recent call last):
File "<stdin>", line 1, in <lambda>
ZeroDivisionError: division by zero
```
The trackback will be shown from the point when any non-PySpark file is seen in the traceback.
### Why are the changes needed?
Without this configuration. such internal tracebacks are exposed to users directly especially for shall or notebook users in PySpark. 99% cases people don't care about the internal Python worker, (de)serialization and related tracebacks. It just makes the exception more difficult to read. For example, one statement of `x/0` above shows a very long traceback and most of them are unnecessary.
This configuration enables the ability to show simplified tracebacks which users will likely be most interested in.
### Does this PR introduce _any_ user-facing change?
By default, no. It adds one configuration that simplifies the exception message. See the example above.
### How was this patch tested?
Manually tested:
```bash
$ pyspark --conf spark.sql.execution.pyspark.udf.simplifiedException.enabled=true
```
```python
from pyspark.sql.functions import udf; spark.sparkContext.setLogLevel("FATAL"); spark.range(10).select(udf(lambda x: x/0)("id")).collect()
```
and unittests were also added.
Closes #30309 from HyukjinKwon/SPARK-33407.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-17 00:15:31 -05:00
|
|
|
|
2021-06-19 22:48:38 -04:00
|
|
|
|
|
|
|
if __name__ == "__main__":
|
[SPARK-33407][PYTHON] Simplify the exception message from Python UDFs (disabled by default)
### What changes were proposed in this pull request?
This PR proposes to simplify the exception messages from Python UDFS.
Currently, the exception message from Python UDFs is as below:
```python
from pyspark.sql.functions import udf; spark.range(10).select(udf(lambda x: x/0)("id")).collect()
```
```python
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../python/pyspark/sql/dataframe.py", line 427, in show
print(self._jdf.showString(n, 20, vertical))
File "/.../python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", line 1305, in __call__
File "/.../python/pyspark/sql/utils.py", line 127, in deco
raise_from(converted)
File "<string>", line 3, in raise_from
pyspark.sql.utils.PythonException:
An exception was thrown from Python worker in the executor:
Traceback (most recent call last):
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 605, in main
process()
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 597, in process
serializer.dump_stream(out_iter, outfile)
File "/.../python/lib/pyspark.zip/pyspark/serializers.py", line 223, in dump_stream
self.serializer.dump_stream(self._batched(iterator), stream)
File "/.../python/lib/pyspark.zip/pyspark/serializers.py", line 141, in dump_stream
for obj in iterator:
File "/.../python/lib/pyspark.zip/pyspark/serializers.py", line 212, in _batched
for item in iterator:
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 450, in mapper
result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in udfs)
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 450, in <genexpr>
result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in udfs)
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 90, in <lambda>
return lambda *a: f(*a)
File "/.../python/lib/pyspark.zip/pyspark/util.py", line 107, in wrapper
return f(*args, **kwargs)
File "<stdin>", line 1, in <lambda>
ZeroDivisionError: division by zero
```
Actually, almost all cases, users only care about `ZeroDivisionError: division by zero`. We don't really have to show the internal stuff in 99% cases.
This PR adds a configuration `spark.sql.execution.pyspark.udf.simplifiedException.enabled` (disabled by default) that hides the internal tracebacks related to Python worker, (de)serialization, etc.
```python
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../python/pyspark/sql/dataframe.py", line 427, in show
print(self._jdf.showString(n, 20, vertical))
File "/.../python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", line 1305, in __call__
File "/.../python/pyspark/sql/utils.py", line 127, in deco
raise_from(converted)
File "<string>", line 3, in raise_from
pyspark.sql.utils.PythonException:
An exception was thrown from Python worker in the executor:
Traceback (most recent call last):
File "<stdin>", line 1, in <lambda>
ZeroDivisionError: division by zero
```
The trackback will be shown from the point when any non-PySpark file is seen in the traceback.
### Why are the changes needed?
Without this configuration. such internal tracebacks are exposed to users directly especially for shall or notebook users in PySpark. 99% cases people don't care about the internal Python worker, (de)serialization and related tracebacks. It just makes the exception more difficult to read. For example, one statement of `x/0` above shows a very long traceback and most of them are unnecessary.
This configuration enables the ability to show simplified tracebacks which users will likely be most interested in.
### Does this PR introduce _any_ user-facing change?
By default, no. It adds one configuration that simplifies the exception message. See the example above.
### How was this patch tested?
Manually tested:
```bash
$ pyspark --conf spark.sql.execution.pyspark.udf.simplifiedException.enabled=true
```
```python
from pyspark.sql.functions import udf; spark.sparkContext.setLogLevel("FATAL"); spark.range(10).select(udf(lambda x: x/0)("id")).collect()
```
and unittests were also added.
Closes #30309 from HyukjinKwon/SPARK-33407.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-17 00:15:31 -05:00
|
|
|
if "pypy" not in platform.python_implementation().lower() and sys.version_info[:2] >= (3, 7):
|
2021-06-19 22:48:38 -04:00
|
|
|
import doctest
|
|
|
|
import pyspark.util
|
|
|
|
from pyspark.context import SparkContext
|
|
|
|
|
|
|
|
globs = pyspark.util.__dict__.copy()
|
|
|
|
globs['sc'] = SparkContext('local[4]', 'PythonTest')
|
|
|
|
(failure_count, test_count) = doctest.testmod(pyspark.util, globs=globs)
|
|
|
|
globs['sc'].stop()
|
|
|
|
|
[SPARK-33407][PYTHON] Simplify the exception message from Python UDFs (disabled by default)
### What changes were proposed in this pull request?
This PR proposes to simplify the exception messages from Python UDFS.
Currently, the exception message from Python UDFs is as below:
```python
from pyspark.sql.functions import udf; spark.range(10).select(udf(lambda x: x/0)("id")).collect()
```
```python
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../python/pyspark/sql/dataframe.py", line 427, in show
print(self._jdf.showString(n, 20, vertical))
File "/.../python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", line 1305, in __call__
File "/.../python/pyspark/sql/utils.py", line 127, in deco
raise_from(converted)
File "<string>", line 3, in raise_from
pyspark.sql.utils.PythonException:
An exception was thrown from Python worker in the executor:
Traceback (most recent call last):
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 605, in main
process()
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 597, in process
serializer.dump_stream(out_iter, outfile)
File "/.../python/lib/pyspark.zip/pyspark/serializers.py", line 223, in dump_stream
self.serializer.dump_stream(self._batched(iterator), stream)
File "/.../python/lib/pyspark.zip/pyspark/serializers.py", line 141, in dump_stream
for obj in iterator:
File "/.../python/lib/pyspark.zip/pyspark/serializers.py", line 212, in _batched
for item in iterator:
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 450, in mapper
result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in udfs)
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 450, in <genexpr>
result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in udfs)
File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 90, in <lambda>
return lambda *a: f(*a)
File "/.../python/lib/pyspark.zip/pyspark/util.py", line 107, in wrapper
return f(*args, **kwargs)
File "<stdin>", line 1, in <lambda>
ZeroDivisionError: division by zero
```
Actually, almost all cases, users only care about `ZeroDivisionError: division by zero`. We don't really have to show the internal stuff in 99% cases.
This PR adds a configuration `spark.sql.execution.pyspark.udf.simplifiedException.enabled` (disabled by default) that hides the internal tracebacks related to Python worker, (de)serialization, etc.
```python
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../python/pyspark/sql/dataframe.py", line 427, in show
print(self._jdf.showString(n, 20, vertical))
File "/.../python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", line 1305, in __call__
File "/.../python/pyspark/sql/utils.py", line 127, in deco
raise_from(converted)
File "<string>", line 3, in raise_from
pyspark.sql.utils.PythonException:
An exception was thrown from Python worker in the executor:
Traceback (most recent call last):
File "<stdin>", line 1, in <lambda>
ZeroDivisionError: division by zero
```
The trackback will be shown from the point when any non-PySpark file is seen in the traceback.
### Why are the changes needed?
Without this configuration. such internal tracebacks are exposed to users directly especially for shall or notebook users in PySpark. 99% cases people don't care about the internal Python worker, (de)serialization and related tracebacks. It just makes the exception more difficult to read. For example, one statement of `x/0` above shows a very long traceback and most of them are unnecessary.
This configuration enables the ability to show simplified tracebacks which users will likely be most interested in.
### Does this PR introduce _any_ user-facing change?
By default, no. It adds one configuration that simplifies the exception message. See the example above.
### How was this patch tested?
Manually tested:
```bash
$ pyspark --conf spark.sql.execution.pyspark.udf.simplifiedException.enabled=true
```
```python
from pyspark.sql.functions import udf; spark.sparkContext.setLogLevel("FATAL"); spark.range(10).select(udf(lambda x: x/0)("id")).collect()
```
and unittests were also added.
Closes #30309 from HyukjinKwon/SPARK-33407.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-11-17 00:15:31 -05:00
|
|
|
if failure_count:
|
|
|
|
sys.exit(-1)
|