ee8d661058
### What changes were proposed in this pull request? This PR proposes to move pandas related functionalities into pandas package. Namely: ```bash pyspark/sql/pandas ├── __init__.py ├── conversion.py # Conversion between pandas <> PySpark DataFrames ├── functions.py # pandas_udf ├── group_ops.py # Grouped UDF / Cogrouped UDF + groupby.apply, groupby.cogroup.apply ├── map_ops.py # Map Iter UDF + mapInPandas ├── serializers.py # pandas <> PyArrow serializers ├── types.py # Type utils between pandas <> PyArrow └── utils.py # Version requirement checks ``` In order to separately locate `groupby.apply`, `groupby.cogroup.apply`, `mapInPandas`, `toPandas`, and `createDataFrame(pdf)` under `pandas` sub-package, I had to use a mix-in approach which Scala side uses often by `trait`, and also pandas itself uses this approach (see `IndexOpsMixin` as an example) to group related functionalities. Currently, you can think it's like Scala's self typed trait. See the structure below: ```python class PandasMapOpsMixin(object): def mapInPandas(self, ...): ... return ... # other Pandas <> PySpark APIs ``` ```python class DataFrame(PandasMapOpsMixin): # other DataFrame APIs equivalent to Scala side. ``` Yes, This is a big PR but they are mostly just moving around except one case `createDataFrame` which I had to split the methods. ### Why are the changes needed? There are pandas functionalities here and there and I myself gets lost where it was. Also, when you have to make a change commonly for all of pandas related features, it's almost impossible now. Also, after this change, `DataFrame` and `SparkSession` become more consistent with Scala side since pandas is specific to Python, and this change separates pandas-specific APIs away from `DataFrame` or `SparkSession`. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests should cover. Also, I manually built the PySpark API documentation and checked. Closes #27109 from HyukjinKwon/pandas-refactoring. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
192 lines
6.1 KiB
Python
192 lines
6.1 KiB
Python
#
|
|
# 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
|
|
import sys
|
|
|
|
if sys.version_info.major >= 3:
|
|
unicode = str
|
|
|
|
|
|
class CapturedException(Exception):
|
|
def __init__(self, desc, stackTrace, cause=None):
|
|
self.desc = desc
|
|
self.stackTrace = stackTrace
|
|
self.cause = convert_exception(cause) if cause is not None else None
|
|
|
|
def __str__(self):
|
|
desc = self.desc
|
|
# encode unicode instance for python2 for human readable description
|
|
if sys.version_info.major < 3 and isinstance(desc, unicode):
|
|
return str(desc.encode('utf-8'))
|
|
else:
|
|
return str(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.
|
|
"""
|
|
|
|
|
|
class UnknownException(CapturedException):
|
|
"""
|
|
None of the above exceptions.
|
|
"""
|
|
|
|
|
|
def convert_exception(e):
|
|
s = e.toString()
|
|
stackTrace = '\n\t at '.join(map(lambda x: x.toString(), e.getStackTrace()))
|
|
c = e.getCause()
|
|
if s.startswith('org.apache.spark.sql.AnalysisException: '):
|
|
return AnalysisException(s.split(': ', 1)[1], stackTrace, c)
|
|
if s.startswith('org.apache.spark.sql.catalyst.analysis'):
|
|
return AnalysisException(s.split(': ', 1)[1], stackTrace, c)
|
|
if s.startswith('org.apache.spark.sql.catalyst.parser.ParseException: '):
|
|
return ParseException(s.split(': ', 1)[1], stackTrace, c)
|
|
if s.startswith('org.apache.spark.sql.streaming.StreamingQueryException: '):
|
|
return StreamingQueryException(s.split(': ', 1)[1], stackTrace, c)
|
|
if s.startswith('org.apache.spark.sql.execution.QueryExecutionException: '):
|
|
return QueryExecutionException(s.split(': ', 1)[1], stackTrace, c)
|
|
if s.startswith('java.lang.IllegalArgumentException: '):
|
|
return IllegalArgumentException(s.split(': ', 1)[1], stackTrace, c)
|
|
return UnknownException(s, stackTrace, c)
|
|
|
|
|
|
def capture_sql_exception(f):
|
|
def deco(*a, **kw):
|
|
try:
|
|
return f(*a, **kw)
|
|
except py4j.protocol.Py4JJavaError as e:
|
|
converted = convert_exception(e.java_exception)
|
|
if not isinstance(converted, UnknownException):
|
|
raise converted
|
|
else:
|
|
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 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
|
|
|
|
|
|
def require_test_compiled():
|
|
""" Raise Exception if test classes are not compiled
|
|
"""
|
|
import os
|
|
import glob
|
|
try:
|
|
spark_home = os.environ['SPARK_HOME']
|
|
except KeyError:
|
|
raise RuntimeError('SPARK_HOME is not defined in environment')
|
|
|
|
test_class_path = os.path.join(
|
|
spark_home, 'sql', 'core', 'target', '*', 'test-classes')
|
|
paths = glob.glob(test_class_path)
|
|
|
|
if len(paths) == 0:
|
|
raise RuntimeError(
|
|
"%s doesn't exist. Spark sql test classes are not compiled." % test_class_path)
|
|
|
|
|
|
class ForeachBatchFunction(object):
|
|
"""
|
|
This is the Python implementation of Java interface 'ForeachBatchFunction'. This wraps
|
|
the user-defined 'foreachBatch' function such that it can be called from the JVM when
|
|
the query is active.
|
|
"""
|
|
|
|
def __init__(self, sql_ctx, func):
|
|
self.sql_ctx = sql_ctx
|
|
self.func = func
|
|
|
|
def call(self, jdf, batch_id):
|
|
from pyspark.sql.dataframe import DataFrame
|
|
try:
|
|
self.func(DataFrame(jdf, self.sql_ctx), batch_id)
|
|
except Exception as e:
|
|
self.error = e
|
|
raise e
|
|
|
|
class Java:
|
|
implements = ['org.apache.spark.sql.execution.streaming.sources.PythonForeachBatchFunction']
|
|
|
|
|
|
def to_str(value):
|
|
"""
|
|
A wrapper over str(), but converts bool values to lower case strings.
|
|
If None is given, just returns None, instead of converting it to string "None".
|
|
"""
|
|
if isinstance(value, bool):
|
|
return str(value).lower()
|
|
elif value is None:
|
|
return value
|
|
else:
|
|
return str(value)
|