Skip to content

Commit 0a180c0

Browse files
sadikoviMaxGekk
authored andcommitted
[SPARK-40292][SQL] Fix column names in "arrays_zip" function when arrays are referenced from nested structs
### What changes were proposed in this pull request? This PR fixes an issue in `arrays_zip` function where a field index was used as a column name in the resulting schema which was a regression from Spark 3.1. With this change, the original behaviour is restored: a corresponding struct field name will be used instead of a field index. Example: ```sql with q as ( select named_struct( 'my_array', array(1, 2, 3), 'my_array2', array(4, 5, 6) ) as my_struct ) select arrays_zip(my_struct.my_array, my_struct.my_array2) from q ``` would return schema: ``` root |-- arrays_zip(my_struct.my_array, my_struct.my_array2): array (nullable = false) | |-- element: struct (containsNull = false) | | |-- 0: integer (nullable = true) | | |-- 1: integer (nullable = true) ``` which is somewhat inaccurate. PR adds handling of `GetStructField` expression to return the struct field names like this: ``` root |-- arrays_zip(my_struct.my_array, my_struct.my_array2): array (nullable = false) | |-- element: struct (containsNull = false) | | |-- my_array: integer (nullable = true) | | |-- my_array2: integer (nullable = true) ``` ### Why are the changes needed? ### Does this PR introduce _any_ user-facing change? Yes, `arrays_zip` function returns struct field names now as in Spark 3.1 instead of field indices. Some users might have worked around this issue so this patch would affect them by bringing back the original behaviour. ### How was this patch tested? Existing unit tests. I also added a test case that reproduces the problem. Closes #37833 from sadikovi/SPARK-40292. Authored-by: Ivan Sadikov <ivan.sadikov@databricks.com> Signed-off-by: Max Gekk <max.gekk@gmail.com> (cherry picked from commit 443eea9) Signed-off-by: Max Gekk <max.gekk@gmail.com>
1 parent 052d60c commit 0a180c0

File tree

2 files changed

+20
-0
lines changed

2 files changed

+20
-0
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala

+1
Original file line numberDiff line numberDiff line change
@@ -267,6 +267,7 @@ case class ArraysZip(children: Seq[Expression], names: Seq[Expression])
267267
case (u: UnresolvedAttribute, _) => Literal(u.nameParts.last)
268268
case (e: NamedExpression, _) if e.resolved => Literal(e.name)
269269
case (e: NamedExpression, _) => NamePlaceholder
270+
case (e: GetStructField, _) => Literal(e.extractFieldName)
270271
case (_, idx) => Literal(idx.toString)
271272
})
272273
}

sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala

+19
Original file line numberDiff line numberDiff line change
@@ -721,6 +721,25 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession {
721721
}
722722
}
723723

724+
test("SPARK-40292: arrays_zip should retain field names in nested structs") {
725+
val df = spark.sql("""
726+
select
727+
named_struct(
728+
'arr_1', array(named_struct('a', 1, 'b', 2)),
729+
'arr_2', array(named_struct('p', 1, 'q', 2)),
730+
'field', named_struct(
731+
'arr_3', array(named_struct('x', 1, 'y', 2))
732+
)
733+
) as obj
734+
""")
735+
736+
val res = df.selectExpr("arrays_zip(obj.arr_1, obj.arr_2, obj.field.arr_3) as arr")
737+
738+
val fieldNames = res.schema.head.dataType.asInstanceOf[ArrayType]
739+
.elementType.asInstanceOf[StructType].fieldNames
740+
assert(fieldNames.toSeq === Seq("arr_1", "arr_2", "arr_3"))
741+
}
742+
724743
def testSizeOfMap(sizeOfNull: Any): Unit = {
725744
val df = Seq(
726745
(Map[Int, Int](1 -> 1, 2 -> 2), "x"),

0 commit comments

Comments
 (0)