You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Shardul Mahadik (Jira)" <ji...@apache.org> on 2021/09/06 08:49:00 UTC

[jira] [Updated] (SPARK-36673) Incorrect Unions of struct with mismatched field name case

     [ https://issues.apache.org/jira/browse/SPARK-36673?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Shardul Mahadik updated SPARK-36673:
------------------------------------
    Description: 
If a nested field has different casing on two sides of the union, the resultant schema of the union will both fields in its schemaa
{code:java}
scala> val df1 = spark.range(2).withColumn("nested", struct(expr("id * 5 AS INNER")))
df1: org.apache.spark.sql.DataFrame = [id: bigint, nested: struct<INNER: bigint>]

val df2 = spark.range(2).withColumn("nested", struct(expr("id * 5 AS inner")))
df2: org.apache.spark.sql.DataFrame = [id: bigint, nested: struct<inner: bigint>]

scala> df1.union(df2).printSchema
root
 |-- id: long (nullable = false)
 |-- nested: struct (nullable = false)
 |    |-- INNER: long (nullable = false)
 |    |-- inner: long (nullable = false)
 {code}
This seems like a bug. I would expect that Spark SQL would either just union by index or if the user has requested {{unionByName}}, then it should matched fields case insensitively if {{spark.sql.caseSensitive}} is {{false}}.

However the output data only has one nested column
{code:java}
scala> df1.union(df2).show()
+---+------+
| id|nested|
+---+------+
|  0|   {0}|
|  1|   {5}|
|  0|   {0}|
|  1|   {5}|
+---+------+
{code}
Trying to project fields of {{nested}} throws an error:
{code:java}
scala> df1.union(df2).select("nested.*").show()
java.lang.ArrayIndexOutOfBoundsException: 1
  at org.apache.spark.sql.types.StructType.apply(StructType.scala:414)
  at org.apache.spark.sql.catalyst.expressions.GetStructField.dataType(complexTypeExtractors.scala:108)
  at org.apache.spark.sql.catalyst.expressions.Alias.toAttribute(namedExpressions.scala:192)
  at org.apache.spark.sql.catalyst.plans.logical.Project.$anonfun$output$1(basicLogicalOperators.scala:63)
  at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
  at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
  at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
  at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
  at scala.collection.TraversableLike.map(TraversableLike.scala:238)
  at scala.collection.TraversableLike.map$(TraversableLike.scala:231)
  at scala.collection.AbstractTraversable.map(Traversable.scala:108)
  at org.apache.spark.sql.catalyst.plans.logical.Project.output(basicLogicalOperators.scala:63)
  at org.apache.spark.sql.catalyst.plans.logical.Union.$anonfun$output$3(basicLogicalOperators.scala:260)
  at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
  at scala.collection.immutable.List.foreach(List.scala:392)
  at scala.collection.TraversableLike.map(TraversableLike.scala:238)
  at scala.collection.TraversableLike.map$(TraversableLike.scala:231)
  at scala.collection.immutable.List.map(List.scala:298)
  at org.apache.spark.sql.catalyst.plans.logical.Union.output(basicLogicalOperators.scala:260)
  at org.apache.spark.sql.catalyst.plans.QueryPlan.outputSet$lzycompute(QueryPlan.scala:49)
  at org.apache.spark.sql.catalyst.plans.QueryPlan.outputSet(QueryPlan.scala:49)
  at org.apache.spark.sql.catalyst.optimizer.ColumnPruning$$anonfun$apply$8.applyOrElse(Optimizer.scala:747)
  at org.apache.spark.sql.catalyst.optimizer.ColumnPruning$$anonfun$apply$8.applyOrElse(Optimizer.scala:695)
  at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$1(TreeNode.scala:316)
  at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:72)
  at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:316)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDown(LogicalPlan.scala:29)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDown(AnalysisHelper.scala:171)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDown$(AnalysisHelper.scala:169)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDown(LogicalPlan.scala:29)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDown(LogicalPlan.scala:29)
  at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$3(TreeNode.scala:321)
  at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$mapChildren$1(TreeNode.scala:406)
  at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:242)
  at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:404)
  at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:357)
  at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:321)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDown(LogicalPlan.scala:29)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDown(AnalysisHelper.scala:171)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDown$(AnalysisHelper.scala:169)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDown(LogicalPlan.scala:29)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDown(LogicalPlan.scala:29)
  at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$3(TreeNode.scala:321)
  at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$mapChildren$1(TreeNode.scala:406)
  at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:242)
  at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:404)
  at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:357)
  at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:321)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDown(LogicalPlan.scala:29)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDown(AnalysisHelper.scala:171)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDown$(AnalysisHelper.scala:169)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDown(LogicalPlan.scala:29)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDown(LogicalPlan.scala:29)
  at org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:305)
  at org.apache.spark.sql.catalyst.optimizer.ColumnPruning$.apply(Optimizer.scala:695)
  at org.apache.spark.sql.catalyst.optimizer.ColumnPruning$.apply(Optimizer.scala:693)
  at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:215)
  at scala.collection.LinearSeqOptimized.foldLeft(LinearSeqOptimized.scala:126)
  at scala.collection.LinearSeqOptimized.foldLeft$(LinearSeqOptimized.scala:122)
  at scala.collection.immutable.List.foldLeft(List.scala:89)
  at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:212)
  at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:204)
  at scala.collection.immutable.List.foreach(List.scala:392)
  at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:204)
  at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:182)
  at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:88)
  at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:182)
  at org.apache.spark.sql.execution.QueryExecution.$anonfun$optimizedPlan$1(QueryExecution.scala:88)
  at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
  at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:144)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:771)
  at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:144)
  at org.apache.spark.sql.execution.QueryExecution.optimizedPlan$lzycompute(QueryExecution.scala:85)
  at org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:85)
  at org.apache.spark.sql.execution.QueryExecution.assertOptimized(QueryExecution.scala:96)
  at org.apache.spark.sql.execution.QueryExecution.executedPlan$lzycompute(QueryExecution.scala:114)
  at org.apache.spark.sql.execution.QueryExecution.executedPlan(QueryExecution.scala:111)
  at org.apache.spark.sql.execution.QueryExecution.$anonfun$simpleString$2(QueryExecution.scala:162)
  at org.apache.spark.sql.execution.ExplainUtils$.processPlan(ExplainUtils.scala:115)
  at org.apache.spark.sql.execution.QueryExecution.simpleString(QueryExecution.scala:162)
  at org.apache.spark.sql.execution.QueryExecution.org$apache$spark$sql$execution$QueryExecution$$explainString(QueryExecution.scala:207)
  at org.apache.spark.sql.execution.QueryExecution.explainString(QueryExecution.scala:176)
  at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:98)
  at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:163)
  at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:90)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:771)
  at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
  at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3703)
  at org.apache.spark.sql.Dataset.head(Dataset.scala:2740)
  at org.apache.spark.sql.Dataset.take(Dataset.scala:2947)
  at org.apache.spark.sql.Dataset.getRows(Dataset.scala:301)
  at org.apache.spark.sql.Dataset.showString(Dataset.scala:340)
  at org.apache.spark.sql.Dataset.show(Dataset.scala:827)
  at org.apache.spark.sql.Dataset.show(Dataset.scala:786)
  at org.apache.spark.sql.Dataset.show(Dataset.scala:795)
  ... 47 elided
{code}

  was:
