### What changes were proposed in this pull request?
Followup to make assertions from recent test consistent with the rest of the module
### Why are the changes needed?
Better to use assertions from `unittest` and be consistent
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Existing tests
Closes#28659 from BryanCutler/arrow-category-test-fix-SPARK-25351.
Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
Handle Pandas category type while converting from python with Arrow enabled. The category column will be converted to whatever type the category elements are as is the case with Arrow disabled.
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
New unit tests were added for `createDataFrame` and scalar `pandas_udf`
Closes#26585 from jalpan-randeri/feature-pyarrow-dictionary-type.
Authored-by: Jalpan Randeri <randerij@amazon.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
### What changes were proposed in this pull request?
This PR proposes to improve the error message from Scalar iterator pandas UDF.
### Why are the changes needed?
To show the correct error messages.
### Does this PR introduce any user-facing change?
Yes, but only in unreleased branches.
```python
import pandas as pd
from pyspark.sql.functions import pandas_udf, PandasUDFType
pandas_udf('long', PandasUDFType.SCALAR_ITER)
def pandas_plus_one(iterator):
for _ in iterator:
yield pd.Series(1)
spark.range(10).repartition(1).select(pandas_plus_one("id")).show()
```
```python
import pandas as pd
from pyspark.sql.functions import pandas_udf, PandasUDFType
pandas_udf('long', PandasUDFType.SCALAR_ITER)
def pandas_plus_one(iterator):
for _ in iterator:
yield pd.Series(list(range(20)))
spark.range(10).repartition(1).select(pandas_plus_one("id")).show()
```
**Before:**
```
RuntimeError: The number of output rows of pandas iterator UDF should
be the same with input rows. The input rows number is 10 but the output
rows number is 1.
```
```
AssertionError: Pandas MAP_ITER UDF outputted more rows than input rows.
```
**After:**
```
RuntimeError: The length of output in Scalar iterator pandas UDF should be
the same with the input's; however, the length of output was 1 and the length
of input was 10.
```
```
AssertionError: Pandas SCALAR_ITER UDF outputted more rows than input rows.
```
### How was this patch tested?
Unittests were fixed accordingly.
Closes#28135 from HyukjinKwon/SPARK-26412-followup.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This PR renames `spark.sql.pandas.udf.buffer.size` to `spark.sql.execution.pandas.udf.buffer.size` to be more consistent with other pandas configuration prefixes, given:
- `spark.sql.execution.pandas.arrowSafeTypeConversion`
- `spark.sql.execution.pandas.respectSessionTimeZone`
- `spark.sql.legacy.execution.pandas.groupedMap.assignColumnsByName`
- other configurations like `spark.sql.execution.arrow.*`.
### Why are the changes needed?
To make configuration names consistent.
### Does this PR introduce any user-facing change?
No because this configuration was not released yet.
### How was this patch tested?
Existing tests should cover.
Closes#27450 from HyukjinKwon/SPARK-27870-followup.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### 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>
### What changes were proposed in this pull request?
In the PR, I propose to remove the SQL config `spark.sql.execution.pandas.respectSessionTimeZone` which has been deprecated since Spark 2.3.
### Why are the changes needed?
To improve code maintainability.
### Does this PR introduce any user-facing change?
Yes.
### How was this patch tested?
by running python tests, https://spark.apache.org/docs/latest/building-spark.html#pyspark-tests-with-maven-or-sbtCloses#27218 from MaxGekk/remove-respectSessionTimeZone.
Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
reorganize the packages of DS v2 interfaces/classes:
1. `org.spark.sql.connector.catalog`: put `TableCatalog`, `Table` and other related interfaces/classes
2. `org.spark.sql.connector.expression`: put `Expression`, `Transform` and other related interfaces/classes
3. `org.spark.sql.connector.read`: put `ScanBuilder`, `Scan` and other related interfaces/classes
4. `org.spark.sql.connector.write`: put `WriteBuilder`, `BatchWrite` and other related interfaces/classes
### Why are the changes needed?
Data Source V2 has evolved a lot. It's a bit weird that `Expression` is in `org.spark.sql.catalog.v2` and `Table` is in `org.spark.sql.sources.v2`.
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
existing tests
Closes#25700 from cloud-fan/package.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
Closes the generator when Python UDFs stop early.
### Manually verification on pandas iterator UDF and mapPartitions
```python
from pyspark.sql import SparkSession
from pyspark.sql.functions import pandas_udf, PandasUDFType
from pyspark.sql.functions import col, udf
from pyspark.taskcontext import TaskContext
import time
import os
spark.conf.set('spark.sql.execution.arrow.maxRecordsPerBatch', '1')
spark.conf.set('spark.sql.pandas.udf.buffer.size', '4')
pandas_udf("int", PandasUDFType.SCALAR_ITER)
def fi1(it):
try:
for batch in it:
yield batch + 100
time.sleep(1.0)
except BaseException as be:
print("Debug: exception raised: " + str(type(be)))
raise be
finally:
open("/tmp/000001.tmp", "a").close()
df1 = spark.range(10).select(col('id').alias('a')).repartition(1)
# will see log Debug: exception raised: <class 'GeneratorExit'>
# and file "/tmp/000001.tmp" generated.
df1.select(col('a'), fi1('a')).limit(2).collect()
def mapper(it):
try:
for batch in it:
yield batch
except BaseException as be:
print("Debug: exception raised: " + str(type(be)))
raise be
finally:
open("/tmp/000002.tmp", "a").close()
df2 = spark.range(10000000).repartition(1)
# will see log Debug: exception raised: <class 'GeneratorExit'>
# and file "/tmp/000002.tmp" generated.
df2.rdd.mapPartitions(mapper).take(2)
```
## How was this patch tested?
Unit test added.
Please review https://spark.apache.org/contributing.html before opening a pull request.
Closes#24986 from WeichenXu123/pandas_iter_udf_limit.
Authored-by: WeichenXu <weichen.xu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
Currently, pretty skipped message added by f7435bec6a mechanism seems not working when xmlrunner is installed apparently.
This PR fixes two things:
1. When `xmlrunner` is installed, seems `xmlrunner` does not respect `vervosity` level in unittests (default is level 1).
So the output looks as below
```
Running tests...
----------------------------------------------------------------------
SSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSS
----------------------------------------------------------------------
```
So it is not caught by our message detection mechanism.
2. If we manually set the `vervocity` level to `xmlrunner`, it prints messages as below:
```
test_mixed_udf (pyspark.sql.tests.test_pandas_udf_scalar.ScalarPandasUDFTests) ... SKIP (0.000s)
test_mixed_udf_and_sql (pyspark.sql.tests.test_pandas_udf_scalar.ScalarPandasUDFTests) ... SKIP (0.000s)
...
```
This is different in our Jenkins machine:
```
test_createDataFrame_column_name_encoding (pyspark.sql.tests.test_arrow.ArrowTests) ... skipped 'Pandas >= 0.23.2 must be installed; however, it was not found.'
test_createDataFrame_does_not_modify_input (pyspark.sql.tests.test_arrow.ArrowTests) ... skipped 'Pandas >= 0.23.2 must be installed; however, it was not found.'
...
```
Note that last `SKIP` is different. This PR fixes the regular expression to catch `SKIP` case as well.
## How was this patch tested?
Manually tested.
**Before:**
```
Starting test(python2.7): pyspark....
Finished test(python2.7): pyspark.... (0s)
...
Tests passed in 562 seconds
========================================================================
...
```
**After:**
```
Starting test(python2.7): pyspark....
Finished test(python2.7): pyspark.... (48s) ... 93 tests were skipped
...
Tests passed in 560 seconds
Skipped tests pyspark.... with python2.7:
pyspark...(...) ... SKIP (0.000s)
...
========================================================================
...
```
Closes#24927 from HyukjinKwon/SPARK-28130.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
Allow Pandas UDF to take an iterator of pd.Series or an iterator of tuple of pd.Series.
Note the UDF input args will be always one iterator:
* if the udf take only column as input, the iterator's element will be pd.Series (corresponding to the column values batch)
* if the udf take multiple columns as inputs, the iterator's element will be a tuple composed of multiple `pd.Series`s, each one corresponding to the multiple columns as inputs (keep the same order). For example:
```
pandas_udf("int", PandasUDFType.SCALAR_ITER)
def the_udf(iterator):
for col1_batch, col2_batch in iterator:
yield col1_batch + col2_batch
df.select(the_udf("col1", "col2"))
```
The udf above will add col1 and col2.
I haven't add unit tests, but manually tests show it works fine. So it is ready for first pass review.
We can test several typical cases:
```
from pyspark.sql import SparkSession
from pyspark.sql.functions import pandas_udf, PandasUDFType
from pyspark.sql.functions import udf
from pyspark.taskcontext import TaskContext
df = spark.createDataFrame([(1, 20), (3, 40)], ["a", "b"])
pandas_udf("int", PandasUDFType.SCALAR_ITER)
def fi1(it):
pid = TaskContext.get().partitionId()
print("DBG: fi1: do init stuff, partitionId=" + str(pid))
for batch in it:
yield batch + 100
print("DBG: fi1: do close stuff, partitionId=" + str(pid))
pandas_udf("int", PandasUDFType.SCALAR_ITER)
def fi2(it):
pid = TaskContext.get().partitionId()
print("DBG: fi2: do init stuff, partitionId=" + str(pid))
for batch in it:
yield batch + 10000
print("DBG: fi2: do close stuff, partitionId=" + str(pid))
pandas_udf("int", PandasUDFType.SCALAR_ITER)
def fi3(it):
pid = TaskContext.get().partitionId()
print("DBG: fi3: do init stuff, partitionId=" + str(pid))
for x, y in it:
yield x + y * 10 + 100000
print("DBG: fi3: do close stuff, partitionId=" + str(pid))
pandas_udf("int", PandasUDFType.SCALAR)
def fp1(x):
return x + 1000
udf("int")
def fu1(x):
return x + 10
# test select "pandas iter udf/pandas udf/sql udf" expressions at the same time.
# Note this case the `fi1("a"), fi2("b"), fi3("a", "b")` will generate only one plan,
# and `fu1("a")`, `fp1("a")` will generate another two separate plans.
df.select(fi1("a"), fi2("b"), fi3("a", "b"), fu1("a"), fp1("a")).show()
# test chain two pandas iter udf together
# Note this case `fi2(fi1("a"))` will generate only one plan
# Also note the init stuff/close stuff call order will be like:
# (debug output following)
# DBG: fi2: do init stuff, partitionId=0
# DBG: fi1: do init stuff, partitionId=0
# DBG: fi1: do close stuff, partitionId=0
# DBG: fi2: do close stuff, partitionId=0
df.select(fi2(fi1("a"))).show()
# test more complex chain
# Note this case `fi1("a"), fi2("a")` will generate one plan,
# and `fi3(fi1_output, fi2_output)` will generate another plan
df.select(fi3(fi1("a"), fi2("a"))).show()
```
## How was this patch tested?
To be added.
Please review http://spark.apache.org/contributing.html before opening a pull request.
Closes#24643 from WeichenXu123/pandas_udf_iter.
Lead-authored-by: WeichenXu <weichen.xu@databricks.com>
Co-authored-by: Xiangrui Meng <meng@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
## What changes were proposed in this pull request?
This increases the minimum support version of pyarrow to 0.12.1 and removes workarounds in pyspark to remain compatible with prior versions. This means that users will need to have at least pyarrow 0.12.1 installed and available in the cluster or an `ImportError` will be raised to indicate an upgrade is needed.
## How was this patch tested?
Existing tests using:
Python 2.7.15, pyarrow 0.12.1, pandas 0.24.2
Python 3.6.7, pyarrow 0.12.1, pandas 0.24.0
Closes#24298 from BryanCutler/arrow-bump-min-pyarrow-SPARK-27276.
Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
Running PySpark tests with Pandas 0.24.x causes a failure in `test_pandas_udf_grouped_map` test_supported_types:
`ValueError: The truth value of an array with more than one element is ambiguous. Use a.any() or a.all()`
This is because a column is an ArrayType and the method `sqlutils ReusedSQLTestCase.assertPandasEqual ` does not properly check this.
This PR removes `assertPandasEqual` and replaces it with the built-in `pandas.util.testing.assert_frame_equal` which can properly handle columns of ArrayType and also prints out better diff between the DataFrames when an error occurs.
Additionally, imports of pandas and pyarrow were moved to the top of related test files to avoid duplicating the same import many times.
## How was this patch tested?
Existing tests
Closes#24306 from BryanCutler/python-pandas-assert_frame_equal-SPARK-27387.
Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
Now that we support returning pandas DataFrame for struct type in Scalar Pandas UDF.
If we chain another Pandas UDF after the Scalar Pandas UDF returning pandas DataFrame, the argument of the chained UDF will be pandas DataFrame, but currently we don't support pandas DataFrame as an argument of Scalar Pandas UDF. That means there is an inconsistency between the chained UDF and the single UDF.
We should support taking pandas DataFrame for struct type argument in Scalar Pandas UDF to be consistent.
Currently pyarrow >=0.11 is supported.
## How was this patch tested?
Modified and added some tests.
Closes#24177 from ueshin/issues/SPARK-27240/structtype_argument.
Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
## What changes were proposed in this pull request?
This change adds support for returning StructType from a scalar Pandas UDF, where the return value of the function is a pandas.DataFrame. Nested structs are not supported and an error will be raised, child types can be any other type currently supported.
## How was this patch tested?
Added additional unit tests to `test_pandas_udf_scalar`
Closes#23900 from BryanCutler/pyspark-support-scalar_udf-StructType-SPARK-23836.
Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
## What changes were proposed in this pull request?
Currently `DataFrame.toPandas()` with arrow enabled or `ArrowStreamPandasSerializer` for pandas UDF with pyarrow<0.12 creates `datetime64[ns]` type series as intermediate data and then convert to `datetime.date` series, but the intermediate `datetime64[ns]` might cause an overflow even if the date is valid.
```
>>> import datetime
>>>
>>> t = [datetime.date(2262, 4, 12), datetime.date(2263, 4, 12)]
>>>
>>> df = spark.createDataFrame(t, 'date')
>>> df.show()
+----------+
| value|
+----------+
|2262-04-12|
|2263-04-12|
+----------+
>>>
>>> spark.conf.set("spark.sql.execution.arrow.enabled", "true")
>>>
>>> df.toPandas()
value
0 1677-09-21
1 1678-09-21
```
We should avoid creating such intermediate data and create `datetime.date` series directly instead.
## How was this patch tested?
Modified some tests to include the date which overflow caused by the intermediate conversion.
Run tests with pyarrow 0.8, 0.10, 0.11, 0.12 in my local environment.
Closes#23795 from ueshin/issues/SPARK-26887/date_as_object.
Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
Clean up unconditional import statements and move them to the top.
Conditional imports (pandas, numpy, pyarrow) are left as-is.
## How was this patch tested?
Exising tests.
Closes#23314 from icexelloss/clean-up-test-imports.
Authored-by: Li Jin <ice.xelloss@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
This PR continues to break down a big large file into smaller files. See https://github.com/apache/spark/pull/23021. It targets to follow https://github.com/numpy/numpy/tree/master/numpy.
Basically this PR proposes to break down `pyspark/tests.py` into ...:
```
pyspark
...
├── testing
...
│ └── utils.py
├── tests
│ ├── __init__.py
│ ├── test_appsubmit.py
│ ├── test_broadcast.py
│ ├── test_conf.py
│ ├── test_context.py
│ ├── test_daemon.py
│ ├── test_join.py
│ ├── test_profiler.py
│ ├── test_rdd.py
│ ├── test_readwrite.py
│ ├── test_serializers.py
│ ├── test_shuffle.py
│ ├── test_taskcontext.py
│ ├── test_util.py
│ └── test_worker.py
...
```
## How was this patch tested?
Existing tests should cover.
`cd python` and .`/run-tests-with-coverage`. Manually checked they are actually being ran.
Each test (not officially) can be ran via:
```bash
SPARK_TESTING=1 ./bin/pyspark pyspark.tests.test_context
```
Note that if you're using Mac and Python 3, you might have to `OBJC_DISABLE_INITIALIZE_FORK_SAFETY=YES`.
Closes#23033 from HyukjinKwon/SPARK-26036.
Authored-by: hyukjinkwon <gurwls223@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
This is the official first attempt to break huge single `tests.py` file - I did it locally before few times and gave up for some reasons. Now, currently it really makes the unittests super hard to read and difficult to check. To me, it even bothers me to to scroll down the big file. It's one single 7000 lines file!
This is not only readability issue. Since one big test takes most of tests time, the tests don't run in parallel fully - although it will costs to start and stop the context.
We could pick up one example and follow. Given my investigation, the current style looks closer to NumPy structure and looks easier to follow. Please see https://github.com/numpy/numpy/tree/master/numpy.
Basically this PR proposes to break down `pyspark/sql/tests.py` into ...:
```bash
pyspark
...
├── sql
...
│ ├── tests # Includes all tests broken down from 'pyspark/sql/tests.py'
│ │ │ # Each matchs to module in 'pyspark/sql'. Additionally, some logical group can
│ │ │ # be added. For instance, 'test_arrow.py', 'test_datasources.py' ...
│ │ ├── __init__.py
│ │ ├── test_appsubmit.py
│ │ ├── test_arrow.py
│ │ ├── test_catalog.py
│ │ ├── test_column.py
│ │ ├── test_conf.py
│ │ ├── test_context.py
│ │ ├── test_dataframe.py
│ │ ├── test_datasources.py
│ │ ├── test_functions.py
│ │ ├── test_group.py
│ │ ├── test_pandas_udf.py
│ │ ├── test_pandas_udf_grouped_agg.py
│ │ ├── test_pandas_udf_grouped_map.py
│ │ ├── test_pandas_udf_scalar.py
│ │ ├── test_pandas_udf_window.py
│ │ ├── test_readwriter.py
│ │ ├── test_serde.py
│ │ ├── test_session.py
│ │ ├── test_streaming.py
│ │ ├── test_types.py
│ │ ├── test_udf.py
│ │ └── test_utils.py
...
├── testing # Includes testing utils that can be used in unittests.
│ ├── __init__.py
│ └── sqlutils.py
...
```
## How was this patch tested?
Existing tests should cover.
`cd python` and `./run-tests-with-coverage`. Manually checked they are actually being ran.
Each test (not officially) can be ran via:
```
SPARK_TESTING=1 ./bin/pyspark pyspark.sql.tests.test_pandas_udf_scalar
```
Note that if you're using Mac and Python 3, you might have to `OBJC_DISABLE_INITIALIZE_FORK_SAFETY=YES`.
Closes#23021 from HyukjinKwon/SPARK-25344.
Authored-by: hyukjinkwon <gurwls223@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>