spark-instrumented-optimizer/python/pyspark/sql/pandas/map_ops.py
HyukjinKwon aa6a60530e [SPARK-30722][PYTHON][DOCS] Update documentation for Pandas UDF with Python type hints
### What changes were proposed in this pull request?

This PR targets to document the Pandas UDF redesign with type hints introduced at SPARK-28264.
Mostly self-describing; however, there are few things to note for reviewers.

1. This PR replace the existing documentation of pandas UDFs to the newer redesign to promote the Python type hints. I added some words that Spark 3.0 still keeps the compatibility though.

2. This PR proposes to name non-pandas UDFs as "Pandas Function API"

3. SCALAR_ITER become two separate sections to reduce confusion:
  - `Iterator[pd.Series]` -> `Iterator[pd.Series]`
  - `Iterator[Tuple[pd.Series, ...]]` -> `Iterator[pd.Series]`

4. I removed some examples that look overkill to me.

5. I also removed some information in the doc, that seems duplicating or too much.

### Why are the changes needed?

To document new redesign in pandas UDF.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Existing tests should cover.

Closes #27466 from HyukjinKwon/SPARK-30722.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-12 10:49:46 +09:00

96 lines
3.6 KiB
Python

#
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
#
import sys
from pyspark import since
from pyspark.rdd import PythonEvalType
class PandasMapOpsMixin(object):
"""
Min-in for pandas map operations. Currently, only :class:`DataFrame`
can use this class.
"""
@since(3.0)
def mapInPandas(self, func, schema):
"""
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`.
The function should take an iterator of `pandas.DataFrame`\\s and return
another iterator of `pandas.DataFrame`\\s. All columns are passed
together as an iterator of `pandas.DataFrame`\\s to the function and the
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`.
:param func: a Python native function that takes an iterator of `pandas.DataFrame`\\s, and
outputs an iterator of `pandas.DataFrame`\\s.
:param schema: the return type of the `func` in PySpark. The value can be either a
:class:`pyspark.sql.types.DataType` object or a DDL-formatted type string.
>>> from pyspark.sql.functions import pandas_udf
>>> df = spark.createDataFrame([(1, 21), (2, 30)], ("id", "age"))
>>> def filter_func(iterator):
... for pdf in iterator:
... yield pdf[pdf.id == 1]
>>> df.mapInPandas(filter_func, df.schema).show() # doctest: +SKIP
+---+---+
| id|age|
+---+---+
| 1| 21|
+---+---+
.. seealso:: :meth:`pyspark.sql.functions.pandas_udf`
.. note:: Experimental
"""
from pyspark.sql import DataFrame
from pyspark.sql.pandas.functions import pandas_udf
assert isinstance(self, DataFrame)
udf = pandas_udf(
func, returnType=schema, functionType=PythonEvalType.SQL_MAP_PANDAS_ITER_UDF)
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()