viirya commented on a change in pull request #29587:
URL: https://github.com/apache/spark/pull/29587#discussion_r487457983



##########
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveUnion.scala
##########
@@ -17,29 +17,101 @@
 
 package org.apache.spark.sql.catalyst.analysis
 
+import scala.collection.mutable
+
 import org.apache.spark.sql.AnalysisException
-import org.apache.spark.sql.catalyst.expressions.{Alias, Literal}
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, 
Expression, Literal, NamedExpression, WithFields}
 import org.apache.spark.sql.catalyst.optimizer.CombineUnions
 import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
Union}
 import org.apache.spark.sql.catalyst.rules.Rule
 import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
 import org.apache.spark.sql.util.SchemaUtils
 
 /**
  * Resolves different children of Union to a common set of columns.
  */
 object ResolveUnion extends Rule[LogicalPlan] {
-  private def unionTwoSides(
+  /**
+   * Adds missing fields recursively into given `col` expression, based on the 
target `StructType`.
+   * For example, given `col` as "a struct<a:int, b:int>, b int" and `target` 
as
+   * "a struct<a:int, b:int, c:long>, b int, c string", this method should add 
`a.c` and `c` to
+   * `col` expression.
+   */
+  private def addFields(col: NamedExpression, target: StructType): 
Option[Expression] = {
+    assert(col.dataType.isInstanceOf[StructType], "Only support StructType.")
+
+    val resolver = SQLConf.get.resolver
+    val missingFields =
+      StructType.findMissingFields(col.dataType.asInstanceOf[StructType], 
target, resolver)
+    if (missingFields.isEmpty) {
+      None
+    } else {
+      missingFields.map(s => addFieldsInto(col, "", s.fields))
+    }
+  }
+
+  /**
+   * Adds missing fields recursively into given `col` expression. The missing 
fields are given
+   * in `fields`. For example, given `col` as "a struct<a:int, b:int>, b int", 
and `fields` is
+   * "a struct<c:long>, c string". This method will add a nested `a.c` field 
and a top-level
+   * `c` field to `col` and fill null values for them.
+   */
+  private def addFieldsInto(col: Expression, base: String, fields: 
Seq[StructField]): Expression = {
+    fields.foldLeft(col) { case (currCol, field) =>
+      field.dataType match {
+        case st: StructType =>
+          val resolver = SQLConf.get.resolver
+          val colField = currCol.dataType.asInstanceOf[StructType]
+            .find(f => resolver(f.name, field.name))
+          if (colField.isEmpty) {
+            // The whole struct is missing. Add a null.
+            WithFields(currCol, s"$base${field.name}", Literal(null, st),
+              sortOutputColumns = true)
+          } else {
+            addFieldsInto(currCol, s"$base${field.name}.", st.fields)
+          }
+        case dt =>
+          // We need to sort columns in result, because we might add another 
column in other side.
+          // E.g., we want to union two structs "a int, b long" and "a int, c 
string".
+          // If we don't sort, we will have "a int, b long, c string" and "a 
int, c string, b long",
+          // which are not compatible.
+          WithFields(currCol, s"$base${field.name}", Literal(null, dt),
+            sortOutputColumns = true)
+      }
+    }
+  }
+
+  private def compareAndAddFields(
       left: LogicalPlan,
       right: LogicalPlan,
-      allowMissingCol: Boolean): LogicalPlan = {
+      allowMissingCol: Boolean): (Seq[NamedExpression], Seq[NamedExpression]) 
= {
     val resolver = SQLConf.get.resolver
     val leftOutputAttrs = left.output
     val rightOutputAttrs = right.output
 
-    // Builds a project list for `right` based on `left` output names
+    val aliased = mutable.ArrayBuffer.empty[Attribute]
+
     val rightProjectList = leftOutputAttrs.map { lattr =>
-      rightOutputAttrs.find { rattr => resolver(lattr.name, rattr.name) 
}.getOrElse {
+      val found = rightOutputAttrs.find { rattr => resolver(lattr.name, 
rattr.name) }
+      if (found.isDefined) {
+        val foundDt = found.get.dataType
+        (foundDt, lattr.dataType) match {
+          case (source: StructType, target: StructType)
+              if allowMissingCol && !source.sameType(target) =>

Review comment:
       I see. I will add more comments explaining this.




----------------------------------------------------------------
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.

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

Reply via email to