Skip to content

Commit

Permalink
[SPARK-45554][PYTHON] Introduce flexible parameter to `assertSchemaEq…
Browse files Browse the repository at this point in the history
…ual`

### What changes were proposed in this pull request?

This PR proposes to add three new parameters to the `assertSchemaEqual`: `ignoreNullable`, `ignoreColumnOrder` and `ignoreColumnName` to provide users with more flexibility in schema testing.

### Why are the changes needed?

To enhance the utility of `assertSchemaEqual` by accommodating various common schema comparison scenarios that users might encounter, without necessitating manual adjustments or workarounds.

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

Yes. `assertDataFrameEqual` now have the option to use the five new parameters:
<!DOCTYPE html>

Parameter | Type | Comment
-- | -- | --
ignoreNullable | Boolean [optional] | Specifies whether a column’s nullable property is included when checking for schema equality.</br></br> When set to True (default), the nullable property of the columns being compared is not taken into account and the columns will be considered equal even if they have different nullable settings.</br></br>When set to False, columns are considered equal only if they have the same nullable setting.
ignoreColumnOrder | Boolean [optional] | Specifies whether to compare columns in the order they appear in the DataFrames or by column name.</br></br> When set to False (default), columns are compared in the order they appear in the DataFrames.</br></br> When set to True, a column in the expected DataFrame is compared to the column with the same name in the actual DataFrame. </br></br>ignoreColumnOrder cannot be set to True if ignoreColumnNames is also set to True.
ignoreColumnName | Boolean [optional] | Specifies whether to fail the initial schema equality check if the column names in the two DataFrames are different.</br></br> When set to False (default), column names are checked and the function fails if they are different.</br></br> When set to True, the function will succeed even if column names are different. Column data types are compared for columns in the order they appear in the DataFrames.</br></br> ignoreColumnNames cannot be set to True if ignoreColumnOrder is also set to True.

### How was this patch tested?

Added usage examples into doctest for each parameter.

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #43450 from itholic/SPARK-45554.

Authored-by: Haejoon Lee <[email protected]>
Signed-off-by: Hyukjin Kwon <[email protected]>
  • Loading branch information
itholic authored and HyukjinKwon committed Oct 30, 2023
1 parent 4af4dde commit 0245b84
Showing 1 changed file with 85 additions and 5 deletions.
90 changes: 85 additions & 5 deletions python/pyspark/testing/utils.py
Original file line number Diff line number Diff line change
Expand Up @@ -293,7 +293,13 @@ def check_error(
)


