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
|
2018-12-13 21:45:24 -05:00
|
|
|
from pyspark.sql.functions import array, explode, col, lit, mean, min, max, rank, \
|
|
|
|
udf, pandas_udf, PandasUDFType
|
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
|
2018-11-14 01:51:11 -05:00
|
|
|
|
2019-04-09 18:50:25 -04:00
|
|
|
if have_pandas:
|
2020-11-30 20:34:40 -05:00
|
|
|
from pandas.testing import assert_frame_equal
|
2019-04-09 18:50:25 -04:00
|
|
|
|
2018-11-14 01:51:11 -05:00
|
|
|
|
|
|
|
@unittest.skipIf(
|
|
|
|
not have_pandas or not have_pyarrow,
|
2020-09-24 01:15:36 -04:00
|
|
|
pandas_requirement_message or pyarrow_requirement_message) # type: ignore[arg-type]
|
2018-11-14 01:51:11 -05:00
|
|
|
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):
|
[SPARK-34610][PYTHON][TEST] Fix Python UDF used in GroupedAggPandasUDFTests
### What changes were proposed in this pull request?
Fixes a Python UDF `plus_one` used in `GroupedAggPandasUDFTests` to always return float (double) values.
### Why are the changes needed?
The Python UDF `plus_one` used in `GroupedAggPandasUDFTests` is always returning `v + 1` regardless of its type. The return type of the UDF is 'double', so if the input is int, the result will be `null`.
```py
>>> df = 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))
>>> udf('double')
... def plus_one(v):
... assert isinstance(v, (int, float))
... return v + 1
...
>>> pandas_udf('double', PandasUDFType.GROUPED_AGG)
... def sum_udf(v):
... return v.sum()
...
>>> df.groupby(plus_one(df.id)).agg(sum_udf(df.v)).show()
+------------+----------+
|plus_one(id)|sum_udf(v)|
+------------+----------+
| null| 2900.0|
+------------+----------+
```
This is meaningless and should be:
```py
>>> udf('double')
... def plus_one(v):
... assert isinstance(v, (int, float))
... return float(v + 1)
...
>>> df.groupby(plus_one(df.id)).agg(sum_udf(df.v)).sort('plus_one(id)').show()
+------------+----------+
|plus_one(id)|sum_udf(v)|
+------------+----------+
| 1.0| 245.0|
| 2.0| 255.0|
| 3.0| 265.0|
| 4.0| 275.0|
| 5.0| 285.0|
| 6.0| 295.0|
| 7.0| 305.0|
| 8.0| 315.0|
| 9.0| 325.0|
| 10.0| 335.0|
+------------+----------+
```
### Does this PR introduce _any_ user-facing change?
No, test-only.
### How was this patch tested?
Fixed the test.
Closes #31730 from ueshin/issues/SPARK-34610/test_pandas_udf_grouped_agg.
Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-03 20:03:54 -05:00
|
|
|
@udf('double')
|
|
|
|
def plus_one(v):
|
|
|
|
assert isinstance(v, float)
|
|
|
|
return v + 1
|
|
|
|
return plus_one
|
2018-11-14 01:51:11 -05:00
|
|
|
|
|
|
|
@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
|
|
|
|
|
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')
|
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)
|
|
|
|
|
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))
|
|
|
|
|
2019-04-09 18:50:25 -04:00
|
|
|
assert_frame_equal(expected1.toPandas(), result1.toPandas())
|
|
|
|
assert_frame_equal(expected2.toPandas(), result2.toPandas())
|
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))
|
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))
|
2018-11-14 01:51:11 -05:00
|
|
|
|
2019-04-09 18:50:25 -04:00
|
|
|
assert_frame_equal(expected1.toPandas(), result1.toPandas())
|
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))
|
|
|
|
|
2019-04-09 18:50:25 -04:00
|
|
|
assert_frame_equal(expected1.toPandas(), result1.toPandas())
|
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)
|
|
|
|
|
2019-04-09 18:50:25 -04:00
|
|
|
assert_frame_equal(expected1.toPandas(), result1.toPandas())
|
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)))
|
|
|
|
|
2019-04-09 18:50:25 -04:00
|
|
|
assert_frame_equal(expected1.toPandas(), result1.toPandas())
|
|
|
|
assert_frame_equal(expected2.toPandas(), result2.toPandas())
|
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))
|
|
|
|
|
2019-04-09 18:50:25 -04:00
|
|
|
assert_frame_equal(expected1.toPandas(), result1.toPandas())
|
|
|
|
assert_frame_equal(expected2.toPandas(), result2.toPandas())
|
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')
|
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))
|
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))
|
2018-11-14 01:51:11 -05:00
|
|
|
|
2019-04-09 18:50:25 -04: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())
|
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))
|
2020-11-30 20:34:40 -05:00
|
|
|
self.assertEqual(result1.first()['v2'], [1.0, 2.0])
|
2018-11-14 01:51:11 -05:00
|
|
|
|
|
|
|
def test_invalid_args(self):
|
|
|
|
df = self.data
|
|
|
|
w = self.unbounded_window
|
|
|
|
|
|
|
|
with QuietTest(self.sc):
|
2020-11-30 20:34:40 -05:00
|
|
|
with self.assertRaisesRegex(
|
2018-11-14 01:51:11 -05:00
|
|
|
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))
|
|
|
|
|
2019-04-09 18:50:25 -04:00
|
|
|
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))
|
|
|
|
|
2019-04-09 18:50:25 -04:00
|
|
|
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))
|
|
|
|
|
2019-04-09 18:50:25 -04:00
|
|
|
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))
|
|
|
|
|
2019-04-09 18:50:25 -04:00
|
|
|
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))
|
|
|
|
|
2019-04-09 18:50:25 -04:00
|
|
|
assert_frame_equal(expected1.toPandas(), result1.toPandas())
|
2018-11-14 01:51:11 -05:00
|
|
|
|
|
|
|
|
|
|
|
if __name__ == "__main__":
|
2020-08-08 11:51:57 -04:00
|
|
|
from pyspark.sql.tests.test_pandas_udf_window import * # noqa: F401
|
2018-11-14 01:51:11 -05:00
|
|
|
|
|
|
|
try:
|
2020-09-24 01:15:36 -04:00
|
|
|
import xmlrunner # type: ignore[import]
|
2019-06-23 20:58:17 -04:00
|
|
|
testRunner = xmlrunner.XMLTestRunner(output='target/test-reports', verbosity=2)
|
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)
|