If a nested field has different casing on two sides of the union, the resultant schema of the union will both fields in its schemaa

{code:java}
scala> val df1 = spark.range(1).withColumn("nested", struct(expr("id * 5 AS INNER")))
df1: org.apache.spark.sql.DataFrame = [id: bigint, nested: struct<INNER: bigint>]

val df2 = spark.range(10).withColumn("nested", struct(expr("id * 5 AS inner")))
df2: org.apache.spark.sql.DataFrame = [id: bigint, nested: struct<inner: bigint>]

scala> df1.union(df2).printSchema
root
 |-- id: long (nullable = false)
 |-- nested: struct (nullable = false)
 |    |-- INNER: long (nullable = false)
 |    |-- inner: long (nullable = false)
 {code}
This seems like a bug. I would expect that Spark SQL would either just union by index or if the user has requested {{unionByName}}, then it should matched fields case insensitively if {{spark.sql.caseSensitive}} is {{false}}.

However the output data only has one nested column
{code}
scala> df1.union(df2).show()
+---+------+
| id|nested|
+---+------+
|  0|   {0}|
|  1|   {5}|
|  0|   {0}|
|  1|   {5}|
+---+------+
{code}

Trying to project fields of {{nested}} throws an error:
{code}
scala> df1.union(df2).select("nested.*").show()
java.lang.ArrayIndexOutOfBoundsException: 1
  at org.apache.spark.sql.types.StructType.apply(StructType.scala:414)
  at org.apache.spark.sql.catalyst.expressions.GetStructField.dataType(complexTypeExtractors.scala:108)
  at org.apache.spark.sql.catalyst.expressions.Alias.toAttribute(namedExpressions.scala:192)
  at org.apache.spark.sql.catalyst.plans.logical.Project.$anonfun$output$1(basicLogicalOperators.scala:63)
  at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
  at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
  at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
  at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
  at scala.collection.TraversableLike.map(TraversableLike.scala:238)
  at scala.collection.TraversableLike.map$(TraversableLike.scala:231)
  at scala.collection.AbstractTraversable.map(Traversable.scala:108)
  at org.apache.spark.sql.catalyst.plans.logical.Project.output(basicLogicalOperators.scala:63)
  at org.apache.spark.sql.catalyst.plans.logical.Union.$anonfun$output$3(basicLogicalOperators.scala:260)
  at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
  at scala.collection.immutable.List.foreach(List.scala:392)
  at scala.collection.TraversableLike.map(TraversableLike.scala:238)
  at scala.collection.TraversableLike.map$(TraversableLike.scala:231)
  at scala.collection.immutable.List.map(List.scala:298)
  at org.apache.spark.sql.catalyst.plans.logical.Union.output(basicLogicalOperators.scala:260)
  at org.apache.spark.sql.catalyst.plans.QueryPlan.outputSet$lzycompute(QueryPlan.scala:49)
  at org.apache.spark.sql.catalyst.plans.QueryPlan.outputSet(QueryPlan.scala:49)
  at org.apache.spark.sql.catalyst.optimizer.ColumnPruning$$anonfun$apply$8.applyOrElse(Optimizer.scala:747)
  at org.apache.spark.sql.catalyst.optimizer.ColumnPruning$$anonfun$apply$8.applyOrElse(Optimizer.scala:695)
  at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$1(TreeNode.scala:316)
  at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:72)
  at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:316)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDown(LogicalPlan.scala:29)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDown(AnalysisHelper.scala:171)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDown$(AnalysisHelper.scala:169)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDown(LogicalPlan.scala:29)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDown(LogicalPlan.scala:29)
  at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$3(TreeNode.scala:321)
  at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$mapChildren$1(TreeNode.scala:406)
  at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:242)
  at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:404)
  at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:357)
  at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:321)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDown(LogicalPlan.scala:29)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDown(AnalysisHelper.scala:171)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDown$(AnalysisHelper.scala:169)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDown(LogicalPlan.scala:29)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDown(LogicalPlan.scala:29)
  at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$3(TreeNode.scala:321)
  at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$mapChildren$1(TreeNode.scala:406)
  at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:242)
  at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:404)
  at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:357)
  at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:321)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDown(LogicalPlan.scala:29)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDown(AnalysisHelper.scala:171)
  at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDown$(AnalysisHelper.scala:169)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDown(LogicalPlan.scala:29)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDown(LogicalPlan.scala:29)
  at org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:305)
  at org.apache.spark.sql.catalyst.optimizer.ColumnPruning$.apply(Optimizer.scala:695)
  at org.apache.spark.sql.catalyst.optimizer.ColumnPruning$.apply(Optimizer.scala:693)
  at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:215)
  at scala.collection.LinearSeqOptimized.foldLeft(LinearSeqOptimized.scala:126)
  at scala.collection.LinearSeqOptimized.foldLeft$(LinearSeqOptimized.scala:122)
  at scala.collection.immutable.List.foldLeft(List.scala:89)
  at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:212)
  at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:204)
  at scala.collection.immutable.List.foreach(List.scala:392)
  at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:204)
  at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:182)
  at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:88)
  at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:182)
  at org.apache.spark.sql.execution.QueryExecution.$anonfun$optimizedPlan$1(QueryExecution.scala:88)
  at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
  at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:144)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:771)
  at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:144)
  at org.apache.spark.sql.execution.QueryExecution.optimizedPlan$lzycompute(QueryExecution.scala:85)
  at org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:85)
  at org.apache.spark.sql.execution.QueryExecution.assertOptimized(QueryExecution.scala:96)
  at org.apache.spark.sql.execution.QueryExecution.executedPlan$lzycompute(QueryExecution.scala:114)
  at org.apache.spark.sql.execution.QueryExecution.executedPlan(QueryExecution.scala:111)
  at org.apache.spark.sql.execution.QueryExecution.$anonfun$simpleString$2(QueryExecution.scala:162)
  at org.apache.spark.sql.execution.ExplainUtils$.processPlan(ExplainUtils.scala:115)
  at org.apache.spark.sql.execution.QueryExecution.simpleString(QueryExecution.scala:162)
  at org.apache.spark.sql.execution.QueryExecution.org$apache$spark$sql$execution$QueryExecution$$explainString(QueryExecution.scala:207)
  at org.apache.spark.sql.execution.QueryExecution.explainString(QueryExecution.scala:176)
  at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:98)
  at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:163)
  at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:90)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:771)
  at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
  at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3703)
  at org.apache.spark.sql.Dataset.head(Dataset.scala:2740)
  at org.apache.spark.sql.Dataset.take(Dataset.scala:2947)
  at org.apache.spark.sql.Dataset.getRows(Dataset.scala:301)
  at org.apache.spark.sql.Dataset.showString(Dataset.scala:340)
  at org.apache.spark.sql.Dataset.show(Dataset.scala:827)
  at org.apache.spark.sql.Dataset.show(Dataset.scala:786)
  at org.apache.spark.sql.Dataset.show(Dataset.scala:795)
  ... 47 elided
{code} 


