spark-instrumented-optimizer/python/pyspark/sql/tests/test_pandas_udf.py

254 lines
9.9 KiB
Python
Raw Normal View History

[SPARK-26032][PYTHON] Break large sql/tests.py files into smaller files ## What changes were proposed in this pull request? This is the official first attempt to break huge single `tests.py` file - I did it locally before few times and gave up for some reasons. Now, currently it really makes the unittests super hard to read and difficult to check. To me, it even bothers me to to scroll down the big file. It's one single 7000 lines file! This is not only readability issue. Since one big test takes most of tests time, the tests don't run in parallel fully - although it will costs to start and stop the context. We could pick up one example and follow. Given my investigation, the current style looks closer to NumPy structure and looks easier to follow. Please see https://github.com/numpy/numpy/tree/master/numpy. Basically this PR proposes to break down `pyspark/sql/tests.py` into ...: ```bash pyspark ... ├── sql ... │   ├── tests # Includes all tests broken down from 'pyspark/sql/tests.py' │ │  │ # Each matchs to module in 'pyspark/sql'. Additionally, some logical group can │ │  │ # be added. For instance, 'test_arrow.py', 'test_datasources.py' ... │   │   ├── __init__.py │   │   ├── test_appsubmit.py │   │   ├── test_arrow.py │   │   ├── test_catalog.py │   │   ├── test_column.py │   │   ├── test_conf.py │   │   ├── test_context.py │   │   ├── test_dataframe.py │   │   ├── test_datasources.py │   │   ├── test_functions.py │   │   ├── test_group.py │   │   ├── test_pandas_udf.py │   │   ├── test_pandas_udf_grouped_agg.py │   │   ├── test_pandas_udf_grouped_map.py │   │   ├── test_pandas_udf_scalar.py │   │   ├── test_pandas_udf_window.py │   │   ├── test_readwriter.py │   │   ├── test_serde.py │   │   ├── test_session.py │   │   ├── test_streaming.py │   │   ├── test_types.py │   │   ├── test_udf.py │   │   └── test_utils.py ... ├── testing # Includes testing utils that can be used in unittests. │   ├── __init__.py │   └── sqlutils.py ... ``` ## How was this patch tested? Existing tests should cover. `cd python` and `./run-tests-with-coverage`. Manually checked they are actually being ran. Each test (not officially) can be ran via: ``` SPARK_TESTING=1 ./bin/pyspark pyspark.sql.tests.test_pandas_udf_scalar ``` Note that if you're using Mac and Python 3, you might have to `OBJC_DISABLE_INITIALIZE_FORK_SAFETY=YES`. Closes #23021 from HyukjinKwon/SPARK-25344. Authored-by: hyukjinkwon <gurwls223@apache.org> Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-11-14 01:51:11 -05:00
#
# 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 unittest
from pyspark.sql.functions import udf, pandas_udf, PandasUDFType
[SPARK-26032][PYTHON] Break large sql/tests.py files into smaller files ## What changes were proposed in this pull request? This is the official first attempt to break huge single `tests.py` file - I did it locally before few times and gave up for some reasons. Now, currently it really makes the unittests super hard to read and difficult to check. To me, it even bothers me to to scroll down the big file. It's one single 7000 lines file! This is not only readability issue. Since one big test takes most of tests time, the tests don't run in parallel fully - although it will costs to start and stop the context. We could pick up one example and follow. Given my investigation, the current style looks closer to NumPy structure and looks easier to follow. Please see https://github.com/numpy/numpy/tree/master/numpy. Basically this PR proposes to break down `pyspark/sql/tests.py` into ...: ```bash pyspark ... ├── sql ... │   ├── tests # Includes all tests broken down from 'pyspark/sql/tests.py' │ │  │ # Each matchs to module in 'pyspark/sql'. Additionally, some logical group can │ │  │ # be added. For instance, 'test_arrow.py', 'test_datasources.py' ... │   │   ├── __init__.py │   │   ├── test_appsubmit.py │   │   ├── test_arrow.py │   │   ├── test_catalog.py │   │   ├── test_column.py │   │   ├── test_conf.py │   │   ├── test_context.py │   │   ├── test_dataframe.py │   │   ├── test_datasources.py │   │   ├── test_functions.py │   │   ├── test_group.py │   │   ├── test_pandas_udf.py │   │   ├── test_pandas_udf_grouped_agg.py │   │   ├── test_pandas_udf_grouped_map.py │   │   ├── test_pandas_udf_scalar.py │   │   ├── test_pandas_udf_window.py │   │   ├── test_readwriter.py │   │   ├── test_serde.py │   │   ├── test_session.py │   │   ├── test_streaming.py │   │   ├── test_types.py │   │   ├── test_udf.py │   │   └── test_utils.py ... ├── testing # Includes testing utils that can be used in unittests. │   ├── __init__.py │   └── sqlutils.py ... ``` ## How was this patch tested? Existing tests should cover. `cd python` and `./run-tests-with-coverage`. Manually checked they are actually being ran. Each test (not officially) can be ran via: ``` SPARK_TESTING=1 ./bin/pyspark pyspark.sql.tests.test_pandas_udf_scalar ``` Note that if you're using Mac and Python 3, you might have to `OBJC_DISABLE_INITIALIZE_FORK_SAFETY=YES`. Closes #23021 from HyukjinKwon/SPARK-25344. Authored-by: hyukjinkwon <gurwls223@apache.org> Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-11-14 01:51:11 -05:00
from pyspark.sql.types import *
from pyspark.sql.utils import ParseException
from pyspark.rdd import PythonEvalType
[SPARK-26032][PYTHON] Break large sql/tests.py files into smaller files ## What changes were proposed in this pull request? This is the official first attempt to break huge single `tests.py` file - I did it locally before few times and gave up for some reasons. Now, currently it really makes the unittests super hard to read and difficult to check. To me, it even bothers me to to scroll down the big file. It's one single 7000 lines file! This is not only readability issue. Since one big test takes most of tests time, the tests don't run in parallel fully - although it will costs to start and stop the context. We could pick up one example and follow. Given my investigation, the current style looks closer to NumPy structure and looks easier to follow. Please see https://github.com/numpy/numpy/tree/master/numpy. Basically this PR proposes to break down `pyspark/sql/tests.py` into ...: ```bash pyspark ... ├── sql ... │   ├── tests # Includes all tests broken down from 'pyspark/sql/tests.py' │ │  │ # Each matchs to module in 'pyspark/sql'. Additionally, some logical group can │ │  │ # be added. For instance, 'test_arrow.py', 'test_datasources.py' ... │   │   ├── __init__.py │   │   ├── test_appsubmit.py │   │   ├── test_arrow.py │   │   ├── test_catalog.py │   │   ├── test_column.py │   │   ├── test_conf.py │   │   ├── test_context.py │   │   ├── test_dataframe.py │   │   ├── test_datasources.py │   │   ├── test_functions.py │   │   ├── test_group.py │   │   ├── test_pandas_udf.py │   │   ├── test_pandas_udf_grouped_agg.py │   │   ├── test_pandas_udf_grouped_map.py │   │   ├── test_pandas_udf_scalar.py │   │   ├── test_pandas_udf_window.py │   │   ├── test_readwriter.py │   │   ├── test_serde.py │   │   ├── test_session.py │   │   ├── test_streaming.py │   │   ├── test_types.py │   │   ├── test_udf.py │   │   └── test_utils.py ... ├── testing # Includes testing utils that can be used in unittests. │   ├── __init__.py │   └── sqlutils.py ... ``` ## How was this patch tested? Existing tests should cover. `cd python` and `./run-tests-with-coverage`. Manually checked they are actually being ran. Each test (not officially) can be ran via: ``` SPARK_TESTING=1 ./bin/pyspark pyspark.sql.tests.test_pandas_udf_scalar ``` Note that if you're using Mac and Python 3, you might have to `OBJC_DISABLE_INITIALIZE_FORK_SAFETY=YES`. Closes #23021 from HyukjinKwon/SPARK-25344. Authored-by: hyukjinkwon <gurwls223@apache.org> Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-11-14 01:51:11 -05:00
from pyspark.testing.sqlutils import ReusedSQLTestCase, have_pandas, have_pyarrow, \
pandas_requirement_message, pyarrow_requirement_message
2018-11-14 23:30:52 -05:00
from pyspark.testing.utils import QuietTest
[SPARK-26032][PYTHON] Break large sql/tests.py files into smaller files ## What changes were proposed in this pull request? This is the official first attempt to break huge single `tests.py` file - I did it locally before few times and gave up for some reasons. Now, currently it really makes the unittests super hard to read and difficult to check. To me, it even bothers me to to scroll down the big file. It's one single 7000 lines file! This is not only readability issue. Since one big test takes most of tests time, the tests don't run in parallel fully - although it will costs to start and stop the context. We could pick up one example and follow. Given my investigation, the current style looks closer to NumPy structure and looks easier to follow. Please see https://github.com/numpy/numpy/tree/master/numpy. Basically this PR proposes to break down `pyspark/sql/tests.py` into ...: ```bash pyspark ... ├── sql ... │   ├── tests # Includes all tests broken down from 'pyspark/sql/tests.py' │ │  │ # Each matchs to module in 'pyspark/sql'. Additionally, some logical group can │ │  │ # be added. For instance, 'test_arrow.py', 'test_datasources.py' ... │   │   ├── __init__.py │   │   ├── test_appsubmit.py │   │   ├── test_arrow.py │   │   ├── test_catalog.py │   │   ├── test_column.py │   │   ├── test_conf.py │   │   ├── test_context.py │   │   ├── test_dataframe.py │   │   ├── test_datasources.py │   │   ├── test_functions.py │   │   ├── test_group.py │   │   ├── test_pandas_udf.py │   │   ├── test_pandas_udf_grouped_agg.py │   │   ├── test_pandas_udf_grouped_map.py │   │   ├── test_pandas_udf_scalar.py │   │   ├── test_pandas_udf_window.py │   │   ├── test_readwriter.py │   │   ├── test_serde.py │   │   ├── test_session.py │   │   ├── test_streaming.py │   │   ├── test_types.py │   │   ├── test_udf.py │   │   └── test_utils.py ... ├── testing # Includes testing utils that can be used in unittests. │   ├── __init__.py │   └── sqlutils.py ... ``` ## How was this patch tested? Existing tests should cover. `cd python` and `./run-tests-with-coverage`. Manually checked they are actually being ran. Each test (not officially) can be ran via: ``` SPARK_TESTING=1 ./bin/pyspark pyspark.sql.tests.test_pandas_udf_scalar ``` Note that if you're using Mac and Python 3, you might have to `OBJC_DISABLE_INITIALIZE_FORK_SAFETY=YES`. Closes #23021 from HyukjinKwon/SPARK-25344. Authored-by: hyukjinkwon <gurwls223@apache.org> Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-11-14 01:51:11 -05:00
from py4j.protocol import Py4JJavaError
[SPARK-26032][PYTHON] Break large sql/tests.py files into smaller files ## What changes were proposed in this pull request? This is the official first attempt to break huge single `tests.py` file - I did it locally before few times and gave up for some reasons. Now, currently it really makes the unittests super hard to read and difficult to check. To me, it even bothers me to to scroll down the big file. It's one single 7000 lines file! This is not only readability issue. Since one big test takes most of tests time, the tests don't run in parallel fully - although it will costs to start and stop the context. We could pick up one example and follow. Given my investigation, the current style looks closer to NumPy structure and looks easier to follow. Please see https://github.com/numpy/numpy/tree/master/numpy. Basically this PR proposes to break down `pyspark/sql/tests.py` into ...: ```bash pyspark ... ├── sql ... │   ├── tests # Includes all tests broken down from 'pyspark/sql/tests.py' │ │  │ # Each matchs to module in 'pyspark/sql'. Additionally, some logical group can │ │  │ # be added. For instance, 'test_arrow.py', 'test_datasources.py' ... │   │   ├── __init__.py │   │   ├── test_appsubmit.py │   │   ├── test_arrow.py │   │   ├── test_catalog.py │   │   ├── test_column.py │   │   ├── test_conf.py │   │   ├── test_context.py │   │   ├── test_dataframe.py │   │   ├── test_datasources.py │   │   ├── test_functions.py │   │   ├── test_group.py │   │   ├── test_pandas_udf.py │   │   ├── test_pandas_udf_grouped_agg.py │   │   ├── test_pandas_udf_grouped_map.py │   │   ├── test_pandas_udf_scalar.py │   │   ├── test_pandas_udf_window.py │   │   ├── test_readwriter.py │   │   ├── test_serde.py │   │   ├── test_session.py │   │   ├── test_streaming.py │   │   ├── test_types.py │   │   ├── test_udf.py │   │   └── test_utils.py ... ├── testing # Includes testing utils that can be used in unittests. │   ├── __init__.py │   └── sqlutils.py ... ``` ## How was this patch tested? Existing tests should cover. `cd python` and `./run-tests-with-coverage`. Manually checked they are actually being ran. Each test (not officially) can be ran via: ``` SPARK_TESTING=1 ./bin/pyspark pyspark.sql.tests.test_pandas_udf_scalar ``` Note that if you're using Mac and Python 3, you might have to `OBJC_DISABLE_INITIALIZE_FORK_SAFETY=YES`. Closes #23021 from HyukjinKwon/SPARK-25344. Authored-by: hyukjinkwon <gurwls223@apache.org> Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-11-14 01:51:11 -05:00
@unittest.skipIf(
not have_pandas or not have_pyarrow,
pandas_requirement_message or pyarrow_requirement_message)
class PandasUDFTests(ReusedSQLTestCase):
def test_pandas_udf_basic(self):
udf = pandas_udf(lambda x: x, DoubleType())
self.assertEqual(udf.returnType, DoubleType())
self.assertEqual(udf.evalType, PythonEvalType.SQL_SCALAR_PANDAS_UDF)
udf = pandas_udf(lambda x: x, DoubleType(), PandasUDFType.SCALAR)
self.assertEqual(udf.returnType, DoubleType())
self.assertEqual(udf.evalType, PythonEvalType.SQL_SCALAR_PANDAS_UDF)
udf = pandas_udf(lambda x: x, 'double', PandasUDFType.SCALAR)
self.assertEqual(udf.returnType, DoubleType())
self.assertEqual(udf.evalType, PythonEvalType.SQL_SCALAR_PANDAS_UDF)
udf = pandas_udf(lambda x: x, StructType([StructField("v", DoubleType())]),
PandasUDFType.GROUPED_MAP)
self.assertEqual(udf.returnType, StructType([StructField("v", DoubleType())]))
self.assertEqual(udf.evalType, PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF)
udf = pandas_udf(lambda x: x, 'v double', PandasUDFType.GROUPED_MAP)
self.assertEqual(udf.returnType, StructType([StructField("v", DoubleType())]))
self.assertEqual(udf.evalType, PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF)
udf = pandas_udf(lambda x: x, 'v double',
functionType=PandasUDFType.GROUPED_MAP)
self.assertEqual(udf.returnType, StructType([StructField("v", DoubleType())]))
self.assertEqual(udf.evalType, PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF)
udf = pandas_udf(lambda x: x, returnType='v double',
functionType=PandasUDFType.GROUPED_MAP)
self.assertEqual(udf.returnType, StructType([StructField("v", DoubleType())]))
self.assertEqual(udf.evalType, PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF)
def test_pandas_udf_decorator(self):
@pandas_udf(DoubleType())
def foo(x):
return x
self.assertEqual(foo.returnType, DoubleType())
self.assertEqual(foo.evalType, PythonEvalType.SQL_SCALAR_PANDAS_UDF)
@pandas_udf(returnType=DoubleType())
def foo(x):
return x
self.assertEqual(foo.returnType, DoubleType())
self.assertEqual(foo.evalType, PythonEvalType.SQL_SCALAR_PANDAS_UDF)
schema = StructType([StructField("v", DoubleType())])
@pandas_udf(schema, PandasUDFType.GROUPED_MAP)
def foo(x):
return x
self.assertEqual(foo.returnType, schema)
self.assertEqual(foo.evalType, PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF)
@pandas_udf('v double', PandasUDFType.GROUPED_MAP)
def foo(x):
return x
self.assertEqual(foo.returnType, schema)
self.assertEqual(foo.evalType, PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF)
@pandas_udf(schema, functionType=PandasUDFType.GROUPED_MAP)
def foo(x):
return x
self.assertEqual(foo.returnType, schema)
self.assertEqual(foo.evalType, PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF)
@pandas_udf(returnType='double', functionType=PandasUDFType.SCALAR)
def foo(x):
return x
self.assertEqual(foo.returnType, DoubleType())
self.assertEqual(foo.evalType, PythonEvalType.SQL_SCALAR_PANDAS_UDF)
@pandas_udf(returnType=schema, functionType=PandasUDFType.GROUPED_MAP)
def foo(x):
return x
self.assertEqual(foo.returnType, schema)
self.assertEqual(foo.evalType, PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF)
def test_udf_wrong_arg(self):
with QuietTest(self.sc):
with self.assertRaises(ParseException):
@pandas_udf('blah')
def foo(x):
return x
[SPARK-28264][PYTHON][SQL] Support type hints in pandas UDF and rename/move inconsistent pandas UDF types ### What changes were proposed in this pull request? This PR proposes to redesign pandas UDFs as described in [the proposal](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit?usp=sharing). ```python from pyspark.sql.functions import pandas_udf import pandas as pd pandas_udf("long") def plug_one(s: pd.Series) -> pd.Series: return s + 1 spark.range(10).select(plug_one("id")).show() ``` ``` +------------+ |plug_one(id)| +------------+ | 1| | 2| | 3| | 4| | 5| | 6| | 7| | 8| | 9| | 10| +------------+ ``` Note that, this PR address one of the future improvements described [here](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit#heading=h.h3ncjpk6ujqu), "A couple of less-intuitive pandas UDF types" (by zero323) together. In short, - Adds new way with type hints as an alternative and experimental way. ```python pandas_udf(schema='...') def func(c1: Series, c2: Series) -> DataFrame: pass ``` - Replace and/or add an alias for three types below from UDF, and make them as separate standalone APIs. So, `pandas_udf` is now consistent with regular `udf`s and other expressions. `df.mapInPandas(udf)` -replace-> `df.mapInPandas(f, schema)` `df.groupby.apply(udf)` -alias-> `df.groupby.applyInPandas(f, schema)` `df.groupby.cogroup.apply(udf)` -replace-> `df.groupby.cogroup.applyInPandas(f, schema)` *`df.groupby.apply` was added from 2.3 while the other were added in the master only. - No deprecation for the existing ways for now. ```python pandas_udf(schema='...', functionType=PandasUDFType.SCALAR) def func(c1, c2): pass ``` If users are happy with this, I plan to deprecate the existing way and declare using type hints is not experimental anymore. One design goal in this PR was that, avoid touching the internal (since we didn't deprecate the old ways for now), but supports type hints with a minimised changes only at the interface. - Once we deprecate or remove the old ways, I think it requires another refactoring for the internal in the future. At the very least, we should rename internal pandas evaluation types. - If users find this experimental type hints isn't quite helpful, we should simply revert the changes at the interface level. ### Why are the changes needed? In order to address old design issues. Please see [the proposal](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit?usp=sharing). ### Does this PR introduce any user-facing change? For behaviour changes, No. It adds new ways to use pandas UDFs by using type hints. See below. **SCALAR**: ```python pandas_udf(schema='...') def func(c1: Series, c2: DataFrame) -> Series: pass # DataFrame represents a struct column ``` **SCALAR_ITER**: ```python pandas_udf(schema='...') def func(iter: Iterator[Tuple[Series, DataFrame, ...]]) -> Iterator[Series]: pass # Same as SCALAR but wrapped by Iterator ``` **GROUPED_AGG**: ```python pandas_udf(schema='...') def func(c1: Series, c2: DataFrame) -> int: pass # DataFrame represents a struct column ``` **GROUPED_MAP**: This was added in Spark 2.3 as of SPARK-20396. As described above, it keeps the existing behaviour. Additionally, we now have a new alias `groupby.applyInPandas` for `groupby.apply`. See the example below: ```python def func(pdf): return pdf df.groupby("...").applyInPandas(func, schema=df.schema) ``` **MAP_ITER**: this is not a pandas UDF anymore This was added in Spark 3.0 as of SPARK-28198; and this PR replaces the usages. See the example below: ```python def func(iter): for df in iter: yield df df.mapInPandas(func, df.schema) ``` **COGROUPED_MAP**: this is not a pandas UDF anymore This was added in Spark 3.0 as of SPARK-27463; and this PR replaces the usages. See the example below: ```python def asof_join(left, right): return pd.merge_asof(left, right, on="...", by="...") df1.groupby("...").cogroup(df2.groupby("...")).applyInPandas(asof_join, schema="...") ``` ### How was this patch tested? Unittests added and tested against Python 2.7, 3.6 and 3.7. Closes #27165 from HyukjinKwon/revisit-pandas. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-22 01:32:58 -05:00
with self.assertRaisesRegexp(ValueError, 'Invalid return type.*None'):
[SPARK-26032][PYTHON] Break large sql/tests.py files into smaller files ## What changes were proposed in this pull request? This is the official first attempt to break huge single `tests.py` file - I did it locally before few times and gave up for some reasons. Now, currently it really makes the unittests super hard to read and difficult to check. To me, it even bothers me to to scroll down the big file. It's one single 7000 lines file! This is not only readability issue. Since one big test takes most of tests time, the tests don't run in parallel fully - although it will costs to start and stop the context. We could pick up one example and follow. Given my investigation, the current style looks closer to NumPy structure and looks easier to follow. Please see https://github.com/numpy/numpy/tree/master/numpy. Basically this PR proposes to break down `pyspark/sql/tests.py` into ...: ```bash pyspark ... ├── sql ... │   ├── tests # Includes all tests broken down from 'pyspark/sql/tests.py' │ │  │ # Each matchs to module in 'pyspark/sql'. Additionally, some logical group can │ │  │ # be added. For instance, 'test_arrow.py', 'test_datasources.py' ... │   │   ├── __init__.py │   │   ├── test_appsubmit.py │   │   ├── test_arrow.py │   │   ├── test_catalog.py │   │   ├── test_column.py │   │   ├── test_conf.py │   │   ├── test_context.py │   │   ├── test_dataframe.py │   │   ├── test_datasources.py │   │   ├── test_functions.py │   │   ├── test_group.py │   │   ├── test_pandas_udf.py │   │   ├── test_pandas_udf_grouped_agg.py │   │   ├── test_pandas_udf_grouped_map.py │   │   ├── test_pandas_udf_scalar.py │   │   ├── test_pandas_udf_window.py │   │   ├── test_readwriter.py │   │   ├── test_serde.py │   │   ├── test_session.py │   │   ├── test_streaming.py │   │   ├── test_types.py │   │   ├── test_udf.py │   │   └── test_utils.py ... ├── testing # Includes testing utils that can be used in unittests. │   ├── __init__.py │   └── sqlutils.py ... ``` ## How was this patch tested? Existing tests should cover. `cd python` and `./run-tests-with-coverage`. Manually checked they are actually being ran. Each test (not officially) can be ran via: ``` SPARK_TESTING=1 ./bin/pyspark pyspark.sql.tests.test_pandas_udf_scalar ``` Note that if you're using Mac and Python 3, you might have to `OBJC_DISABLE_INITIALIZE_FORK_SAFETY=YES`. Closes #23021 from HyukjinKwon/SPARK-25344. Authored-by: hyukjinkwon <gurwls223@apache.org> Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-11-14 01:51:11 -05:00
@pandas_udf(functionType=PandasUDFType.SCALAR)
def foo(x):
return x
[SPARK-28264][PYTHON][SQL] Support type hints in pandas UDF and rename/move inconsistent pandas UDF types ### What changes were proposed in this pull request? This PR proposes to redesign pandas UDFs as described in [the proposal](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit?usp=sharing). ```python from pyspark.sql.functions import pandas_udf import pandas as pd pandas_udf("long") def plug_one(s: pd.Series) -> pd.Series: return s + 1 spark.range(10).select(plug_one("id")).show() ``` ``` +------------+ |plug_one(id)| +------------+ | 1| | 2| | 3| | 4| | 5| | 6| | 7| | 8| | 9| | 10| +------------+ ``` Note that, this PR address one of the future improvements described [here](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit#heading=h.h3ncjpk6ujqu), "A couple of less-intuitive pandas UDF types" (by zero323) together. In short, - Adds new way with type hints as an alternative and experimental way. ```python pandas_udf(schema='...') def func(c1: Series, c2: Series) -> DataFrame: pass ``` - Replace and/or add an alias for three types below from UDF, and make them as separate standalone APIs. So, `pandas_udf` is now consistent with regular `udf`s and other expressions. `df.mapInPandas(udf)` -replace-> `df.mapInPandas(f, schema)` `df.groupby.apply(udf)` -alias-> `df.groupby.applyInPandas(f, schema)` `df.groupby.cogroup.apply(udf)` -replace-> `df.groupby.cogroup.applyInPandas(f, schema)` *`df.groupby.apply` was added from 2.3 while the other were added in the master only. - No deprecation for the existing ways for now. ```python pandas_udf(schema='...', functionType=PandasUDFType.SCALAR) def func(c1, c2): pass ``` If users are happy with this, I plan to deprecate the existing way and declare using type hints is not experimental anymore. One design goal in this PR was that, avoid touching the internal (since we didn't deprecate the old ways for now), but supports type hints with a minimised changes only at the interface. - Once we deprecate or remove the old ways, I think it requires another refactoring for the internal in the future. At the very least, we should rename internal pandas evaluation types. - If users find this experimental type hints isn't quite helpful, we should simply revert the changes at the interface level. ### Why are the changes needed? In order to address old design issues. Please see [the proposal](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit?usp=sharing). ### Does this PR introduce any user-facing change? For behaviour changes, No. It adds new ways to use pandas UDFs by using type hints. See below. **SCALAR**: ```python pandas_udf(schema='...') def func(c1: Series, c2: DataFrame) -> Series: pass # DataFrame represents a struct column ``` **SCALAR_ITER**: ```python pandas_udf(schema='...') def func(iter: Iterator[Tuple[Series, DataFrame, ...]]) -> Iterator[Series]: pass # Same as SCALAR but wrapped by Iterator ``` **GROUPED_AGG**: ```python pandas_udf(schema='...') def func(c1: Series, c2: DataFrame) -> int: pass # DataFrame represents a struct column ``` **GROUPED_MAP**: This was added in Spark 2.3 as of SPARK-20396. As described above, it keeps the existing behaviour. Additionally, we now have a new alias `groupby.applyInPandas` for `groupby.apply`. See the example below: ```python def func(pdf): return pdf df.groupby("...").applyInPandas(func, schema=df.schema) ``` **MAP_ITER**: this is not a pandas UDF anymore This was added in Spark 3.0 as of SPARK-28198; and this PR replaces the usages. See the example below: ```python def func(iter): for df in iter: yield df df.mapInPandas(func, df.schema) ``` **COGROUPED_MAP**: this is not a pandas UDF anymore This was added in Spark 3.0 as of SPARK-27463; and this PR replaces the usages. See the example below: ```python def asof_join(left, right): return pd.merge_asof(left, right, on="...", by="...") df1.groupby("...").cogroup(df2.groupby("...")).applyInPandas(asof_join, schema="...") ``` ### How was this patch tested? Unittests added and tested against Python 2.7, 3.6 and 3.7. Closes #27165 from HyukjinKwon/revisit-pandas. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-22 01:32:58 -05:00
with self.assertRaisesRegexp(ValueError, 'Invalid function'):
[SPARK-26032][PYTHON] Break large sql/tests.py files into smaller files ## What changes were proposed in this pull request? This is the official first attempt to break huge single `tests.py` file - I did it locally before few times and gave up for some reasons. Now, currently it really makes the unittests super hard to read and difficult to check. To me, it even bothers me to to scroll down the big file. It's one single 7000 lines file! This is not only readability issue. Since one big test takes most of tests time, the tests don't run in parallel fully - although it will costs to start and stop the context. We could pick up one example and follow. Given my investigation, the current style looks closer to NumPy structure and looks easier to follow. Please see https://github.com/numpy/numpy/tree/master/numpy. Basically this PR proposes to break down `pyspark/sql/tests.py` into ...: ```bash pyspark ... ├── sql ... │   ├── tests # Includes all tests broken down from 'pyspark/sql/tests.py' │ │  │ # Each matchs to module in 'pyspark/sql'. Additionally, some logical group can │ │  │ # be added. For instance, 'test_arrow.py', 'test_datasources.py' ... │   │   ├── __init__.py │   │   ├── test_appsubmit.py │   │   ├── test_arrow.py │   │   ├── test_catalog.py │   │   ├── test_column.py │   │   ├── test_conf.py │   │   ├── test_context.py │   │   ├── test_dataframe.py │   │   ├── test_datasources.py │   │   ├── test_functions.py │   │   ├── test_group.py │   │   ├── test_pandas_udf.py │   │   ├── test_pandas_udf_grouped_agg.py │   │   ├── test_pandas_udf_grouped_map.py │   │   ├── test_pandas_udf_scalar.py │   │   ├── test_pandas_udf_window.py │   │   ├── test_readwriter.py │   │   ├── test_serde.py │   │   ├── test_session.py │   │   ├── test_streaming.py │   │   ├── test_types.py │   │   ├── test_udf.py │   │   └── test_utils.py ... ├── testing # Includes testing utils that can be used in unittests. │   ├── __init__.py │   └── sqlutils.py ... ``` ## How was this patch tested? Existing tests should cover. `cd python` and `./run-tests-with-coverage`. Manually checked they are actually being ran. Each test (not officially) can be ran via: ``` SPARK_TESTING=1 ./bin/pyspark pyspark.sql.tests.test_pandas_udf_scalar ``` Note that if you're using Mac and Python 3, you might have to `OBJC_DISABLE_INITIALIZE_FORK_SAFETY=YES`. Closes #23021 from HyukjinKwon/SPARK-25344. Authored-by: hyukjinkwon <gurwls223@apache.org> Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-11-14 01:51:11 -05:00
@pandas_udf('double', 100)
def foo(x):
return x
with self.assertRaisesRegexp(ValueError, '0-arg pandas_udfs.*not.*supported'):
pandas_udf(lambda: 1, LongType(), PandasUDFType.SCALAR)
with self.assertRaisesRegexp(ValueError, '0-arg pandas_udfs.*not.*supported'):
@pandas_udf(LongType(), PandasUDFType.SCALAR)
def zero_with_type():
return 1
[SPARK-28264][PYTHON][SQL] Support type hints in pandas UDF and rename/move inconsistent pandas UDF types ### What changes were proposed in this pull request? This PR proposes to redesign pandas UDFs as described in [the proposal](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit?usp=sharing). ```python from pyspark.sql.functions import pandas_udf import pandas as pd pandas_udf("long") def plug_one(s: pd.Series) -> pd.Series: return s + 1 spark.range(10).select(plug_one("id")).show() ``` ``` +------------+ |plug_one(id)| +------------+ | 1| | 2| | 3| | 4| | 5| | 6| | 7| | 8| | 9| | 10| +------------+ ``` Note that, this PR address one of the future improvements described [here](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit#heading=h.h3ncjpk6ujqu), "A couple of less-intuitive pandas UDF types" (by zero323) together. In short, - Adds new way with type hints as an alternative and experimental way. ```python pandas_udf(schema='...') def func(c1: Series, c2: Series) -> DataFrame: pass ``` - Replace and/or add an alias for three types below from UDF, and make them as separate standalone APIs. So, `pandas_udf` is now consistent with regular `udf`s and other expressions. `df.mapInPandas(udf)` -replace-> `df.mapInPandas(f, schema)` `df.groupby.apply(udf)` -alias-> `df.groupby.applyInPandas(f, schema)` `df.groupby.cogroup.apply(udf)` -replace-> `df.groupby.cogroup.applyInPandas(f, schema)` *`df.groupby.apply` was added from 2.3 while the other were added in the master only. - No deprecation for the existing ways for now. ```python pandas_udf(schema='...', functionType=PandasUDFType.SCALAR) def func(c1, c2): pass ``` If users are happy with this, I plan to deprecate the existing way and declare using type hints is not experimental anymore. One design goal in this PR was that, avoid touching the internal (since we didn't deprecate the old ways for now), but supports type hints with a minimised changes only at the interface. - Once we deprecate or remove the old ways, I think it requires another refactoring for the internal in the future. At the very least, we should rename internal pandas evaluation types. - If users find this experimental type hints isn't quite helpful, we should simply revert the changes at the interface level. ### Why are the changes needed? In order to address old design issues. Please see [the proposal](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit?usp=sharing). ### Does this PR introduce any user-facing change? For behaviour changes, No. It adds new ways to use pandas UDFs by using type hints. See below. **SCALAR**: ```python pandas_udf(schema='...') def func(c1: Series, c2: DataFrame) -> Series: pass # DataFrame represents a struct column ``` **SCALAR_ITER**: ```python pandas_udf(schema='...') def func(iter: Iterator[Tuple[Series, DataFrame, ...]]) -> Iterator[Series]: pass # Same as SCALAR but wrapped by Iterator ``` **GROUPED_AGG**: ```python pandas_udf(schema='...') def func(c1: Series, c2: DataFrame) -> int: pass # DataFrame represents a struct column ``` **GROUPED_MAP**: This was added in Spark 2.3 as of SPARK-20396. As described above, it keeps the existing behaviour. Additionally, we now have a new alias `groupby.applyInPandas` for `groupby.apply`. See the example below: ```python def func(pdf): return pdf df.groupby("...").applyInPandas(func, schema=df.schema) ``` **MAP_ITER**: this is not a pandas UDF anymore This was added in Spark 3.0 as of SPARK-28198; and this PR replaces the usages. See the example below: ```python def func(iter): for df in iter: yield df df.mapInPandas(func, df.schema) ``` **COGROUPED_MAP**: this is not a pandas UDF anymore This was added in Spark 3.0 as of SPARK-27463; and this PR replaces the usages. See the example below: ```python def asof_join(left, right): return pd.merge_asof(left, right, on="...", by="...") df1.groupby("...").cogroup(df2.groupby("...")).applyInPandas(asof_join, schema="...") ``` ### How was this patch tested? Unittests added and tested against Python 2.7, 3.6 and 3.7. Closes #27165 from HyukjinKwon/revisit-pandas. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-22 01:32:58 -05:00
with self.assertRaisesRegexp(TypeError, 'Invalid return type'):
[SPARK-26032][PYTHON] Break large sql/tests.py files into smaller files ## What changes were proposed in this pull request? This is the official first attempt to break huge single `tests.py` file - I did it locally before few times and gave up for some reasons. Now, currently it really makes the unittests super hard to read and difficult to check. To me, it even bothers me to to scroll down the big file. It's one single 7000 lines file! This is not only readability issue. Since one big test takes most of tests time, the tests don't run in parallel fully - although it will costs to start and stop the context. We could pick up one example and follow. Given my investigation, the current style looks closer to NumPy structure and looks easier to follow. Please see https://github.com/numpy/numpy/tree/master/numpy. Basically this PR proposes to break down `pyspark/sql/tests.py` into ...: ```bash pyspark ... ├── sql ... │   ├── tests # Includes all tests broken down from 'pyspark/sql/tests.py' │ │  │ # Each matchs to module in 'pyspark/sql'. Additionally, some logical group can │ │  │ # be added. For instance, 'test_arrow.py', 'test_datasources.py' ... │   │   ├── __init__.py │   │   ├── test_appsubmit.py │   │   ├── test_arrow.py │   │   ├── test_catalog.py │   │   ├── test_column.py │   │   ├── test_conf.py │   │   ├── test_context.py │   │   ├── test_dataframe.py │   │   ├── test_datasources.py │   │   ├── test_functions.py │   │   ├── test_group.py │   │   ├── test_pandas_udf.py │   │   ├── test_pandas_udf_grouped_agg.py │   │   ├── test_pandas_udf_grouped_map.py │   │   ├── test_pandas_udf_scalar.py │   │   ├── test_pandas_udf_window.py │   │   ├── test_readwriter.py │   │   ├── test_serde.py │   │   ├── test_session.py │   │   ├── test_streaming.py │   │   ├── test_types.py │   │   ├── test_udf.py │   │   └── test_utils.py ... ├── testing # Includes testing utils that can be used in unittests. │   ├── __init__.py │   └── sqlutils.py ... ``` ## How was this patch tested? Existing tests should cover. `cd python` and `./run-tests-with-coverage`. Manually checked they are actually being ran. Each test (not officially) can be ran via: ``` SPARK_TESTING=1 ./bin/pyspark pyspark.sql.tests.test_pandas_udf_scalar ``` Note that if you're using Mac and Python 3, you might have to `OBJC_DISABLE_INITIALIZE_FORK_SAFETY=YES`. Closes #23021 from HyukjinKwon/SPARK-25344. Authored-by: hyukjinkwon <gurwls223@apache.org> Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-11-14 01:51:11 -05:00
@pandas_udf(returnType=PandasUDFType.GROUPED_MAP)
def foo(df):
return df
[SPARK-28264][PYTHON][SQL] Support type hints in pandas UDF and rename/move inconsistent pandas UDF types ### What changes were proposed in this pull request? This PR proposes to redesign pandas UDFs as described in [the proposal](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit?usp=sharing). ```python from pyspark.sql.functions import pandas_udf import pandas as pd pandas_udf("long") def plug_one(s: pd.Series) -> pd.Series: return s + 1 spark.range(10).select(plug_one("id")).show() ``` ``` +------------+ |plug_one(id)| +------------+ | 1| | 2| | 3| | 4| | 5| | 6| | 7| | 8| | 9| | 10| +------------+ ``` Note that, this PR address one of the future improvements described [here](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit#heading=h.h3ncjpk6ujqu), "A couple of less-intuitive pandas UDF types" (by zero323) together. In short, - Adds new way with type hints as an alternative and experimental way. ```python pandas_udf(schema='...') def func(c1: Series, c2: Series) -> DataFrame: pass ``` - Replace and/or add an alias for three types below from UDF, and make them as separate standalone APIs. So, `pandas_udf` is now consistent with regular `udf`s and other expressions. `df.mapInPandas(udf)` -replace-> `df.mapInPandas(f, schema)` `df.groupby.apply(udf)` -alias-> `df.groupby.applyInPandas(f, schema)` `df.groupby.cogroup.apply(udf)` -replace-> `df.groupby.cogroup.applyInPandas(f, schema)` *`df.groupby.apply` was added from 2.3 while the other were added in the master only. - No deprecation for the existing ways for now. ```python pandas_udf(schema='...', functionType=PandasUDFType.SCALAR) def func(c1, c2): pass ``` If users are happy with this, I plan to deprecate the existing way and declare using type hints is not experimental anymore. One design goal in this PR was that, avoid touching the internal (since we didn't deprecate the old ways for now), but supports type hints with a minimised changes only at the interface. - Once we deprecate or remove the old ways, I think it requires another refactoring for the internal in the future. At the very least, we should rename internal pandas evaluation types. - If users find this experimental type hints isn't quite helpful, we should simply revert the changes at the interface level. ### Why are the changes needed? In order to address old design issues. Please see [the proposal](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit?usp=sharing). ### Does this PR introduce any user-facing change? For behaviour changes, No. It adds new ways to use pandas UDFs by using type hints. See below. **SCALAR**: ```python pandas_udf(schema='...') def func(c1: Series, c2: DataFrame) -> Series: pass # DataFrame represents a struct column ``` **SCALAR_ITER**: ```python pandas_udf(schema='...') def func(iter: Iterator[Tuple[Series, DataFrame, ...]]) -> Iterator[Series]: pass # Same as SCALAR but wrapped by Iterator ``` **GROUPED_AGG**: ```python pandas_udf(schema='...') def func(c1: Series, c2: DataFrame) -> int: pass # DataFrame represents a struct column ``` **GROUPED_MAP**: This was added in Spark 2.3 as of SPARK-20396. As described above, it keeps the existing behaviour. Additionally, we now have a new alias `groupby.applyInPandas` for `groupby.apply`. See the example below: ```python def func(pdf): return pdf df.groupby("...").applyInPandas(func, schema=df.schema) ``` **MAP_ITER**: this is not a pandas UDF anymore This was added in Spark 3.0 as of SPARK-28198; and this PR replaces the usages. See the example below: ```python def func(iter): for df in iter: yield df df.mapInPandas(func, df.schema) ``` **COGROUPED_MAP**: this is not a pandas UDF anymore This was added in Spark 3.0 as of SPARK-27463; and this PR replaces the usages. See the example below: ```python def asof_join(left, right): return pd.merge_asof(left, right, on="...", by="...") df1.groupby("...").cogroup(df2.groupby("...")).applyInPandas(asof_join, schema="...") ``` ### How was this patch tested? Unittests added and tested against Python 2.7, 3.6 and 3.7. Closes #27165 from HyukjinKwon/revisit-pandas. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-22 01:32:58 -05:00
with self.assertRaisesRegexp(TypeError, 'Invalid return type'):
[SPARK-26032][PYTHON] Break large sql/tests.py files into smaller files ## What changes were proposed in this pull request? This is the official first attempt to break huge single `tests.py` file - I did it locally before few times and gave up for some reasons. Now, currently it really makes the unittests super hard to read and difficult to check. To me, it even bothers me to to scroll down the big file. It's one single 7000 lines file! This is not only readability issue. Since one big test takes most of tests time, the tests don't run in parallel fully - although it will costs to start and stop the context. We could pick up one example and follow. Given my investigation, the current style looks closer to NumPy structure and looks easier to follow. Please see https://github.com/numpy/numpy/tree/master/numpy. Basically this PR proposes to break down `pyspark/sql/tests.py` into ...: ```bash pyspark ... ├── sql ... │   ├── tests # Includes all tests broken down from 'pyspark/sql/tests.py' │ │  │ # Each matchs to module in 'pyspark/sql'. Additionally, some logical group can │ │  │ # be added. For instance, 'test_arrow.py', 'test_datasources.py' ... │   │   ├── __init__.py │   │   ├── test_appsubmit.py │   │   ├── test_arrow.py │   │   ├── test_catalog.py │   │   ├── test_column.py │   │   ├── test_conf.py │   │   ├── test_context.py │   │   ├── test_dataframe.py │   │   ├── test_datasources.py │   │   ├── test_functions.py │   │   ├── test_group.py │   │   ├── test_pandas_udf.py │   │   ├── test_pandas_udf_grouped_agg.py │   │   ├── test_pandas_udf_grouped_map.py │   │   ├── test_pandas_udf_scalar.py │   │   ├── test_pandas_udf_window.py │   │   ├── test_readwriter.py │   │   ├── test_serde.py │   │   ├── test_session.py │   │   ├── test_streaming.py │   │   ├── test_types.py │   │   ├── test_udf.py │   │   └── test_utils.py ... ├── testing # Includes testing utils that can be used in unittests. │   ├── __init__.py │   └── sqlutils.py ... ``` ## How was this patch tested? Existing tests should cover. `cd python` and `./run-tests-with-coverage`. Manually checked they are actually being ran. Each test (not officially) can be ran via: ``` SPARK_TESTING=1 ./bin/pyspark pyspark.sql.tests.test_pandas_udf_scalar ``` Note that if you're using Mac and Python 3, you might have to `OBJC_DISABLE_INITIALIZE_FORK_SAFETY=YES`. Closes #23021 from HyukjinKwon/SPARK-25344. Authored-by: hyukjinkwon <gurwls223@apache.org> Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-11-14 01:51:11 -05:00
@pandas_udf(returnType='double', functionType=PandasUDFType.GROUPED_MAP)
def foo(df):
return df
with self.assertRaisesRegexp(ValueError, 'Invalid function'):
@pandas_udf(returnType='k int, v double', functionType=PandasUDFType.GROUPED_MAP)
def foo(k, v, w):
return k
def test_stopiteration_in_udf(self):
def foo(x):
raise StopIteration()
def foofoo(x, y):
raise StopIteration()
exc_message = "Caught StopIteration thrown from user's code; failing the task"
df = self.spark.range(0, 100)
# plain udf (test for SPARK-23754)
self.assertRaisesRegexp(
Py4JJavaError,
exc_message,
df.withColumn('v', udf(foo)('id')).collect
)
# pandas scalar udf
self.assertRaisesRegexp(
Py4JJavaError,
exc_message,
df.withColumn(
'v', pandas_udf(foo, 'double', PandasUDFType.SCALAR)('id')
).collect
)
# pandas grouped map
self.assertRaisesRegexp(
Py4JJavaError,
exc_message,
df.groupBy('id').apply(
pandas_udf(foo, df.schema, PandasUDFType.GROUPED_MAP)
).collect
)
self.assertRaisesRegexp(
Py4JJavaError,
exc_message,
df.groupBy('id').apply(
pandas_udf(foofoo, df.schema, PandasUDFType.GROUPED_MAP)
).collect
)
# pandas grouped agg
self.assertRaisesRegexp(
Py4JJavaError,
exc_message,
df.groupBy('id').agg(
pandas_udf(foo, 'double', PandasUDFType.GROUPED_AGG)('id')
).collect
)
def test_pandas_udf_detect_unsafe_type_conversion(self):
import pandas as pd
import numpy as np
values = [1.0] * 3
pdf = pd.DataFrame({'A': values})
df = self.spark.createDataFrame(pdf).repartition(1)
@pandas_udf(returnType="int")
def udf(column):
return pd.Series(np.linspace(0, 1, len(column)))
# Since 0.11.0, PyArrow supports the feature to raise an error for unsafe cast.
with self.sql_conf({
"spark.sql.execution.pandas.arrowSafeTypeConversion": True}):
with self.assertRaisesRegexp(Exception,
"Exception thrown when converting pandas.Series"):
df.select(['A']).withColumn('udf', udf('A')).collect()
# Disabling Arrow safe type check.
with self.sql_conf({
"spark.sql.execution.pandas.arrowSafeTypeConversion": False}):
df.select(['A']).withColumn('udf', udf('A')).collect()
def test_pandas_udf_arrow_overflow(self):
import pandas as pd
df = self.spark.range(0, 1)
@pandas_udf(returnType="byte")
def udf(column):
return pd.Series([128] * len(column))
# When enabling safe type check, Arrow 0.11.0+ disallows overflow cast.
with self.sql_conf({
"spark.sql.execution.pandas.arrowSafeTypeConversion": True}):
with self.assertRaisesRegexp(Exception,
"Exception thrown when converting pandas.Series"):
df.withColumn('udf', udf('id')).collect()
# Disabling safe type check, let Arrow do the cast anyway.
with self.sql_conf({"spark.sql.execution.pandas.arrowSafeTypeConversion": False}):
df.withColumn('udf', udf('id')).collect()
[SPARK-26032][PYTHON] Break large sql/tests.py files into smaller files ## What changes were proposed in this pull request? This is the official first attempt to break huge single `tests.py` file - I did it locally before few times and gave up for some reasons. Now, currently it really makes the unittests super hard to read and difficult to check. To me, it even bothers me to to scroll down the big file. It's one single 7000 lines file! This is not only readability issue. Since one big test takes most of tests time, the tests don't run in parallel fully - although it will costs to start and stop the context. We could pick up one example and follow. Given my investigation, the current style looks closer to NumPy structure and looks easier to follow. Please see https://github.com/numpy/numpy/tree/master/numpy. Basically this PR proposes to break down `pyspark/sql/tests.py` into ...: ```bash pyspark ... ├── sql ... │   ├── tests # Includes all tests broken down from 'pyspark/sql/tests.py' │ │  │ # Each matchs to module in 'pyspark/sql'. Additionally, some logical group can │ │  │ # be added. For instance, 'test_arrow.py', 'test_datasources.py' ... │   │   ├── __init__.py │   │   ├── test_appsubmit.py │   │   ├── test_arrow.py │   │   ├── test_catalog.py │   │   ├── test_column.py │   │   ├── test_conf.py │   │   ├── test_context.py │   │   ├── test_dataframe.py │   │   ├── test_datasources.py │   │   ├── test_functions.py │   │   ├── test_group.py │   │   ├── test_pandas_udf.py │   │   ├── test_pandas_udf_grouped_agg.py │   │   ├── test_pandas_udf_grouped_map.py │   │   ├── test_pandas_udf_scalar.py │   │   ├── test_pandas_udf_window.py │   │   ├── test_readwriter.py │   │   ├── test_serde.py │   │   ├── test_session.py │   │   ├── test_streaming.py │   │   ├── test_types.py │   │   ├── test_udf.py │   │   └── test_utils.py ... ├── testing # Includes testing utils that can be used in unittests. │   ├── __init__.py │   └── sqlutils.py ... ``` ## How was this patch tested? Existing tests should cover. `cd python` and `./run-tests-with-coverage`. Manually checked they are actually being ran. Each test (not officially) can be ran via: ``` SPARK_TESTING=1 ./bin/pyspark pyspark.sql.tests.test_pandas_udf_scalar ``` Note that if you're using Mac and Python 3, you might have to `OBJC_DISABLE_INITIALIZE_FORK_SAFETY=YES`. Closes #23021 from HyukjinKwon/SPARK-25344. Authored-by: hyukjinkwon <gurwls223@apache.org> Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-11-14 01:51:11 -05:00
if __name__ == "__main__":
from pyspark.sql.tests.test_pandas_udf import *
try:
import xmlrunner
[SPARK-28130][PYTHON] Print pretty messages for skipped tests when xmlrunner is available in PySpark ## What changes were proposed in this pull request? Currently, pretty skipped message added by https://github.com/apache/spark/commit/f7435bec6a9348cfbbe26b13c230c08545d16067 mechanism seems not working when xmlrunner is installed apparently. This PR fixes two things: 1. When `xmlrunner` is installed, seems `xmlrunner` does not respect `vervosity` level in unittests (default is level 1). So the output looks as below ``` Running tests... ---------------------------------------------------------------------- SSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSS ---------------------------------------------------------------------- ``` So it is not caught by our message detection mechanism. 2. If we manually set the `vervocity` level to `xmlrunner`, it prints messages as below: ``` test_mixed_udf (pyspark.sql.tests.test_pandas_udf_scalar.ScalarPandasUDFTests) ... SKIP (0.000s) test_mixed_udf_and_sql (pyspark.sql.tests.test_pandas_udf_scalar.ScalarPandasUDFTests) ... SKIP (0.000s) ... ``` This is different in our Jenkins machine: ``` test_createDataFrame_column_name_encoding (pyspark.sql.tests.test_arrow.ArrowTests) ... skipped 'Pandas >= 0.23.2 must be installed; however, it was not found.' test_createDataFrame_does_not_modify_input (pyspark.sql.tests.test_arrow.ArrowTests) ... skipped 'Pandas >= 0.23.2 must be installed; however, it was not found.' ... ``` Note that last `SKIP` is different. This PR fixes the regular expression to catch `SKIP` case as well. ## How was this patch tested? Manually tested. **Before:** ``` Starting test(python2.7): pyspark.... Finished test(python2.7): pyspark.... (0s) ... Tests passed in 562 seconds ======================================================================== ... ``` **After:** ``` Starting test(python2.7): pyspark.... Finished test(python2.7): pyspark.... (48s) ... 93 tests were skipped ... Tests passed in 560 seconds Skipped tests pyspark.... with python2.7: pyspark...(...) ... SKIP (0.000s) ... ======================================================================== ... ``` Closes #24927 from HyukjinKwon/SPARK-28130. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-23 20:58:17 -04:00
testRunner = xmlrunner.XMLTestRunner(output='target/test-reports', verbosity=2)
[SPARK-26032][PYTHON] Break large sql/tests.py files into smaller files ## What changes were proposed in this pull request? This is the official first attempt to break huge single `tests.py` file - I did it locally before few times and gave up for some reasons. Now, currently it really makes the unittests super hard to read and difficult to check. To me, it even bothers me to to scroll down the big file. It's one single 7000 lines file! This is not only readability issue. Since one big test takes most of tests time, the tests don't run in parallel fully - although it will costs to start and stop the context. We could pick up one example and follow. Given my investigation, the current style looks closer to NumPy structure and looks easier to follow. Please see https://github.com/numpy/numpy/tree/master/numpy. Basically this PR proposes to break down `pyspark/sql/tests.py` into ...: ```bash pyspark ... ├── sql ... │   ├── tests # Includes all tests broken down from 'pyspark/sql/tests.py' │ │  │ # Each matchs to module in 'pyspark/sql'. Additionally, some logical group can │ │  │ # be added. For instance, 'test_arrow.py', 'test_datasources.py' ... │   │   ├── __init__.py │   │   ├── test_appsubmit.py │   │   ├── test_arrow.py │   │   ├── test_catalog.py │   │   ├── test_column.py │   │   ├── test_conf.py │   │   ├── test_context.py │   │   ├── test_dataframe.py │   │   ├── test_datasources.py │   │   ├── test_functions.py │   │   ├── test_group.py │   │   ├── test_pandas_udf.py │   │   ├── test_pandas_udf_grouped_agg.py │   │   ├── test_pandas_udf_grouped_map.py │   │   ├── test_pandas_udf_scalar.py │   │   ├── test_pandas_udf_window.py │   │   ├── test_readwriter.py │   │   ├── test_serde.py │   │   ├── test_session.py │   │   ├── test_streaming.py │   │   ├── test_types.py │   │   ├── test_udf.py │   │   └── test_utils.py ... ├── testing # Includes testing utils that can be used in unittests. │   ├── __init__.py │   └── sqlutils.py ... ``` ## How was this patch tested? Existing tests should cover. `cd python` and `./run-tests-with-coverage`. Manually checked they are actually being ran. Each test (not officially) can be ran via: ``` SPARK_TESTING=1 ./bin/pyspark pyspark.sql.tests.test_pandas_udf_scalar ``` Note that if you're using Mac and Python 3, you might have to `OBJC_DISABLE_INITIALIZE_FORK_SAFETY=YES`. Closes #23021 from HyukjinKwon/SPARK-25344. Authored-by: hyukjinkwon <gurwls223@apache.org> Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-11-14 01:51:11 -05:00
except ImportError:
2018-11-14 23:30:52 -05:00
testRunner = None
unittest.main(testRunner=testRunner, verbosity=2)