def assertSchemaEqual(actual: StructType, expected: StructType):
def assertSchemaEqual(
actual: StructType,
expected: StructType,
ignoreNullable: bool = True,
ignoreColumnOrder: bool = False,
ignoreColumnName: bool = False,
):
r"""
A util function to assert equality between DataFrame schemas `actual` and `expected`.
Expand All @@ -305,6 +311,31 @@ def assertSchemaEqual(actual: StructType, expected: StructType):
The DataFrame schema that is being compared or tested.
expected : StructType
The expected schema, for comparison with the actual schema.
ignoreNullable : bool, default True
Specifies whether a column’s nullable property is included when checking for
schema equality.
When set to `True` (default), the nullable property of the columns being compared
is not taken into account and the columns will be considered equal even if they have
different nullable settings.
When set to `False`, columns are considered equal only if they have the same nullable
setting.
.. versionadded:: 4.0.0
ignoreColumnOrder : bool, default False
Specifies whether to compare columns in the order they appear in the DataFrame or by
column name.
If set to `False` (default), columns are compared in the order they appear in the
DataFrames.
When set to `True`, a column in the expected DataFrame is compared to the column with the
same name in the actual DataFrame.
.. versionadded:: 4.0.0
ignoreColumnName : bool, default False
Specifies whether to fail the initial schema equality check if the column names in the two
DataFrames are different.
When set to `False` (default), column names are checked and the function fails if they are
different.
When set to `True`, the function will succeed even if column names are different.
Column data types are compared for columns in the order they appear in the DataFrames.
.. versionadded:: 4.0.0
Notes
-----
Expand All @@ -318,6 +349,21 @@ def assertSchemaEqual(actual: StructType, expected: StructType):
>>> s2 = StructType([StructField("names", ArrayType(DoubleType(), True), True)])
>>> assertSchemaEqual(s1, s2) # pass, schemas are identical
Different schemas with `ignoreNullable=False` would fail.
>>> s3 = StructType([StructField("names", ArrayType(DoubleType(), True), False)])
>>> assertSchemaEqual(s1, s3, ignoreNullable=False) # doctest: +IGNORE_EXCEPTION_DETAIL
Traceback (most recent call last):
...
PySparkAssertionError: [DIFFERENT_SCHEMA] Schemas do not match.
--- actual
+++ expected
- StructType([StructField('names', ArrayType(DoubleType(), True), True)])
? ^^^
+ StructType([StructField('names', ArrayType(DoubleType(), True), False)])
? ^^^^
>>> df1 = spark.createDataFrame(data=[(1, 1000), (2, 3000)], schema=["id", "number"])
>>> df2 = spark.createDataFrame(data=[("1", 1000), ("2", 5000)], schema=["id", "amount"])
>>> assertSchemaEqual(df1.schema, df2.schema) # doctest: +IGNORE_EXCEPTION_DETAIL
Expand All @@ -330,6 +376,26 @@ def assertSchemaEqual(actual: StructType, expected: StructType):
? ^^ ^^^^^
+ StructType([StructField('id', StringType(), True), StructField('amount', LongType(), True)])
? ^^^^ ++++ ^
Compare two schemas ignoring the column order.
>>> s1 = StructType(
... [StructField("a", IntegerType(), True), StructField("b", DoubleType(), True)]
... )
>>> s2 = StructType(
... [StructField("b", DoubleType(), True), StructField("a", IntegerType(), True)]
... )
>>> assertSchemaEqual(s1, s2, ignoreColumnOrder=True)
Compare two schemas ignoring the column names.
>>> s1 = StructType(
... [StructField("a", IntegerType(), True), StructField("c", DoubleType(), True)]
... )
>>> s2 = StructType(
... [StructField("b", IntegerType(), True), StructField("d", DoubleType(), True)]
... )
>>> assertSchemaEqual(s1, s2, ignoreColumnName=True)
"""
if not isinstance(actual, StructType):
raise PySparkAssertionError(
Expand Down Expand Up @@ -373,12 +439,26 @@ def compare_datatypes_ignore_nullable(dt1: Any, dt2: Any):
else:
return False

# ignore nullable flag by default
if not compare_schemas_ignore_nullable(actual, expected):
generated_diff = difflib.ndiff(str(actual).splitlines(), str(expected).splitlines())
if ignoreColumnOrder:
actual = StructType(sorted(actual, key=lambda x: x.name))
expected = StructType(sorted(expected, key=lambda x: x.name))

error_msg = "\n".join(generated_diff)
if ignoreColumnName:
actual = StructType(
[StructField(str(i), field.dataType, field.nullable) for i, field in enumerate(actual)]
)
expected = StructType(
[
StructField(str(i), field.dataType, field.nullable)
for i, field in enumerate(expected)
]
)

if (ignoreNullable and not compare_schemas_ignore_nullable(actual, expected)) or (
not ignoreNullable and actual != expected
):
generated_diff = difflib.ndiff(str(actual).splitlines(), str(expected).splitlines())
error_msg = "\n".join(generated_diff)
raise PySparkAssertionError(
error_class="DIFFERENT_SCHEMA",
message_parameters={"error_msg": error_msg},
Expand Down

0 comments on commit 0245b84

Please sign in to comment.