aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark/sql/utils.py
blob: 2a85ec01bc92a6a810586097acc6b85b851b9594 (plain) (blame)
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
#
# 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.
#

import py4j


class CapturedException(Exception):
    def __init__(self, desc, stackTrace):
        self.desc = desc
        self.stackTrace = stackTrace

    def __str__(self):
        return repr(self.desc)


class AnalysisException(CapturedException):
    """
    Failed to analyze a SQL query plan.
    """


class ParseException(CapturedException):
    """
    Failed to parse a SQL command.
    """


class IllegalArgumentException(CapturedException):
    """
    Passed an illegal or inappropriate argument.
    """


class StreamingQueryException(CapturedException):
    """
    Exception that stopped a :class:`StreamingQuery`.
    """


class QueryExecutionException(CapturedException):
    """
    Failed to execute a query.
    """


def capture_sql_exception(f):
    def deco(*a, **kw):
        try:
            return f(*a, **kw)
        except py4j.protocol.Py4JJavaError as e:
            s = e.java_exception.toString()
            stackTrace = '\n\t at '.join(map(lambda x: x.toString(),
                                             e.java_exception.getStackTrace()))
            if s.startswith('org.apache.spark.sql.AnalysisException: '):
                raise AnalysisException(s.split(': ', 1)[1], stackTrace)
            if s.startswith('org.apache.spark.sql.catalyst.analysis'):
                raise AnalysisException(s.split(': ', 1)[1], stackTrace)
            if s.startswith('org.apache.spark.sql.catalyst.parser.ParseException: '):
                raise ParseException(s.split(': ', 1)[1], stackTrace)
            if s.startswith('org.apache.spark.sql.streaming.StreamingQueryException: '):
                raise StreamingQueryException(s.split(': ', 1)[1], stackTrace)
            if s.startswith('org.apache.spark.sql.execution.QueryExecutionException: '):
                raise QueryExecutionException(s.split(': ', 1)[1], stackTrace)
            if s.startswith('java.lang.IllegalArgumentException: '):
                raise IllegalArgumentException(s.split(': ', 1)[1], stackTrace)
            raise
    return deco


def install_exception_handler():
    """
    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 in py4j.java_gateway (call Java API)
    py4j.java_gateway.get_return_value = patched


def toJArray(gateway, jtype, arr):
    """
    Convert python list to java type array
    :param gateway: Py4j Gateway
    :param jtype: java type of element in array
    :param arr: python type list
    """
    jarr = gateway.new_array(jtype, len(arr))
    for i in range(0, len(arr)):
        jarr[i] = arr[i]
    return jarr