spark-instrumented-optimizer/python/pyspark/sql/pandas/map_ops.py

106 lines
3.7 KiB
Python
Raw Normal View History

[SPARK-30434][PYTHON][SQL] Move pandas related functionalities into 'pandas' sub-package ### What changes were proposed in this pull request? This PR proposes to move pandas related functionalities into pandas package. Namely: ```bash pyspark/sql/pandas ├── __init__.py ├── conversion.py # Conversion between pandas <> PySpark DataFrames ├── functions.py # pandas_udf ├── group_ops.py # Grouped UDF / Cogrouped UDF + groupby.apply, groupby.cogroup.apply ├── map_ops.py # Map Iter UDF + mapInPandas ├── serializers.py # pandas <> PyArrow serializers ├── types.py # Type utils between pandas <> PyArrow └── utils.py # Version requirement checks ``` In order to separately locate `groupby.apply`, `groupby.cogroup.apply`, `mapInPandas`, `toPandas`, and `createDataFrame(pdf)` under `pandas` sub-package, I had to use a mix-in approach which Scala side uses often by `trait`, and also pandas itself uses this approach (see `IndexOpsMixin` as an example) to group related functionalities. Currently, you can think it's like Scala's self typed trait. See the structure below: ```python class PandasMapOpsMixin(object): def mapInPandas(self, ...): ... return ... # other Pandas <> PySpark APIs ``` ```python class DataFrame(PandasMapOpsMixin): # other DataFrame APIs equivalent to Scala side. ``` Yes, This is a big PR but they are mostly just moving around except one case `createDataFrame` which I had to split the methods. ### Why are the changes needed? There are pandas functionalities here and there and I myself gets lost where it was. Also, when you have to make a change commonly for all of pandas related features, it's almost impossible now. Also, after this change, `DataFrame` and `SparkSession` become more consistent with Scala side since pandas is specific to Python, and this change separates pandas-specific APIs away from `DataFrame` or `SparkSession`. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests should cover. Also, I manually built the PySpark API documentation and checked. Closes #27109 from HyukjinKwon/pandas-refactoring. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-08 20:22:50 -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 sys
from pyspark.rdd import PythonEvalType
class PandasMapOpsMixin(object):
"""
Min-in for pandas map operations. Currently, only :class:`DataFrame`
can use this class.
"""
[SPARK-28264][PYTHON][SQL] Support type hints in pandas UDF and rename/move inconsistent pandas UDF types ### What changes were proposed in this pull request? This PR proposes to redesign pandas UDFs as described in [the proposal](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit?usp=sharing). ```python from pyspark.sql.functions import pandas_udf import pandas as pd pandas_udf("long") def plug_one(s: pd.Series) -> pd.Series: return s + 1 spark.range(10).select(plug_one("id")).show() ``` ``` +------------+ |plug_one(id)| +------------+ | 1| | 2| | 3| | 4| | 5| | 6| | 7| | 8| | 9| | 10| +------------+ ``` Note that, this PR address one of the future improvements described [here](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit#heading=h.h3ncjpk6ujqu), "A couple of less-intuitive pandas UDF types" (by zero323) together. In short, - Adds new way with type hints as an alternative and experimental way. ```python pandas_udf(schema='...') def func(c1: Series, c2: Series) -> DataFrame: pass ``` - Replace and/or add an alias for three types below from UDF, and make them as separate standalone APIs. So, `pandas_udf` is now consistent with regular `udf`s and other expressions. `df.mapInPandas(udf)` -replace-> `df.mapInPandas(f, schema)` `df.groupby.apply(udf)` -alias-> `df.groupby.applyInPandas(f, schema)` `df.groupby.cogroup.apply(udf)` -replace-> `df.groupby.cogroup.applyInPandas(f, schema)` *`df.groupby.apply` was added from 2.3 while the other were added in the master only. - No deprecation for the existing ways for now. ```python pandas_udf(schema='...', functionType=PandasUDFType.SCALAR) def func(c1, c2): pass ``` If users are happy with this, I plan to deprecate the existing way and declare using type hints is not experimental anymore. One design goal in this PR was that, avoid touching the internal (since we didn't deprecate the old ways for now), but supports type hints with a minimised changes only at the interface. - Once we deprecate or remove the old ways, I think it requires another refactoring for the internal in the future. At the very least, we should rename internal pandas evaluation types. - If users find this experimental type hints isn't quite helpful, we should simply revert the changes at the interface level. ### Why are the changes needed? In order to address old design issues. Please see [the proposal](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit?usp=sharing). ### Does this PR introduce any user-facing change? For behaviour changes, No. It adds new ways to use pandas UDFs by using type hints. See below. **SCALAR**: ```python pandas_udf(schema='...') def func(c1: Series, c2: DataFrame) -> Series: pass # DataFrame represents a struct column ``` **SCALAR_ITER**: ```python pandas_udf(schema='...') def func(iter: Iterator[Tuple[Series, DataFrame, ...]]) -> Iterator[Series]: pass # Same as SCALAR but wrapped by Iterator ``` **GROUPED_AGG**: ```python pandas_udf(schema='...') def func(c1: Series, c2: DataFrame) -> int: pass # DataFrame represents a struct column ``` **GROUPED_MAP**: This was added in Spark 2.3 as of SPARK-20396. As described above, it keeps the existing behaviour. Additionally, we now have a new alias `groupby.applyInPandas` for `groupby.apply`. See the example below: ```python def func(pdf): return pdf df.groupby("...").applyInPandas(func, schema=df.schema) ``` **MAP_ITER**: this is not a pandas UDF anymore This was added in Spark 3.0 as of SPARK-28198; and this PR replaces the usages. See the example below: ```python def func(iter): for df in iter: yield df df.mapInPandas(func, df.schema) ``` **COGROUPED_MAP**: this is not a pandas UDF anymore This was added in Spark 3.0 as of SPARK-27463; and this PR replaces the usages. See the example below: ```python def asof_join(left, right): return pd.merge_asof(left, right, on="...", by="...") df1.groupby("...").cogroup(df2.groupby("...")).applyInPandas(asof_join, schema="...") ``` ### How was this patch tested? Unittests added and tested against Python 2.7, 3.6 and 3.7. Closes #27165 from HyukjinKwon/revisit-pandas. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-22 01:32:58 -05:00
def mapInPandas(self, func, schema):
[SPARK-30434][PYTHON][SQL] Move pandas related functionalities into 'pandas' sub-package ### What changes were proposed in this pull request? This PR proposes to move pandas related functionalities into pandas package. Namely: ```bash pyspark/sql/pandas ├── __init__.py ├── conversion.py # Conversion between pandas <> PySpark DataFrames ├── functions.py # pandas_udf ├── group_ops.py # Grouped UDF / Cogrouped UDF + groupby.apply, groupby.cogroup.apply ├── map_ops.py # Map Iter UDF + mapInPandas ├── serializers.py # pandas <> PyArrow serializers ├── types.py # Type utils between pandas <> PyArrow └── utils.py # Version requirement checks ``` In order to separately locate `groupby.apply`, `groupby.cogroup.apply`, `mapInPandas`, `toPandas`, and `createDataFrame(pdf)` under `pandas` sub-package, I had to use a mix-in approach which Scala side uses often by `trait`, and also pandas itself uses this approach (see `IndexOpsMixin` as an example) to group related functionalities. Currently, you can think it's like Scala's self typed trait. See the structure below: ```python class PandasMapOpsMixin(object): def mapInPandas(self, ...): ... return ... # other Pandas <> PySpark APIs ``` ```python class DataFrame(PandasMapOpsMixin): # other DataFrame APIs equivalent to Scala side. ``` Yes, This is a big PR but they are mostly just moving around except one case `createDataFrame` which I had to split the methods. ### Why are the changes needed? There are pandas functionalities here and there and I myself gets lost where it was. Also, when you have to make a change commonly for all of pandas related features, it's almost impossible now. Also, after this change, `DataFrame` and `SparkSession` become more consistent with Scala side since pandas is specific to Python, and this change separates pandas-specific APIs away from `DataFrame` or `SparkSession`. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests should cover. Also, I manually built the PySpark API documentation and checked. Closes #27109 from HyukjinKwon/pandas-refactoring. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-08 20:22:50 -05:00
"""
[SPARK-28264][PYTHON][SQL] Support type hints in pandas UDF and rename/move inconsistent pandas UDF types ### What changes were proposed in this pull request? This PR proposes to redesign pandas UDFs as described in [the proposal](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit?usp=sharing). ```python from pyspark.sql.functions import pandas_udf import pandas as pd pandas_udf("long") def plug_one(s: pd.Series) -> pd.Series: return s + 1 spark.range(10).select(plug_one("id")).show() ``` ``` +------------+ |plug_one(id)| +------------+ | 1| | 2| | 3| | 4| | 5| | 6| | 7| | 8| | 9| | 10| +------------+ ``` Note that, this PR address one of the future improvements described [here](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit#heading=h.h3ncjpk6ujqu), "A couple of less-intuitive pandas UDF types" (by zero323) together. In short, - Adds new way with type hints as an alternative and experimental way. ```python pandas_udf(schema='...') def func(c1: Series, c2: Series) -> DataFrame: pass ``` - Replace and/or add an alias for three types below from UDF, and make them as separate standalone APIs. So, `pandas_udf` is now consistent with regular `udf`s and other expressions. `df.mapInPandas(udf)` -replace-> `df.mapInPandas(f, schema)` `df.groupby.apply(udf)` -alias-> `df.groupby.applyInPandas(f, schema)` `df.groupby.cogroup.apply(udf)` -replace-> `df.groupby.cogroup.applyInPandas(f, schema)` *`df.groupby.apply` was added from 2.3 while the other were added in the master only. - No deprecation for the existing ways for now. ```python pandas_udf(schema='...', functionType=PandasUDFType.SCALAR) def func(c1, c2): pass ``` If users are happy with this, I plan to deprecate the existing way and declare using type hints is not experimental anymore. One design goal in this PR was that, avoid touching the internal (since we didn't deprecate the old ways for now), but supports type hints with a minimised changes only at the interface. - Once we deprecate or remove the old ways, I think it requires another refactoring for the internal in the future. At the very least, we should rename internal pandas evaluation types. - If users find this experimental type hints isn't quite helpful, we should simply revert the changes at the interface level. ### Why are the changes needed? In order to address old design issues. Please see [the proposal](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit?usp=sharing). ### Does this PR introduce any user-facing change? For behaviour changes, No. It adds new ways to use pandas UDFs by using type hints. See below. **SCALAR**: ```python pandas_udf(schema='...') def func(c1: Series, c2: DataFrame) -> Series: pass # DataFrame represents a struct column ``` **SCALAR_ITER**: ```python pandas_udf(schema='...') def func(iter: Iterator[Tuple[Series, DataFrame, ...]]) -> Iterator[Series]: pass # Same as SCALAR but wrapped by Iterator ``` **GROUPED_AGG**: ```python pandas_udf(schema='...') def func(c1: Series, c2: DataFrame) -> int: pass # DataFrame represents a struct column ``` **GROUPED_MAP**: This was added in Spark 2.3 as of SPARK-20396. As described above, it keeps the existing behaviour. Additionally, we now have a new alias `groupby.applyInPandas` for `groupby.apply`. See the example below: ```python def func(pdf): return pdf df.groupby("...").applyInPandas(func, schema=df.schema) ``` **MAP_ITER**: this is not a pandas UDF anymore This was added in Spark 3.0 as of SPARK-28198; and this PR replaces the usages. See the example below: ```python def func(iter): for df in iter: yield df df.mapInPandas(func, df.schema) ``` **COGROUPED_MAP**: this is not a pandas UDF anymore This was added in Spark 3.0 as of SPARK-27463; and this PR replaces the usages. See the example below: ```python def asof_join(left, right): return pd.merge_asof(left, right, on="...", by="...") df1.groupby("...").cogroup(df2.groupby("...")).applyInPandas(asof_join, schema="...") ``` ### How was this patch tested? Unittests added and tested against Python 2.7, 3.6 and 3.7. Closes #27165 from HyukjinKwon/revisit-pandas. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-22 01:32:58 -05:00
Maps an iterator of batches in the current :class:`DataFrame` using a Python native
function that takes and outputs a pandas DataFrame, and returns the result as a
:class:`DataFrame`.
[SPARK-30434][PYTHON][SQL] Move pandas related functionalities into 'pandas' sub-package ### What changes were proposed in this pull request? This PR proposes to move pandas related functionalities into pandas package. Namely: ```bash pyspark/sql/pandas ├── __init__.py ├── conversion.py # Conversion between pandas <> PySpark DataFrames ├── functions.py # pandas_udf ├── group_ops.py # Grouped UDF / Cogrouped UDF + groupby.apply, groupby.cogroup.apply ├── map_ops.py # Map Iter UDF + mapInPandas ├── serializers.py # pandas <> PyArrow serializers ├── types.py # Type utils between pandas <> PyArrow └── utils.py # Version requirement checks ``` In order to separately locate `groupby.apply`, `groupby.cogroup.apply`, `mapInPandas`, `toPandas`, and `createDataFrame(pdf)` under `pandas` sub-package, I had to use a mix-in approach which Scala side uses often by `trait`, and also pandas itself uses this approach (see `IndexOpsMixin` as an example) to group related functionalities. Currently, you can think it's like Scala's self typed trait. See the structure below: ```python class PandasMapOpsMixin(object): def mapInPandas(self, ...): ... return ... # other Pandas <> PySpark APIs ``` ```python class DataFrame(PandasMapOpsMixin): # other DataFrame APIs equivalent to Scala side. ``` Yes, This is a big PR but they are mostly just moving around except one case `createDataFrame` which I had to split the methods. ### Why are the changes needed? There are pandas functionalities here and there and I myself gets lost where it was. Also, when you have to make a change commonly for all of pandas related features, it's almost impossible now. Also, after this change, `DataFrame` and `SparkSession` become more consistent with Scala side since pandas is specific to Python, and this change separates pandas-specific APIs away from `DataFrame` or `SparkSession`. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests should cover. Also, I manually built the PySpark API documentation and checked. Closes #27109 from HyukjinKwon/pandas-refactoring. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-08 20:22:50 -05:00
[SPARK-28264][PYTHON][SQL] Support type hints in pandas UDF and rename/move inconsistent pandas UDF types ### What changes were proposed in this pull request? This PR proposes to redesign pandas UDFs as described in [the proposal](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit?usp=sharing). ```python from pyspark.sql.functions import pandas_udf import pandas as pd pandas_udf("long") def plug_one(s: pd.Series) -> pd.Series: return s + 1 spark.range(10).select(plug_one("id")).show() ``` ``` +------------+ |plug_one(id)| +------------+ | 1| | 2| | 3| | 4| | 5| | 6| | 7| | 8| | 9| | 10| +------------+ ``` Note that, this PR address one of the future improvements described [here](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit#heading=h.h3ncjpk6ujqu), "A couple of less-intuitive pandas UDF types" (by zero323) together. In short, - Adds new way with type hints as an alternative and experimental way. ```python pandas_udf(schema='...') def func(c1: Series, c2: Series) -> DataFrame: pass ``` - Replace and/or add an alias for three types below from UDF, and make them as separate standalone APIs. So, `pandas_udf` is now consistent with regular `udf`s and other expressions. `df.mapInPandas(udf)` -replace-> `df.mapInPandas(f, schema)` `df.groupby.apply(udf)` -alias-> `df.groupby.applyInPandas(f, schema)` `df.groupby.cogroup.apply(udf)` -replace-> `df.groupby.cogroup.applyInPandas(f, schema)` *`df.groupby.apply` was added from 2.3 while the other were added in the master only. - No deprecation for the existing ways for now. ```python pandas_udf(schema='...', functionType=PandasUDFType.SCALAR) def func(c1, c2): pass ``` If users are happy with this, I plan to deprecate the existing way and declare using type hints is not experimental anymore. One design goal in this PR was that, avoid touching the internal (since we didn't deprecate the old ways for now), but supports type hints with a minimised changes only at the interface. - Once we deprecate or remove the old ways, I think it requires another refactoring for the internal in the future. At the very least, we should rename internal pandas evaluation types. - If users find this experimental type hints isn't quite helpful, we should simply revert the changes at the interface level. ### Why are the changes needed? In order to address old design issues. Please see [the proposal](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit?usp=sharing). ### Does this PR introduce any user-facing change? For behaviour changes, No. It adds new ways to use pandas UDFs by using type hints. See below. **SCALAR**: ```python pandas_udf(schema='...') def func(c1: Series, c2: DataFrame) -> Series: pass # DataFrame represents a struct column ``` **SCALAR_ITER**: ```python pandas_udf(schema='...') def func(iter: Iterator[Tuple[Series, DataFrame, ...]]) -> Iterator[Series]: pass # Same as SCALAR but wrapped by Iterator ``` **GROUPED_AGG**: ```python pandas_udf(schema='...') def func(c1: Series, c2: DataFrame) -> int: pass # DataFrame represents a struct column ``` **GROUPED_MAP**: This was added in Spark 2.3 as of SPARK-20396. As described above, it keeps the existing behaviour. Additionally, we now have a new alias `groupby.applyInPandas` for `groupby.apply`. See the example below: ```python def func(pdf): return pdf df.groupby("...").applyInPandas(func, schema=df.schema) ``` **MAP_ITER**: this is not a pandas UDF anymore This was added in Spark 3.0 as of SPARK-28198; and this PR replaces the usages. See the example below: ```python def func(iter): for df in iter: yield df df.mapInPandas(func, df.schema) ``` **COGROUPED_MAP**: this is not a pandas UDF anymore This was added in Spark 3.0 as of SPARK-27463; and this PR replaces the usages. See the example below: ```python def asof_join(left, right): return pd.merge_asof(left, right, on="...", by="...") df1.groupby("...").cogroup(df2.groupby("...")).applyInPandas(asof_join, schema="...") ``` ### How was this patch tested? Unittests added and tested against Python 2.7, 3.6 and 3.7. Closes #27165 from HyukjinKwon/revisit-pandas. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-22 01:32:58 -05:00
The function should take an iterator of `pandas.DataFrame`\\s and return
[SPARK-30434][PYTHON][SQL] Move pandas related functionalities into 'pandas' sub-package ### What changes were proposed in this pull request? This PR proposes to move pandas related functionalities into pandas package. Namely: ```bash pyspark/sql/pandas ├── __init__.py ├── conversion.py # Conversion between pandas <> PySpark DataFrames ├── functions.py # pandas_udf ├── group_ops.py # Grouped UDF / Cogrouped UDF + groupby.apply, groupby.cogroup.apply ├── map_ops.py # Map Iter UDF + mapInPandas ├── serializers.py # pandas <> PyArrow serializers ├── types.py # Type utils between pandas <> PyArrow └── utils.py # Version requirement checks ``` In order to separately locate `groupby.apply`, `groupby.cogroup.apply`, `mapInPandas`, `toPandas`, and `createDataFrame(pdf)` under `pandas` sub-package, I had to use a mix-in approach which Scala side uses often by `trait`, and also pandas itself uses this approach (see `IndexOpsMixin` as an example) to group related functionalities. Currently, you can think it's like Scala's self typed trait. See the structure below: ```python class PandasMapOpsMixin(object): def mapInPandas(self, ...): ... return ... # other Pandas <> PySpark APIs ``` ```python class DataFrame(PandasMapOpsMixin): # other DataFrame APIs equivalent to Scala side. ``` Yes, This is a big PR but they are mostly just moving around except one case `createDataFrame` which I had to split the methods. ### Why are the changes needed? There are pandas functionalities here and there and I myself gets lost where it was. Also, when you have to make a change commonly for all of pandas related features, it's almost impossible now. Also, after this change, `DataFrame` and `SparkSession` become more consistent with Scala side since pandas is specific to Python, and this change separates pandas-specific APIs away from `DataFrame` or `SparkSession`. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests should cover. Also, I manually built the PySpark API documentation and checked. Closes #27109 from HyukjinKwon/pandas-refactoring. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-08 20:22:50 -05:00
another iterator of `pandas.DataFrame`\\s. All columns are passed
[SPARK-28264][PYTHON][SQL] Support type hints in pandas UDF and rename/move inconsistent pandas UDF types ### What changes were proposed in this pull request? This PR proposes to redesign pandas UDFs as described in [the proposal](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit?usp=sharing). ```python from pyspark.sql.functions import pandas_udf import pandas as pd pandas_udf("long") def plug_one(s: pd.Series) -> pd.Series: return s + 1 spark.range(10).select(plug_one("id")).show() ``` ``` +------------+ |plug_one(id)| +------------+ | 1| | 2| | 3| | 4| | 5| | 6| | 7| | 8| | 9| | 10| +------------+ ``` Note that, this PR address one of the future improvements described [here](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit#heading=h.h3ncjpk6ujqu), "A couple of less-intuitive pandas UDF types" (by zero323) together. In short, - Adds new way with type hints as an alternative and experimental way. ```python pandas_udf(schema='...') def func(c1: Series, c2: Series) -> DataFrame: pass ``` - Replace and/or add an alias for three types below from UDF, and make them as separate standalone APIs. So, `pandas_udf` is now consistent with regular `udf`s and other expressions. `df.mapInPandas(udf)` -replace-> `df.mapInPandas(f, schema)` `df.groupby.apply(udf)` -alias-> `df.groupby.applyInPandas(f, schema)` `df.groupby.cogroup.apply(udf)` -replace-> `df.groupby.cogroup.applyInPandas(f, schema)` *`df.groupby.apply` was added from 2.3 while the other were added in the master only. - No deprecation for the existing ways for now. ```python pandas_udf(schema='...', functionType=PandasUDFType.SCALAR) def func(c1, c2): pass ``` If users are happy with this, I plan to deprecate the existing way and declare using type hints is not experimental anymore. One design goal in this PR was that, avoid touching the internal (since we didn't deprecate the old ways for now), but supports type hints with a minimised changes only at the interface. - Once we deprecate or remove the old ways, I think it requires another refactoring for the internal in the future. At the very least, we should rename internal pandas evaluation types. - If users find this experimental type hints isn't quite helpful, we should simply revert the changes at the interface level. ### Why are the changes needed? In order to address old design issues. Please see [the proposal](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit?usp=sharing). ### Does this PR introduce any user-facing change? For behaviour changes, No. It adds new ways to use pandas UDFs by using type hints. See below. **SCALAR**: ```python pandas_udf(schema='...') def func(c1: Series, c2: DataFrame) -> Series: pass # DataFrame represents a struct column ``` **SCALAR_ITER**: ```python pandas_udf(schema='...') def func(iter: Iterator[Tuple[Series, DataFrame, ...]]) -> Iterator[Series]: pass # Same as SCALAR but wrapped by Iterator ``` **GROUPED_AGG**: ```python pandas_udf(schema='...') def func(c1: Series, c2: DataFrame) -> int: pass # DataFrame represents a struct column ``` **GROUPED_MAP**: This was added in Spark 2.3 as of SPARK-20396. As described above, it keeps the existing behaviour. Additionally, we now have a new alias `groupby.applyInPandas` for `groupby.apply`. See the example below: ```python def func(pdf): return pdf df.groupby("...").applyInPandas(func, schema=df.schema) ``` **MAP_ITER**: this is not a pandas UDF anymore This was added in Spark 3.0 as of SPARK-28198; and this PR replaces the usages. See the example below: ```python def func(iter): for df in iter: yield df df.mapInPandas(func, df.schema) ``` **COGROUPED_MAP**: this is not a pandas UDF anymore This was added in Spark 3.0 as of SPARK-27463; and this PR replaces the usages. See the example below: ```python def asof_join(left, right): return pd.merge_asof(left, right, on="...", by="...") df1.groupby("...").cogroup(df2.groupby("...")).applyInPandas(asof_join, schema="...") ``` ### How was this patch tested? Unittests added and tested against Python 2.7, 3.6 and 3.7. Closes #27165 from HyukjinKwon/revisit-pandas. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-22 01:32:58 -05:00
together as an iterator of `pandas.DataFrame`\\s to the function and the
[SPARK-30434][PYTHON][SQL] Move pandas related functionalities into 'pandas' sub-package ### What changes were proposed in this pull request? This PR proposes to move pandas related functionalities into pandas package. Namely: ```bash pyspark/sql/pandas ├── __init__.py ├── conversion.py # Conversion between pandas <> PySpark DataFrames ├── functions.py # pandas_udf ├── group_ops.py # Grouped UDF / Cogrouped UDF + groupby.apply, groupby.cogroup.apply ├── map_ops.py # Map Iter UDF + mapInPandas ├── serializers.py # pandas <> PyArrow serializers ├── types.py # Type utils between pandas <> PyArrow └── utils.py # Version requirement checks ``` In order to separately locate `groupby.apply`, `groupby.cogroup.apply`, `mapInPandas`, `toPandas`, and `createDataFrame(pdf)` under `pandas` sub-package, I had to use a mix-in approach which Scala side uses often by `trait`, and also pandas itself uses this approach (see `IndexOpsMixin` as an example) to group related functionalities. Currently, you can think it's like Scala's self typed trait. See the structure below: ```python class PandasMapOpsMixin(object): def mapInPandas(self, ...): ... return ... # other Pandas <> PySpark APIs ``` ```python class DataFrame(PandasMapOpsMixin): # other DataFrame APIs equivalent to Scala side. ``` Yes, This is a big PR but they are mostly just moving around except one case `createDataFrame` which I had to split the methods. ### Why are the changes needed? There are pandas functionalities here and there and I myself gets lost where it was. Also, when you have to make a change commonly for all of pandas related features, it's almost impossible now. Also, after this change, `DataFrame` and `SparkSession` become more consistent with Scala side since pandas is specific to Python, and this change separates pandas-specific APIs away from `DataFrame` or `SparkSession`. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests should cover. Also, I manually built the PySpark API documentation and checked. Closes #27109 from HyukjinKwon/pandas-refactoring. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-08 20:22:50 -05:00
returned iterator of `pandas.DataFrame`\\s are combined as a :class:`DataFrame`.
Each `pandas.DataFrame` size can be controlled by
`spark.sql.execution.arrow.maxRecordsPerBatch`.
.. versionadded:: 3.0.0
Parameters
----------
func : function
a Python native function that takes an iterator of `pandas.DataFrame`\\s, and
[SPARK-28264][PYTHON][SQL] Support type hints in pandas UDF and rename/move inconsistent pandas UDF types ### What changes were proposed in this pull request? This PR proposes to redesign pandas UDFs as described in [the proposal](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit?usp=sharing). ```python from pyspark.sql.functions import pandas_udf import pandas as pd pandas_udf("long") def plug_one(s: pd.Series) -> pd.Series: return s + 1 spark.range(10).select(plug_one("id")).show() ``` ``` +------------+ |plug_one(id)| +------------+ | 1| | 2| | 3| | 4| | 5| | 6| | 7| | 8| | 9| | 10| +------------+ ``` Note that, this PR address one of the future improvements described [here](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit#heading=h.h3ncjpk6ujqu), "A couple of less-intuitive pandas UDF types" (by zero323) together. In short, - Adds new way with type hints as an alternative and experimental way. ```python pandas_udf(schema='...') def func(c1: Series, c2: Series) -> DataFrame: pass ``` - Replace and/or add an alias for three types below from UDF, and make them as separate standalone APIs. So, `pandas_udf` is now consistent with regular `udf`s and other expressions. `df.mapInPandas(udf)` -replace-> `df.mapInPandas(f, schema)` `df.groupby.apply(udf)` -alias-> `df.groupby.applyInPandas(f, schema)` `df.groupby.cogroup.apply(udf)` -replace-> `df.groupby.cogroup.applyInPandas(f, schema)` *`df.groupby.apply` was added from 2.3 while the other were added in the master only. - No deprecation for the existing ways for now. ```python pandas_udf(schema='...', functionType=PandasUDFType.SCALAR) def func(c1, c2): pass ``` If users are happy with this, I plan to deprecate the existing way and declare using type hints is not experimental anymore. One design goal in this PR was that, avoid touching the internal (since we didn't deprecate the old ways for now), but supports type hints with a minimised changes only at the interface. - Once we deprecate or remove the old ways, I think it requires another refactoring for the internal in the future. At the very least, we should rename internal pandas evaluation types. - If users find this experimental type hints isn't quite helpful, we should simply revert the changes at the interface level. ### Why are the changes needed? In order to address old design issues. Please see [the proposal](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit?usp=sharing). ### Does this PR introduce any user-facing change? For behaviour changes, No. It adds new ways to use pandas UDFs by using type hints. See below. **SCALAR**: ```python pandas_udf(schema='...') def func(c1: Series, c2: DataFrame) -> Series: pass # DataFrame represents a struct column ``` **SCALAR_ITER**: ```python pandas_udf(schema='...') def func(iter: Iterator[Tuple[Series, DataFrame, ...]]) -> Iterator[Series]: pass # Same as SCALAR but wrapped by Iterator ``` **GROUPED_AGG**: ```python pandas_udf(schema='...') def func(c1: Series, c2: DataFrame) -> int: pass # DataFrame represents a struct column ``` **GROUPED_MAP**: This was added in Spark 2.3 as of SPARK-20396. As described above, it keeps the existing behaviour. Additionally, we now have a new alias `groupby.applyInPandas` for `groupby.apply`. See the example below: ```python def func(pdf): return pdf df.groupby("...").applyInPandas(func, schema=df.schema) ``` **MAP_ITER**: this is not a pandas UDF anymore This was added in Spark 3.0 as of SPARK-28198; and this PR replaces the usages. See the example below: ```python def func(iter): for df in iter: yield df df.mapInPandas(func, df.schema) ``` **COGROUPED_MAP**: this is not a pandas UDF anymore This was added in Spark 3.0 as of SPARK-27463; and this PR replaces the usages. See the example below: ```python def asof_join(left, right): return pd.merge_asof(left, right, on="...", by="...") df1.groupby("...").cogroup(df2.groupby("...")).applyInPandas(asof_join, schema="...") ``` ### How was this patch tested? Unittests added and tested against Python 2.7, 3.6 and 3.7. Closes #27165 from HyukjinKwon/revisit-pandas. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-22 01:32:58 -05:00
outputs an iterator of `pandas.DataFrame`\\s.
schema : :class:`pyspark.sql.types.DataType` or str
the return type of the `func` in PySpark. The value can be either a
[SPARK-28264][PYTHON][SQL] Support type hints in pandas UDF and rename/move inconsistent pandas UDF types ### What changes were proposed in this pull request? This PR proposes to redesign pandas UDFs as described in [the proposal](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit?usp=sharing). ```python from pyspark.sql.functions import pandas_udf import pandas as pd pandas_udf("long") def plug_one(s: pd.Series) -> pd.Series: return s + 1 spark.range(10).select(plug_one("id")).show() ``` ``` +------------+ |plug_one(id)| +------------+ | 1| | 2| | 3| | 4| | 5| | 6| | 7| | 8| | 9| | 10| +------------+ ``` Note that, this PR address one of the future improvements described [here](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit#heading=h.h3ncjpk6ujqu), "A couple of less-intuitive pandas UDF types" (by zero323) together. In short, - Adds new way with type hints as an alternative and experimental way. ```python pandas_udf(schema='...') def func(c1: Series, c2: Series) -> DataFrame: pass ``` - Replace and/or add an alias for three types below from UDF, and make them as separate standalone APIs. So, `pandas_udf` is now consistent with regular `udf`s and other expressions. `df.mapInPandas(udf)` -replace-> `df.mapInPandas(f, schema)` `df.groupby.apply(udf)` -alias-> `df.groupby.applyInPandas(f, schema)` `df.groupby.cogroup.apply(udf)` -replace-> `df.groupby.cogroup.applyInPandas(f, schema)` *`df.groupby.apply` was added from 2.3 while the other were added in the master only. - No deprecation for the existing ways for now. ```python pandas_udf(schema='...', functionType=PandasUDFType.SCALAR) def func(c1, c2): pass ``` If users are happy with this, I plan to deprecate the existing way and declare using type hints is not experimental anymore. One design goal in this PR was that, avoid touching the internal (since we didn't deprecate the old ways for now), but supports type hints with a minimised changes only at the interface. - Once we deprecate or remove the old ways, I think it requires another refactoring for the internal in the future. At the very least, we should rename internal pandas evaluation types. - If users find this experimental type hints isn't quite helpful, we should simply revert the changes at the interface level. ### Why are the changes needed? In order to address old design issues. Please see [the proposal](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit?usp=sharing). ### Does this PR introduce any user-facing change? For behaviour changes, No. It adds new ways to use pandas UDFs by using type hints. See below. **SCALAR**: ```python pandas_udf(schema='...') def func(c1: Series, c2: DataFrame) -> Series: pass # DataFrame represents a struct column ``` **SCALAR_ITER**: ```python pandas_udf(schema='...') def func(iter: Iterator[Tuple[Series, DataFrame, ...]]) -> Iterator[Series]: pass # Same as SCALAR but wrapped by Iterator ``` **GROUPED_AGG**: ```python pandas_udf(schema='...') def func(c1: Series, c2: DataFrame) -> int: pass # DataFrame represents a struct column ``` **GROUPED_MAP**: This was added in Spark 2.3 as of SPARK-20396. As described above, it keeps the existing behaviour. Additionally, we now have a new alias `groupby.applyInPandas` for `groupby.apply`. See the example below: ```python def func(pdf): return pdf df.groupby("...").applyInPandas(func, schema=df.schema) ``` **MAP_ITER**: this is not a pandas UDF anymore This was added in Spark 3.0 as of SPARK-28198; and this PR replaces the usages. See the example below: ```python def func(iter): for df in iter: yield df df.mapInPandas(func, df.schema) ``` **COGROUPED_MAP**: this is not a pandas UDF anymore This was added in Spark 3.0 as of SPARK-27463; and this PR replaces the usages. See the example below: ```python def asof_join(left, right): return pd.merge_asof(left, right, on="...", by="...") df1.groupby("...").cogroup(df2.groupby("...")).applyInPandas(asof_join, schema="...") ``` ### How was this patch tested? Unittests added and tested against Python 2.7, 3.6 and 3.7. Closes #27165 from HyukjinKwon/revisit-pandas. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-22 01:32:58 -05:00
:class:`pyspark.sql.types.DataType` object or a DDL-formatted type string.
[SPARK-30434][PYTHON][SQL] Move pandas related functionalities into 'pandas' sub-package ### What changes were proposed in this pull request? This PR proposes to move pandas related functionalities into pandas package. Namely: ```bash pyspark/sql/pandas ├── __init__.py ├── conversion.py # Conversion between pandas <> PySpark DataFrames ├── functions.py # pandas_udf ├── group_ops.py # Grouped UDF / Cogrouped UDF + groupby.apply, groupby.cogroup.apply ├── map_ops.py # Map Iter UDF + mapInPandas ├── serializers.py # pandas <> PyArrow serializers ├── types.py # Type utils between pandas <> PyArrow └── utils.py # Version requirement checks ``` In order to separately locate `groupby.apply`, `groupby.cogroup.apply`, `mapInPandas`, `toPandas`, and `createDataFrame(pdf)` under `pandas` sub-package, I had to use a mix-in approach which Scala side uses often by `trait`, and also pandas itself uses this approach (see `IndexOpsMixin` as an example) to group related functionalities. Currently, you can think it's like Scala's self typed trait. See the structure below: ```python class PandasMapOpsMixin(object): def mapInPandas(self, ...): ... return ... # other Pandas <> PySpark APIs ``` ```python class DataFrame(PandasMapOpsMixin): # other DataFrame APIs equivalent to Scala side. ``` Yes, This is a big PR but they are mostly just moving around except one case `createDataFrame` which I had to split the methods. ### Why are the changes needed? There are pandas functionalities here and there and I myself gets lost where it was. Also, when you have to make a change commonly for all of pandas related features, it's almost impossible now. Also, after this change, `DataFrame` and `SparkSession` become more consistent with Scala side since pandas is specific to Python, and this change separates pandas-specific APIs away from `DataFrame` or `SparkSession`. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests should cover. Also, I manually built the PySpark API documentation and checked. Closes #27109 from HyukjinKwon/pandas-refactoring. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-08 20:22:50 -05:00
Examples
--------
>>> from pyspark.sql.functions import pandas_udf
[SPARK-28264][PYTHON][SQL] Support type hints in pandas UDF and rename/move inconsistent pandas UDF types ### What changes were proposed in this pull request? This PR proposes to redesign pandas UDFs as described in [the proposal](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit?usp=sharing). ```python from pyspark.sql.functions import pandas_udf import pandas as pd pandas_udf("long") def plug_one(s: pd.Series) -> pd.Series: return s + 1 spark.range(10).select(plug_one("id")).show() ``` ``` +------------+ |plug_one(id)| +------------+ | 1| | 2| | 3| | 4| | 5| | 6| | 7| | 8| | 9| | 10| +------------+ ``` Note that, this PR address one of the future improvements described [here](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit#heading=h.h3ncjpk6ujqu), "A couple of less-intuitive pandas UDF types" (by zero323) together. In short, - Adds new way with type hints as an alternative and experimental way. ```python pandas_udf(schema='...') def func(c1: Series, c2: Series) -> DataFrame: pass ``` - Replace and/or add an alias for three types below from UDF, and make them as separate standalone APIs. So, `pandas_udf` is now consistent with regular `udf`s and other expressions. `df.mapInPandas(udf)` -replace-> `df.mapInPandas(f, schema)` `df.groupby.apply(udf)` -alias-> `df.groupby.applyInPandas(f, schema)` `df.groupby.cogroup.apply(udf)` -replace-> `df.groupby.cogroup.applyInPandas(f, schema)` *`df.groupby.apply` was added from 2.3 while the other were added in the master only. - No deprecation for the existing ways for now. ```python pandas_udf(schema='...', functionType=PandasUDFType.SCALAR) def func(c1, c2): pass ``` If users are happy with this, I plan to deprecate the existing way and declare using type hints is not experimental anymore. One design goal in this PR was that, avoid touching the internal (since we didn't deprecate the old ways for now), but supports type hints with a minimised changes only at the interface. - Once we deprecate or remove the old ways, I think it requires another refactoring for the internal in the future. At the very least, we should rename internal pandas evaluation types. - If users find this experimental type hints isn't quite helpful, we should simply revert the changes at the interface level. ### Why are the changes needed? In order to address old design issues. Please see [the proposal](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit?usp=sharing). ### Does this PR introduce any user-facing change? For behaviour changes, No. It adds new ways to use pandas UDFs by using type hints. See below. **SCALAR**: ```python pandas_udf(schema='...') def func(c1: Series, c2: DataFrame) -> Series: pass # DataFrame represents a struct column ``` **SCALAR_ITER**: ```python pandas_udf(schema='...') def func(iter: Iterator[Tuple[Series, DataFrame, ...]]) -> Iterator[Series]: pass # Same as SCALAR but wrapped by Iterator ``` **GROUPED_AGG**: ```python pandas_udf(schema='...') def func(c1: Series, c2: DataFrame) -> int: pass # DataFrame represents a struct column ``` **GROUPED_MAP**: This was added in Spark 2.3 as of SPARK-20396. As described above, it keeps the existing behaviour. Additionally, we now have a new alias `groupby.applyInPandas` for `groupby.apply`. See the example below: ```python def func(pdf): return pdf df.groupby("...").applyInPandas(func, schema=df.schema) ``` **MAP_ITER**: this is not a pandas UDF anymore This was added in Spark 3.0 as of SPARK-28198; and this PR replaces the usages. See the example below: ```python def func(iter): for df in iter: yield df df.mapInPandas(func, df.schema) ``` **COGROUPED_MAP**: this is not a pandas UDF anymore This was added in Spark 3.0 as of SPARK-27463; and this PR replaces the usages. See the example below: ```python def asof_join(left, right): return pd.merge_asof(left, right, on="...", by="...") df1.groupby("...").cogroup(df2.groupby("...")).applyInPandas(asof_join, schema="...") ``` ### How was this patch tested? Unittests added and tested against Python 2.7, 3.6 and 3.7. Closes #27165 from HyukjinKwon/revisit-pandas. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-22 01:32:58 -05:00
>>> df = spark.createDataFrame([(1, 21), (2, 30)], ("id", "age"))
>>> def filter_func(iterator):
... for pdf in iterator:
[SPARK-30434][PYTHON][SQL] Move pandas related functionalities into 'pandas' sub-package ### What changes were proposed in this pull request? This PR proposes to move pandas related functionalities into pandas package. Namely: ```bash pyspark/sql/pandas ├── __init__.py ├── conversion.py # Conversion between pandas <> PySpark DataFrames ├── functions.py # pandas_udf ├── group_ops.py # Grouped UDF / Cogrouped UDF + groupby.apply, groupby.cogroup.apply ├── map_ops.py # Map Iter UDF + mapInPandas ├── serializers.py # pandas <> PyArrow serializers ├── types.py # Type utils between pandas <> PyArrow └── utils.py # Version requirement checks ``` In order to separately locate `groupby.apply`, `groupby.cogroup.apply`, `mapInPandas`, `toPandas`, and `createDataFrame(pdf)` under `pandas` sub-package, I had to use a mix-in approach which Scala side uses often by `trait`, and also pandas itself uses this approach (see `IndexOpsMixin` as an example) to group related functionalities. Currently, you can think it's like Scala's self typed trait. See the structure below: ```python class PandasMapOpsMixin(object): def mapInPandas(self, ...): ... return ... # other Pandas <> PySpark APIs ``` ```python class DataFrame(PandasMapOpsMixin): # other DataFrame APIs equivalent to Scala side. ``` Yes, This is a big PR but they are mostly just moving around except one case `createDataFrame` which I had to split the methods. ### Why are the changes needed? There are pandas functionalities here and there and I myself gets lost where it was. Also, when you have to make a change commonly for all of pandas related features, it's almost impossible now. Also, after this change, `DataFrame` and `SparkSession` become more consistent with Scala side since pandas is specific to Python, and this change separates pandas-specific APIs away from `DataFrame` or `SparkSession`. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests should cover. Also, I manually built the PySpark API documentation and checked. Closes #27109 from HyukjinKwon/pandas-refactoring. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-08 20:22:50 -05:00
... yield pdf[pdf.id == 1]
[SPARK-28264][PYTHON][SQL] Support type hints in pandas UDF and rename/move inconsistent pandas UDF types ### What changes were proposed in this pull request? This PR proposes to redesign pandas UDFs as described in [the proposal](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit?usp=sharing). ```python from pyspark.sql.functions import pandas_udf import pandas as pd pandas_udf("long") def plug_one(s: pd.Series) -> pd.Series: return s + 1 spark.range(10).select(plug_one("id")).show() ``` ``` +------------+ |plug_one(id)| +------------+ | 1| | 2| | 3| | 4| | 5| | 6| | 7| | 8| | 9| | 10| +------------+ ``` Note that, this PR address one of the future improvements described [here](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit#heading=h.h3ncjpk6ujqu), "A couple of less-intuitive pandas UDF types" (by zero323) together. In short, - Adds new way with type hints as an alternative and experimental way. ```python pandas_udf(schema='...') def func(c1: Series, c2: Series) -> DataFrame: pass ``` - Replace and/or add an alias for three types below from UDF, and make them as separate standalone APIs. So, `pandas_udf` is now consistent with regular `udf`s and other expressions. `df.mapInPandas(udf)` -replace-> `df.mapInPandas(f, schema)` `df.groupby.apply(udf)` -alias-> `df.groupby.applyInPandas(f, schema)` `df.groupby.cogroup.apply(udf)` -replace-> `df.groupby.cogroup.applyInPandas(f, schema)` *`df.groupby.apply` was added from 2.3 while the other were added in the master only. - No deprecation for the existing ways for now. ```python pandas_udf(schema='...', functionType=PandasUDFType.SCALAR) def func(c1, c2): pass ``` If users are happy with this, I plan to deprecate the existing way and declare using type hints is not experimental anymore. One design goal in this PR was that, avoid touching the internal (since we didn't deprecate the old ways for now), but supports type hints with a minimised changes only at the interface. - Once we deprecate or remove the old ways, I think it requires another refactoring for the internal in the future. At the very least, we should rename internal pandas evaluation types. - If users find this experimental type hints isn't quite helpful, we should simply revert the changes at the interface level. ### Why are the changes needed? In order to address old design issues. Please see [the proposal](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit?usp=sharing). ### Does this PR introduce any user-facing change? For behaviour changes, No. It adds new ways to use pandas UDFs by using type hints. See below. **SCALAR**: ```python pandas_udf(schema='...') def func(c1: Series, c2: DataFrame) -> Series: pass # DataFrame represents a struct column ``` **SCALAR_ITER**: ```python pandas_udf(schema='...') def func(iter: Iterator[Tuple[Series, DataFrame, ...]]) -> Iterator[Series]: pass # Same as SCALAR but wrapped by Iterator ``` **GROUPED_AGG**: ```python pandas_udf(schema='...') def func(c1: Series, c2: DataFrame) -> int: pass # DataFrame represents a struct column ``` **GROUPED_MAP**: This was added in Spark 2.3 as of SPARK-20396. As described above, it keeps the existing behaviour. Additionally, we now have a new alias `groupby.applyInPandas` for `groupby.apply`. See the example below: ```python def func(pdf): return pdf df.groupby("...").applyInPandas(func, schema=df.schema) ``` **MAP_ITER**: this is not a pandas UDF anymore This was added in Spark 3.0 as of SPARK-28198; and this PR replaces the usages. See the example below: ```python def func(iter): for df in iter: yield df df.mapInPandas(func, df.schema) ``` **COGROUPED_MAP**: this is not a pandas UDF anymore This was added in Spark 3.0 as of SPARK-27463; and this PR replaces the usages. See the example below: ```python def asof_join(left, right): return pd.merge_asof(left, right, on="...", by="...") df1.groupby("...").cogroup(df2.groupby("...")).applyInPandas(asof_join, schema="...") ``` ### How was this patch tested? Unittests added and tested against Python 2.7, 3.6 and 3.7. Closes #27165 from HyukjinKwon/revisit-pandas. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-22 01:32:58 -05:00
>>> df.mapInPandas(filter_func, df.schema).show() # doctest: +SKIP
[SPARK-30434][PYTHON][SQL] Move pandas related functionalities into 'pandas' sub-package ### What changes were proposed in this pull request? This PR proposes to move pandas related functionalities into pandas package. Namely: ```bash pyspark/sql/pandas ├── __init__.py ├── conversion.py # Conversion between pandas <> PySpark DataFrames ├── functions.py # pandas_udf ├── group_ops.py # Grouped UDF / Cogrouped UDF + groupby.apply, groupby.cogroup.apply ├── map_ops.py # Map Iter UDF + mapInPandas ├── serializers.py # pandas <> PyArrow serializers ├── types.py # Type utils between pandas <> PyArrow └── utils.py # Version requirement checks ``` In order to separately locate `groupby.apply`, `groupby.cogroup.apply`, `mapInPandas`, `toPandas`, and `createDataFrame(pdf)` under `pandas` sub-package, I had to use a mix-in approach which Scala side uses often by `trait`, and also pandas itself uses this approach (see `IndexOpsMixin` as an example) to group related functionalities. Currently, you can think it's like Scala's self typed trait. See the structure below: ```python class PandasMapOpsMixin(object): def mapInPandas(self, ...): ... return ... # other Pandas <> PySpark APIs ``` ```python class DataFrame(PandasMapOpsMixin): # other DataFrame APIs equivalent to Scala side. ``` Yes, This is a big PR but they are mostly just moving around except one case `createDataFrame` which I had to split the methods. ### Why are the changes needed? There are pandas functionalities here and there and I myself gets lost where it was. Also, when you have to make a change commonly for all of pandas related features, it's almost impossible now. Also, after this change, `DataFrame` and `SparkSession` become more consistent with Scala side since pandas is specific to Python, and this change separates pandas-specific APIs away from `DataFrame` or `SparkSession`. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests should cover. Also, I manually built the PySpark API documentation and checked. Closes #27109 from HyukjinKwon/pandas-refactoring. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-08 20:22:50 -05:00
+---+---+
| id|age|
+---+---+
| 1| 21|
+---+---+
Notes
-----
This API is experimental
[SPARK-30434][PYTHON][SQL] Move pandas related functionalities into 'pandas' sub-package ### What changes were proposed in this pull request? This PR proposes to move pandas related functionalities into pandas package. Namely: ```bash pyspark/sql/pandas ├── __init__.py ├── conversion.py # Conversion between pandas <> PySpark DataFrames ├── functions.py # pandas_udf ├── group_ops.py # Grouped UDF / Cogrouped UDF + groupby.apply, groupby.cogroup.apply ├── map_ops.py # Map Iter UDF + mapInPandas ├── serializers.py # pandas <> PyArrow serializers ├── types.py # Type utils between pandas <> PyArrow └── utils.py # Version requirement checks ``` In order to separately locate `groupby.apply`, `groupby.cogroup.apply`, `mapInPandas`, `toPandas`, and `createDataFrame(pdf)` under `pandas` sub-package, I had to use a mix-in approach which Scala side uses often by `trait`, and also pandas itself uses this approach (see `IndexOpsMixin` as an example) to group related functionalities. Currently, you can think it's like Scala's self typed trait. See the structure below: ```python class PandasMapOpsMixin(object): def mapInPandas(self, ...): ... return ... # other Pandas <> PySpark APIs ``` ```python class DataFrame(PandasMapOpsMixin): # other DataFrame APIs equivalent to Scala side. ``` Yes, This is a big PR but they are mostly just moving around except one case `createDataFrame` which I had to split the methods. ### Why are the changes needed? There are pandas functionalities here and there and I myself gets lost where it was. Also, when you have to make a change commonly for all of pandas related features, it's almost impossible now. Also, after this change, `DataFrame` and `SparkSession` become more consistent with Scala side since pandas is specific to Python, and this change separates pandas-specific APIs away from `DataFrame` or `SparkSession`. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests should cover. Also, I manually built the PySpark API documentation and checked. Closes #27109 from HyukjinKwon/pandas-refactoring. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-08 20:22:50 -05:00
See Also
--------
pyspark.sql.functions.pandas_udf
[SPARK-30434][PYTHON][SQL] Move pandas related functionalities into 'pandas' sub-package ### What changes were proposed in this pull request? This PR proposes to move pandas related functionalities into pandas package. Namely: ```bash pyspark/sql/pandas ├── __init__.py ├── conversion.py # Conversion between pandas <> PySpark DataFrames ├── functions.py # pandas_udf ├── group_ops.py # Grouped UDF / Cogrouped UDF + groupby.apply, groupby.cogroup.apply ├── map_ops.py # Map Iter UDF + mapInPandas ├── serializers.py # pandas <> PyArrow serializers ├── types.py # Type utils between pandas <> PyArrow └── utils.py # Version requirement checks ``` In order to separately locate `groupby.apply`, `groupby.cogroup.apply`, `mapInPandas`, `toPandas`, and `createDataFrame(pdf)` under `pandas` sub-package, I had to use a mix-in approach which Scala side uses often by `trait`, and also pandas itself uses this approach (see `IndexOpsMixin` as an example) to group related functionalities. Currently, you can think it's like Scala's self typed trait. See the structure below: ```python class PandasMapOpsMixin(object): def mapInPandas(self, ...): ... return ... # other Pandas <> PySpark APIs ``` ```python class DataFrame(PandasMapOpsMixin): # other DataFrame APIs equivalent to Scala side. ``` Yes, This is a big PR but they are mostly just moving around except one case `createDataFrame` which I had to split the methods. ### Why are the changes needed? There are pandas functionalities here and there and I myself gets lost where it was. Also, when you have to make a change commonly for all of pandas related features, it's almost impossible now. Also, after this change, `DataFrame` and `SparkSession` become more consistent with Scala side since pandas is specific to Python, and this change separates pandas-specific APIs away from `DataFrame` or `SparkSession`. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests should cover. Also, I manually built the PySpark API documentation and checked. Closes #27109 from HyukjinKwon/pandas-refactoring. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-08 20:22:50 -05:00
"""
[SPARK-28264][PYTHON][SQL] Support type hints in pandas UDF and rename/move inconsistent pandas UDF types ### What changes were proposed in this pull request? This PR proposes to redesign pandas UDFs as described in [the proposal](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit?usp=sharing). ```python from pyspark.sql.functions import pandas_udf import pandas as pd pandas_udf("long") def plug_one(s: pd.Series) -> pd.Series: return s + 1 spark.range(10).select(plug_one("id")).show() ``` ``` +------------+ |plug_one(id)| +------------+ | 1| | 2| | 3| | 4| | 5| | 6| | 7| | 8| | 9| | 10| +------------+ ``` Note that, this PR address one of the future improvements described [here](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit#heading=h.h3ncjpk6ujqu), "A couple of less-intuitive pandas UDF types" (by zero323) together. In short, - Adds new way with type hints as an alternative and experimental way. ```python pandas_udf(schema='...') def func(c1: Series, c2: Series) -> DataFrame: pass ``` - Replace and/or add an alias for three types below from UDF, and make them as separate standalone APIs. So, `pandas_udf` is now consistent with regular `udf`s and other expressions. `df.mapInPandas(udf)` -replace-> `df.mapInPandas(f, schema)` `df.groupby.apply(udf)` -alias-> `df.groupby.applyInPandas(f, schema)` `df.groupby.cogroup.apply(udf)` -replace-> `df.groupby.cogroup.applyInPandas(f, schema)` *`df.groupby.apply` was added from 2.3 while the other were added in the master only. - No deprecation for the existing ways for now. ```python pandas_udf(schema='...', functionType=PandasUDFType.SCALAR) def func(c1, c2): pass ``` If users are happy with this, I plan to deprecate the existing way and declare using type hints is not experimental anymore. One design goal in this PR was that, avoid touching the internal (since we didn't deprecate the old ways for now), but supports type hints with a minimised changes only at the interface. - Once we deprecate or remove the old ways, I think it requires another refactoring for the internal in the future. At the very least, we should rename internal pandas evaluation types. - If users find this experimental type hints isn't quite helpful, we should simply revert the changes at the interface level. ### Why are the changes needed? In order to address old design issues. Please see [the proposal](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit?usp=sharing). ### Does this PR introduce any user-facing change? For behaviour changes, No. It adds new ways to use pandas UDFs by using type hints. See below. **SCALAR**: ```python pandas_udf(schema='...') def func(c1: Series, c2: DataFrame) -> Series: pass # DataFrame represents a struct column ``` **SCALAR_ITER**: ```python pandas_udf(schema='...') def func(iter: Iterator[Tuple[Series, DataFrame, ...]]) -> Iterator[Series]: pass # Same as SCALAR but wrapped by Iterator ``` **GROUPED_AGG**: ```python pandas_udf(schema='...') def func(c1: Series, c2: DataFrame) -> int: pass # DataFrame represents a struct column ``` **GROUPED_MAP**: This was added in Spark 2.3 as of SPARK-20396. As described above, it keeps the existing behaviour. Additionally, we now have a new alias `groupby.applyInPandas` for `groupby.apply`. See the example below: ```python def func(pdf): return pdf df.groupby("...").applyInPandas(func, schema=df.schema) ``` **MAP_ITER**: this is not a pandas UDF anymore This was added in Spark 3.0 as of SPARK-28198; and this PR replaces the usages. See the example below: ```python def func(iter): for df in iter: yield df df.mapInPandas(func, df.schema) ``` **COGROUPED_MAP**: this is not a pandas UDF anymore This was added in Spark 3.0 as of SPARK-27463; and this PR replaces the usages. See the example below: ```python def asof_join(left, right): return pd.merge_asof(left, right, on="...", by="...") df1.groupby("...").cogroup(df2.groupby("...")).applyInPandas(asof_join, schema="...") ``` ### How was this patch tested? Unittests added and tested against Python 2.7, 3.6 and 3.7. Closes #27165 from HyukjinKwon/revisit-pandas. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-22 01:32:58 -05:00
from pyspark.sql import DataFrame
from pyspark.sql.pandas.functions import pandas_udf
[SPARK-30434][PYTHON][SQL] Move pandas related functionalities into 'pandas' sub-package ### What changes were proposed in this pull request? This PR proposes to move pandas related functionalities into pandas package. Namely: ```bash pyspark/sql/pandas ├── __init__.py ├── conversion.py # Conversion between pandas <> PySpark DataFrames ├── functions.py # pandas_udf ├── group_ops.py # Grouped UDF / Cogrouped UDF + groupby.apply, groupby.cogroup.apply ├── map_ops.py # Map Iter UDF + mapInPandas ├── serializers.py # pandas <> PyArrow serializers ├── types.py # Type utils between pandas <> PyArrow └── utils.py # Version requirement checks ``` In order to separately locate `groupby.apply`, `groupby.cogroup.apply`, `mapInPandas`, `toPandas`, and `createDataFrame(pdf)` under `pandas` sub-package, I had to use a mix-in approach which Scala side uses often by `trait`, and also pandas itself uses this approach (see `IndexOpsMixin` as an example) to group related functionalities. Currently, you can think it's like Scala's self typed trait. See the structure below: ```python class PandasMapOpsMixin(object): def mapInPandas(self, ...): ... return ... # other Pandas <> PySpark APIs ``` ```python class DataFrame(PandasMapOpsMixin): # other DataFrame APIs equivalent to Scala side. ``` Yes, This is a big PR but they are mostly just moving around except one case `createDataFrame` which I had to split the methods. ### Why are the changes needed? There are pandas functionalities here and there and I myself gets lost where it was. Also, when you have to make a change commonly for all of pandas related features, it's almost impossible now. Also, after this change, `DataFrame` and `SparkSession` become more consistent with Scala side since pandas is specific to Python, and this change separates pandas-specific APIs away from `DataFrame` or `SparkSession`. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests should cover. Also, I manually built the PySpark API documentation and checked. Closes #27109 from HyukjinKwon/pandas-refactoring. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-08 20:22:50 -05:00
assert isinstance(self, DataFrame)
[SPARK-28264][PYTHON][SQL] Support type hints in pandas UDF and rename/move inconsistent pandas UDF types ### What changes were proposed in this pull request? This PR proposes to redesign pandas UDFs as described in [the proposal](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit?usp=sharing). ```python from pyspark.sql.functions import pandas_udf import pandas as pd pandas_udf("long") def plug_one(s: pd.Series) -> pd.Series: return s + 1 spark.range(10).select(plug_one("id")).show() ``` ``` +------------+ |plug_one(id)| +------------+ | 1| | 2| | 3| | 4| | 5| | 6| | 7| | 8| | 9| | 10| +------------+ ``` Note that, this PR address one of the future improvements described [here](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit#heading=h.h3ncjpk6ujqu), "A couple of less-intuitive pandas UDF types" (by zero323) together. In short, - Adds new way with type hints as an alternative and experimental way. ```python pandas_udf(schema='...') def func(c1: Series, c2: Series) -> DataFrame: pass ``` - Replace and/or add an alias for three types below from UDF, and make them as separate standalone APIs. So, `pandas_udf` is now consistent with regular `udf`s and other expressions. `df.mapInPandas(udf)` -replace-> `df.mapInPandas(f, schema)` `df.groupby.apply(udf)` -alias-> `df.groupby.applyInPandas(f, schema)` `df.groupby.cogroup.apply(udf)` -replace-> `df.groupby.cogroup.applyInPandas(f, schema)` *`df.groupby.apply` was added from 2.3 while the other were added in the master only. - No deprecation for the existing ways for now. ```python pandas_udf(schema='...', functionType=PandasUDFType.SCALAR) def func(c1, c2): pass ``` If users are happy with this, I plan to deprecate the existing way and declare using type hints is not experimental anymore. One design goal in this PR was that, avoid touching the internal (since we didn't deprecate the old ways for now), but supports type hints with a minimised changes only at the interface. - Once we deprecate or remove the old ways, I think it requires another refactoring for the internal in the future. At the very least, we should rename internal pandas evaluation types. - If users find this experimental type hints isn't quite helpful, we should simply revert the changes at the interface level. ### Why are the changes needed? In order to address old design issues. Please see [the proposal](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit?usp=sharing). ### Does this PR introduce any user-facing change? For behaviour changes, No. It adds new ways to use pandas UDFs by using type hints. See below. **SCALAR**: ```python pandas_udf(schema='...') def func(c1: Series, c2: DataFrame) -> Series: pass # DataFrame represents a struct column ``` **SCALAR_ITER**: ```python pandas_udf(schema='...') def func(iter: Iterator[Tuple[Series, DataFrame, ...]]) -> Iterator[Series]: pass # Same as SCALAR but wrapped by Iterator ``` **GROUPED_AGG**: ```python pandas_udf(schema='...') def func(c1: Series, c2: DataFrame) -> int: pass # DataFrame represents a struct column ``` **GROUPED_MAP**: This was added in Spark 2.3 as of SPARK-20396. As described above, it keeps the existing behaviour. Additionally, we now have a new alias `groupby.applyInPandas` for `groupby.apply`. See the example below: ```python def func(pdf): return pdf df.groupby("...").applyInPandas(func, schema=df.schema) ``` **MAP_ITER**: this is not a pandas UDF anymore This was added in Spark 3.0 as of SPARK-28198; and this PR replaces the usages. See the example below: ```python def func(iter): for df in iter: yield df df.mapInPandas(func, df.schema) ``` **COGROUPED_MAP**: this is not a pandas UDF anymore This was added in Spark 3.0 as of SPARK-27463; and this PR replaces the usages. See the example below: ```python def asof_join(left, right): return pd.merge_asof(left, right, on="...", by="...") df1.groupby("...").cogroup(df2.groupby("...")).applyInPandas(asof_join, schema="...") ``` ### How was this patch tested? Unittests added and tested against Python 2.7, 3.6 and 3.7. Closes #27165 from HyukjinKwon/revisit-pandas. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-22 01:32:58 -05:00
udf = pandas_udf(
func, returnType=schema, functionType=PythonEvalType.SQL_MAP_PANDAS_ITER_UDF)
[SPARK-30434][PYTHON][SQL] Move pandas related functionalities into 'pandas' sub-package ### What changes were proposed in this pull request? This PR proposes to move pandas related functionalities into pandas package. Namely: ```bash pyspark/sql/pandas ├── __init__.py ├── conversion.py # Conversion between pandas <> PySpark DataFrames ├── functions.py # pandas_udf ├── group_ops.py # Grouped UDF / Cogrouped UDF + groupby.apply, groupby.cogroup.apply ├── map_ops.py # Map Iter UDF + mapInPandas ├── serializers.py # pandas <> PyArrow serializers ├── types.py # Type utils between pandas <> PyArrow └── utils.py # Version requirement checks ``` In order to separately locate `groupby.apply`, `groupby.cogroup.apply`, `mapInPandas`, `toPandas`, and `createDataFrame(pdf)` under `pandas` sub-package, I had to use a mix-in approach which Scala side uses often by `trait`, and also pandas itself uses this approach (see `IndexOpsMixin` as an example) to group related functionalities. Currently, you can think it's like Scala's self typed trait. See the structure below: ```python class PandasMapOpsMixin(object): def mapInPandas(self, ...): ... return ... # other Pandas <> PySpark APIs ``` ```python class DataFrame(PandasMapOpsMixin): # other DataFrame APIs equivalent to Scala side. ``` Yes, This is a big PR but they are mostly just moving around except one case `createDataFrame` which I had to split the methods. ### Why are the changes needed? There are pandas functionalities here and there and I myself gets lost where it was. Also, when you have to make a change commonly for all of pandas related features, it's almost impossible now. Also, after this change, `DataFrame` and `SparkSession` become more consistent with Scala side since pandas is specific to Python, and this change separates pandas-specific APIs away from `DataFrame` or `SparkSession`. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests should cover. Also, I manually built the PySpark API documentation and checked. Closes #27109 from HyukjinKwon/pandas-refactoring. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-08 20:22:50 -05:00
udf_column = udf(*[self[col] for col in self.columns])
jdf = self._jdf.mapInPandas(udf_column._jc.expr())
return DataFrame(jdf, self.sql_ctx)
def _test():
import doctest
from pyspark.sql import SparkSession
import pyspark.sql.pandas.map_ops
globs = pyspark.sql.pandas.map_ops.__dict__.copy()
spark = SparkSession.builder\
.master("local[4]")\
.appName("sql.pandas.map_ops tests")\
.getOrCreate()
globs['spark'] = spark
(failure_count, test_count) = doctest.testmod(
pyspark.sql.pandas.map_ops, globs=globs,
optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE | doctest.REPORT_NDIFF)
spark.stop()
if failure_count:
sys.exit(-1)
if __name__ == "__main__":
_test()