-
Notifications
You must be signed in to change notification settings - Fork 28.5k
/
Copy pathcaptured.py
307 lines (249 loc) · 10.6 KB
/
captured.py
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
#
# 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.
#
from contextlib import contextmanager
from typing import Any, Callable, Dict, Iterator, Optional, cast
import py4j
from py4j.protocol import Py4JJavaError
from py4j.java_gateway import is_instance_of
from pyspark import SparkContext
from pyspark.errors.exceptions.base import (
AnalysisException as BaseAnalysisException,
IllegalArgumentException as BaseIllegalArgumentException,
ArithmeticException as BaseArithmeticException,
UnsupportedOperationException as BaseUnsupportedOperationException,
ArrayIndexOutOfBoundsException as BaseArrayIndexOutOfBoundsException,
DateTimeException as BaseDateTimeException,
NumberFormatException as BaseNumberFormatException,
ParseException as BaseParseException,
PySparkException,
PythonException as BasePythonException,
QueryExecutionException as BaseQueryExecutionException,
SparkRuntimeException as BaseSparkRuntimeException,
SparkUpgradeException as BaseSparkUpgradeException,
StreamingQueryException as BaseStreamingQueryException,
UnknownException as BaseUnknownException,
)
class CapturedException(PySparkException):
def __init__(
self,
desc: Optional[str] = None,
stackTrace: Optional[str] = None,
cause: Optional[Py4JJavaError] = None,
origin: Optional[Py4JJavaError] = None,
):
# desc & stackTrace vs origin are mutually exclusive.
# cause is optional.
assert (origin is not None and desc is None and stackTrace is None) or (
origin is None and desc is not None and stackTrace is not None
)
self.desc = desc if desc is not None else cast(Py4JJavaError, origin).getMessage()
assert SparkContext._jvm is not None
self.stackTrace = (
stackTrace
if stackTrace is not None
else (SparkContext._jvm.org.apache.spark.util.Utils.exceptionString(origin))
)
self.cause = convert_exception(cause) if cause is not None else None
if self.cause is None and origin is not None and origin.getCause() is not None:
self.cause = convert_exception(origin.getCause())
self._origin = origin
def __str__(self) -> str:
assert SparkContext._jvm is not None
jvm = SparkContext._jvm
# SPARK-42752: default to True to see issues with initialization
debug_enabled = True
try:
sql_conf = jvm.org.apache.spark.sql.internal.SQLConf.get()
debug_enabled = sql_conf.pysparkJVMStacktraceEnabled()
except BaseException:
pass
desc = self.desc
if debug_enabled:
desc = desc + "\n\nJVM stacktrace:\n%s" % self.stackTrace
return str(desc)
def getErrorClass(self) -> Optional[str]:
assert SparkContext._gateway is not None
gw = SparkContext._gateway
if self._origin is not None and is_instance_of(
gw, self._origin, "org.apache.spark.SparkThrowable"
):
return self._origin.getErrorClass()
else:
return None
def getMessageParameters(self) -> Optional[Dict[str, str]]:
assert SparkContext._gateway is not None
gw = SparkContext._gateway
if self._origin is not None and is_instance_of(
gw, self._origin, "org.apache.spark.SparkThrowable"
):
return self._origin.getMessageParameters()
else:
return None
def getSqlState(self) -> Optional[str]: # type: ignore[override]
assert SparkContext._gateway is not None
gw = SparkContext._gateway
if self._origin is not None and is_instance_of(
gw, self._origin, "org.apache.spark.SparkThrowable"
):
return self._origin.getSqlState()
else:
return None
def convert_exception(e: Py4JJavaError) -> CapturedException:
assert e is not None
assert SparkContext._jvm is not None
assert SparkContext._gateway is not None
jvm = SparkContext._jvm
gw = SparkContext._gateway
if is_instance_of(gw, e, "org.apache.spark.sql.catalyst.parser.ParseException"):
return ParseException(origin=e)
# Order matters. ParseException inherits AnalysisException.
elif is_instance_of(gw, e, "org.apache.spark.sql.AnalysisException"):
return AnalysisException(origin=e)
elif is_instance_of(gw, e, "org.apache.spark.sql.streaming.StreamingQueryException"):
return StreamingQueryException(origin=e)
elif is_instance_of(gw, e, "org.apache.spark.sql.execution.QueryExecutionException"):
return QueryExecutionException(origin=e)
# Order matters. NumberFormatException inherits IllegalArgumentException.
elif is_instance_of(gw, e, "java.lang.NumberFormatException"):
return NumberFormatException(origin=e)
elif is_instance_of(gw, e, "java.lang.IllegalArgumentException"):
return IllegalArgumentException(origin=e)
elif is_instance_of(gw, e, "java.lang.ArithmeticException"):
return ArithmeticException(origin=e)
elif is_instance_of(gw, e, "java.lang.UnsupportedOperationException"):
return UnsupportedOperationException(origin=e)
elif is_instance_of(gw, e, "java.lang.ArrayIndexOutOfBoundsException"):
return ArrayIndexOutOfBoundsException(origin=e)
elif is_instance_of(gw, e, "java.time.DateTimeException"):
return DateTimeException(origin=e)
elif is_instance_of(gw, e, "org.apache.spark.SparkRuntimeException"):
return SparkRuntimeException(origin=e)
elif is_instance_of(gw, e, "org.apache.spark.SparkUpgradeException"):
return SparkUpgradeException(origin=e)
c: Py4JJavaError = e.getCause()
stacktrace: str = jvm.org.apache.spark.util.Utils.exceptionString(e)
if c is not None and (
is_instance_of(gw, c, "org.apache.spark.api.python.PythonException")
# To make sure this only catches Python UDFs.
and any(
map(
lambda v: "org.apache.spark.sql.execution.python" in v.toString(), c.getStackTrace()
)
)
):
msg = (
"\n An exception was thrown from the Python worker. "
"Please see the stack trace below.\n%s" % c.getMessage()
)
return PythonException(msg, stacktrace)
return UnknownException(desc=e.toString(), stackTrace=stacktrace, cause=c)
def capture_sql_exception(f: Callable[..., Any]) -> Callable[..., Any]:
def deco(*a: Any, **kw: Any) -> Any:
try:
return f(*a, **kw)
except Py4JJavaError as e:
converted = convert_exception(e.java_exception)
if not isinstance(converted, UnknownException):
# Hide where the exception came from that shows a non-Pythonic
# JVM exception message.
raise converted from None
else:
raise
return deco
@contextmanager
def unwrap_spark_exception() -> Iterator[Any]:
assert SparkContext._gateway is not None
gw = SparkContext._gateway
try:
yield
except Py4JJavaError as e:
je: Py4JJavaError = e.java_exception
if je is not None and is_instance_of(gw, je, "org.apache.spark.SparkException"):
converted = convert_exception(je.getCause())
if not isinstance(converted, UnknownException):
raise converted from None
raise
def install_exception_handler() -> None:
"""
Hook an exception handler into Py4j, which could capture some SQL exceptions in Java.
When calling Java API, it will call `get_return_value` to parse the returned object.
If any exception happened in JVM, the result will be Java exception object, it raise
py4j.protocol.Py4JJavaError. We replace the original `get_return_value` with one that
could capture the Java exception and throw a Python one (with the same error message).
It's idempotent, could be called multiple times.
"""
original = py4j.protocol.get_return_value
# The original `get_return_value` is not patched, it's idempotent.
patched = capture_sql_exception(original)
# only patch the one used in py4j.java_gateway (call Java API)
py4j.java_gateway.get_return_value = patched
class AnalysisException(CapturedException, BaseAnalysisException):
"""
Failed to analyze a SQL query plan.
"""
class ParseException(AnalysisException, BaseParseException):
"""
Failed to parse a SQL command.
"""
class IllegalArgumentException(CapturedException, BaseIllegalArgumentException):
"""
Passed an illegal or inappropriate argument.
"""
class StreamingQueryException(CapturedException, BaseStreamingQueryException):
"""
Exception that stopped a :class:`StreamingQuery`.
"""
class QueryExecutionException(CapturedException, BaseQueryExecutionException):
"""
Failed to execute a query.
"""
class PythonException(CapturedException, BasePythonException):
"""
Exceptions thrown from Python workers.
"""
class ArithmeticException(CapturedException, BaseArithmeticException):
"""
Arithmetic exception.
"""
class UnsupportedOperationException(CapturedException, BaseUnsupportedOperationException):
"""
Unsupported operation exception.
"""
class ArrayIndexOutOfBoundsException(CapturedException, BaseArrayIndexOutOfBoundsException):
"""
Array index out of bounds exception.
"""
class DateTimeException(CapturedException, BaseDateTimeException):
"""
Datetime exception.
"""
class NumberFormatException(IllegalArgumentException, BaseNumberFormatException):
"""
Number format exception.
"""
class SparkRuntimeException(CapturedException, BaseSparkRuntimeException):
"""
Runtime exception.
"""
class SparkUpgradeException(CapturedException, BaseSparkUpgradeException):
"""
Exception thrown because of Spark upgrade.
"""
class UnknownException(CapturedException, BaseUnknownException):
"""
None of the above exceptions.
"""