SparkApplicationMaster opened a new issue, #2036: URL: https://github.com/apache/datafusion-comet/issues/2036
### Describe the bug Spark doc says: arrays_overlap(a1, a2) - Returns true if a1 contains at least a non-null element present also in a2. **If the arrays have no common element and they are both non-empty and either of them contains a null element null is returned**, false otherwise. https://spark.apache.org/docs/latest/api/sql/index.html#arrays_overlap But comet returns True instead of NULL if the arrays intersect only by NULL ### Steps to reproduce ```python #change this to your path comet_jar_path = "/home/x/comet-spark-spark3.5_2.12-0.9.0.jar" from pyspark.sql import SparkSession # uncomment to reproduce vanilla spark behavior: #spark = SparkSession.builder.appName("test").getOrCreate() # comment if upper is uncommented spark = SparkSession.builder.appName("test")\ .config("spark.jars", comet_jar_path)\ .config("spark.driver.extraClassPath", comet_jar_path)\ .config("spark.executor.extraClassPath", comet_jar_path)\ .config("spark.plugins", "org.apache.spark.CometPlugin")\ .config("spark.shuffle.manager", "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager")\ .config("spark.comet.explainFallback.enabled", "true")\ .config("spark.comet.expression.allowIncompatible", "true")\ .getOrCreate() df = spark.createDataFrame([ (["a", None], ["b", "c", None]), ], ['x', 'y'] ) df.write.mode("overwrite").save("/tmp/tmp_parquet") spark.read.parquet("/tmp/tmp_parquet").createOrReplaceTempView("t1") spark.sql("select * from t1").show() spark.sql("SELECT arrays_overlap(x, y) from t1").show() spark.sql("SELECT arrays_overlap(x, y) from t1").explain(True) ``` comet returns true if the arrays intersect only by NULL value: ``` +---------+------------+ | x| y| +---------+------------+ |[a, NULL]|[b, c, NULL]| +---------+------------+ +--------------------+ |arrays_overlap(x, y)| +--------------------+ | true| +--------------------+ == Physical Plan == *(1) CometColumnarToRow +- CometProject [arrays_overlap(x, y)#56], [arrays_overlap(x#34, y#35) AS arrays_overlap(x, y)#56] +- CometScan parquet [x#34,y#35] Batched: true, DataFilters: [], Format: CometParquet, Location: InMemoryFileIndex(1 paths)[file:/tmp/tmp_parquet], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<x:array<string>,y:array<string>> ``` ### Expected behavior spark 3.5.6 returns NULL if the arrays intersect only by NULL value: ``` +---------+------------+ | x| y| +---------+------------+ |[a, NULL]|[b, c, NULL]| +---------+------------+ +--------------------+ |arrays_overlap(x, y)| +--------------------+ | NULL| +--------------------+ ``` ### Additional context This is because the `arrays_overlap` call propagates to datafusion `` call, which returns True in this case Maybe it needs to be changed in datafusion https://github.com/apache/datafusion/blob/8a3ea8739c36e5cafffff195ffc42ae1fc1e8dde/datafusion/functions-nested/src/array_has.rs#L652 Because if not, then there is no good solution to provide compartibility -- 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: github-unsubscr...@datafusion.apache.org.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: github-unsubscr...@datafusion.apache.org For additional commands, e-mail: github-h...@datafusion.apache.org