Description
This JIRA targets to improve Python test coverage in particular about ExtractPythonUDFs.
This rule has caused many regressions or issues such as SPARK-27803, SPARK-26147, SPARK-26864, SPARK-26293, SPARK-25314 and SPARK-24721.
We should convert *.sql test cases that can be affected by this rule ExtractPythonUDFs like https://github.com/apache/spark/blob/f5317f10b25bd193cf5026a8f4fd1cd1ded8f5b4/sql/core/src/test/resources/sql-tests/inputs/udf/udf-inner-join.sql
Namely most of plan related test cases might have to be converted.
Here is the rough contribution guide to follow:
Make sure you have Python with Pandas 0.23.2+ and PyArrow 0.12.1+. Check if you're able to do this:
>>> import pandas >>> pandas.__version__ '0.23.4' >>> import pyarrow >>> pyarrow.__version__ '0.13.0' >>> pyarrow.Table.from_pandas(pandas.DataFrame({'a': [1,2,3]})) pyarrow.Table a: int64 metadata -------- OrderedDict([(b'pandas', b'{"index_columns": [{"kind": "range", "name": null, "start": ' b'0, "stop": 3, "step": 1}], "column_indexes": [{"name": null,' b' "field_name": null, "pandas_type": "unicode", "numpy_type":' b' "object", "metadata": {"encoding": "UTF-8"}}], "columns": [' b'{"name": "a", "field_name": "a", "pandas_type": "int64", "nu' b'mpy_type": "int64", "metadata": null}], "creator": {"library' b'": "pyarrow", "version": "0.13.0"}, "pandas_version": null}')])
1. Copy and paste sql/core/src/test/resources/sql-tests/inputs/xxx.sql file into sql/core/src/test/resources/sql-tests/inputs/udf/udf-xxx.sql
2. Keep the comments and state that this file was copied from sql/core/src/test/resources/sql-tests/inputs/xxx.sql, for now.
For instance, let's add a comment as below on the top:
-- This test file was converted from xxx.sql.
3. Run it below:
SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/test-only *SQLQueryTestSuite -- -z udf/udf-xxx.sql"
git add .
4. Insert one or multiple udf(...) into each statement. It is not required to add more combinations.
And it is not strict about where to insert. Ideally, we should try to put udf differently for each statement.
5. Run it below again:
SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/test-only *SQLQueryTestSuite -- -z udf/udf-xxx.sql"
git diff
# or git diff --no-index sql/core/src/test/resources/sql-tests/results/xxx.sql.out sql/core/src/test/resources/sql-tests/results/udf/xxx.sql.out
6. Compare results with original file, sql/core/src/test/resources/sql-tests/results/xxx.sql.out
7. If there are diff, analyze it, file or find the JIRA, skip the tests with comments. Please see this comment when you file a JIRA.
It's more than perfect if you are even able to fix an issue found but this can be done separately. There is a great example to check and follow at SPARK-28323, done by viirya
8. Run without generating golden files and check:
build/sbt "sql/test-only *SQLQueryTestSuite -- -z udf/udf-xxx.sql"
9. When you open a PR. please attach git diff --no-index sql/core/src/test/resources/sql-tests/results/xxx.sql.out sql/core/src/test/resources/sql-tests/results/udf/xxx.sql.out in the PR description with the template below:
<details><summary>Diff comparing to 'xxx.sql'</summary> <p> ```diff ... # here you put 'git diff' results ``` </p> </details>
10. You're ready. Please go for a PR! If the PR contains other minor fixes, use [SPARK-XXXXX][SQL][PYTHON] prefix in the PR title. If the PR is purely about tests, use [SPARK-XXXXX][SQL][PYTHON][TESTS].
See https://github.com/apache/spark/pull/25069 as an example.
Note that registered UDFs all return strings - so there are some differences are expected.
Note that this JIRA targets plan specific cases in general.
Note that one output.sql.out file is shared for three UDF test cases (Scala UDF, Python UDF, and Pandas UDF). Beware of it when you fix the tests.
Note that this guide is supposed to be updated continuously given how it goes.
Note that this test case uses the integrated UDF test base. See https://github.com/apache/spark/pull/24752 if you're interested in it or find an issue.
Attachments
Issue Links
- is related to
-
SPARK-28359 Make integrated UDF tests robust by making them no-op
- Resolved
-
SPARK-28342 Replace REL_12_BETA1 to REL_12_BETA2 in PostgresSQL SQL tests
- Resolved
- relates to
-
SPARK-27893 Create an integrated test base for Python, Scalar Pandas, Scala UDF by sql files
- Resolved
-
SPARK-27763 Port test cases from PostgreSQL to Spark SQL
- Resolved