You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "asl3 (via GitHub)" <gi...@apache.org> on 2023/07/10 20:47:55 UTC

[GitHub] [spark] asl3 opened a new pull request, #41927: [SPARK-44216] [PYTHON] Make assertSchemaEqual API with ignore_nullable optional flag

asl3 opened a new pull request, #41927:
URL: https://github.com/apache/spark/pull/41927

   ### What changes were proposed in this pull request?
   This PR implements and exposes the PySpark util function `assertSchemaEqual` to test for DataFrame schema equality. It allows for an optional `ignore_nullable` argument.
   
   SPIP: https://docs.google.com/document/d/1OkyBn3JbEHkkQgSQ45Lq82esXjr9rm2Vj7Ih_4zycRc/edit#heading=h.f5f0u2riv07v
   
   
   ### Why are the changes needed?
   The `assertSchemaEqual` function compares schema equality with the option to ignore the nullable flag in schemas, simplifying the testing process for PySpark users.
   
   
   ### Does this PR introduce _any_ user-facing change?
   Yes, the PR exposes a user-facing PySpark util function `assertSchemaEqual`.
   
   
   ### How was this patch tested?
   Added tests to `runtime/python/pyspark/sql/tests/test_utils.py` and `runtime/python/pyspark/sql/tests/connect/test_utils.py`
   
   Sample schema inequality error message:
   
   ![](https://user-images.githubusercontent.com/68875504/252451592-092e0b71-139d-46fb-a32f-599224f5ffc8.png)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #41927: [SPARK-44216] [PYTHON] Make assertSchemaEqual API public

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on code in PR #41927:
URL: https://github.com/apache/spark/pull/41927#discussion_r1263144877


##########
python/pyspark/testing/utils.py:
##########
@@ -252,31 +351,25 @@ def assertDataFrameEqual(df: DataFrame, expected: DataFrame, check_row_order: bo
     >>> df1 = spark.createDataFrame(data=[("1", 1000), ("2", 3000)], schema=["id", "amount"])
     >>> df2 = spark.createDataFrame(data=[("1", 1000), ("2", 3000)], schema=["id", "amount"])
     >>> assertDataFrameEqual(df1, df2) # pass
-    >>> df1 = spark.createDataFrame(data=[("1", 1000.00), ("2", 3000.00), ("3", 2000.00)], \
-        schema=["id", "amount"])
-    >>> df2 = spark.createDataFrame(data=[("1", 1001.00), ("2", 3000.00), ("3", 2003.00)], \
-        schema=["id", "amount"])
+    >>> df1 = spark.createDataFrame(data=[("1", 1000.00), ("2", 3000.00), ("3", 2000.00)],
+    ... schema=["id", "amount"])
+    >>> df2 = spark.createDataFrame(data=[("1", 1001.00), ("2", 3000.00), ("3", 2003.00)],
+    ... schema=["id", "amount"])

Review Comment:
   ```suggestion
       >>> df1 = spark.createDataFrame(
       ...     data=[("1", 1000.00), ("2", 3000.00), ("3", 2000.00)], schema=["id", "amount"])
       >>> df2 = spark.createDataFrame(
       ...     data=[("1", 1001.00), ("2", 3000.00), ("3", 2003.00)], schema=["id", "amount"])
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HyukjinKwon commented on pull request #41927: [SPARK-44216] [PYTHON] Make assertSchemaEqual API public

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on PR #41927:
URL: https://github.com/apache/spark/pull/41927#issuecomment-1637219242

   Merged to master.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonwang-db commented on a diff in pull request #41927: [SPARK-44216] [PYTHON] Make assertSchemaEqual API public

Posted by "allisonwang-db (via GitHub)" <gi...@apache.org>.
allisonwang-db commented on code in PR #41927:
URL: https://github.com/apache/spark/pull/41927#discussion_r1263911806


##########
python/pyspark/testing/utils.py:
##########
@@ -30,6 +30,7 @@
     Tuple,
 )
 from itertools import zip_longest
+import difflib

Review Comment:
   super nit: group it with the other `import ...` statement?



##########
python/pyspark/sql/tests/test_utils.py:
##########
@@ -921,22 +931,186 @@ def test_diff_schema_lens(self):
             schema=["id", "amount", "letter"],
         )
 
+        if is_remote():

Review Comment:
   Instead of checking `is_remote` here, can we split it into two test cases extending from this mixin?
   
   ```
   class BaseUtilsTestsMixin:
       def _get_schema_tree_string(self):
           ...
   
   class UtilsTests(BaseUtilsTestsMixin, ReusedSQLTestCase):
       ...
   
   # Under the connect test directory: python/pyspark/sql/tests/connect/test_parity_utils.py
   class UtilsParityTests(BaseUDFTestsMixin, ReusedConnectTestCase):
       def _get_schema_tree_string(self):
           ...
   ```
    And we can add helper functions to get the schema string and override it in connect test cases.



##########
python/pyspark/testing/utils.py:
##########
@@ -222,6 +223,114 @@ def check_error(
         )
 
 
+def assertSchemaEqual(actual: StructType, expected: StructType):
+    """
+    A util function to assert equality between DataFrame schemas `actual` and `expected`.

Review Comment:
   Maybe we should also update the param name for `assertDataFrameEqual`.



##########
python/pyspark/testing/utils.py:
##########
@@ -222,6 +223,114 @@ def check_error(
         )
 
 
+def assertSchemaEqual(actual: StructType, expected: StructType):
+    """
+    A util function to assert equality between DataFrame schemas `actual` and `expected`.
+
+    .. versionadded:: 3.5.0
+
+    Parameters
+    ----------
+    actual : StructType
+        The DataFrame schema that is being compared or tested.
+    expected : StructType
+        The expected schema, for comparison with the actual schema.
+
+    Examples
+    --------
+    >>> from pyspark.sql.types import StructType, StructField, ArrayType, IntegerType, DoubleType
+    >>> s1 = StructType([StructField("names", ArrayType(DoubleType(), True), True)])
+    >>> s2 = StructType([StructField("names", ArrayType(DoubleType(), True), True)])
+    >>> assertSchemaEqual(s1, s2)
+
+    Pass, schemas are identical
+
+    >>> s1 = StructType([StructField("names", StructType(
+    ...                                         [StructField("age", DoubleType(), True)]), True)])
+    >>> s2 = StructType([StructField("first name", StructType([StructField(
+    ...                                                      "age", IntegerType(), True)]), True)])
+    >>> assertSchemaEqual(s1, s2)  # fail  # doctest: +IGNORE_EXCEPTION_DETAIL
+    Traceback (most recent call last):
+    ...
+    PySparkAssertionError: [DIFFERENT_SCHEMA] Schemas do not match.
+    The diff below overlays `actual` and `expected` schema tree strings.
+    - indicates a line that should be removed from `actual` to match `expected`.
+    + indicates a line that should be added to `actual` to match `expected`.
+      root
+    -  |-- names: struct (nullable = true)
+    ?          -
+
+    +  |-- first name: struct (nullable = true)
+    ?     ++++++
+
+    -  |    |-- age: double (nullable = true)
+    ?                ^^^^^
+
+    +  |    |-- age: integer (nullable = true)
+    ?                ^^^ +++

Review Comment:
   Hmm why do we have both `^^^` and `+++`?



##########
python/pyspark/testing/utils.py:
##########
@@ -222,6 +223,114 @@ def check_error(
         )
 
 
+def assertSchemaEqual(actual: StructType, expected: StructType):
+    """
+    A util function to assert equality between DataFrame schemas `actual` and `expected`.
+
+    .. versionadded:: 3.5.0
+
+    Parameters
+    ----------
+    actual : StructType
+        The DataFrame schema that is being compared or tested.
+    expected : StructType

Review Comment:
   We can also make it support a DDL string in the future :) 



##########
python/pyspark/testing/utils.py:
##########
@@ -222,6 +223,114 @@ def check_error(
         )
 
 
+def assertSchemaEqual(actual: StructType, expected: StructType):
+    """
+    A util function to assert equality between DataFrame schemas `actual` and `expected`.
+
+    .. versionadded:: 3.5.0
+
+    Parameters
+    ----------
+    actual : StructType
+        The DataFrame schema that is being compared or tested.
+    expected : StructType
+        The expected schema, for comparison with the actual schema.
+
+    Examples
+    --------
+    >>> from pyspark.sql.types import StructType, StructField, ArrayType, IntegerType, DoubleType
+    >>> s1 = StructType([StructField("names", ArrayType(DoubleType(), True), True)])
+    >>> s2 = StructType([StructField("names", ArrayType(DoubleType(), True), True)])
+    >>> assertSchemaEqual(s1, s2)
+
+    Pass, schemas are identical
+
+    >>> s1 = StructType([StructField("names", StructType(
+    ...                                         [StructField("age", DoubleType(), True)]), True)])
+    >>> s2 = StructType([StructField("first name", StructType([StructField(
+    ...                                                      "age", IntegerType(), True)]), True)])
+    >>> assertSchemaEqual(s1, s2)  # fail  # doctest: +IGNORE_EXCEPTION_DETAIL
+    Traceback (most recent call last):
+    ...
+    PySparkAssertionError: [DIFFERENT_SCHEMA] Schemas do not match.
+    The diff below overlays `actual` and `expected` schema tree strings.
+    - indicates a line that should be removed from `actual` to match `expected`.
+    + indicates a line that should be added to `actual` to match `expected`.
+      root
+    -  |-- names: struct (nullable = true)
+    ?          -
+
+    +  |-- first name: struct (nullable = true)
+    ?     ++++++
+
+    -  |    |-- age: double (nullable = true)
+    ?                ^^^^^
+
+    +  |    |-- age: integer (nullable = true)
+    ?                ^^^ +++
+    """
+    if not isinstance(actual, StructType):
+        raise PySparkAssertionError(
+            error_class="UNSUPPORTED_DATA_TYPE",

Review Comment:
   Let's not use the error class UNSUPPORTED_DATA_TYPE here. We can just throw a pyspark type error here and tell users how to make it right (expect a structtype, but got... )



##########
python/pyspark/testing/utils.py:
##########
@@ -222,6 +223,114 @@ def check_error(
         )
 
 
+def assertSchemaEqual(actual: StructType, expected: StructType):
+    """
+    A util function to assert equality between DataFrame schemas `actual` and `expected`.
+
+    .. versionadded:: 3.5.0
+
+    Parameters
+    ----------
+    actual : StructType
+        The DataFrame schema that is being compared or tested.
+    expected : StructType
+        The expected schema, for comparison with the actual schema.
+
+    Examples
+    --------
+    >>> from pyspark.sql.types import StructType, StructField, ArrayType, IntegerType, DoubleType
+    >>> s1 = StructType([StructField("names", ArrayType(DoubleType(), True), True)])
+    >>> s2 = StructType([StructField("names", ArrayType(DoubleType(), True), True)])
+    >>> assertSchemaEqual(s1, s2)
+
+    Pass, schemas are identical
+
+    >>> s1 = StructType([StructField("names", StructType(
+    ...                                         [StructField("age", DoubleType(), True)]), True)])
+    >>> s2 = StructType([StructField("first name", StructType([StructField(
+    ...                                                      "age", IntegerType(), True)]), True)])
+    >>> assertSchemaEqual(s1, s2)  # fail  # doctest: +IGNORE_EXCEPTION_DETAIL
+    Traceback (most recent call last):
+    ...
+    PySparkAssertionError: [DIFFERENT_SCHEMA] Schemas do not match.
+    The diff below overlays `actual` and `expected` schema tree strings.
+    - indicates a line that should be removed from `actual` to match `expected`.
+    + indicates a line that should be added to `actual` to match `expected`.
+      root
+    -  |-- names: struct (nullable = true)
+    ?          -
+
+    +  |-- first name: struct (nullable = true)
+    ?     ++++++
+
+    -  |    |-- age: double (nullable = true)
+    ?                ^^^^^
+
+    +  |    |-- age: integer (nullable = true)
+    ?                ^^^ +++
+    """
+    if not isinstance(actual, StructType):
+        raise PySparkAssertionError(
+            error_class="UNSUPPORTED_DATA_TYPE",
+            message_parameters={"data_type": type(actual)},
+        )
+    if not isinstance(expected, StructType):
+        raise PySparkAssertionError(
+            error_class="UNSUPPORTED_DATA_TYPE",
+            message_parameters={"data_type": type(expected)},
+        )
+
+    from pyspark.sql import SparkSession
+
+    spark = SparkSession.builder.appName("assertSchemaEqual").getOrCreate()
+
+    def compare_schemas_ignore_nullable(s1, s2):
+        if len(s1) != len(s2):
+            return False
+        zipped = zip_longest(s1, s2)
+        for sf1, sf2 in zipped:
+            if not compare_structfields_ignore_nullable(sf1, sf2):
+                return False
+        return True
+
+    def compare_structfields_ignore_nullable(actualSF, expectedSF):
+        if actualSF is None and expectedSF is None:
+            return True
+        elif actualSF is None or expectedSF is None:
+            return False
+        if actualSF.name != expectedSF.name:
+            return False
+        else:
+            return compare_datatypes_ignore_nullable(actualSF.dataType, expectedSF.dataType)
+
+    def compare_datatypes_ignore_nullable(dt1, dt2):

Review Comment:
   nit: can we add type hints for all these functions?  



##########
python/pyspark/testing/utils.py:
##########
@@ -222,6 +223,114 @@ def check_error(
         )
 
 
+def assertSchemaEqual(actual: StructType, expected: StructType):
+    """
+    A util function to assert equality between DataFrame schemas `actual` and `expected`.
+
+    .. versionadded:: 3.5.0
+
+    Parameters
+    ----------
+    actual : StructType
+        The DataFrame schema that is being compared or tested.
+    expected : StructType
+        The expected schema, for comparison with the actual schema.
+
+    Examples
+    --------
+    >>> from pyspark.sql.types import StructType, StructField, ArrayType, IntegerType, DoubleType
+    >>> s1 = StructType([StructField("names", ArrayType(DoubleType(), True), True)])
+    >>> s2 = StructType([StructField("names", ArrayType(DoubleType(), True), True)])
+    >>> assertSchemaEqual(s1, s2)
+
+    Pass, schemas are identical
+
+    >>> s1 = StructType([StructField("names", StructType(
+    ...                                         [StructField("age", DoubleType(), True)]), True)])
+    >>> s2 = StructType([StructField("first name", StructType([StructField(
+    ...                                                      "age", IntegerType(), True)]), True)])

Review Comment:
   nit: indent



##########
python/pyspark/testing/utils.py:
##########
@@ -222,6 +223,114 @@ def check_error(
         )
 
 
+def assertSchemaEqual(actual: StructType, expected: StructType):
+    """
+    A util function to assert equality between DataFrame schemas `actual` and `expected`.
+
+    .. versionadded:: 3.5.0
+
+    Parameters
+    ----------
+    actual : StructType
+        The DataFrame schema that is being compared or tested.
+    expected : StructType
+        The expected schema, for comparison with the actual schema.
+
+    Examples
+    --------
+    >>> from pyspark.sql.types import StructType, StructField, ArrayType, IntegerType, DoubleType
+    >>> s1 = StructType([StructField("names", ArrayType(DoubleType(), True), True)])
+    >>> s2 = StructType([StructField("names", ArrayType(DoubleType(), True), True)])
+    >>> assertSchemaEqual(s1, s2)
+
+    Pass, schemas are identical
+
+    >>> s1 = StructType([StructField("names", StructType(
+    ...                                         [StructField("age", DoubleType(), True)]), True)])
+    >>> s2 = StructType([StructField("first name", StructType([StructField(
+    ...                                                      "age", IntegerType(), True)]), True)])
+    >>> assertSchemaEqual(s1, s2)  # fail  # doctest: +IGNORE_EXCEPTION_DETAIL
+    Traceback (most recent call last):
+    ...
+    PySparkAssertionError: [DIFFERENT_SCHEMA] Schemas do not match.
+    The diff below overlays `actual` and `expected` schema tree strings.
+    - indicates a line that should be removed from `actual` to match `expected`.
+    + indicates a line that should be added to `actual` to match `expected`.
+      root
+    -  |-- names: struct (nullable = true)
+    ?          -
+
+    +  |-- first name: struct (nullable = true)
+    ?     ++++++
+
+    -  |    |-- age: double (nullable = true)
+    ?                ^^^^^
+
+    +  |    |-- age: integer (nullable = true)
+    ?                ^^^ +++
+    """
+    if not isinstance(actual, StructType):
+        raise PySparkAssertionError(
+            error_class="UNSUPPORTED_DATA_TYPE",
+            message_parameters={"data_type": type(actual)},
+        )
+    if not isinstance(expected, StructType):
+        raise PySparkAssertionError(
+            error_class="UNSUPPORTED_DATA_TYPE",
+            message_parameters={"data_type": type(expected)},
+        )
+
+    from pyspark.sql import SparkSession
+
+    spark = SparkSession.builder.appName("assertSchemaEqual").getOrCreate()

Review Comment:
   Hmm, is it necessary to instantiate a Spark session here? Doing so might make this utility function quite heavy. If possible, we should directly check the StructType of the schema, rather than using a Spark session.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonwang-db commented on a diff in pull request #41927: [SPARK-44216] [PYTHON] Make assertSchemaEqual API with ignore_nullable optional flag

Posted by "allisonwang-db (via GitHub)" <gi...@apache.org>.
allisonwang-db commented on code in PR #41927:
URL: https://github.com/apache/spark/pull/41927#discussion_r1261735884


##########
python/pyspark/testing/utils.py:
##########
@@ -221,7 +221,130 @@ def check_error(
         )
 
 
-def assertDataFrameEqual(df: DataFrame, expected: DataFrame, check_row_order: bool = False):
+def assertSchemaEqual(
+    df_schema: StructType, expected_schema: StructType, ignore_nullable: bool = False
+):
+    """
+    A util function to assert equality between DataFrame schemas `df_schema`
+    and `expected_schema`, with optional parameter `ignore_nullable`.
+
+    .. versionadded:: 3.5.0
+
+    Parameters
+    ----------
+    df_schema : StructType
+        The DataFrame schema that is being compared or tested.
+
+    expected_schema : StructType
+        The expected schema, for comparison with the actual schema.
+
+    ignore_nullable : bool, optional
+        A flag indicating whether the nullable flag should be ignored in schema comparison.
+        If set to `False` (default), the nullable flag in the schemas is not taken into account.
+        If set to `True`, the nullable flag will be checked during schema comparison.
+
+    Examples
+    --------
+    >>> from pyspark.sql.types import StructType, StructField, ArrayType, IntegerType, DoubleType
+    >>> s1 = StructType([StructField("names", ArrayType(DoubleType(), True), True)])
+    >>> s2 = StructType([StructField("names", ArrayType(DoubleType(), True), True)])
+    >>> assertSchemaEqual(s1, s2) # pass
+    >>> s1 = StructType([StructField("names", ArrayType(IntegerType(), True), True)])
+    >>> s2 = StructType([StructField("names", ArrayType(DoubleType(), False), True)])
+    >>> assertSchemaEqual(s1, s2) # fail  # doctest: +IGNORE_EXCEPTION_DETAIL
+    Traceback (most recent call last):
+    ...
+    PySparkAssertionError: [DIFFERENT_SCHEMA] Schemas do not match:
+    [df]
+    StructField("names", ArrayType(IntegerType(), True), True)
+    <BLANKLINE>
+    [expected]
+    StructField("names", ArrayType(DoubleType(), False), True)
+    <BLANKLINE>
+    """
+
+    def compare_schemas_ignore_nullable(s1, s2):
+        if len(s1) != len(s2):
+            return False
+        zipped = zip_longest(s1, s2)
+        for sf1, sf2 in zipped:
+            if not compare_structfields_ignore_nullable(sf1, sf2):
+                return False
+        return True
+
+    def compare_structfields_ignore_nullable(df_structfield, expected_structfield):
+        if ignore_nullable:
+            if df_structfield is None and expected_structfield is None:
+                return True
+            elif df_structfield is None or expected_structfield is None:
+                return False
+            if df_structfield.name != expected_structfield.name:
+                return False
+            else:
+                return compare_datatypes_ignore_nullable(
+                    df_structfield.dataType, expected_structfield.dataType
+                )
+        else:
+            return df_structfield == expected_structfield
+
+    def compare_datatypes_ignore_nullable(dt1, dt2):
+        # checks datatype equality, using recursion to ignore nullable
+        if dt1.typeName() == dt2.typeName():
+            if dt1.typeName() == "array":
+                return compare_datatypes_ignore_nullable(dt1.elementType, dt2.elementType)
+            elif dt1.typeName() == "struct":
+                return compare_schemas_ignore_nullable(dt1, dt2)
+            else:
+                return True
+        else:
+            return False
+
+    schemas_equal = True
+    error_msg = "Schemas do not match: \n"
+
+    if ignore_nullable:
+        if not compare_schemas_ignore_nullable(df_schema, expected_schema):
+            zipped = zip_longest(df_schema, expected_schema)
+            for df_structfield, expected_structfield in zipped:
+                if not compare_structfields_ignore_nullable(df_structfield, expected_structfield):
+                    schemas_equal = False
+                    error_msg += (
+                        "[df]"
+                        + "\n"
+                        + str(df_structfield)
+                        + "\n\n"
+                        + "[expected]"
+                        + "\n"
+                        + str(expected_structfield)
+                        + "\n\n"
+                    )
+    else:
+        if df_schema != expected_schema:
+            schemas_equal = False
+            zipped = zip_longest(df_schema, expected_schema)
+            for df_structfield, expected_structfield in zipped:
+                if df_structfield != expected_structfield:
+                    error_msg += (
+                        "[df]"
+                        + "\n"
+                        + str(df_structfield)
+                        + "\n\n"
+                        + "[expected]"
+                        + "\n"
+                        + str(expected_structfield)
+                        + "\n\n"
+                    )
+
+    if not schemas_equal:
+        raise PySparkAssertionError(
+            error_class="DIFFERENT_SCHEMA",
+            message_parameters={"error_msg": error_msg},
+        )
+
+
+def assertDataFrameEqual(
+    df: DataFrame, expected: DataFrame, check_row_order: bool = False, ignore_nullable: bool = False

Review Comment:
   +1 on excluding the nullability check for this PR.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #41927: [SPARK-44216] [PYTHON] Make assertSchemaEqual API with ignore_nullable optional flag

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on code in PR #41927:
URL: https://github.com/apache/spark/pull/41927#discussion_r1260475301


##########
python/pyspark/testing/utils.py:
##########
@@ -221,7 +221,130 @@ def check_error(
         )
 
 
-def assertDataFrameEqual(df: DataFrame, expected: DataFrame, check_row_order: bool = False):
+def assertSchemaEqual(
+    df_schema: StructType, expected_schema: StructType, ignore_nullable: bool = False
+):
+    """
+    A util function to assert equality between DataFrame schemas `df_schema`
+    and `expected_schema`, with optional parameter `ignore_nullable`.
+
+    .. versionadded:: 3.5.0
+
+    Parameters
+    ----------
+    df_schema : StructType
+        The DataFrame schema that is being compared or tested.
+
+    expected_schema : StructType
+        The expected schema, for comparison with the actual schema.
+
+    ignore_nullable : bool, optional
+        A flag indicating whether the nullable flag should be ignored in schema comparison.
+        If set to `False` (default), the nullable flag in the schemas is not taken into account.
+        If set to `True`, the nullable flag will be checked during schema comparison.
+
+    Examples
+    --------
+    >>> from pyspark.sql.types import StructType, StructField, ArrayType, IntegerType, DoubleType
+    >>> s1 = StructType([StructField("names", ArrayType(DoubleType(), True), True)])
+    >>> s2 = StructType([StructField("names", ArrayType(DoubleType(), True), True)])
+    >>> assertSchemaEqual(s1, s2) # pass
+    >>> s1 = StructType([StructField("names", ArrayType(IntegerType(), True), True)])
+    >>> s2 = StructType([StructField("names", ArrayType(DoubleType(), False), True)])
+    >>> assertSchemaEqual(s1, s2) # fail  # doctest: +IGNORE_EXCEPTION_DETAIL
+    Traceback (most recent call last):
+    ...
+    PySparkAssertionError: [DIFFERENT_SCHEMA] Schemas do not match:
+    [df]
+    StructField("names", ArrayType(IntegerType(), True), True)
+    <BLANKLINE>
+    [expected]
+    StructField("names", ArrayType(DoubleType(), False), True)
+    <BLANKLINE>
+    """
+
+    def compare_schemas_ignore_nullable(s1, s2):
+        if len(s1) != len(s2):
+            return False
+        zipped = zip_longest(s1, s2)
+        for sf1, sf2 in zipped:
+            if not compare_structfields_ignore_nullable(sf1, sf2):
+                return False
+        return True
+
+    def compare_structfields_ignore_nullable(df_structfield, expected_structfield):
+        if ignore_nullable:
+            if df_structfield is None and expected_structfield is None:
+                return True
+            elif df_structfield is None or expected_structfield is None:
+                return False
+            if df_structfield.name != expected_structfield.name:
+                return False
+            else:
+                return compare_datatypes_ignore_nullable(
+                    df_structfield.dataType, expected_structfield.dataType
+                )
+        else:
+            return df_structfield == expected_structfield
+
+    def compare_datatypes_ignore_nullable(dt1, dt2):
+        # checks datatype equality, using recursion to ignore nullable
+        if dt1.typeName() == dt2.typeName():
+            if dt1.typeName() == "array":
+                return compare_datatypes_ignore_nullable(dt1.elementType, dt2.elementType)
+            elif dt1.typeName() == "struct":
+                return compare_schemas_ignore_nullable(dt1, dt2)
+            else:
+                return True
+        else:
+            return False
+
+    schemas_equal = True
+    error_msg = "Schemas do not match: \n"
+
+    if ignore_nullable:
+        if not compare_schemas_ignore_nullable(df_schema, expected_schema):
+            zipped = zip_longest(df_schema, expected_schema)
+            for df_structfield, expected_structfield in zipped:
+                if not compare_structfields_ignore_nullable(df_structfield, expected_structfield):
+                    schemas_equal = False
+                    error_msg += (
+                        "[df]"
+                        + "\n"
+                        + str(df_structfield)
+                        + "\n\n"
+                        + "[expected]"
+                        + "\n"
+                        + str(expected_structfield)
+                        + "\n\n"
+                    )
+    else:
+        if df_schema != expected_schema:
+            schemas_equal = False
+            zipped = zip_longest(df_schema, expected_schema)
+            for df_structfield, expected_structfield in zipped:
+                if df_structfield != expected_structfield:
+                    error_msg += (
+                        "[df]"
+                        + "\n"
+                        + str(df_structfield)
+                        + "\n\n"
+                        + "[expected]"
+                        + "\n"
+                        + str(expected_structfield)
+                        + "\n\n"
+                    )
+
+    if not schemas_equal:
+        raise PySparkAssertionError(
+            error_class="DIFFERENT_SCHEMA",
+            message_parameters={"error_msg": error_msg},
+        )
+
+
+def assertDataFrameEqual(
+    df: DataFrame, expected: DataFrame, check_row_order: bool = False, ignore_nullable: bool = False

Review Comment:
   I am good with excluding nullability in this PR scope for now, but wonder if we can have a good error message for deeply nested type. That's actually my major concern.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #41927: [SPARK-44216] [PYTHON] Make assertSchemaEqual API with ignore_nullable optional flag

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on code in PR #41927:
URL: https://github.com/apache/spark/pull/41927#discussion_r1259014832


##########
python/pyspark/testing/utils.py:
##########
@@ -221,7 +221,130 @@ def check_error(
         )
 
 
-def assertDataFrameEqual(df: DataFrame, expected: DataFrame, check_row_order: bool = False):
+def assertSchemaEqual(
+    df_schema: StructType, expected_schema: StructType, ignore_nullable: bool = False
+):
+    """
+    A util function to assert equality between DataFrame schemas `df_schema`
+    and `expected_schema`, with optional parameter `ignore_nullable`.
+
+    .. versionadded:: 3.5.0
+
+    Parameters
+    ----------
+    df_schema : StructType
+        The DataFrame schema that is being compared or tested.
+
+    expected_schema : StructType
+        The expected schema, for comparison with the actual schema.
+
+    ignore_nullable : bool, optional
+        A flag indicating whether the nullable flag should be ignored in schema comparison.
+        If set to `False` (default), the nullable flag in the schemas is not taken into account.
+        If set to `True`, the nullable flag will be checked during schema comparison.
+
+    Examples
+    --------
+    >>> from pyspark.sql.types import StructType, StructField, ArrayType, IntegerType, DoubleType
+    >>> s1 = StructType([StructField("names", ArrayType(DoubleType(), True), True)])
+    >>> s2 = StructType([StructField("names", ArrayType(DoubleType(), True), True)])
+    >>> assertSchemaEqual(s1, s2) # pass
+    >>> s1 = StructType([StructField("names", ArrayType(IntegerType(), True), True)])
+    >>> s2 = StructType([StructField("names", ArrayType(DoubleType(), False), True)])
+    >>> assertSchemaEqual(s1, s2) # fail  # doctest: +IGNORE_EXCEPTION_DETAIL
+    Traceback (most recent call last):
+    ...
+    PySparkAssertionError: [DIFFERENT_SCHEMA] Schemas do not match:
+    [df]
+    StructField("names", ArrayType(IntegerType(), True), True)
+    <BLANKLINE>
+    [expected]
+    StructField("names", ArrayType(DoubleType(), False), True)
+    <BLANKLINE>
+    """
+
+    def compare_schemas_ignore_nullable(s1, s2):

Review Comment:
   There are some codes to refer to provide a nice message for nested types, e.g., https://github.com/apache/spark/blob/master/python/pyspark/sql/types.py#L1906-L2189



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonwang-db commented on a diff in pull request #41927: [SPARK-44216] [PYTHON] Make assertSchemaEqual API public

Posted by "allisonwang-db (via GitHub)" <gi...@apache.org>.
allisonwang-db commented on code in PR #41927:
URL: https://github.com/apache/spark/pull/41927#discussion_r1264721096


##########
python/pyspark/testing/utils.py:
##########
@@ -267,49 +358,40 @@ def assertDataFrameEqual(
     --------
     >>> df1 = spark.createDataFrame(data=[("1", 1000), ("2", 3000)], schema=["id", "amount"])
     >>> df2 = spark.createDataFrame(data=[("1", 1000), ("2", 3000)], schema=["id", "amount"])
-    >>> assertDataFrameEqual(df1, df2)
-
-    Pass, DataFrames are identical
-
+    >>> assertDataFrameEqual(df1, df2)  # pass, DataFrames are identical
     >>> df1 = spark.createDataFrame(data=[("1", 0.1), ("2", 3.23)], schema=["id", "amount"])
     >>> df2 = spark.createDataFrame(data=[("1", 0.109), ("2", 3.23)], schema=["id", "amount"])
-    >>> assertDataFrameEqual(df1, df2, rtol=1e-1)
-
-    Pass, DataFrames are approx equal by rtol
-
-    >>> df1 = spark.createDataFrame(data=[("1", 1000.00), ("2", 3000.00), ("3", 2000.00)],
-    ... schema=["id", "amount"])
-    >>> df2 = spark.createDataFrame(data=[("1", 1001.00), ("2", 3000.00), ("3", 2003.00)],
-    ... schema=["id", "amount"])
+    >>> assertDataFrameEqual(df1, df2, rtol=1e-1)  # pass, DataFrames are approx equal by rtol
+    >>> df1 = spark.createDataFrame(
+    ...     data=[("1", 1000.00), ("2", 3000.00), ("3", 2000.00)], schema=["id", "amount"])
+    >>> df2 = spark.createDataFrame(
+    ...     data=[("1", 1001.00), ("2", 3000.00), ("3", 2003.00)], schema=["id", "amount"])

Review Comment:
   We should add another email using list[Row]



##########
python/pyspark/testing/utils.py:
##########
@@ -333,8 +415,8 @@ def assertDataFrameEqual(
             )
 
     # special cases: empty datasets, datasets with 0 columns
-    if (df.first() is None and expected.first() is None) or (
-        len(df.columns) == 0 and len(expected.columns) == 0
+    if (actual.first() is None and expected.first() is None) or (

Review Comment:
   This will fail when expected is a list. We can have a follow up PR to fix this.



##########
python/pyspark/testing/utils.py:
##########
@@ -222,22 +223,112 @@ def check_error(
         )
 
 
+def assertSchemaEqual(actual: StructType, expected: StructType):
+    r"""
+    A util function to assert equality between DataFrame schemas `actual` and `expected`.
+
+    .. versionadded:: 3.5.0
+
+    Parameters
+    ----------
+    actual : StructType
+        The DataFrame schema that is being compared or tested.
+    expected : StructType
+        The expected schema, for comparison with the actual schema.
+
+    Notes
+    -----
+    When assertSchemaEqual fails, the error message uses the Python `difflib` library to display
+    a diff log of the `actual` and `expected` schemas.
+
+    Examples
+    --------
+    >>> from pyspark.sql.types import StructType, StructField, ArrayType, IntegerType, DoubleType
+    >>> s1 = StructType([StructField("names", ArrayType(DoubleType(), True), True)])
+    >>> s2 = StructType([StructField("names", ArrayType(DoubleType(), True), True)])
+    >>> assertSchemaEqual(s1, s2)  # pass, schemas are identical
+    >>> 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
+    Traceback (most recent call last):
+    ...
+    PySparkAssertionError: [DIFFERENT_SCHEMA] Schemas do not match.
+    --- actual
+    +++ expected
+    - StructType([StructField('id', LongType(), True), StructField('number', LongType(), True)])
+    ?                               ^^                               ^^^^^
+    + StructType([StructField('id', StringType(), True), StructField('amount', LongType(), True)])
+    ?                               ^^^^                              ++++ ^
+    """
+    if not isinstance(actual, StructType):
+        raise PySparkAssertionError(
+            error_class="UNSUPPORTED_DATA_TYPE",
+            message_parameters={"data_type": type(actual)},
+        )
+    if not isinstance(expected, StructType):
+        raise PySparkAssertionError(
+            error_class="UNSUPPORTED_DATA_TYPE",
+            message_parameters={"data_type": type(expected)},
+        )
+
+    def compare_schemas_ignore_nullable(s1: StructType, s2: StructType):
+        if len(s1) != len(s2):
+            return False
+        zipped = zip_longest(s1, s2)
+        for sf1, sf2 in zipped:
+            if not compare_structfields_ignore_nullable(sf1, sf2):
+                return False
+        return True
+
+    def compare_structfields_ignore_nullable(actualSF: StructField, expectedSF: StructField):
+        if actualSF is None and expectedSF is None:
+            return True
+        elif actualSF is None or expectedSF is None:
+            return False
+        if actualSF.name != expectedSF.name:
+            return False
+        else:
+            return compare_datatypes_ignore_nullable(actualSF.dataType, expectedSF.dataType)
+
+    def compare_datatypes_ignore_nullable(dt1: Any, dt2: Any):
+        # checks datatype equality, using recursion to ignore nullable
+        if dt1.typeName() == dt2.typeName():
+            if dt1.typeName() == "array":
+                return compare_datatypes_ignore_nullable(dt1.elementType, dt2.elementType)
+            elif dt1.typeName() == "struct":
+                return compare_schemas_ignore_nullable(dt1, dt2)
+            else:
+                return True
+        else:
+            return False
+
+    if not compare_schemas_ignore_nullable(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},
+        )
+
+
 def assertDataFrameEqual(
-    df: DataFrame,
+    actual: DataFrame,
     expected: Union[DataFrame, List[Row]],
     checkRowOrder: bool = False,

Review Comment:
   We can add a flag to not check the schema. And in the docstring we should mention that we only check schema when `expect` is a Dataframe (not a list of rows).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HyukjinKwon closed pull request #41927: [SPARK-44216] [PYTHON] Make assertSchemaEqual API public

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon closed pull request #41927: [SPARK-44216] [PYTHON] Make assertSchemaEqual API public
URL: https://github.com/apache/spark/pull/41927


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #41927: [SPARK-44216] [PYTHON] Make assertSchemaEqual API with ignore_nullable optional flag

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on code in PR #41927:
URL: https://github.com/apache/spark/pull/41927#discussion_r1259015012


##########
python/pyspark/testing/utils.py:
##########
@@ -221,7 +221,130 @@ def check_error(
         )
 
 
-def assertDataFrameEqual(df: DataFrame, expected: DataFrame, check_row_order: bool = False):
+def assertSchemaEqual(

Review Comment:
   ignore_nullable -> `ignoreNullable`. or `nullability`? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #41927: [SPARK-44216] [PYTHON] Make assertSchemaEqual API with ignore_nullable optional flag

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on code in PR #41927:
URL: https://github.com/apache/spark/pull/41927#discussion_r1260473784


##########
python/pyspark/testing/utils.py:
##########
@@ -221,7 +221,130 @@ def check_error(
         )
 
 
-def assertDataFrameEqual(df: DataFrame, expected: DataFrame, check_row_order: bool = False):
+def assertSchemaEqual(
+    df_schema: StructType, expected_schema: StructType, ignore_nullable: bool = False
+):
+    """
+    A util function to assert equality between DataFrame schemas `df_schema`
+    and `expected_schema`, with optional parameter `ignore_nullable`.
+
+    .. versionadded:: 3.5.0
+
+    Parameters
+    ----------
+    df_schema : StructType
+        The DataFrame schema that is being compared or tested.
+
+    expected_schema : StructType
+        The expected schema, for comparison with the actual schema.
+
+    ignore_nullable : bool, optional
+        A flag indicating whether the nullable flag should be ignored in schema comparison.
+        If set to `False` (default), the nullable flag in the schemas is not taken into account.
+        If set to `True`, the nullable flag will be checked during schema comparison.
+
+    Examples
+    --------
+    >>> from pyspark.sql.types import StructType, StructField, ArrayType, IntegerType, DoubleType
+    >>> s1 = StructType([StructField("names", ArrayType(DoubleType(), True), True)])
+    >>> s2 = StructType([StructField("names", ArrayType(DoubleType(), True), True)])
+    >>> assertSchemaEqual(s1, s2) # pass
+    >>> s1 = StructType([StructField("names", ArrayType(IntegerType(), True), True)])
+    >>> s2 = StructType([StructField("names", ArrayType(DoubleType(), False), True)])
+    >>> assertSchemaEqual(s1, s2) # fail  # doctest: +IGNORE_EXCEPTION_DETAIL
+    Traceback (most recent call last):
+    ...
+    PySparkAssertionError: [DIFFERENT_SCHEMA] Schemas do not match:
+    [df]
+    StructField("names", ArrayType(IntegerType(), True), True)
+    <BLANKLINE>
+    [expected]
+    StructField("names", ArrayType(DoubleType(), False), True)
+    <BLANKLINE>
+    """
+
+    def compare_schemas_ignore_nullable(s1, s2):
+        if len(s1) != len(s2):
+            return False
+        zipped = zip_longest(s1, s2)
+        for sf1, sf2 in zipped:
+            if not compare_structfields_ignore_nullable(sf1, sf2):
+                return False
+        return True
+
+    def compare_structfields_ignore_nullable(df_structfield, expected_structfield):
+        if ignore_nullable:
+            if df_structfield is None and expected_structfield is None:
+                return True
+            elif df_structfield is None or expected_structfield is None:
+                return False
+            if df_structfield.name != expected_structfield.name:
+                return False
+            else:
+                return compare_datatypes_ignore_nullable(
+                    df_structfield.dataType, expected_structfield.dataType
+                )
+        else:
+            return df_structfield == expected_structfield
+
+    def compare_datatypes_ignore_nullable(dt1, dt2):
+        # checks datatype equality, using recursion to ignore nullable
+        if dt1.typeName() == dt2.typeName():
+            if dt1.typeName() == "array":
+                return compare_datatypes_ignore_nullable(dt1.elementType, dt2.elementType)
+            elif dt1.typeName() == "struct":
+                return compare_schemas_ignore_nullable(dt1, dt2)
+            else:
+                return True
+        else:
+            return False
+
+    schemas_equal = True
+    error_msg = "Schemas do not match: \n"
+
+    if ignore_nullable:
+        if not compare_schemas_ignore_nullable(df_schema, expected_schema):
+            zipped = zip_longest(df_schema, expected_schema)
+            for df_structfield, expected_structfield in zipped:
+                if not compare_structfields_ignore_nullable(df_structfield, expected_structfield):
+                    schemas_equal = False
+                    error_msg += (
+                        "[df]"
+                        + "\n"
+                        + str(df_structfield)
+                        + "\n\n"
+                        + "[expected]"
+                        + "\n"
+                        + str(expected_structfield)
+                        + "\n\n"
+                    )
+    else:
+        if df_schema != expected_schema:
+            schemas_equal = False
+            zipped = zip_longest(df_schema, expected_schema)
+            for df_structfield, expected_structfield in zipped:
+                if df_structfield != expected_structfield:
+                    error_msg += (
+                        "[df]"
+                        + "\n"
+                        + str(df_structfield)
+                        + "\n\n"
+                        + "[expected]"
+                        + "\n"
+                        + str(expected_structfield)
+                        + "\n\n"
+                    )
+
+    if not schemas_equal:
+        raise PySparkAssertionError(
+            error_class="DIFFERENT_SCHEMA",
+            message_parameters={"error_msg": error_msg},
+        )
+
+
+def assertDataFrameEqual(
+    df: DataFrame, expected: DataFrame, check_row_order: bool = False, ignore_nullable: bool = False

Review Comment:
   One option is to add a sth like `StructType.asNullable` that Scala side has, (and document that this is Unstable API). Then users can call
   
   ```python
   assertDataFrameEqual(schema1.asNullable, schema2.asNullable)
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonwang-db commented on a diff in pull request #41927: [SPARK-44216] [PYTHON] Make assertSchemaEqual API with ignore_nullable optional flag

Posted by "allisonwang-db (via GitHub)" <gi...@apache.org>.
allisonwang-db commented on code in PR #41927:
URL: https://github.com/apache/spark/pull/41927#discussion_r1261736414


##########
python/pyspark/errors/error_classes.py:
##########
@@ -171,9 +171,7 @@
   },
   "DIFFERENT_SCHEMA" : {
     "message" : [
-      "Schemas do not match:",
-      "df schema: <df_schema>",
-      "expected schema: <expected_schema>"
+      "<error_msg>"

Review Comment:
   Seems like we can have `Schemas do not match` in the error message here.



##########
python/pyspark/testing/utils.py:
##########
@@ -221,7 +221,127 @@ def check_error(
         )
 
 
-def assertDataFrameEqual(df: DataFrame, expected: DataFrame, check_row_order: bool = False):
+def assertSchemaEqual(actual: StructType, expected: StructType, check_nullability: bool = True):

Review Comment:
   Can the expected schema be a string, like `a: int, b: int`?



##########
python/pyspark/testing/utils.py:
##########
@@ -221,7 +221,127 @@ def check_error(
         )
 
 
-def assertDataFrameEqual(df: DataFrame, expected: DataFrame, check_row_order: bool = False):
+def assertSchemaEqual(actual: StructType, expected: StructType, check_nullability: bool = True):
+    """
+    A util function to assert equality between DataFrame schemas `actual`
+    and `expected`, with optional parameter `check_nullability`.
+
+    .. versionadded:: 3.5.0
+
+    Parameters
+    ----------
+    actual : StructType
+        The DataFrame schema that is being compared or tested.
+    expected : StructType
+        The expected schema, for comparison with the actual schema.
+    check_nullability : bool, optional
+        A flag indicating whether the nullable flag should be ignored in schema comparison.
+        If set to `False`, the nullable flag in the schemas is not taken into account.
+        If set to `True` (default), the nullable flag will be checked during schema comparison.
+
+    Examples
+    --------
+    >>> from pyspark.sql.types import StructType, StructField, ArrayType, IntegerType, DoubleType
+    >>> s1 = StructType([StructField("names", ArrayType(DoubleType(), True), True)])
+    >>> s2 = StructType([StructField("names", ArrayType(DoubleType(), True), True)])
+    >>> assertSchemaEqual(s1, s2) # pass
+    >>> s1 = StructType([StructField("names", ArrayType(IntegerType(), True), True)])
+    >>> s2 = StructType([StructField("names", ArrayType(DoubleType(), False), True)])
+    >>> assertSchemaEqual(s1, s2) # fail  # doctest: +IGNORE_EXCEPTION_DETAIL +NORMALIZE_WHITESPACE
+    Traceback (most recent call last):
+    ...
+    PySparkAssertionError: [DIFFERENT_SCHEMA] Schemas do not match:
+    [df]
+    StructField("names", ArrayType(IntegerType(), True), True)
+
+    [expected]
+    StructField("names", ArrayType(DoubleType(), False), True)
+
+    """
+
+    def compare_schemas_ignore_nullable(s1, s2):
+        if len(s1) != len(s2):
+            return False
+        zipped = zip_longest(s1, s2)
+        for sf1, sf2 in zipped:
+            if not compare_structfields_ignore_nullable(sf1, sf2):
+                return False
+        return True
+
+    def compare_structfields_ignore_nullable(actualSF, expectedSF):
+        if not check_nullability:
+            if actualSF is None and expectedSF is None:
+                return True
+            elif actualSF is None or expectedSF is None:
+                return False
+            if actualSF.name != expectedSF.name:
+                return False
+            else:
+                return compare_datatypes_ignore_nullable(actualSF.dataType, expectedSF.dataType)
+        else:
+            return actualSF == expectedSF
+
+    def compare_datatypes_ignore_nullable(dt1, dt2):
+        # checks datatype equality, using recursion to ignore nullable
+        if dt1.typeName() == dt2.typeName():
+            if dt1.typeName() == "array":
+                return compare_datatypes_ignore_nullable(dt1.elementType, dt2.elementType)
+            elif dt1.typeName() == "struct":
+                return compare_schemas_ignore_nullable(dt1, dt2)
+            else:
+                return True
+        else:
+            return False
+
+    schemas_equal = True
+    error_msg = "Schemas do not match: \n"
+
+    if not check_nullability:
+        if not compare_schemas_ignore_nullable(actual, expected):
+            zipped = zip_longest(actual, expected)
+            for actualSF, expectedSF in zipped:
+                if not compare_structfields_ignore_nullable(actualSF, expectedSF):
+                    schemas_equal = False
+                    error_msg += (
+                        "[df]"
+                        + "\n"
+                        + str(actualSF)
+                        + "\n\n"
+                        + "[expected]"
+                        + "\n"
+                        + str(expectedSF)
+                        + "\n\n"
+                    )
+    else:
+        if actual != expected:

Review Comment:
   Should we also check if both actual and expected are StructType?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonwang-db commented on a diff in pull request #41927: [SPARK-44216] [PYTHON] Make assertSchemaEqual API with ignore_nullable optional flag

Posted by "allisonwang-db (via GitHub)" <gi...@apache.org>.
allisonwang-db commented on code in PR #41927:
URL: https://github.com/apache/spark/pull/41927#discussion_r1260330870


##########
python/pyspark/testing/utils.py:
##########
@@ -221,7 +221,130 @@ def check_error(
         )
 
 
-def assertDataFrameEqual(df: DataFrame, expected: DataFrame, check_row_order: bool = False):
+def assertSchemaEqual(
+    df_schema: StructType, expected_schema: StructType, ignore_nullable: bool = False
+):
+    """
+    A util function to assert equality between DataFrame schemas `df_schema`
+    and `expected_schema`, with optional parameter `ignore_nullable`.
+
+    .. versionadded:: 3.5.0
+
+    Parameters
+    ----------
+    df_schema : StructType
+        The DataFrame schema that is being compared or tested.
+
+    expected_schema : StructType
+        The expected schema, for comparison with the actual schema.
+
+    ignore_nullable : bool, optional
+        A flag indicating whether the nullable flag should be ignored in schema comparison.
+        If set to `False` (default), the nullable flag in the schemas is not taken into account.
+        If set to `True`, the nullable flag will be checked during schema comparison.
+
+    Examples
+    --------
+    >>> from pyspark.sql.types import StructType, StructField, ArrayType, IntegerType, DoubleType
+    >>> s1 = StructType([StructField("names", ArrayType(DoubleType(), True), True)])
+    >>> s2 = StructType([StructField("names", ArrayType(DoubleType(), True), True)])
+    >>> assertSchemaEqual(s1, s2) # pass
+    >>> s1 = StructType([StructField("names", ArrayType(IntegerType(), True), True)])
+    >>> s2 = StructType([StructField("names", ArrayType(DoubleType(), False), True)])
+    >>> assertSchemaEqual(s1, s2) # fail  # doctest: +IGNORE_EXCEPTION_DETAIL
+    Traceback (most recent call last):
+    ...
+    PySparkAssertionError: [DIFFERENT_SCHEMA] Schemas do not match:
+    [df]
+    StructField("names", ArrayType(IntegerType(), True), True)
+    <BLANKLINE>
+    [expected]
+    StructField("names", ArrayType(DoubleType(), False), True)
+    <BLANKLINE>
+    """
+
+    def compare_schemas_ignore_nullable(s1, s2):
+        if len(s1) != len(s2):
+            return False
+        zipped = zip_longest(s1, s2)
+        for sf1, sf2 in zipped:
+            if not compare_structfields_ignore_nullable(sf1, sf2):
+                return False
+        return True
+
+    def compare_structfields_ignore_nullable(df_structfield, expected_structfield):
+        if ignore_nullable:
+            if df_structfield is None and expected_structfield is None:
+                return True
+            elif df_structfield is None or expected_structfield is None:
+                return False
+            if df_structfield.name != expected_structfield.name:
+                return False
+            else:
+                return compare_datatypes_ignore_nullable(
+                    df_structfield.dataType, expected_structfield.dataType
+                )
+        else:
+            return df_structfield == expected_structfield
+
+    def compare_datatypes_ignore_nullable(dt1, dt2):
+        # checks datatype equality, using recursion to ignore nullable
+        if dt1.typeName() == dt2.typeName():
+            if dt1.typeName() == "array":
+                return compare_datatypes_ignore_nullable(dt1.elementType, dt2.elementType)
+            elif dt1.typeName() == "struct":
+                return compare_schemas_ignore_nullable(dt1, dt2)
+            else:
+                return True
+        else:
+            return False
+
+    schemas_equal = True
+    error_msg = "Schemas do not match: \n"
+
+    if ignore_nullable:
+        if not compare_schemas_ignore_nullable(df_schema, expected_schema):
+            zipped = zip_longest(df_schema, expected_schema)
+            for df_structfield, expected_structfield in zipped:
+                if not compare_structfields_ignore_nullable(df_structfield, expected_structfield):
+                    schemas_equal = False
+                    error_msg += (
+                        "[df]"
+                        + "\n"
+                        + str(df_structfield)
+                        + "\n\n"
+                        + "[expected]"
+                        + "\n"
+                        + str(expected_structfield)
+                        + "\n\n"
+                    )
+    else:
+        if df_schema != expected_schema:
+            schemas_equal = False
+            zipped = zip_longest(df_schema, expected_schema)
+            for df_structfield, expected_structfield in zipped:
+                if df_structfield != expected_structfield:
+                    error_msg += (
+                        "[df]"
+                        + "\n"
+                        + str(df_structfield)
+                        + "\n\n"
+                        + "[expected]"
+                        + "\n"
+                        + str(expected_structfield)
+                        + "\n\n"
+                    )
+
+    if not schemas_equal:
+        raise PySparkAssertionError(
+            error_class="DIFFERENT_SCHEMA",
+            message_parameters={"error_msg": error_msg},
+        )
+
+
+def assertDataFrameEqual(
+    df: DataFrame, expected: DataFrame, check_row_order: bool = False, ignore_nullable: bool = False

Review Comment:
   I'm wondering if the `check_nullability` option could be confusing for users, since it's unclear what the nullability is referring to. Is this option necessary? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #41927: [SPARK-44216] [PYTHON] Make assertSchemaEqual API with ignore_nullable optional flag

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on code in PR #41927:
URL: https://github.com/apache/spark/pull/41927#discussion_r1259015331


##########
python/pyspark/testing/utils.py:
##########
@@ -221,7 +221,130 @@ def check_error(
         )
 
 
-def assertDataFrameEqual(df: DataFrame, expected: DataFrame, check_row_order: bool = False):
+def assertSchemaEqual(
+    df_schema: StructType, expected_schema: StructType, ignore_nullable: bool = False

Review Comment:
   I would name it like `this` vs `that` or `schema1` `schema2` or `actualSchema` vs `expectedSchema` or `actual` vs `expected`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] asl3 commented on pull request #41927: [SPARK-44216] [PYTHON] Make assertSchemaEqual API with ignore_nullable optional flag

Posted by "asl3 (via GitHub)" <gi...@apache.org>.
asl3 commented on PR #41927:
URL: https://github.com/apache/spark/pull/41927#issuecomment-1629878767

   cc @HyukjinKwon @ueshin @allisonwang-db


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] asl3 commented on a diff in pull request #41927: [SPARK-44216] [PYTHON] Make assertSchemaEqual API with ignore_nullable optional flag

Posted by "asl3 (via GitHub)" <gi...@apache.org>.
asl3 commented on code in PR #41927:
URL: https://github.com/apache/spark/pull/41927#discussion_r1259848389


##########
python/pyspark/testing/utils.py:
##########
@@ -221,7 +221,130 @@ def check_error(
         )
 
 
-def assertDataFrameEqual(df: DataFrame, expected: DataFrame, check_row_order: bool = False):
+def assertSchemaEqual(
+    df_schema: StructType, expected_schema: StructType, ignore_nullable: bool = False
+):
+    """
+    A util function to assert equality between DataFrame schemas `df_schema`
+    and `expected_schema`, with optional parameter `ignore_nullable`.
+
+    .. versionadded:: 3.5.0
+
+    Parameters
+    ----------
+    df_schema : StructType
+        The DataFrame schema that is being compared or tested.
+
+    expected_schema : StructType
+        The expected schema, for comparison with the actual schema.
+
+    ignore_nullable : bool, optional
+        A flag indicating whether the nullable flag should be ignored in schema comparison.
+        If set to `False` (default), the nullable flag in the schemas is not taken into account.
+        If set to `True`, the nullable flag will be checked during schema comparison.
+
+    Examples
+    --------
+    >>> from pyspark.sql.types import StructType, StructField, ArrayType, IntegerType, DoubleType
+    >>> s1 = StructType([StructField("names", ArrayType(DoubleType(), True), True)])
+    >>> s2 = StructType([StructField("names", ArrayType(DoubleType(), True), True)])
+    >>> assertSchemaEqual(s1, s2) # pass
+    >>> s1 = StructType([StructField("names", ArrayType(IntegerType(), True), True)])
+    >>> s2 = StructType([StructField("names", ArrayType(DoubleType(), False), True)])
+    >>> assertSchemaEqual(s1, s2) # fail  # doctest: +IGNORE_EXCEPTION_DETAIL
+    Traceback (most recent call last):
+    ...
+    PySparkAssertionError: [DIFFERENT_SCHEMA] Schemas do not match:
+    [df]
+    StructField("names", ArrayType(IntegerType(), True), True)
+    <BLANKLINE>
+    [expected]
+    StructField("names", ArrayType(DoubleType(), False), True)
+    <BLANKLINE>
+    """
+
+    def compare_schemas_ignore_nullable(s1, s2):
+        if len(s1) != len(s2):
+            return False
+        zipped = zip_longest(s1, s2)
+        for sf1, sf2 in zipped:
+            if not compare_structfields_ignore_nullable(sf1, sf2):
+                return False
+        return True
+
+    def compare_structfields_ignore_nullable(df_structfield, expected_structfield):
+        if ignore_nullable:
+            if df_structfield is None and expected_structfield is None:
+                return True
+            elif df_structfield is None or expected_structfield is None:
+                return False
+            if df_structfield.name != expected_structfield.name:
+                return False
+            else:
+                return compare_datatypes_ignore_nullable(
+                    df_structfield.dataType, expected_structfield.dataType
+                )
+        else:
+            return df_structfield == expected_structfield
+
+    def compare_datatypes_ignore_nullable(dt1, dt2):
+        # checks datatype equality, using recursion to ignore nullable
+        if dt1.typeName() == dt2.typeName():
+            if dt1.typeName() == "array":
+                return compare_datatypes_ignore_nullable(dt1.elementType, dt2.elementType)
+            elif dt1.typeName() == "struct":
+                return compare_schemas_ignore_nullable(dt1, dt2)
+            else:
+                return True
+        else:
+            return False
+
+    schemas_equal = True
+    error_msg = "Schemas do not match: \n"
+
+    if ignore_nullable:
+        if not compare_schemas_ignore_nullable(df_schema, expected_schema):
+            zipped = zip_longest(df_schema, expected_schema)
+            for df_structfield, expected_structfield in zipped:
+                if not compare_structfields_ignore_nullable(df_structfield, expected_structfield):
+                    schemas_equal = False
+                    error_msg += (
+                        "[df]"
+                        + "\n"
+                        + str(df_structfield)
+                        + "\n\n"
+                        + "[expected]"
+                        + "\n"
+                        + str(expected_structfield)
+                        + "\n\n"
+                    )
+    else:
+        if df_schema != expected_schema:
+            schemas_equal = False
+            zipped = zip_longest(df_schema, expected_schema)
+            for df_structfield, expected_structfield in zipped:
+                if df_structfield != expected_structfield:
+                    error_msg += (
+                        "[df]"
+                        + "\n"
+                        + str(df_structfield)
+                        + "\n\n"
+                        + "[expected]"
+                        + "\n"
+                        + str(expected_structfield)
+                        + "\n\n"
+                    )
+
+    if not schemas_equal:
+        raise PySparkAssertionError(
+            error_class="DIFFERENT_SCHEMA",
+            message_parameters={"error_msg": error_msg},
+        )
+
+
+def assertDataFrameEqual(
+    df: DataFrame, expected: DataFrame, check_row_order: bool = False, ignore_nullable: bool = False

Review Comment:
   sounds good, how about `check_nullability` -- if set to False, it also will ignore nullable flag in Maps, Arrays, and Structs



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonwang-db commented on a diff in pull request #41927: [SPARK-44216] [PYTHON] Make assertSchemaEqual API with ignore_nullable optional flag

Posted by "allisonwang-db (via GitHub)" <gi...@apache.org>.
allisonwang-db commented on code in PR #41927:
URL: https://github.com/apache/spark/pull/41927#discussion_r1259106431


##########
python/pyspark/testing/utils.py:
##########
@@ -221,7 +221,130 @@ def check_error(
         )
 
 
-def assertDataFrameEqual(df: DataFrame, expected: DataFrame, check_row_order: bool = False):
+def assertSchemaEqual(
+    df_schema: StructType, expected_schema: StructType, ignore_nullable: bool = False
+):
+    """
+    A util function to assert equality between DataFrame schemas `df_schema`
+    and `expected_schema`, with optional parameter `ignore_nullable`.
+
+    .. versionadded:: 3.5.0
+
+    Parameters
+    ----------
+    df_schema : StructType
+        The DataFrame schema that is being compared or tested.
+
+    expected_schema : StructType
+        The expected schema, for comparison with the actual schema.
+
+    ignore_nullable : bool, optional
+        A flag indicating whether the nullable flag should be ignored in schema comparison.
+        If set to `False` (default), the nullable flag in the schemas is not taken into account.
+        If set to `True`, the nullable flag will be checked during schema comparison.
+
+    Examples
+    --------
+    >>> from pyspark.sql.types import StructType, StructField, ArrayType, IntegerType, DoubleType
+    >>> s1 = StructType([StructField("names", ArrayType(DoubleType(), True), True)])
+    >>> s2 = StructType([StructField("names", ArrayType(DoubleType(), True), True)])
+    >>> assertSchemaEqual(s1, s2) # pass
+    >>> s1 = StructType([StructField("names", ArrayType(IntegerType(), True), True)])
+    >>> s2 = StructType([StructField("names", ArrayType(DoubleType(), False), True)])
+    >>> assertSchemaEqual(s1, s2) # fail  # doctest: +IGNORE_EXCEPTION_DETAIL
+    Traceback (most recent call last):
+    ...
+    PySparkAssertionError: [DIFFERENT_SCHEMA] Schemas do not match:
+    [df]
+    StructField("names", ArrayType(IntegerType(), True), True)
+    <BLANKLINE>
+    [expected]
+    StructField("names", ArrayType(DoubleType(), False), True)
+    <BLANKLINE>
+    """
+
+    def compare_schemas_ignore_nullable(s1, s2):
+        if len(s1) != len(s2):
+            return False
+        zipped = zip_longest(s1, s2)
+        for sf1, sf2 in zipped:
+            if not compare_structfields_ignore_nullable(sf1, sf2):
+                return False
+        return True
+
+    def compare_structfields_ignore_nullable(df_structfield, expected_structfield):
+        if ignore_nullable:
+            if df_structfield is None and expected_structfield is None:
+                return True
+            elif df_structfield is None or expected_structfield is None:
+                return False
+            if df_structfield.name != expected_structfield.name:
+                return False
+            else:
+                return compare_datatypes_ignore_nullable(
+                    df_structfield.dataType, expected_structfield.dataType
+                )
+        else:
+            return df_structfield == expected_structfield
+
+    def compare_datatypes_ignore_nullable(dt1, dt2):
+        # checks datatype equality, using recursion to ignore nullable
+        if dt1.typeName() == dt2.typeName():
+            if dt1.typeName() == "array":
+                return compare_datatypes_ignore_nullable(dt1.elementType, dt2.elementType)
+            elif dt1.typeName() == "struct":
+                return compare_schemas_ignore_nullable(dt1, dt2)
+            else:
+                return True
+        else:
+            return False
+
+    schemas_equal = True
+    error_msg = "Schemas do not match: \n"
+
+    if ignore_nullable:
+        if not compare_schemas_ignore_nullable(df_schema, expected_schema):
+            zipped = zip_longest(df_schema, expected_schema)
+            for df_structfield, expected_structfield in zipped:
+                if not compare_structfields_ignore_nullable(df_structfield, expected_structfield):
+                    schemas_equal = False
+                    error_msg += (
+                        "[df]"
+                        + "\n"
+                        + str(df_structfield)
+                        + "\n\n"
+                        + "[expected]"
+                        + "\n"
+                        + str(expected_structfield)
+                        + "\n\n"
+                    )
+    else:
+        if df_schema != expected_schema:
+            schemas_equal = False
+            zipped = zip_longest(df_schema, expected_schema)
+            for df_structfield, expected_structfield in zipped:
+                if df_structfield != expected_structfield:
+                    error_msg += (
+                        "[df]"
+                        + "\n"
+                        + str(df_structfield)
+                        + "\n\n"
+                        + "[expected]"
+                        + "\n"
+                        + str(expected_structfield)
+                        + "\n\n"
+                    )
+
+    if not schemas_equal:
+        raise PySparkAssertionError(
+            error_class="DIFFERENT_SCHEMA",
+            message_parameters={"error_msg": error_msg},
+        )
+
+
+def assertDataFrameEqual(
+    df: DataFrame, expected: DataFrame, check_row_order: bool = False, ignore_nullable: bool = False

Review Comment:
   ignore_nullable -> check_schema_nullability? To make it consistent with `check_row_order`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] allisonwang-db commented on a diff in pull request #41927: [SPARK-44216] [PYTHON] Make assertSchemaEqual API with ignore_nullable optional flag

Posted by "allisonwang-db (via GitHub)" <gi...@apache.org>.
allisonwang-db commented on code in PR #41927:
URL: https://github.com/apache/spark/pull/41927#discussion_r1260330870


##########
python/pyspark/testing/utils.py:
##########
@@ -221,7 +221,130 @@ def check_error(
         )
 
 
-def assertDataFrameEqual(df: DataFrame, expected: DataFrame, check_row_order: bool = False):
+def assertSchemaEqual(
+    df_schema: StructType, expected_schema: StructType, ignore_nullable: bool = False
+):
+    """
+    A util function to assert equality between DataFrame schemas `df_schema`
+    and `expected_schema`, with optional parameter `ignore_nullable`.
+
+    .. versionadded:: 3.5.0
+
+    Parameters
+    ----------
+    df_schema : StructType
+        The DataFrame schema that is being compared or tested.
+
+    expected_schema : StructType
+        The expected schema, for comparison with the actual schema.
+
+    ignore_nullable : bool, optional
+        A flag indicating whether the nullable flag should be ignored in schema comparison.
+        If set to `False` (default), the nullable flag in the schemas is not taken into account.
+        If set to `True`, the nullable flag will be checked during schema comparison.
+
+    Examples
+    --------
+    >>> from pyspark.sql.types import StructType, StructField, ArrayType, IntegerType, DoubleType
+    >>> s1 = StructType([StructField("names", ArrayType(DoubleType(), True), True)])
+    >>> s2 = StructType([StructField("names", ArrayType(DoubleType(), True), True)])
+    >>> assertSchemaEqual(s1, s2) # pass
+    >>> s1 = StructType([StructField("names", ArrayType(IntegerType(), True), True)])
+    >>> s2 = StructType([StructField("names", ArrayType(DoubleType(), False), True)])
+    >>> assertSchemaEqual(s1, s2) # fail  # doctest: +IGNORE_EXCEPTION_DETAIL
+    Traceback (most recent call last):
+    ...
+    PySparkAssertionError: [DIFFERENT_SCHEMA] Schemas do not match:
+    [df]
+    StructField("names", ArrayType(IntegerType(), True), True)
+    <BLANKLINE>
+    [expected]
+    StructField("names", ArrayType(DoubleType(), False), True)
+    <BLANKLINE>
+    """
+
+    def compare_schemas_ignore_nullable(s1, s2):
+        if len(s1) != len(s2):
+            return False
+        zipped = zip_longest(s1, s2)
+        for sf1, sf2 in zipped:
+            if not compare_structfields_ignore_nullable(sf1, sf2):
+                return False
+        return True
+
+    def compare_structfields_ignore_nullable(df_structfield, expected_structfield):
+        if ignore_nullable:
+            if df_structfield is None and expected_structfield is None:
+                return True
+            elif df_structfield is None or expected_structfield is None:
+                return False
+            if df_structfield.name != expected_structfield.name:
+                return False
+            else:
+                return compare_datatypes_ignore_nullable(
+                    df_structfield.dataType, expected_structfield.dataType
+                )
+        else:
+            return df_structfield == expected_structfield
+
+    def compare_datatypes_ignore_nullable(dt1, dt2):
+        # checks datatype equality, using recursion to ignore nullable
+        if dt1.typeName() == dt2.typeName():
+            if dt1.typeName() == "array":
+                return compare_datatypes_ignore_nullable(dt1.elementType, dt2.elementType)
+            elif dt1.typeName() == "struct":
+                return compare_schemas_ignore_nullable(dt1, dt2)
+            else:
+                return True
+        else:
+            return False
+
+    schemas_equal = True
+    error_msg = "Schemas do not match: \n"
+
+    if ignore_nullable:
+        if not compare_schemas_ignore_nullable(df_schema, expected_schema):
+            zipped = zip_longest(df_schema, expected_schema)
+            for df_structfield, expected_structfield in zipped:
+                if not compare_structfields_ignore_nullable(df_structfield, expected_structfield):
+                    schemas_equal = False
+                    error_msg += (
+                        "[df]"
+                        + "\n"
+                        + str(df_structfield)
+                        + "\n\n"
+                        + "[expected]"
+                        + "\n"
+                        + str(expected_structfield)
+                        + "\n\n"
+                    )
+    else:
+        if df_schema != expected_schema:
+            schemas_equal = False
+            zipped = zip_longest(df_schema, expected_schema)
+            for df_structfield, expected_structfield in zipped:
+                if df_structfield != expected_structfield:
+                    error_msg += (
+                        "[df]"
+                        + "\n"
+                        + str(df_structfield)
+                        + "\n\n"
+                        + "[expected]"
+                        + "\n"
+                        + str(expected_structfield)
+                        + "\n\n"
+                    )
+
+    if not schemas_equal:
+        raise PySparkAssertionError(
+            error_class="DIFFERENT_SCHEMA",
+            message_parameters={"error_msg": error_msg},
+        )
+
+
+def assertDataFrameEqual(
+    df: DataFrame, expected: DataFrame, check_row_order: bool = False, ignore_nullable: bool = False

Review Comment:
   I'm wondering if the `check_nullability` option could be confusing for users, since it's unclear what the nullability is referring to. Is this option necessary? When do we need to set it to True?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org