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

363 lines
12 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.utils import AnalysisException
from pyspark.sql.functions import array, explode, col, lit, mean, min, max, rank, \
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.window import Window
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
if have_pandas:
from pandas.util.testing import assert_frame_equal
[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 WindowPandasUDFTests(ReusedSQLTestCase):
@property
def data(self):
return self.spark.range(10).toDF('id') \
.withColumn("vs", array([lit(i * 1.0) + col('id') for i in range(20, 30)])) \
.withColumn("v", explode(col('vs'))) \
.drop('vs') \
.withColumn('w', lit(1.0))
@property
def python_plus_one(self):
return udf(lambda v: v + 1, 'double')
@property
def pandas_scalar_time_two(self):
return pandas_udf(lambda v: v * 2, 'double')
[SPARK-24561][SQL][PYTHON] User-defined window aggregation functions with Pandas UDF (bounded window) ## What changes were proposed in this pull request? This PR implements a new feature - window aggregation Pandas UDF for bounded window. #### Doc: https://docs.google.com/document/d/14EjeY5z4-NC27-SmIP9CsMPCANeTcvxN44a7SIJtZPc/edit#heading=h.c87w44wcj3wj #### Example: ``` from pyspark.sql.functions import pandas_udf, PandasUDFType from pyspark.sql.window import Window df = spark.range(0, 10, 2).toDF('v') w1 = Window.partitionBy().orderBy('v').rangeBetween(-2, 4) w2 = Window.partitionBy().orderBy('v').rowsBetween(-2, 2) pandas_udf('double', PandasUDFType.GROUPED_AGG) def avg(v): return v.mean() df.withColumn('v_mean', avg(df['v']).over(w1)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 1.0| # | 2| 2.0| # | 4| 4.0| # | 6| 6.0| # | 8| 7.0| # +---+------+ df.withColumn('v_mean', avg(df['v']).over(w2)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 2.0| # | 2| 3.0| # | 4| 4.0| # | 6| 5.0| # | 8| 6.0| # +---+------+ ``` #### High level changes: This PR modifies the existing WindowInPandasExec physical node to deal with unbounded (growing, shrinking and sliding) windows. * `WindowInPandasExec` now share the same base class as `WindowExec` and share utility functions. See `WindowExecBase` * `WindowFunctionFrame` now has two new functions `currentLowerBound` and `currentUpperBound` - to return the lower and upper window bound for the current output row. It is also modified to allow `AggregateProcessor` == null. Null aggregator processor is used for `WindowInPandasExec` where we don't have an aggregator and only uses lower and upper bound functions from `WindowFunctionFrame` * The biggest change is in `WindowInPandasExec`, where it is modified to take `currentLowerBound` and `currentUpperBound` and write those values together with the input data to the python process for rolling window aggregation. See `WindowInPandasExec` for more details. #### Discussion In benchmarking, I found numpy variant of the rolling window UDF is much faster than the pandas version: Spark SQL window function: 20s Pandas variant: ~80s Numpy variant: 10s Numpy variant with numba: 4s Allowing numpy variant of the vectorized UDFs is something I want to discuss because of the performance improvement, but doesn't have to be in this PR. ## How was this patch tested? New tests Closes #22305 from icexelloss/SPARK-24561-bounded-window-udf. Authored-by: Li Jin <ice.xelloss@gmail.com> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-17 20:15:21 -05:00
@property
def pandas_agg_count_udf(self):
@pandas_udf('long', PandasUDFType.GROUPED_AGG)
def count(v):
return len(v)
return count
[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
@property
def pandas_agg_mean_udf(self):
@pandas_udf('double', PandasUDFType.GROUPED_AGG)
def avg(v):
return v.mean()
return avg
@property
def pandas_agg_max_udf(self):
@pandas_udf('double', PandasUDFType.GROUPED_AGG)
def max(v):
return v.max()
return max
@property
def pandas_agg_min_udf(self):
@pandas_udf('double', PandasUDFType.GROUPED_AGG)
def min(v):
return v.min()
return min
@property
def unbounded_window(self):
return Window.partitionBy('id') \
[SPARK-24561][SQL][PYTHON] User-defined window aggregation functions with Pandas UDF (bounded window) ## What changes were proposed in this pull request? This PR implements a new feature - window aggregation Pandas UDF for bounded window. #### Doc: https://docs.google.com/document/d/14EjeY5z4-NC27-SmIP9CsMPCANeTcvxN44a7SIJtZPc/edit#heading=h.c87w44wcj3wj #### Example: ``` from pyspark.sql.functions import pandas_udf, PandasUDFType from pyspark.sql.window import Window df = spark.range(0, 10, 2).toDF('v') w1 = Window.partitionBy().orderBy('v').rangeBetween(-2, 4) w2 = Window.partitionBy().orderBy('v').rowsBetween(-2, 2) pandas_udf('double', PandasUDFType.GROUPED_AGG) def avg(v): return v.mean() df.withColumn('v_mean', avg(df['v']).over(w1)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 1.0| # | 2| 2.0| # | 4| 4.0| # | 6| 6.0| # | 8| 7.0| # +---+------+ df.withColumn('v_mean', avg(df['v']).over(w2)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 2.0| # | 2| 3.0| # | 4| 4.0| # | 6| 5.0| # | 8| 6.0| # +---+------+ ``` #### High level changes: This PR modifies the existing WindowInPandasExec physical node to deal with unbounded (growing, shrinking and sliding) windows. * `WindowInPandasExec` now share the same base class as `WindowExec` and share utility functions. See `WindowExecBase` * `WindowFunctionFrame` now has two new functions `currentLowerBound` and `currentUpperBound` - to return the lower and upper window bound for the current output row. It is also modified to allow `AggregateProcessor` == null. Null aggregator processor is used for `WindowInPandasExec` where we don't have an aggregator and only uses lower and upper bound functions from `WindowFunctionFrame` * The biggest change is in `WindowInPandasExec`, where it is modified to take `currentLowerBound` and `currentUpperBound` and write those values together with the input data to the python process for rolling window aggregation. See `WindowInPandasExec` for more details. #### Discussion In benchmarking, I found numpy variant of the rolling window UDF is much faster than the pandas version: Spark SQL window function: 20s Pandas variant: ~80s Numpy variant: 10s Numpy variant with numba: 4s Allowing numpy variant of the vectorized UDFs is something I want to discuss because of the performance improvement, but doesn't have to be in this PR. ## How was this patch tested? New tests Closes #22305 from icexelloss/SPARK-24561-bounded-window-udf. Authored-by: Li Jin <ice.xelloss@gmail.com> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-17 20:15:21 -05:00
.rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing).orderBy('v')
[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
@property
def ordered_window(self):
return Window.partitionBy('id').orderBy('v')
@property
def unpartitioned_window(self):
return Window.partitionBy()
[SPARK-24561][SQL][PYTHON] User-defined window aggregation functions with Pandas UDF (bounded window) ## What changes were proposed in this pull request? This PR implements a new feature - window aggregation Pandas UDF for bounded window. #### Doc: https://docs.google.com/document/d/14EjeY5z4-NC27-SmIP9CsMPCANeTcvxN44a7SIJtZPc/edit#heading=h.c87w44wcj3wj #### Example: ``` from pyspark.sql.functions import pandas_udf, PandasUDFType from pyspark.sql.window import Window df = spark.range(0, 10, 2).toDF('v') w1 = Window.partitionBy().orderBy('v').rangeBetween(-2, 4) w2 = Window.partitionBy().orderBy('v').rowsBetween(-2, 2) pandas_udf('double', PandasUDFType.GROUPED_AGG) def avg(v): return v.mean() df.withColumn('v_mean', avg(df['v']).over(w1)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 1.0| # | 2| 2.0| # | 4| 4.0| # | 6| 6.0| # | 8| 7.0| # +---+------+ df.withColumn('v_mean', avg(df['v']).over(w2)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 2.0| # | 2| 3.0| # | 4| 4.0| # | 6| 5.0| # | 8| 6.0| # +---+------+ ``` #### High level changes: This PR modifies the existing WindowInPandasExec physical node to deal with unbounded (growing, shrinking and sliding) windows. * `WindowInPandasExec` now share the same base class as `WindowExec` and share utility functions. See `WindowExecBase` * `WindowFunctionFrame` now has two new functions `currentLowerBound` and `currentUpperBound` - to return the lower and upper window bound for the current output row. It is also modified to allow `AggregateProcessor` == null. Null aggregator processor is used for `WindowInPandasExec` where we don't have an aggregator and only uses lower and upper bound functions from `WindowFunctionFrame` * The biggest change is in `WindowInPandasExec`, where it is modified to take `currentLowerBound` and `currentUpperBound` and write those values together with the input data to the python process for rolling window aggregation. See `WindowInPandasExec` for more details. #### Discussion In benchmarking, I found numpy variant of the rolling window UDF is much faster than the pandas version: Spark SQL window function: 20s Pandas variant: ~80s Numpy variant: 10s Numpy variant with numba: 4s Allowing numpy variant of the vectorized UDFs is something I want to discuss because of the performance improvement, but doesn't have to be in this PR. ## How was this patch tested? New tests Closes #22305 from icexelloss/SPARK-24561-bounded-window-udf. Authored-by: Li Jin <ice.xelloss@gmail.com> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-17 20:15:21 -05:00
@property
def sliding_row_window(self):
return Window.partitionBy('id').orderBy('v').rowsBetween(-2, 1)
@property
def sliding_range_window(self):
return Window.partitionBy('id').orderBy('v').rangeBetween(-2, 4)
@property
def growing_row_window(self):
return Window.partitionBy('id').orderBy('v').rowsBetween(Window.unboundedPreceding, 3)
@property
def growing_range_window(self):
return Window.partitionBy('id').orderBy('v') \
.rangeBetween(Window.unboundedPreceding, 4)
@property
def shrinking_row_window(self):
return Window.partitionBy('id').orderBy('v').rowsBetween(-2, Window.unboundedFollowing)
@property
def shrinking_range_window(self):
return Window.partitionBy('id').orderBy('v') \
.rangeBetween(-3, Window.unboundedFollowing)
[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
def test_simple(self):
df = self.data
w = self.unbounded_window
mean_udf = self.pandas_agg_mean_udf
result1 = df.withColumn('mean_v', mean_udf(df['v']).over(w))
expected1 = df.withColumn('mean_v', mean(df['v']).over(w))
result2 = df.select(mean_udf(df['v']).over(w))
expected2 = df.select(mean(df['v']).over(w))
assert_frame_equal(expected1.toPandas(), result1.toPandas())
assert_frame_equal(expected2.toPandas(), result2.toPandas())
[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
def test_multiple_udfs(self):
df = self.data
w = self.unbounded_window
result1 = df.withColumn('mean_v', self.pandas_agg_mean_udf(df['v']).over(w)) \
[SPARK-24561][SQL][PYTHON] User-defined window aggregation functions with Pandas UDF (bounded window) ## What changes were proposed in this pull request? This PR implements a new feature - window aggregation Pandas UDF for bounded window. #### Doc: https://docs.google.com/document/d/14EjeY5z4-NC27-SmIP9CsMPCANeTcvxN44a7SIJtZPc/edit#heading=h.c87w44wcj3wj #### Example: ``` from pyspark.sql.functions import pandas_udf, PandasUDFType from pyspark.sql.window import Window df = spark.range(0, 10, 2).toDF('v') w1 = Window.partitionBy().orderBy('v').rangeBetween(-2, 4) w2 = Window.partitionBy().orderBy('v').rowsBetween(-2, 2) pandas_udf('double', PandasUDFType.GROUPED_AGG) def avg(v): return v.mean() df.withColumn('v_mean', avg(df['v']).over(w1)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 1.0| # | 2| 2.0| # | 4| 4.0| # | 6| 6.0| # | 8| 7.0| # +---+------+ df.withColumn('v_mean', avg(df['v']).over(w2)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 2.0| # | 2| 3.0| # | 4| 4.0| # | 6| 5.0| # | 8| 6.0| # +---+------+ ``` #### High level changes: This PR modifies the existing WindowInPandasExec physical node to deal with unbounded (growing, shrinking and sliding) windows. * `WindowInPandasExec` now share the same base class as `WindowExec` and share utility functions. See `WindowExecBase` * `WindowFunctionFrame` now has two new functions `currentLowerBound` and `currentUpperBound` - to return the lower and upper window bound for the current output row. It is also modified to allow `AggregateProcessor` == null. Null aggregator processor is used for `WindowInPandasExec` where we don't have an aggregator and only uses lower and upper bound functions from `WindowFunctionFrame` * The biggest change is in `WindowInPandasExec`, where it is modified to take `currentLowerBound` and `currentUpperBound` and write those values together with the input data to the python process for rolling window aggregation. See `WindowInPandasExec` for more details. #### Discussion In benchmarking, I found numpy variant of the rolling window UDF is much faster than the pandas version: Spark SQL window function: 20s Pandas variant: ~80s Numpy variant: 10s Numpy variant with numba: 4s Allowing numpy variant of the vectorized UDFs is something I want to discuss because of the performance improvement, but doesn't have to be in this PR. ## How was this patch tested? New tests Closes #22305 from icexelloss/SPARK-24561-bounded-window-udf. Authored-by: Li Jin <ice.xelloss@gmail.com> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-17 20:15:21 -05:00
.withColumn('max_v', self.pandas_agg_max_udf(df['v']).over(w)) \
.withColumn('min_w', self.pandas_agg_min_udf(df['w']).over(w))
[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
expected1 = df.withColumn('mean_v', mean(df['v']).over(w)) \
[SPARK-24561][SQL][PYTHON] User-defined window aggregation functions with Pandas UDF (bounded window) ## What changes were proposed in this pull request? This PR implements a new feature - window aggregation Pandas UDF for bounded window. #### Doc: https://docs.google.com/document/d/14EjeY5z4-NC27-SmIP9CsMPCANeTcvxN44a7SIJtZPc/edit#heading=h.c87w44wcj3wj #### Example: ``` from pyspark.sql.functions import pandas_udf, PandasUDFType from pyspark.sql.window import Window df = spark.range(0, 10, 2).toDF('v') w1 = Window.partitionBy().orderBy('v').rangeBetween(-2, 4) w2 = Window.partitionBy().orderBy('v').rowsBetween(-2, 2) pandas_udf('double', PandasUDFType.GROUPED_AGG) def avg(v): return v.mean() df.withColumn('v_mean', avg(df['v']).over(w1)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 1.0| # | 2| 2.0| # | 4| 4.0| # | 6| 6.0| # | 8| 7.0| # +---+------+ df.withColumn('v_mean', avg(df['v']).over(w2)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 2.0| # | 2| 3.0| # | 4| 4.0| # | 6| 5.0| # | 8| 6.0| # +---+------+ ``` #### High level changes: This PR modifies the existing WindowInPandasExec physical node to deal with unbounded (growing, shrinking and sliding) windows. * `WindowInPandasExec` now share the same base class as `WindowExec` and share utility functions. See `WindowExecBase` * `WindowFunctionFrame` now has two new functions `currentLowerBound` and `currentUpperBound` - to return the lower and upper window bound for the current output row. It is also modified to allow `AggregateProcessor` == null. Null aggregator processor is used for `WindowInPandasExec` where we don't have an aggregator and only uses lower and upper bound functions from `WindowFunctionFrame` * The biggest change is in `WindowInPandasExec`, where it is modified to take `currentLowerBound` and `currentUpperBound` and write those values together with the input data to the python process for rolling window aggregation. See `WindowInPandasExec` for more details. #### Discussion In benchmarking, I found numpy variant of the rolling window UDF is much faster than the pandas version: Spark SQL window function: 20s Pandas variant: ~80s Numpy variant: 10s Numpy variant with numba: 4s Allowing numpy variant of the vectorized UDFs is something I want to discuss because of the performance improvement, but doesn't have to be in this PR. ## How was this patch tested? New tests Closes #22305 from icexelloss/SPARK-24561-bounded-window-udf. Authored-by: Li Jin <ice.xelloss@gmail.com> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-17 20:15:21 -05:00
.withColumn('max_v', max(df['v']).over(w)) \
.withColumn('min_w', min(df['w']).over(w))
[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
assert_frame_equal(expected1.toPandas(), result1.toPandas())
[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
def test_replace_existing(self):
df = self.data
w = self.unbounded_window
result1 = df.withColumn('v', self.pandas_agg_mean_udf(df['v']).over(w))
expected1 = df.withColumn('v', mean(df['v']).over(w))
assert_frame_equal(expected1.toPandas(), result1.toPandas())
[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
def test_mixed_sql(self):
df = self.data
w = self.unbounded_window
mean_udf = self.pandas_agg_mean_udf
result1 = df.withColumn('v', mean_udf(df['v'] * 2).over(w) + 1)
expected1 = df.withColumn('v', mean(df['v'] * 2).over(w) + 1)
assert_frame_equal(expected1.toPandas(), result1.toPandas())
[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
def test_mixed_udf(self):
df = self.data
w = self.unbounded_window
plus_one = self.python_plus_one
time_two = self.pandas_scalar_time_two
mean_udf = self.pandas_agg_mean_udf
result1 = df.withColumn(
'v2',
plus_one(mean_udf(plus_one(df['v'])).over(w)))
expected1 = df.withColumn(
'v2',
plus_one(mean(plus_one(df['v'])).over(w)))
result2 = df.withColumn(
'v2',
time_two(mean_udf(time_two(df['v'])).over(w)))
expected2 = df.withColumn(
'v2',
time_two(mean(time_two(df['v'])).over(w)))
assert_frame_equal(expected1.toPandas(), result1.toPandas())
assert_frame_equal(expected2.toPandas(), result2.toPandas())
[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
def test_without_partitionBy(self):
df = self.data
w = self.unpartitioned_window
mean_udf = self.pandas_agg_mean_udf
result1 = df.withColumn('v2', mean_udf(df['v']).over(w))
expected1 = df.withColumn('v2', mean(df['v']).over(w))
result2 = df.select(mean_udf(df['v']).over(w))
expected2 = df.select(mean(df['v']).over(w))
assert_frame_equal(expected1.toPandas(), result1.toPandas())
assert_frame_equal(expected2.toPandas(), result2.toPandas())
[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
def test_mixed_sql_and_udf(self):
df = self.data
w = self.unbounded_window
ow = self.ordered_window
max_udf = self.pandas_agg_max_udf
min_udf = self.pandas_agg_min_udf
result1 = df.withColumn('v_diff', max_udf(df['v']).over(w) - min_udf(df['v']).over(w))
expected1 = df.withColumn('v_diff', max(df['v']).over(w) - min(df['v']).over(w))
# Test mixing sql window function and window udf in the same expression
result2 = df.withColumn('v_diff', max_udf(df['v']).over(w) - min(df['v']).over(w))
expected2 = expected1
# Test chaining sql aggregate function and udf
result3 = df.withColumn('max_v', max_udf(df['v']).over(w)) \
[SPARK-24561][SQL][PYTHON] User-defined window aggregation functions with Pandas UDF (bounded window) ## What changes were proposed in this pull request? This PR implements a new feature - window aggregation Pandas UDF for bounded window. #### Doc: https://docs.google.com/document/d/14EjeY5z4-NC27-SmIP9CsMPCANeTcvxN44a7SIJtZPc/edit#heading=h.c87w44wcj3wj #### Example: ``` from pyspark.sql.functions import pandas_udf, PandasUDFType from pyspark.sql.window import Window df = spark.range(0, 10, 2).toDF('v') w1 = Window.partitionBy().orderBy('v').rangeBetween(-2, 4) w2 = Window.partitionBy().orderBy('v').rowsBetween(-2, 2) pandas_udf('double', PandasUDFType.GROUPED_AGG) def avg(v): return v.mean() df.withColumn('v_mean', avg(df['v']).over(w1)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 1.0| # | 2| 2.0| # | 4| 4.0| # | 6| 6.0| # | 8| 7.0| # +---+------+ df.withColumn('v_mean', avg(df['v']).over(w2)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 2.0| # | 2| 3.0| # | 4| 4.0| # | 6| 5.0| # | 8| 6.0| # +---+------+ ``` #### High level changes: This PR modifies the existing WindowInPandasExec physical node to deal with unbounded (growing, shrinking and sliding) windows. * `WindowInPandasExec` now share the same base class as `WindowExec` and share utility functions. See `WindowExecBase` * `WindowFunctionFrame` now has two new functions `currentLowerBound` and `currentUpperBound` - to return the lower and upper window bound for the current output row. It is also modified to allow `AggregateProcessor` == null. Null aggregator processor is used for `WindowInPandasExec` where we don't have an aggregator and only uses lower and upper bound functions from `WindowFunctionFrame` * The biggest change is in `WindowInPandasExec`, where it is modified to take `currentLowerBound` and `currentUpperBound` and write those values together with the input data to the python process for rolling window aggregation. See `WindowInPandasExec` for more details. #### Discussion In benchmarking, I found numpy variant of the rolling window UDF is much faster than the pandas version: Spark SQL window function: 20s Pandas variant: ~80s Numpy variant: 10s Numpy variant with numba: 4s Allowing numpy variant of the vectorized UDFs is something I want to discuss because of the performance improvement, but doesn't have to be in this PR. ## How was this patch tested? New tests Closes #22305 from icexelloss/SPARK-24561-bounded-window-udf. Authored-by: Li Jin <ice.xelloss@gmail.com> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-17 20:15:21 -05:00
.withColumn('min_v', min(df['v']).over(w)) \
.withColumn('v_diff', col('max_v') - col('min_v')) \
.drop('max_v', 'min_v')
[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
expected3 = expected1
# Test mixing sql window function and udf
result4 = df.withColumn('max_v', max_udf(df['v']).over(w)) \
[SPARK-24561][SQL][PYTHON] User-defined window aggregation functions with Pandas UDF (bounded window) ## What changes were proposed in this pull request? This PR implements a new feature - window aggregation Pandas UDF for bounded window. #### Doc: https://docs.google.com/document/d/14EjeY5z4-NC27-SmIP9CsMPCANeTcvxN44a7SIJtZPc/edit#heading=h.c87w44wcj3wj #### Example: ``` from pyspark.sql.functions import pandas_udf, PandasUDFType from pyspark.sql.window import Window df = spark.range(0, 10, 2).toDF('v') w1 = Window.partitionBy().orderBy('v').rangeBetween(-2, 4) w2 = Window.partitionBy().orderBy('v').rowsBetween(-2, 2) pandas_udf('double', PandasUDFType.GROUPED_AGG) def avg(v): return v.mean() df.withColumn('v_mean', avg(df['v']).over(w1)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 1.0| # | 2| 2.0| # | 4| 4.0| # | 6| 6.0| # | 8| 7.0| # +---+------+ df.withColumn('v_mean', avg(df['v']).over(w2)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 2.0| # | 2| 3.0| # | 4| 4.0| # | 6| 5.0| # | 8| 6.0| # +---+------+ ``` #### High level changes: This PR modifies the existing WindowInPandasExec physical node to deal with unbounded (growing, shrinking and sliding) windows. * `WindowInPandasExec` now share the same base class as `WindowExec` and share utility functions. See `WindowExecBase` * `WindowFunctionFrame` now has two new functions `currentLowerBound` and `currentUpperBound` - to return the lower and upper window bound for the current output row. It is also modified to allow `AggregateProcessor` == null. Null aggregator processor is used for `WindowInPandasExec` where we don't have an aggregator and only uses lower and upper bound functions from `WindowFunctionFrame` * The biggest change is in `WindowInPandasExec`, where it is modified to take `currentLowerBound` and `currentUpperBound` and write those values together with the input data to the python process for rolling window aggregation. See `WindowInPandasExec` for more details. #### Discussion In benchmarking, I found numpy variant of the rolling window UDF is much faster than the pandas version: Spark SQL window function: 20s Pandas variant: ~80s Numpy variant: 10s Numpy variant with numba: 4s Allowing numpy variant of the vectorized UDFs is something I want to discuss because of the performance improvement, but doesn't have to be in this PR. ## How was this patch tested? New tests Closes #22305 from icexelloss/SPARK-24561-bounded-window-udf. Authored-by: Li Jin <ice.xelloss@gmail.com> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-17 20:15:21 -05:00
.withColumn('rank', rank().over(ow))
[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
expected4 = df.withColumn('max_v', max(df['v']).over(w)) \
[SPARK-24561][SQL][PYTHON] User-defined window aggregation functions with Pandas UDF (bounded window) ## What changes were proposed in this pull request? This PR implements a new feature - window aggregation Pandas UDF for bounded window. #### Doc: https://docs.google.com/document/d/14EjeY5z4-NC27-SmIP9CsMPCANeTcvxN44a7SIJtZPc/edit#heading=h.c87w44wcj3wj #### Example: ``` from pyspark.sql.functions import pandas_udf, PandasUDFType from pyspark.sql.window import Window df = spark.range(0, 10, 2).toDF('v') w1 = Window.partitionBy().orderBy('v').rangeBetween(-2, 4) w2 = Window.partitionBy().orderBy('v').rowsBetween(-2, 2) pandas_udf('double', PandasUDFType.GROUPED_AGG) def avg(v): return v.mean() df.withColumn('v_mean', avg(df['v']).over(w1)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 1.0| # | 2| 2.0| # | 4| 4.0| # | 6| 6.0| # | 8| 7.0| # +---+------+ df.withColumn('v_mean', avg(df['v']).over(w2)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 2.0| # | 2| 3.0| # | 4| 4.0| # | 6| 5.0| # | 8| 6.0| # +---+------+ ``` #### High level changes: This PR modifies the existing WindowInPandasExec physical node to deal with unbounded (growing, shrinking and sliding) windows. * `WindowInPandasExec` now share the same base class as `WindowExec` and share utility functions. See `WindowExecBase` * `WindowFunctionFrame` now has two new functions `currentLowerBound` and `currentUpperBound` - to return the lower and upper window bound for the current output row. It is also modified to allow `AggregateProcessor` == null. Null aggregator processor is used for `WindowInPandasExec` where we don't have an aggregator and only uses lower and upper bound functions from `WindowFunctionFrame` * The biggest change is in `WindowInPandasExec`, where it is modified to take `currentLowerBound` and `currentUpperBound` and write those values together with the input data to the python process for rolling window aggregation. See `WindowInPandasExec` for more details. #### Discussion In benchmarking, I found numpy variant of the rolling window UDF is much faster than the pandas version: Spark SQL window function: 20s Pandas variant: ~80s Numpy variant: 10s Numpy variant with numba: 4s Allowing numpy variant of the vectorized UDFs is something I want to discuss because of the performance improvement, but doesn't have to be in this PR. ## How was this patch tested? New tests Closes #22305 from icexelloss/SPARK-24561-bounded-window-udf. Authored-by: Li Jin <ice.xelloss@gmail.com> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-17 20:15:21 -05:00
.withColumn('rank', rank().over(ow))
[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
assert_frame_equal(expected1.toPandas(), result1.toPandas())
assert_frame_equal(expected2.toPandas(), result2.toPandas())
assert_frame_equal(expected3.toPandas(), result3.toPandas())
assert_frame_equal(expected4.toPandas(), result4.toPandas())
[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
def test_array_type(self):
df = self.data
w = self.unbounded_window
array_udf = pandas_udf(lambda x: [1.0, 2.0], 'array<double>', PandasUDFType.GROUPED_AGG)
result1 = df.withColumn('v2', array_udf(df['v']).over(w))
self.assertEquals(result1.first()['v2'], [1.0, 2.0])
def test_invalid_args(self):
df = self.data
w = self.unbounded_window
with QuietTest(self.sc):
with self.assertRaisesRegexp(
AnalysisException,
'.*not supported within a window function'):
foo_udf = pandas_udf(lambda x: x, 'v double', PandasUDFType.GROUPED_MAP)
df.withColumn('v2', foo_udf(df['v']).over(w))
[SPARK-24561][SQL][PYTHON] User-defined window aggregation functions with Pandas UDF (bounded window) ## What changes were proposed in this pull request? This PR implements a new feature - window aggregation Pandas UDF for bounded window. #### Doc: https://docs.google.com/document/d/14EjeY5z4-NC27-SmIP9CsMPCANeTcvxN44a7SIJtZPc/edit#heading=h.c87w44wcj3wj #### Example: ``` from pyspark.sql.functions import pandas_udf, PandasUDFType from pyspark.sql.window import Window df = spark.range(0, 10, 2).toDF('v') w1 = Window.partitionBy().orderBy('v').rangeBetween(-2, 4) w2 = Window.partitionBy().orderBy('v').rowsBetween(-2, 2) pandas_udf('double', PandasUDFType.GROUPED_AGG) def avg(v): return v.mean() df.withColumn('v_mean', avg(df['v']).over(w1)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 1.0| # | 2| 2.0| # | 4| 4.0| # | 6| 6.0| # | 8| 7.0| # +---+------+ df.withColumn('v_mean', avg(df['v']).over(w2)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 2.0| # | 2| 3.0| # | 4| 4.0| # | 6| 5.0| # | 8| 6.0| # +---+------+ ``` #### High level changes: This PR modifies the existing WindowInPandasExec physical node to deal with unbounded (growing, shrinking and sliding) windows. * `WindowInPandasExec` now share the same base class as `WindowExec` and share utility functions. See `WindowExecBase` * `WindowFunctionFrame` now has two new functions `currentLowerBound` and `currentUpperBound` - to return the lower and upper window bound for the current output row. It is also modified to allow `AggregateProcessor` == null. Null aggregator processor is used for `WindowInPandasExec` where we don't have an aggregator and only uses lower and upper bound functions from `WindowFunctionFrame` * The biggest change is in `WindowInPandasExec`, where it is modified to take `currentLowerBound` and `currentUpperBound` and write those values together with the input data to the python process for rolling window aggregation. See `WindowInPandasExec` for more details. #### Discussion In benchmarking, I found numpy variant of the rolling window UDF is much faster than the pandas version: Spark SQL window function: 20s Pandas variant: ~80s Numpy variant: 10s Numpy variant with numba: 4s Allowing numpy variant of the vectorized UDFs is something I want to discuss because of the performance improvement, but doesn't have to be in this PR. ## How was this patch tested? New tests Closes #22305 from icexelloss/SPARK-24561-bounded-window-udf. Authored-by: Li Jin <ice.xelloss@gmail.com> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-17 20:15:21 -05:00
def test_bounded_simple(self):
from pyspark.sql.functions import mean, max, min, count
df = self.data
w1 = self.sliding_row_window
w2 = self.shrinking_range_window
plus_one = self.python_plus_one
count_udf = self.pandas_agg_count_udf
mean_udf = self.pandas_agg_mean_udf
max_udf = self.pandas_agg_max_udf
min_udf = self.pandas_agg_min_udf
result1 = df.withColumn('mean_v', mean_udf(plus_one(df['v'])).over(w1)) \
.withColumn('count_v', count_udf(df['v']).over(w2)) \
.withColumn('max_v', max_udf(df['v']).over(w2)) \
.withColumn('min_v', min_udf(df['v']).over(w1))
expected1 = df.withColumn('mean_v', mean(plus_one(df['v'])).over(w1)) \
.withColumn('count_v', count(df['v']).over(w2)) \
.withColumn('max_v', max(df['v']).over(w2)) \
.withColumn('min_v', min(df['v']).over(w1))
assert_frame_equal(expected1.toPandas(), result1.toPandas())
[SPARK-24561][SQL][PYTHON] User-defined window aggregation functions with Pandas UDF (bounded window) ## What changes were proposed in this pull request? This PR implements a new feature - window aggregation Pandas UDF for bounded window. #### Doc: https://docs.google.com/document/d/14EjeY5z4-NC27-SmIP9CsMPCANeTcvxN44a7SIJtZPc/edit#heading=h.c87w44wcj3wj #### Example: ``` from pyspark.sql.functions import pandas_udf, PandasUDFType from pyspark.sql.window import Window df = spark.range(0, 10, 2).toDF('v') w1 = Window.partitionBy().orderBy('v').rangeBetween(-2, 4) w2 = Window.partitionBy().orderBy('v').rowsBetween(-2, 2) pandas_udf('double', PandasUDFType.GROUPED_AGG) def avg(v): return v.mean() df.withColumn('v_mean', avg(df['v']).over(w1)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 1.0| # | 2| 2.0| # | 4| 4.0| # | 6| 6.0| # | 8| 7.0| # +---+------+ df.withColumn('v_mean', avg(df['v']).over(w2)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 2.0| # | 2| 3.0| # | 4| 4.0| # | 6| 5.0| # | 8| 6.0| # +---+------+ ``` #### High level changes: This PR modifies the existing WindowInPandasExec physical node to deal with unbounded (growing, shrinking and sliding) windows. * `WindowInPandasExec` now share the same base class as `WindowExec` and share utility functions. See `WindowExecBase` * `WindowFunctionFrame` now has two new functions `currentLowerBound` and `currentUpperBound` - to return the lower and upper window bound for the current output row. It is also modified to allow `AggregateProcessor` == null. Null aggregator processor is used for `WindowInPandasExec` where we don't have an aggregator and only uses lower and upper bound functions from `WindowFunctionFrame` * The biggest change is in `WindowInPandasExec`, where it is modified to take `currentLowerBound` and `currentUpperBound` and write those values together with the input data to the python process for rolling window aggregation. See `WindowInPandasExec` for more details. #### Discussion In benchmarking, I found numpy variant of the rolling window UDF is much faster than the pandas version: Spark SQL window function: 20s Pandas variant: ~80s Numpy variant: 10s Numpy variant with numba: 4s Allowing numpy variant of the vectorized UDFs is something I want to discuss because of the performance improvement, but doesn't have to be in this PR. ## How was this patch tested? New tests Closes #22305 from icexelloss/SPARK-24561-bounded-window-udf. Authored-by: Li Jin <ice.xelloss@gmail.com> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-17 20:15:21 -05:00
def test_growing_window(self):
from pyspark.sql.functions import mean
df = self.data
w1 = self.growing_row_window
w2 = self.growing_range_window
mean_udf = self.pandas_agg_mean_udf
result1 = df.withColumn('m1', mean_udf(df['v']).over(w1)) \
.withColumn('m2', mean_udf(df['v']).over(w2))
expected1 = df.withColumn('m1', mean(df['v']).over(w1)) \
.withColumn('m2', mean(df['v']).over(w2))
assert_frame_equal(expected1.toPandas(), result1.toPandas())
[SPARK-24561][SQL][PYTHON] User-defined window aggregation functions with Pandas UDF (bounded window) ## What changes were proposed in this pull request? This PR implements a new feature - window aggregation Pandas UDF for bounded window. #### Doc: https://docs.google.com/document/d/14EjeY5z4-NC27-SmIP9CsMPCANeTcvxN44a7SIJtZPc/edit#heading=h.c87w44wcj3wj #### Example: ``` from pyspark.sql.functions import pandas_udf, PandasUDFType from pyspark.sql.window import Window df = spark.range(0, 10, 2).toDF('v') w1 = Window.partitionBy().orderBy('v').rangeBetween(-2, 4) w2 = Window.partitionBy().orderBy('v').rowsBetween(-2, 2) pandas_udf('double', PandasUDFType.GROUPED_AGG) def avg(v): return v.mean() df.withColumn('v_mean', avg(df['v']).over(w1)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 1.0| # | 2| 2.0| # | 4| 4.0| # | 6| 6.0| # | 8| 7.0| # +---+------+ df.withColumn('v_mean', avg(df['v']).over(w2)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 2.0| # | 2| 3.0| # | 4| 4.0| # | 6| 5.0| # | 8| 6.0| # +---+------+ ``` #### High level changes: This PR modifies the existing WindowInPandasExec physical node to deal with unbounded (growing, shrinking and sliding) windows. * `WindowInPandasExec` now share the same base class as `WindowExec` and share utility functions. See `WindowExecBase` * `WindowFunctionFrame` now has two new functions `currentLowerBound` and `currentUpperBound` - to return the lower and upper window bound for the current output row. It is also modified to allow `AggregateProcessor` == null. Null aggregator processor is used for `WindowInPandasExec` where we don't have an aggregator and only uses lower and upper bound functions from `WindowFunctionFrame` * The biggest change is in `WindowInPandasExec`, where it is modified to take `currentLowerBound` and `currentUpperBound` and write those values together with the input data to the python process for rolling window aggregation. See `WindowInPandasExec` for more details. #### Discussion In benchmarking, I found numpy variant of the rolling window UDF is much faster than the pandas version: Spark SQL window function: 20s Pandas variant: ~80s Numpy variant: 10s Numpy variant with numba: 4s Allowing numpy variant of the vectorized UDFs is something I want to discuss because of the performance improvement, but doesn't have to be in this PR. ## How was this patch tested? New tests Closes #22305 from icexelloss/SPARK-24561-bounded-window-udf. Authored-by: Li Jin <ice.xelloss@gmail.com> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-17 20:15:21 -05:00
def test_sliding_window(self):
from pyspark.sql.functions import mean
df = self.data
w1 = self.sliding_row_window
w2 = self.sliding_range_window
mean_udf = self.pandas_agg_mean_udf
result1 = df.withColumn('m1', mean_udf(df['v']).over(w1)) \
.withColumn('m2', mean_udf(df['v']).over(w2))
expected1 = df.withColumn('m1', mean(df['v']).over(w1)) \
.withColumn('m2', mean(df['v']).over(w2))
assert_frame_equal(expected1.toPandas(), result1.toPandas())
[SPARK-24561][SQL][PYTHON] User-defined window aggregation functions with Pandas UDF (bounded window) ## What changes were proposed in this pull request? This PR implements a new feature - window aggregation Pandas UDF for bounded window. #### Doc: https://docs.google.com/document/d/14EjeY5z4-NC27-SmIP9CsMPCANeTcvxN44a7SIJtZPc/edit#heading=h.c87w44wcj3wj #### Example: ``` from pyspark.sql.functions import pandas_udf, PandasUDFType from pyspark.sql.window import Window df = spark.range(0, 10, 2).toDF('v') w1 = Window.partitionBy().orderBy('v').rangeBetween(-2, 4) w2 = Window.partitionBy().orderBy('v').rowsBetween(-2, 2) pandas_udf('double', PandasUDFType.GROUPED_AGG) def avg(v): return v.mean() df.withColumn('v_mean', avg(df['v']).over(w1)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 1.0| # | 2| 2.0| # | 4| 4.0| # | 6| 6.0| # | 8| 7.0| # +---+------+ df.withColumn('v_mean', avg(df['v']).over(w2)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 2.0| # | 2| 3.0| # | 4| 4.0| # | 6| 5.0| # | 8| 6.0| # +---+------+ ``` #### High level changes: This PR modifies the existing WindowInPandasExec physical node to deal with unbounded (growing, shrinking and sliding) windows. * `WindowInPandasExec` now share the same base class as `WindowExec` and share utility functions. See `WindowExecBase` * `WindowFunctionFrame` now has two new functions `currentLowerBound` and `currentUpperBound` - to return the lower and upper window bound for the current output row. It is also modified to allow `AggregateProcessor` == null. Null aggregator processor is used for `WindowInPandasExec` where we don't have an aggregator and only uses lower and upper bound functions from `WindowFunctionFrame` * The biggest change is in `WindowInPandasExec`, where it is modified to take `currentLowerBound` and `currentUpperBound` and write those values together with the input data to the python process for rolling window aggregation. See `WindowInPandasExec` for more details. #### Discussion In benchmarking, I found numpy variant of the rolling window UDF is much faster than the pandas version: Spark SQL window function: 20s Pandas variant: ~80s Numpy variant: 10s Numpy variant with numba: 4s Allowing numpy variant of the vectorized UDFs is something I want to discuss because of the performance improvement, but doesn't have to be in this PR. ## How was this patch tested? New tests Closes #22305 from icexelloss/SPARK-24561-bounded-window-udf. Authored-by: Li Jin <ice.xelloss@gmail.com> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-17 20:15:21 -05:00
def test_shrinking_window(self):
from pyspark.sql.functions import mean
df = self.data
w1 = self.shrinking_row_window
w2 = self.shrinking_range_window
mean_udf = self.pandas_agg_mean_udf
result1 = df.withColumn('m1', mean_udf(df['v']).over(w1)) \
.withColumn('m2', mean_udf(df['v']).over(w2))
expected1 = df.withColumn('m1', mean(df['v']).over(w1)) \
.withColumn('m2', mean(df['v']).over(w2))
assert_frame_equal(expected1.toPandas(), result1.toPandas())
[SPARK-24561][SQL][PYTHON] User-defined window aggregation functions with Pandas UDF (bounded window) ## What changes were proposed in this pull request? This PR implements a new feature - window aggregation Pandas UDF for bounded window. #### Doc: https://docs.google.com/document/d/14EjeY5z4-NC27-SmIP9CsMPCANeTcvxN44a7SIJtZPc/edit#heading=h.c87w44wcj3wj #### Example: ``` from pyspark.sql.functions import pandas_udf, PandasUDFType from pyspark.sql.window import Window df = spark.range(0, 10, 2).toDF('v') w1 = Window.partitionBy().orderBy('v').rangeBetween(-2, 4) w2 = Window.partitionBy().orderBy('v').rowsBetween(-2, 2) pandas_udf('double', PandasUDFType.GROUPED_AGG) def avg(v): return v.mean() df.withColumn('v_mean', avg(df['v']).over(w1)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 1.0| # | 2| 2.0| # | 4| 4.0| # | 6| 6.0| # | 8| 7.0| # +---+------+ df.withColumn('v_mean', avg(df['v']).over(w2)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 2.0| # | 2| 3.0| # | 4| 4.0| # | 6| 5.0| # | 8| 6.0| # +---+------+ ``` #### High level changes: This PR modifies the existing WindowInPandasExec physical node to deal with unbounded (growing, shrinking and sliding) windows. * `WindowInPandasExec` now share the same base class as `WindowExec` and share utility functions. See `WindowExecBase` * `WindowFunctionFrame` now has two new functions `currentLowerBound` and `currentUpperBound` - to return the lower and upper window bound for the current output row. It is also modified to allow `AggregateProcessor` == null. Null aggregator processor is used for `WindowInPandasExec` where we don't have an aggregator and only uses lower and upper bound functions from `WindowFunctionFrame` * The biggest change is in `WindowInPandasExec`, where it is modified to take `currentLowerBound` and `currentUpperBound` and write those values together with the input data to the python process for rolling window aggregation. See `WindowInPandasExec` for more details. #### Discussion In benchmarking, I found numpy variant of the rolling window UDF is much faster than the pandas version: Spark SQL window function: 20s Pandas variant: ~80s Numpy variant: 10s Numpy variant with numba: 4s Allowing numpy variant of the vectorized UDFs is something I want to discuss because of the performance improvement, but doesn't have to be in this PR. ## How was this patch tested? New tests Closes #22305 from icexelloss/SPARK-24561-bounded-window-udf. Authored-by: Li Jin <ice.xelloss@gmail.com> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-17 20:15:21 -05:00
def test_bounded_mixed(self):
from pyspark.sql.functions import mean, max
df = self.data
w1 = self.sliding_row_window
w2 = self.unbounded_window
mean_udf = self.pandas_agg_mean_udf
max_udf = self.pandas_agg_max_udf
result1 = df.withColumn('mean_v', mean_udf(df['v']).over(w1)) \
.withColumn('max_v', max_udf(df['v']).over(w2)) \
.withColumn('mean_unbounded_v', mean_udf(df['v']).over(w1))
expected1 = df.withColumn('mean_v', mean(df['v']).over(w1)) \
.withColumn('max_v', max(df['v']).over(w2)) \
.withColumn('mean_unbounded_v', mean(df['v']).over(w1))
assert_frame_equal(expected1.toPandas(), result1.toPandas())
[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_window import *
try:
import xmlrunner
2018-11-14 23:30:52 -05:00
testRunner = xmlrunner.XMLTestRunner(output='target/test-reports')
[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)