You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2021/04/23 03:30:13 UTC

[GitHub] [spark] allisonwang-db commented on a change in pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

allisonwang-db commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r618914763



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -133,82 +140,84 @@ object NestedColumnAliasing {
   }
 
   /**
-   * Return root references that are individually accessed as a whole, and `GetStructField`s
-   * or `GetArrayStructField`s which on top of other `ExtractValue`s or special expressions.
-   * Check `SelectedField` to see which expressions should be listed here.
+   * Check [[SelectedField]] to see which expressions should be listed here.
    */
-  private def collectRootReferenceAndExtractValue(e: Expression): Seq[Expression] = e match {
-    case _: AttributeReference => Seq(e)
-    case GetStructField(_: ExtractValue | _: AttributeReference, _, _) => Seq(e)
+  private def isSelectedField(e: Expression): Boolean = e match {
+    case GetStructField(_: ExtractValue | _: AttributeReference, _, _) => true
     case GetArrayStructFields(_: MapValues |
                               _: MapKeys |
                               _: ExtractValue |
-                              _: AttributeReference, _, _, _, _) => Seq(e)
-    case es if es.children.nonEmpty => es.children.flatMap(collectRootReferenceAndExtractValue)
+                              _: AttributeReference, _, _, _, _) => true
+    case _ => false
+  }
+
+  /**
+   * Return root references that are individually accessed.
+   */
+  private def collectAttributeReference(e: Expression): Seq[AttributeReference] = e match {
+    case a: AttributeReference => Seq(a)
+    case g if isSelectedField(g) => Seq.empty
+    case es if es.children.nonEmpty => es.children.flatMap(collectAttributeReference)
     case _ => Seq.empty
   }
 
   /**
-   * Return two maps in order to replace nested fields to aliases.
-   *
-   * If `exclusiveAttrs` is given, any nested field accessors of these attributes
-   * won't be considered in nested fields aliasing.
-   *
-   * 1. ExtractValue -> Alias: A new alias is created for each nested field.
-   * 2. ExprId -> Seq[Alias]: A reference attribute has multiple aliases pointing it.
+   * Return [[GetStructField]] or [[GetArrayStructFields]] on top of other [[ExtractValue]]s
+   * or special expressions.
    */
-  def getAliasSubMap(exprList: Seq[Expression], exclusiveAttrs: Seq[Attribute] = Seq.empty)
-    : Option[(Map[ExtractValue, Alias], Map[ExprId, Seq[Alias]])] = {
-    val (nestedFieldReferences, otherRootReferences) =
-      exprList.flatMap(collectRootReferenceAndExtractValue).partition {
-        case _: ExtractValue => true
-        case _ => false
-      }
+  private def collectExtractValue(e: Expression): Seq[ExtractValue] = e match {
+    case g if isSelectedField(g) => Seq(g.asInstanceOf[ExtractValue])
+    case es if es.children.nonEmpty => es.children.flatMap(collectExtractValue)
+    case _ => Seq.empty
+  }
 
-    // Note that when we group by extractors with their references, we should remove
-    // cosmetic variations.
+  /**
+   * Creates a map from root [[Attribute]]s to non-redundant nested [[ExtractValue]]s in the
+   * case that only a subset of the nested fields are used.
+   * Nested field accessors of `exclusiveAttrs` are not considered in nested fields aliasing.
+   */
+  def getAttributeToExtractValues(
+      exprList: Seq[Expression],
+      exclusiveAttrs: Seq[Attribute]): Option[Map[Attribute, Seq[ExtractValue]]] = {
+
+    val nestedFieldReferences = exprList.flatMap(collectExtractValue)
+    val otherRootReferences = exprList.flatMap(collectAttributeReference)
     val exclusiveAttrSet = AttributeSet(exclusiveAttrs ++ otherRootReferences)
-    val aliasSub = nestedFieldReferences.asInstanceOf[Seq[ExtractValue]]
+
+    // Remove cosmetic variations when we group extractors by their references
+    val attributeToExtractValues = nestedFieldReferences
       .filter(!_.references.subsetOf(exclusiveAttrSet))
       .groupBy(_.references.head.canonicalized.asInstanceOf[Attribute])
-      .flatMap { case (attr, nestedFields: Seq[ExtractValue]) =>
-        // Remove redundant `ExtractValue`s if they share the same parent nest field.
+      .flatMap { case (attr: Attribute, nestedFields: Seq[ExtractValue]) =>
+        // Remove redundant [[ExtractValue]]s if they share the same parent nest field.
         // For example, when `a.b` and `a.b.c` are in project list, we only need to alias `a.b`.
-        // We only need to deal with two `ExtractValue`: `GetArrayStructFields` and
-        // `GetStructField`. Please refer to the method `collectRootReferenceAndExtractValue`.
         val dedupNestedFields = nestedFields.filter {
-          case e @ (_: GetStructField | _: GetArrayStructFields) =>
-            val child = e.children.head
+          // See [[collectExtractValue]]: we only need to deal with [[GetArrayStructFields]] and
+          // [[GetStructField]]
+          case GetStructField(child, _, _) =>

Review comment:
       Ah I meant this
   ```scala
   case e @ (_: GetStructField | _: GetArrayStructFields) =>
     val child = e.children.head
     nestedFields.forall(f => child.find(_.semanticEquals(f)).isEmpty)
   ```
   to this
   ```scala
   case GetStructField(child, _, _) =>
     nestedFields.forall(f => child.find(_.semanticEquals(f)).isEmpty)
   case GetArrayStructFields(child, _, _, _, _) =>
     nestedFields.forall(f => child.find(_.semanticEquals(f)).isEmpty)
   ```




-- 
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:
users@infra.apache.org



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