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-unsubscr...@spark.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org