You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by gu...@apache.org on 2023/03/23 02:15:17 UTC

[spark] branch master updated: [SPARK-42899][SQL] Fix DataFrame.to(schema) to handle the case where there is a non-nullable nested field in a nullable field

This is an automated email from the ASF dual-hosted git repository.

gurwls223 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new 4052058f9d1 [SPARK-42899][SQL] Fix DataFrame.to(schema) to handle the case where there is a non-nullable nested field in a nullable field
4052058f9d1 is described below

commit 4052058f9d1f60f1539dd227614cd459bfdfd31f
Author: Takuya UESHIN <ue...@databricks.com>
AuthorDate: Thu Mar 23 11:14:59 2023 +0900

    [SPARK-42899][SQL] Fix DataFrame.to(schema) to handle the case where there is a non-nullable nested field in a nullable field
    
    ### What changes were proposed in this pull request?
    
    Fixes `DataFrame.to(schema)` to handle the case where there is a non-nullable nested field in a nullable field.
    
    ### Why are the changes needed?
    
    `DataFrame.to(schema)` fails when it contains non-nullable nested field in nullable field:
    
    ```scala
    scala> val df = spark.sql("VALUES (1, STRUCT(1 as i)), (NULL, NULL) as t(a, b)")
    df: org.apache.spark.sql.DataFrame = [a: int, b: struct<i: int>]
    scala> df.printSchema()
    root
     |-- a: integer (nullable = true)
     |-- b: struct (nullable = true)
     |    |-- i: integer (nullable = false)
    
    scala> df.to(df.schema)
    org.apache.spark.sql.AnalysisException: [NULLABLE_COLUMN_OR_FIELD] Column or field `b`.`i` is nullable while it's required to be non-nullable.
    ```
    
    ### Does this PR introduce _any_ user-facing change?
    
    No.
    
    ### How was this patch tested?
    
    Added the related tests.
    
    Closes #40526 from ueshin/issues/SPARK-42899/to_schema.
    
    Authored-by: Takuya UESHIN <ue...@databricks.com>
    Signed-off-by: Hyukjin Kwon <gu...@apache.org>
---
 .../plans/logical/basicLogicalOperators.scala      |  3 +-
 .../apache/spark/sql/DataFrameToSchemaSuite.scala  | 33 ++++++++++++++++++++++
 2 files changed, 35 insertions(+), 1 deletion(-)

diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
index 1eddd7ed24d..b77370fc5e7 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
@@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable
 import org.apache.spark.sql.catalyst.catalog.CatalogTable.VIEW_STORING_ANALYZED_PLAN
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, TypedImperativeAggregate}
+import org.apache.spark.sql.catalyst.expressions.objects.AssertNotNull
 import org.apache.spark.sql.catalyst.plans._
 import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, RangePartitioning, RoundRobinPartitioning, SinglePartition}
 import org.apache.spark.sql.catalyst.trees.TreeNodeTag
@@ -119,7 +120,7 @@ object Project {
       case (StructType(fields), expected: StructType) =>
         val newFields = reorderFields(
           fields.zipWithIndex.map { case (f, index) =>
-            (f.name, GetStructField(col, index))
+            (f.name, GetStructField(AssertNotNull(col, columnPath), index))
           },
           expected.fields,
           columnPath,
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameToSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameToSchemaSuite.scala
index 5bbaebbd9ce..160f583c983 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameToSchemaSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameToSchemaSuite.scala
@@ -180,6 +180,17 @@ class DataFrameToSchemaSuite extends QueryTest with SharedSparkSession {
     checkAnswer(df, Row(Row(1)))
   }
 
+  test("struct value: compatible field nullability") {
+    val innerFields = new StructType().add("i", LongType, nullable = false)
+    val schema = new StructType().add("a", LongType).add("b", innerFields)
+    val data = sql("VALUES (1, STRUCT(1 as i)), (NULL, NULL) as t(a, b)")
+    assert(data.schema.fields(1).nullable)
+    assert(!data.schema.fields(1).dataType.asInstanceOf[StructType].fields(0).nullable)
+    val df = data.to(schema)
+    assert(df.schema == schema)
+    checkAnswer(df, Seq(Row(1, Row(1)), Row(null, null)))
+  }
+
   test("negative: incompatible field nullability") {
     val innerFields = new StructType().add("i", IntegerType, nullable = false)
     val schema = new StructType().add("struct", innerFields)
@@ -254,6 +265,17 @@ class DataFrameToSchemaSuite extends QueryTest with SharedSparkSession {
     checkAnswer(df, Row(Seq(Row(1L))))
   }
 
+  test("array element: compatible field nullability") {
+    val innerFields = ArrayType(LongType, containsNull = false)
+    val schema = new StructType().add("a", LongType).add("b", innerFields)
+    val data = sql("VALUES (1, ARRAY(1, 2)), (NULL, NULL) as t(a, b)")
+    assert(data.schema.fields(1).nullable)
+    assert(!data.schema.fields(1).dataType.asInstanceOf[ArrayType].containsNull)
+    val df = data.to(schema)
+    assert(df.schema == schema)
+    checkAnswer(df, Seq(Row(1, Seq(1, 2)), Row(null, null)))
+  }
+
   test("array element: incompatible array nullability") {
     val arr = ArrayType(IntegerType, containsNull = false)
     val schema = new StructType().add("arr", arr)
@@ -321,6 +343,17 @@ class DataFrameToSchemaSuite extends QueryTest with SharedSparkSession {
     checkAnswer(df, Row(Map("a" -> Row("b", "a"))))
   }
 
+  test("map value: compatible field nullability") {
+    val innerFields = MapType(StringType, LongType, valueContainsNull = false)
+    val schema = new StructType().add("a", LongType).add("b", innerFields)
+    val data = sql("VALUES (1, MAP('a', 1, 'b', 2)), (NULL, NULL) as t(a, b)")
+    assert(data.schema.fields(1).nullable)
+    assert(!data.schema.fields(1).dataType.asInstanceOf[MapType].valueContainsNull)
+    val df = data.to(schema)
+    assert(df.schema == schema)
+    checkAnswer(df, Seq(Row(1, Map("a" -> 1, "b" -> 2)), Row(null, null)))
+  }
+
   test("map value: incompatible map nullability") {
     val m = MapType(StringType, StringType, valueContainsNull = false)
     val schema = new StructType().add("map", m, nullable = false)


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