> Incorrect Unions of struct with mismatched field name case
> ----------------------------------------------------------
>
>                 Key: SPARK-36673
>                 URL: https://issues.apache.org/jira/browse/SPARK-36673
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 3.1.1, 3.2.0
>            Reporter: Shardul Mahadik
>            Priority: Major
>
> If a nested field has different casing on two sides of the union, the resultant schema of the union will both fields in its schemaa
> {code:java}
> scala> val df1 = spark.range(2).withColumn("nested", struct(expr("id * 5 AS INNER")))
> df1: org.apache.spark.sql.DataFrame = [id: bigint, nested: struct<INNER: bigint>]
> val df2 = spark.range(2).withColumn("nested", struct(expr("id * 5 AS inner")))
> df2: org.apache.spark.sql.DataFrame = [id: bigint, nested: struct<inner: bigint>]
> scala> df1.union(df2).printSchema
> root
>  |-- id: long (nullable = false)
>  |-- nested: struct (nullable = false)
>  |    |-- INNER: long (nullable = false)
>  |    |-- inner: long (nullable = false)
>  {code}
> This seems like a bug. I would expect that Spark SQL would either just union by index or if the user has requested {{unionByName}}, then it should matched fields case insensitively if {{spark.sql.caseSensitive}} is {{false}}.
> However the output data only has one nested column
> {code:java}
> scala> df1.union(df2).show()
> +---+------+
> | id|nested|
> +---+------+
> |  0|   {0}|
> |  1|   {5}|
> |  0|   {0}|
> |  1|   {5}|
> +---+------+
> {code}
> Trying to project fields of {{nested}} throws an error:
> {code:java}
> scala> df1.union(df2).select("nested.*").show()
> java.lang.ArrayIndexOutOfBoundsException: 1
>   at org.apache.spark.sql.types.StructType.apply(StructType.scala:414)
>   at org.apache.spark.sql.catalyst.expressions.GetStructField.dataType(complexTypeExtractors.scala:108)
>   at org.apache.spark.sql.catalyst.expressions.Alias.toAttribute(namedExpressions.scala:192)
>   at org.apache.spark.sql.catalyst.plans.logical.Project.$anonfun$output$1(basicLogicalOperators.scala:63)
>   at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
>   at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
>   at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
>   at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
>   at scala.collection.TraversableLike.map(TraversableLike.scala:238)
>   at scala.collection.TraversableLike.map$(TraversableLike.scala:231)
>   at scala.collection.AbstractTraversable.map(Traversable.scala:108)
>   at org.apache.spark.sql.catalyst.plans.logical.Project.output(basicLogicalOperators.scala:63)
>   at org.apache.spark.sql.catalyst.plans.logical.Union.$anonfun$output$3(basicLogicalOperators.scala:260)
>   at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
>   at scala.collection.immutable.List.foreach(List.scala:392)
>   at scala.collection.TraversableLike.map(TraversableLike.scala:238)
>   at scala.collection.TraversableLike.map$(TraversableLike.scala:231)
>   at scala.collection.immutable.List.map(List.scala:298)
>   at org.apache.spark.sql.catalyst.plans.logical.Union.output(basicLogicalOperators.scala:260)
>   at org.apache.spark.sql.catalyst.plans.QueryPlan.outputSet$lzycompute(QueryPlan.scala:49)
>   at org.apache.spark.sql.catalyst.plans.QueryPlan.outputSet(QueryPlan.scala:49)
>   at org.apache.spark.sql.catalyst.optimizer.ColumnPruning$$anonfun$apply$8.applyOrElse(Optimizer.scala:747)
>   at org.apache.spark.sql.catalyst.optimizer.ColumnPruning$$anonfun$apply$8.applyOrElse(Optimizer.scala:695)
>   at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$1(TreeNode.scala:316)
>   at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:72)
>   at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:316)
>   at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDown(LogicalPlan.scala:29)
>   at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDown(AnalysisHelper.scala:171)
>   at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDown$(AnalysisHelper.scala:169)
>   at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDown(LogicalPlan.scala:29)
>   at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDown(LogicalPlan.scala:29)
>   at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$3(TreeNode.scala:321)
>   at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$mapChildren$1(TreeNode.scala:406)
>   at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:242)
>   at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:404)
>   at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:357)
>   at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:321)
>   at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDown(LogicalPlan.scala:29)
>   at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDown(AnalysisHelper.scala:171)
>   at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDown$(AnalysisHelper.scala:169)
>   at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDown(LogicalPlan.scala:29)
>   at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDown(LogicalPlan.scala:29)
>   at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$3(TreeNode.scala:321)
>   at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$mapChildren$1(TreeNode.scala:406)
>   at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:242)
>   at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:404)
>   at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:357)
>   at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:321)
>   at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDown(LogicalPlan.scala:29)
>   at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDown(AnalysisHelper.scala:171)
>   at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDown$(AnalysisHelper.scala:169)
>   at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDown(LogicalPlan.scala:29)
>   at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDown(LogicalPlan.scala:29)
>   at org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:305)
>   at org.apache.spark.sql.catalyst.optimizer.ColumnPruning$.apply(Optimizer.scala:695)
>   at org.apache.spark.sql.catalyst.optimizer.ColumnPruning$.apply(Optimizer.scala:693)
>   at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:215)
>   at scala.collection.LinearSeqOptimized.foldLeft(LinearSeqOptimized.scala:126)
>   at scala.collection.LinearSeqOptimized.foldLeft$(LinearSeqOptimized.scala:122)
>   at scala.collection.immutable.List.foldLeft(List.scala:89)
>   at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:212)
>   at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:204)
>   at scala.collection.immutable.List.foreach(List.scala:392)
>   at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:204)
>   at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:182)
>   at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:88)
>   at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:182)
>   at org.apache.spark.sql.execution.QueryExecution.$anonfun$optimizedPlan$1(QueryExecution.scala:88)
>   at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
>   at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:144)
>   at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:771)
>   at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:144)
>   at org.apache.spark.sql.execution.QueryExecution.optimizedPlan$lzycompute(QueryExecution.scala:85)
>   at org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:85)
>   at org.apache.spark.sql.execution.QueryExecution.assertOptimized(QueryExecution.scala:96)
>   at org.apache.spark.sql.execution.QueryExecution.executedPlan$lzycompute(QueryExecution.scala:114)
>   at org.apache.spark.sql.execution.QueryExecution.executedPlan(QueryExecution.scala:111)
>   at org.apache.spark.sql.execution.QueryExecution.$anonfun$simpleString$2(QueryExecution.scala:162)
>   at org.apache.spark.sql.execution.ExplainUtils$.processPlan(ExplainUtils.scala:115)
>   at org.apache.spark.sql.execution.QueryExecution.simpleString(QueryExecution.scala:162)
>   at org.apache.spark.sql.execution.QueryExecution.org$apache$spark$sql$execution$QueryExecution$$explainString(QueryExecution.scala:207)
>   at org.apache.spark.sql.execution.QueryExecution.explainString(QueryExecution.scala:176)
>   at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:98)
>   at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:163)
>   at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:90)
>   at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:771)
>   at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
>   at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3703)
>   at org.apache.spark.sql.Dataset.head(Dataset.scala:2740)
>   at org.apache.spark.sql.Dataset.take(Dataset.scala:2947)
>   at org.apache.spark.sql.Dataset.getRows(Dataset.scala:301)
>   at org.apache.spark.sql.Dataset.showString(Dataset.scala:340)
>   at org.apache.spark.sql.Dataset.show(Dataset.scala:827)
>   at org.apache.spark.sql.Dataset.show(Dataset.scala:786)
>   at org.apache.spark.sql.Dataset.show(Dataset.scala:795)
>   ... 47 elided
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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