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/22 17:20:16 UTC

[GitHub] [spark] karenfeng opened a new pull request #32301: [SPARK-35194] Refactor nested column aliasing for readability

karenfeng opened a new pull request #32301:
URL: https://github.com/apache/spark/pull/32301


   ### What changes were proposed in this pull request?
   
   Refactors `NestedColumnAliasing` and `GeneratorNestedColumnAliasing` for readability.
   
   ### Why are the changes needed?
   
   Improves readability for future maintenance.
   
   ### Does this PR introduce _any_ user-facing change?
   
   No.
   
   ### How was this patch tested?
   
   Existing tests.


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


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

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r622812276



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -17,71 +17,148 @@
 
 package org.apache.spark.sql.catalyst.optimizer
 
+import scala.collection.mutable
+
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 
 /**
- * This aims to handle a nested column aliasing pattern inside the `ColumnPruning` optimizer rule.
- * If a project or its child references to nested fields, and not all the fields
- * in a nested attribute are used, we can substitute them by alias attributes; then a project
- * of the nested fields as aliases on the children of the child will be created.
+ * This aims to handle a nested column aliasing pattern inside the [[ColumnPruning]] optimizer rule.
+ * If:
+ * - A [[Project]] or its child references nested fields
+ * - Not all of the fields in a nested attribute are used
+ * Then:
+ * - Substitute the nested field references with alias attributes
+ * - Add grandchild [[Project]]s transforming the nested fields to aliases
+ *
+ * Example 1: Project
+ * ------------------
+ * Before:
+ * +- Project [concat_ws(s#0.a, s#0.b) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [concat_ws(_gen_alias_2#2, _gen_alias_3#3) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _gen_alias_2#2, s#0.b AS _gen_alias_3#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * Example 2: Project above Filter
+ * -------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1]
+ *   +- Filter (length(s#0.b) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_gen_alias_2#2 AS s.a#1]
+ *   +- Filter (length(_gen_alias_3#3) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- Project [s#0.a AS _gen_alias_2#2, s#0.b AS _gen_alias_3#3]
+ *           +- LocalRelation <empty>, [s#0]
+ *
+ * Example 3: Nested columns in nested columns
+ * -------------------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1, s#0.a.a1 AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_gen_alias_3#3 AS s.a#1, _gen_alias_3#3.name AS s.a.a1#2]

Review comment:
       `_gen_alias_3#3.name`? `_gen_alias_3#3.a1`?




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


[GitHub] [spark] karenfeng edited a comment on pull request #32301: [SPARK-35194] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
karenfeng edited a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-825042480


   @viirya, this will either block/be blocked by your [ongoing PR](https://github.com/apache/spark/pull/31966). Let me know what you think - I think this will improve the readability.


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-842737668


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/138642/
   


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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r638921071



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -149,67 +234,52 @@ object NestedColumnAliasing {
   }
 
   /**
-   * 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.
+   * Creates a map from root [[Attribute]]s to non-redundant nested [[ExtractValue]]s.
+   * Nested field accessors of `exclusiveAttrs` are not considered in nested fields aliasing.
    */
-  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
+  def getAttributeToExtractValues(
+      exprList: Seq[Expression],
+      exclusiveAttrs: Seq[Attribute]): Map[Attribute, Seq[ExtractValue]] = {
+
+    val nestedFieldReferences = new mutable.ArrayBuffer[ExtractValue]()
+    val otherRootReferences = new mutable.ArrayBuffer[AttributeReference]()
+    exprList.foreach { e =>
+      collectRootReferenceAndExtractValue(e).foreach {
+        case ev: ExtractValue => nestedFieldReferences.append(ev)

Review comment:
       shall we add an assert here to make sure `ev` only referecnes to one attribute? To make sure https://github.com/apache/spark/pull/32301/files#diff-966b594a855ef6f4ee560a4b156755243a1734bb1dfb2f6f4bc92a2f7ba0c2c3R257 is reasonable.




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


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

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r640174485



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -17,71 +17,155 @@
 
 package org.apache.spark.sql.catalyst.optimizer
 
+import scala.collection.mutable
+
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 
 /**
- * This aims to handle a nested column aliasing pattern inside the `ColumnPruning` optimizer rule.
- * If a project or its child references to nested fields, and not all the fields
- * in a nested attribute are used, we can substitute them by alias attributes; then a project
- * of the nested fields as aliases on the children of the child will be created.
+ * This aims to handle a nested column aliasing pattern inside the [[ColumnPruning]] optimizer rule.
+ * If:
+ * - A [[Project]] or its child references nested fields
+ * - Not all of the fields in a nested attribute are used
+ * Then:
+ * - Substitute the nested field references with alias attributes
+ * - Add grandchild [[Project]]s transforming the nested fields to aliases

Review comment:
       Yes, this is how nested column pruning works now. The added Project with nested column references will be push down through other nodes until Scan.
   
   If there are any nodes we cannot push through, then nested column pruning doesn't work.




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


[GitHub] [spark] AmplabJenkins commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-842689667






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


[GitHub] [spark] cloud-fan closed pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
cloud-fan closed pull request #32301:
URL: https://github.com/apache/spark/pull/32301


   


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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-842679062


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/43164/
   


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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-849327556


   **[Test build #138997 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138997/testReport)** for PR 32301 at commit [`83e2611`](https://github.com/apache/spark/commit/83e2611f47b86b1b35ba4862fc183e5ce6485a0f).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


[GitHub] [spark] viirya commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
viirya commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-849407388


   > java.lang.AssertionError: assertion failed: struct(col1, 1, col2, struct(col1, a, col2, 1.5)).col2.col1 should have one reference, but found {}
   
   Oh, it doesn't have more than one references, but has no reference...
   
   


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-828072391


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42531/
   


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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-842622211


   Kubernetes integration test unable to build dist.
   
   exiting with code: 1
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/43162/
   


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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-828144237


   **[Test build #138012 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138012/testReport)** for PR 32301 at commit [`9d41de6`](https://github.com/apache/spark/commit/9d41de6b6c793a414133f36d4113689861e47610).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


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

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r640182803



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -149,67 +234,52 @@ object NestedColumnAliasing {
   }
 
   /**
-   * 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.
+   * Creates a map from root [[Attribute]]s to non-redundant nested [[ExtractValue]]s.
+   * Nested field accessors of `exclusiveAttrs` are not considered in nested fields aliasing.
    */
-  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
+  def getAttributeToExtractValues(
+      exprList: Seq[Expression],
+      exclusiveAttrs: Seq[Attribute]): Map[Attribute, Seq[ExtractValue]] = {
+
+    val nestedFieldReferences = new mutable.ArrayBuffer[ExtractValue]()
+    val otherRootReferences = new mutable.ArrayBuffer[AttributeReference]()
+    exprList.foreach { e =>
+      collectRootReferenceAndExtractValue(e).foreach {
+        case ev: ExtractValue => nestedFieldReferences.append(ev)

Review comment:
       `struct(col1, 1, col2, struct(col1, a, col2, 1.5)).col2.col1`? seems there is only `a`? Oh, `a` is duplicate there.
   
   Can we do distinct?
   
   




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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-825951124


   **[Test build #137871 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137871/testReport)** for PR 32301 at commit [`a95360a`](https://github.com/apache/spark/commit/a95360a69907fc6fbb047490f4c448dc69659321).


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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r638911496



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -17,71 +17,155 @@
 
 package org.apache.spark.sql.catalyst.optimizer
 
+import scala.collection.mutable
+
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 
 /**
- * This aims to handle a nested column aliasing pattern inside the `ColumnPruning` optimizer rule.
- * If a project or its child references to nested fields, and not all the fields
- * in a nested attribute are used, we can substitute them by alias attributes; then a project
- * of the nested fields as aliases on the children of the child will be created.
+ * This aims to handle a nested column aliasing pattern inside the [[ColumnPruning]] optimizer rule.
+ * If:
+ * - A [[Project]] or its child references nested fields
+ * - Not all of the fields in a nested attribute are used
+ * Then:
+ * - Substitute the nested field references with alias attributes
+ * - Add grandchild [[Project]]s transforming the nested fields to aliases
+ *
+ * Example 1: Project
+ * ------------------
+ * Before:
+ * +- Project [concat_ws(s#0.a, s#0.b) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [concat_ws(_extract_a#2, _extract_b#3) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _extract_a#2, s#0.b AS _extract_b#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * Example 2: Project above Filter
+ * -------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1]
+ *   +- Filter (length(s#0.b) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_extract_a#2 AS s.a#1]
+ *   +- Filter (length(_extract_b#3) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- Project [s#0.a AS _extract_a#2, s#0.b AS _extract_b#3]
+ *           +- LocalRelation <empty>, [s#0]
+ *
+ * Example 3: Nested fields with referenced parents
+ * ------------------------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1, s#0.a.a1 AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_extract_a#3 AS s.a#1, _extract_a#3.name AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _extract_a#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * The schema of the datasource relation will be pruned in the [[SchemaPruning]] optimizer rule.
  */
 object NestedColumnAliasing {
 
   def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match {
     /**
      * This pattern is needed to support [[Filter]] plan cases like
-     * [[Project]]->[[Filter]]->listed plan in `canProjectPushThrough` (e.g., [[Window]]).
-     * The reason why we don't simply add [[Filter]] in `canProjectPushThrough` is that
+     * [[Project]]->[[Filter]]->listed plan in [[canProjectPushThrough]] (e.g., [[Window]]).
+     * The reason why we don't simply add [[Filter]] in [[canProjectPushThrough]] is that
      * the optimizer can hit an infinite loop during the [[PushDownPredicates]] rule.
      */
-    case Project(projectList, Filter(condition, child))
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
-      val exprCandidatesToPrune = projectList ++ Seq(condition) ++ child.expressions
-      getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases)
-      }
+    case Project(projectList, Filter(condition, child)) if
+        SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
+      rewritePlanIfSubsetFieldsUsed(
+        plan, projectList ++ Seq(condition) ++ child.expressions, child.producedAttributes.toSeq)
 
-    case Project(projectList, child)
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
-      val exprCandidatesToPrune = projectList ++ child.expressions
-      getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases)
-      }
+    case Project(projectList, child) if
+        SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>

Review comment:
       ditto




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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r638943696



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -227,137 +297,114 @@ object NestedColumnAliasing {
 }
 
 /**
- * This prunes unnecessary nested columns from `Generate` and optional `Project` on top
- * of it.
+ * This prunes unnecessary nested columns from [[Generate]], or [[Project]] -> [[Generate]]
  */
 object GeneratorNestedColumnAliasing {
-  // Partitions `attrToAliases` based on whether the attribute is in Generator's output.
-  private def aliasesOnGeneratorOutput(
-      attrToAliases: Map[ExprId, Seq[Alias]],
-      generatorOutput: Seq[Attribute]) = {
-    val generatorOutputExprId = generatorOutput.map(_.exprId)
-    attrToAliases.partition { k =>
-      generatorOutputExprId.contains(k._1)
-    }
-  }
-
-  // Partitions `nestedFieldToAlias` based on whether the attribute of nested field extractor
-  // is in Generator's output.
-  private def nestedFieldOnGeneratorOutput(
-      nestedFieldToAlias: Map[ExtractValue, Alias],
-      generatorOutput: Seq[Attribute]) = {
-    val generatorOutputSet = AttributeSet(generatorOutput)
-    nestedFieldToAlias.partition { pair =>
-      pair._1.references.subsetOf(generatorOutputSet)
-    }
-  }
-
   def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match {
     // Either `nestedPruningOnExpressions` or `nestedSchemaPruningEnabled` is enabled, we
     // need to prune nested columns through Project and under Generate. The difference is
     // when `nestedSchemaPruningEnabled` is on, nested columns will be pruned further at
     // file format readers if it is supported.
     case Project(projectList, g: Generate) if (SQLConf.get.nestedPruningOnExpressions ||
-        SQLConf.get.nestedSchemaPruningEnabled) && canPruneGenerator(g.generator) =>
+      SQLConf.get.nestedSchemaPruningEnabled) && canPruneGenerator(g.generator) =>
       // On top on `Generate`, a `Project` that might have nested column accessors.
       // We try to get alias maps for both project list and generator's children expressions.
-      val exprsToPrune = projectList ++ g.generator.children
-      NestedColumnAliasing.getAliasSubMap(exprsToPrune).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          val (nestedFieldsOnGenerator, nestedFieldsNotOnGenerator) =
-            nestedFieldOnGeneratorOutput(nestedFieldToAlias, g.qualifiedGeneratorOutput)
-          val (attrToAliasesOnGenerator, attrToAliasesNotOnGenerator) =
-            aliasesOnGeneratorOutput(attrToAliases, g.qualifiedGeneratorOutput)
-
-          // Push nested column accessors through `Generator`.
-          // Defer updating `Generate.unrequiredChildIndex` to next round of `ColumnPruning`.
-          val newChild = NestedColumnAliasing.replaceWithAliases(g,
-            nestedFieldsNotOnGenerator, attrToAliasesNotOnGenerator)
-          val pushedThrough = Project(NestedColumnAliasing
-            .getNewProjectList(projectList, nestedFieldsNotOnGenerator), newChild)
-
-          // If the generator output is `ArrayType`, we cannot push through the extractor.
-          // It is because we don't allow field extractor on two-level array,
-          // i.e., attr.field when attr is a ArrayType(ArrayType(...)).
-          // Similarily, we also cannot push through if the child of generator is `MapType`.
-          g.generator.children.head.dataType match {
-            case _: MapType => return Some(pushedThrough)
-            case ArrayType(_: ArrayType, _) => return Some(pushedThrough)
-            case _ =>
-          }
-
-          // Pruning on `Generator`'s output. We only process single field case.
-          // For multiple field case, we cannot directly move field extractor into
-          // the generator expression. A workaround is to re-construct array of struct
-          // from multiple fields. But it will be more complicated and may not worth.
-          // TODO(SPARK-34956): support multiple fields.
-          if (nestedFieldsOnGenerator.size > 1 || nestedFieldsOnGenerator.isEmpty) {
-            pushedThrough
-          } else {
-            // Only one nested column accessor.
-            // E.g., df.select(explode($"items").as("item")).select($"item.a")
-            pushedThrough match {
-              case p @ Project(_, newG: Generate) =>
-                // Replace the child expression of `ExplodeBase` generator with
-                // nested column accessor.
-                // E.g., df.select(explode($"items").as("item")).select($"item.a") =>
-                //       df.select(explode($"items.a").as("item.a"))
-                val rewrittenG = newG.transformExpressions {
-                  case e: ExplodeBase =>
-                    val extractor = nestedFieldsOnGenerator.head._1.transformUp {
-                      case _: Attribute =>
-                        e.child
-                      case g: GetStructField =>
-                        ExtractValue(g.child, Literal(g.extractFieldName), SQLConf.get.resolver)
-                    }
-                    e.withNewChildren(Seq(extractor))
-                }
+      val attrToExtractValues = NestedColumnAliasing.getAttributeToExtractValues(
+        projectList ++ g.generator.children, Seq.empty)
+      if (attrToExtractValues.isEmpty) {
+        return None
+      }
+      val generatorOutputSet = AttributeSet(g.qualifiedGeneratorOutput)
+      val (attrToExtractValuesOnGenerator, attrToExtractValuesNotOnGenerator) =
+        attrToExtractValues.partition { case (attr, _) =>
+          attr.references.subsetOf(generatorOutputSet) }
+
+      val pushedThrough = NestedColumnAliasing.rewritePlanWithAliases(
+        plan, attrToExtractValuesNotOnGenerator)
+
+      // If the generator output is `ArrayType`, we cannot push through the extractor.
+      // It is because we don't allow field extractor on two-level array,
+      // i.e., attr.field when attr is a ArrayType(ArrayType(...)).
+      // Similarily, we also cannot push through if the child of generator is `MapType`.
+      g.generator.children.head.dataType match {
+        case _: MapType => return Some(pushedThrough)
+        case ArrayType(_: ArrayType, _) => return Some(pushedThrough)
+        case _ =>
+      }
 
-                // As we change the child of the generator, its output data type must be updated.
-                val updatedGeneratorOutput = rewrittenG.generatorOutput
-                  .zip(rewrittenG.generator.elementSchema.toAttributes)
-                  .map { case (oldAttr, newAttr) =>
-                  newAttr.withExprId(oldAttr.exprId).withName(oldAttr.name)
-                }
-                assert(updatedGeneratorOutput.length == rewrittenG.generatorOutput.length,
-                  "Updated generator output must have the same length " +
-                    "with original generator output.")
-                val updatedGenerate = rewrittenG.copy(generatorOutput = updatedGeneratorOutput)
-
-                // Replace nested column accessor with generator output.
-                p.withNewChildren(Seq(updatedGenerate)).transformExpressions {
-                  case f: ExtractValue if nestedFieldsOnGenerator.contains(f) =>
-                    updatedGenerate.output
-                      .find(a => attrToAliasesOnGenerator.contains(a.exprId))
-                      .getOrElse(f)
+      // Pruning on `Generator`'s output. We only process single field case.
+      // For multiple field case, we cannot directly move field extractor into
+      // the generator expression. A workaround is to re-construct array of struct
+      // from multiple fields. But it will be more complicated and may not worth.
+      // TODO(SPARK-34956): support multiple fields.
+      val nestedFieldsOnGenerator = attrToExtractValuesOnGenerator.values.flatten.toSet
+      if (nestedFieldsOnGenerator.size > 1 || nestedFieldsOnGenerator.isEmpty) {
+        Some(pushedThrough)
+      } else {
+        // Only one nested column accessor.

Review comment:
       I didn't carefully review the code in the else branch. I assume it's the same as the previous code?




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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-849270337


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/43516/
   


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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-829069238


   **[Test build #138067 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138067/testReport)** for PR 32301 at commit [`d74b569`](https://github.com/apache/spark/commit/d74b5698616df1410fdc8cfb0e5db991ba84d321).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-842546164






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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-826025512


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137871/
   


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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-828051767


   **[Test build #138012 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138012/testReport)** for PR 32301 at commit [`9d41de6`](https://github.com/apache/spark/commit/9d41de6b6c793a414133f36d4113689861e47610).


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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r620136649



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -133,83 +204,77 @@ 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

Review comment:
       Not related to this PR: I don't get the reason to match `ExtractValue`. For `GetStructField(GetArrayItem(Attribute, index), fieldName)`, how can the data source support it?




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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-825982840






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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r639948954



##########
File path: sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala
##########
@@ -23,6 +23,10 @@ package org.apache.spark.sql.catalyst.expressions
  * of the name, or the expected nullability).
  */
 object AttributeMap {
+  def apply[A](kvs: Map[Attribute, A]): AttributeMap[A] = {

Review comment:
       Ah got it, this is more efficient




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


[GitHub] [spark] viirya commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
viirya commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-849906469


   > @viirya - in the case that the number of references is `!=1`, should we exclude the `ExtractValue` from `nestedFieldReferences`?
   
   I think so, it should be safer approach to exclude them from pruning candidates.


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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-826022450


   **[Test build #137871 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137871/testReport)** for PR 32301 at commit [`a95360a`](https://github.com/apache/spark/commit/a95360a69907fc6fbb047490f4c448dc69659321).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


[GitHub] [spark] AmplabJenkins commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-828155775


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/138012/
   


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


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

Posted by GitBox <gi...@apache.org>.
karenfeng commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r619490311



##########
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 just found `e.children.head` confusing. I'll revert this, I don't have that strong of an opinion.




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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-828639757






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


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

Posted by GitBox <gi...@apache.org>.
karenfeng commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r631957610



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -17,71 +17,148 @@
 
 package org.apache.spark.sql.catalyst.optimizer
 
+import scala.collection.mutable
+
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 
 /**
- * This aims to handle a nested column aliasing pattern inside the `ColumnPruning` optimizer rule.
- * If a project or its child references to nested fields, and not all the fields
- * in a nested attribute are used, we can substitute them by alias attributes; then a project
- * of the nested fields as aliases on the children of the child will be created.
+ * This aims to handle a nested column aliasing pattern inside the [[ColumnPruning]] optimizer rule.
+ * If:
+ * - A [[Project]] or its child references nested fields
+ * - Not all of the fields in a nested attribute are used
+ * Then:
+ * - Substitute the nested field references with alias attributes
+ * - Add grandchild [[Project]]s transforming the nested fields to aliases
+ *
+ * Example 1: Project
+ * ------------------
+ * Before:
+ * +- Project [concat_ws(s#0.a, s#0.b) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [concat_ws(_gen_alias_2#2, _gen_alias_3#3) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _gen_alias_2#2, s#0.b AS _gen_alias_3#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * Example 2: Project above Filter
+ * -------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1]
+ *   +- Filter (length(s#0.b) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_gen_alias_2#2 AS s.a#1]
+ *   +- Filter (length(_gen_alias_3#3) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- Project [s#0.a AS _gen_alias_2#2, s#0.b AS _gen_alias_3#3]
+ *           +- LocalRelation <empty>, [s#0]
+ *
+ * Example 3: Nested columns in nested columns
+ * -------------------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1, s#0.a.a1 AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_gen_alias_3#3 AS s.a#1, _gen_alias_3#3.name AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _gen_alias_3#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * The schema of the datasource relation will be pruned in the [[SchemaPruning]] optimizer rule.
  */
 object NestedColumnAliasing {
 
   def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match {
     /**
      * This pattern is needed to support [[Filter]] plan cases like
-     * [[Project]]->[[Filter]]->listed plan in `canProjectPushThrough` (e.g., [[Window]]).
-     * The reason why we don't simply add [[Filter]] in `canProjectPushThrough` is that
+     * [[Project]]->[[Filter]]->listed plan in [[canProjectPushThrough]] (e.g., [[Window]]).
+     * The reason why we don't simply add [[Filter]] in [[canProjectPushThrough]] is that
      * the optimizer can hit an infinite loop during the [[PushDownPredicates]] rule.
      */
-    case Project(projectList, Filter(condition, child))
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
-      val exprCandidatesToPrune = projectList ++ Seq(condition) ++ child.expressions
-      getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases)
-      }
+    case Project(projectList, Filter(condition, child)) if
+        SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
+      rewritePlanIfSubsetFieldsUsed(
+        plan, projectList ++ Seq(condition) ++ child.expressions, child.producedAttributes.toSeq)
 
-    case Project(projectList, child)
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
-      val exprCandidatesToPrune = projectList ++ child.expressions
-      getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases)
-      }
+    case Project(projectList, child) if
+        SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
+      rewritePlanIfSubsetFieldsUsed(
+        plan, projectList ++ child.expressions, child.producedAttributes.toSeq)
 
     case p if SQLConf.get.nestedSchemaPruningEnabled && canPruneOn(p) =>
-      val exprCandidatesToPrune = p.expressions
-      getAliasSubMap(exprCandidatesToPrune, p.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(p, nestedFieldToAlias, attrToAliases)
-      }
+      rewritePlanIfSubsetFieldsUsed(
+        plan, p.expressions, p.producedAttributes.toSeq)
 
     case _ => None
   }
 
+  /**
+   * Rewrites a plan with aliases if only a subset of the nested fields are used.
+   */
+  def rewritePlanIfSubsetFieldsUsed(
+      plan: LogicalPlan,
+      exprList: Seq[Expression],
+      exclusiveAttrs: Seq[Attribute]): Option[LogicalPlan] = {
+    val attrToExtractValues = getAttributeToExtractValues(exprList, exclusiveAttrs)
+    if (attrToExtractValues.isEmpty) {
+      None
+    } else {
+      Some(rewritePlanWithAliases(plan, attrToExtractValues))
+    }
+  }
+
   /**
    * Replace nested columns to prune unused nested columns later.
    */
-  private def replaceToAliases(
+  def rewritePlanWithAliases(
       plan: LogicalPlan,
-      nestedFieldToAlias: Map[ExtractValue, Alias],
-      attrToAliases: Map[ExprId, Seq[Alias]]): LogicalPlan = plan match {
-    case Project(projectList, child) =>
-      Project(
-        getNewProjectList(projectList, nestedFieldToAlias),
-        replaceWithAliases(child, nestedFieldToAlias, attrToAliases))
-
-    // The operators reaching here was already guarded by `canPruneOn`.
-    case other =>
-      replaceWithAliases(other, nestedFieldToAlias, attrToAliases)
+      attributeToExtractValues: Map[Attribute, Seq[ExtractValue]]): LogicalPlan = {
+    // Each expression can contain multiple nested fields.
+    // Note that we keep the original names to deliver to parquet in a case-sensitive way.
+    // A new alias is created for each nested field.
+    val nestedFieldToAlias = attributeToExtractValues.flatMap { case (_, nestedFields) =>
+      nestedFields.map { f =>
+        val exprId = NamedExpression.newExprId
+        val fieldName = f match {
+          case g: GetStructField => g.extractFieldName
+          case g: GetArrayStructFields => g.field.name
+        }
+        f -> Alias(f, s"_extract_${fieldName}")(exprId, Seq.empty, None)

Review comment:
       @cloud-fan correct me if I'm wrong, but the plan should also include the exprId (eg. `_extract_a#18`), right?




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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-843410544


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/43210/
   


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


[GitHub] [spark] SparkQA removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-842512761


   **[Test build #138640 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138640/testReport)** for PR 32301 at commit [`351e74a`](https://github.com/apache/spark/commit/351e74ade348fddc04dd2cc86bd5bc9bdf2d538f).


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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-842729686


   **[Test build #138642 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138642/testReport)** for PR 32301 at commit [`c7e5c5a`](https://github.com/apache/spark/commit/c7e5c5a2228364859e47834c690b87b1fbdb7095).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


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

Posted by GitBox <gi...@apache.org>.
karenfeng commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r623282261



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -17,71 +17,148 @@
 
 package org.apache.spark.sql.catalyst.optimizer
 
+import scala.collection.mutable
+
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 
 /**
- * This aims to handle a nested column aliasing pattern inside the `ColumnPruning` optimizer rule.
- * If a project or its child references to nested fields, and not all the fields
- * in a nested attribute are used, we can substitute them by alias attributes; then a project
- * of the nested fields as aliases on the children of the child will be created.
+ * This aims to handle a nested column aliasing pattern inside the [[ColumnPruning]] optimizer rule.
+ * If:
+ * - A [[Project]] or its child references nested fields
+ * - Not all of the fields in a nested attribute are used
+ * Then:
+ * - Substitute the nested field references with alias attributes
+ * - Add grandchild [[Project]]s transforming the nested fields to aliases
+ *
+ * Example 1: Project
+ * ------------------
+ * Before:
+ * +- Project [concat_ws(s#0.a, s#0.b) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [concat_ws(_gen_alias_2#2, _gen_alias_3#3) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _gen_alias_2#2, s#0.b AS _gen_alias_3#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * Example 2: Project above Filter
+ * -------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1]
+ *   +- Filter (length(s#0.b) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_gen_alias_2#2 AS s.a#1]
+ *   +- Filter (length(_gen_alias_3#3) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- Project [s#0.a AS _gen_alias_2#2, s#0.b AS _gen_alias_3#3]
+ *           +- LocalRelation <empty>, [s#0]
+ *
+ * Example 3: Nested columns in nested columns
+ * -------------------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1, s#0.a.a1 AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_gen_alias_3#3 AS s.a#1, _gen_alias_3#3.name AS s.a.a1#2]

Review comment:
       Ah, good catch. Thanks!




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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r638934740



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -149,67 +234,52 @@ object NestedColumnAliasing {
   }
 
   /**
-   * 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.
+   * Creates a map from root [[Attribute]]s to non-redundant nested [[ExtractValue]]s.
+   * Nested field accessors of `exclusiveAttrs` are not considered in nested fields aliasing.
    */
-  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
+  def getAttributeToExtractValues(
+      exprList: Seq[Expression],
+      exclusiveAttrs: Seq[Attribute]): Map[Attribute, Seq[ExtractValue]] = {
+
+    val nestedFieldReferences = new mutable.ArrayBuffer[ExtractValue]()
+    val otherRootReferences = new mutable.ArrayBuffer[AttributeReference]()
+    exprList.foreach { e =>
+      collectRootReferenceAndExtractValue(e).foreach {
+        case ev: ExtractValue => nestedFieldReferences.append(ev)
+        case ar: AttributeReference => otherRootReferences.append(ar)
       }
-
-    // Note that when we group by extractors with their references, we should remove
-    // cosmetic variations.
+    }
     val exclusiveAttrSet = AttributeSet(exclusiveAttrs ++ otherRootReferences)
-    val aliasSub = nestedFieldReferences.asInstanceOf[Seq[ExtractValue]]
+
+    // Remove cosmetic variations when we group extractors by their references
+    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`.
+        // Because `a.b` requires all of the inner fields of `b`, we cannot prune `a.b.c`.
         val dedupNestedFields = nestedFields.filter {
+          // See [[collectExtractValue]]: we only need to deal with [[GetArrayStructFields]] and
+          // [[GetStructField]]
           case e @ (_: GetStructField | _: GetArrayStructFields) =>
             val child = e.children.head
             nestedFields.forall(f => child.find(_.semanticEquals(f)).isEmpty)
           case _ => true
-        }
-
-        // Each expression can contain multiple nested fields.
-        // Note that we keep the original names to deliver to parquet in a case-sensitive way.
-        val nestedFieldToAlias = dedupNestedFields.distinct.map { f =>
-          val exprId = NamedExpression.newExprId
-          (f, Alias(f, s"_gen_alias_${exprId.id}")(exprId, Seq.empty, None))
-        }
+        }.distinct

Review comment:
       Sorry I'm wrong. We can't do this as the caller side will look up the map from extract value to alias using the original expression.




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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-828642681


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42569/
   


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


[GitHub] [spark] AmplabJenkins commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-848445220


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/138955/
   


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


[GitHub] [spark] AmplabJenkins commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-825987652


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42401/
   


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


[GitHub] [spark] AmplabJenkins commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-842625011


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/43162/
   


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


[GitHub] [spark] AmplabJenkins commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-826025512


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137871/
   


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


[GitHub] [spark] AmplabJenkins commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-828072391


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42531/
   


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-827978228






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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-827944457






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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-848444122


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/43475/
   


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-849993315


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/43553/
   


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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-848427649


   **[Test build #138955 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138955/testReport)** for PR 32301 at commit [`31f8bd0`](https://github.com/apache/spark/commit/31f8bd0952f8680fd78e7c8b3579369482d4dd34).


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


[GitHub] [spark] SparkQA removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-842634306


   **[Test build #138643 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138643/testReport)** for PR 32301 at commit [`f49663c`](https://github.com/apache/spark/commit/f49663c11e79f19aa250c50b6f5ce787c987cad7).


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


[GitHub] [spark] karenfeng commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
karenfeng commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-828836408


   @viirya, I refactored the code that you merged in #31966 to fix conflicts. Can you take a look?


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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r622728793



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -17,71 +17,148 @@
 
 package org.apache.spark.sql.catalyst.optimizer
 
+import scala.collection.mutable
+
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 
 /**
- * This aims to handle a nested column aliasing pattern inside the `ColumnPruning` optimizer rule.
- * If a project or its child references to nested fields, and not all the fields
- * in a nested attribute are used, we can substitute them by alias attributes; then a project
- * of the nested fields as aliases on the children of the child will be created.
+ * This aims to handle a nested column aliasing pattern inside the [[ColumnPruning]] optimizer rule.
+ * If:
+ * - A [[Project]] or its child references nested fields
+ * - Not all of the fields in a nested attribute are used
+ * Then:
+ * - Substitute the nested field references with alias attributes
+ * - Add grandchild [[Project]]s transforming the nested fields to aliases
+ *
+ * Example 1: Project
+ * ------------------
+ * Before:
+ * +- Project [concat_ws(s#0.a, s#0.b) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [concat_ws(_gen_alias_2#2, _gen_alias_3#3) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _gen_alias_2#2, s#0.b AS _gen_alias_3#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * Example 2: Project above Filter
+ * -------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1]
+ *   +- Filter (length(s#0.b) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_gen_alias_2#2 AS s.a#1]
+ *   +- Filter (length(_gen_alias_3#3) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- Project [s#0.a AS _gen_alias_2#2, s#0.b AS _gen_alias_3#3]
+ *           +- LocalRelation <empty>, [s#0]
+ *
+ * Example 3: Nested columns in nested columns
+ * -------------------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1, s#0.a.a1 AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_gen_alias_3#3 AS s.a#1, _gen_alias_3#3.name AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _gen_alias_3#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * The schema of the datasource relation will be pruned in the [[SchemaPruning]] optimizer rule.
  */
 object NestedColumnAliasing {
 
   def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match {
     /**
      * This pattern is needed to support [[Filter]] plan cases like
-     * [[Project]]->[[Filter]]->listed plan in `canProjectPushThrough` (e.g., [[Window]]).
-     * The reason why we don't simply add [[Filter]] in `canProjectPushThrough` is that
+     * [[Project]]->[[Filter]]->listed plan in [[canProjectPushThrough]] (e.g., [[Window]]).
+     * The reason why we don't simply add [[Filter]] in [[canProjectPushThrough]] is that
      * the optimizer can hit an infinite loop during the [[PushDownPredicates]] rule.
      */
-    case Project(projectList, Filter(condition, child))
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
-      val exprCandidatesToPrune = projectList ++ Seq(condition) ++ child.expressions
-      getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases)
-      }
+    case Project(projectList, Filter(condition, child)) if
+        SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
+      rewritePlanIfSubsetFieldsUsed(
+        plan, projectList ++ Seq(condition) ++ child.expressions, child.producedAttributes.toSeq)
 
-    case Project(projectList, child)
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
-      val exprCandidatesToPrune = projectList ++ child.expressions
-      getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases)
-      }
+    case Project(projectList, child) if
+        SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
+      rewritePlanIfSubsetFieldsUsed(
+        plan, projectList ++ child.expressions, child.producedAttributes.toSeq)
 
     case p if SQLConf.get.nestedSchemaPruningEnabled && canPruneOn(p) =>
-      val exprCandidatesToPrune = p.expressions
-      getAliasSubMap(exprCandidatesToPrune, p.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(p, nestedFieldToAlias, attrToAliases)
-      }
+      rewritePlanIfSubsetFieldsUsed(
+        plan, p.expressions, p.producedAttributes.toSeq)
 
     case _ => None
   }
 
+  /**
+   * Rewrites a plan with aliases if only a subset of the nested fields are used.
+   */
+  def rewritePlanIfSubsetFieldsUsed(
+      plan: LogicalPlan,
+      exprList: Seq[Expression],
+      exclusiveAttrs: Seq[Attribute]): Option[LogicalPlan] = {
+    val attrToExtractValues = getAttributeToExtractValues(exprList, exclusiveAttrs)
+    if (attrToExtractValues.isEmpty) {
+      None
+    } else {
+      Some(rewritePlanWithAliases(plan, attrToExtractValues))
+    }
+  }
+
   /**
    * Replace nested columns to prune unused nested columns later.
    */
-  private def replaceToAliases(
+  def rewritePlanWithAliases(
       plan: LogicalPlan,
-      nestedFieldToAlias: Map[ExtractValue, Alias],
-      attrToAliases: Map[ExprId, Seq[Alias]]): LogicalPlan = plan match {
-    case Project(projectList, child) =>
-      Project(
-        getNewProjectList(projectList, nestedFieldToAlias),
-        replaceWithAliases(child, nestedFieldToAlias, attrToAliases))
-
-    // The operators reaching here was already guarded by `canPruneOn`.
-    case other =>
-      replaceWithAliases(other, nestedFieldToAlias, attrToAliases)
+      attributeToExtractValues: Map[Attribute, Seq[ExtractValue]]): LogicalPlan = {
+    // Each expression can contain multiple nested fields.
+    // Note that we keep the original names to deliver to parquet in a case-sensitive way.
+    // A new alias is created for each nested field.
+    val nestedFieldToAlias = attributeToExtractValues.flatMap { case (_, nestedFields) =>
+      nestedFields.map { f =>
+        val exprId = NamedExpression.newExprId
+        val fieldName = f match {
+          case g: GetStructField => g.extractFieldName
+          case g: GetArrayStructFields => g.field.name
+        }
+        f -> Alias(f, s"_extract_${fieldName}")(exprId, Seq.empty, None)
+      }
+    }
+
+    // A reference attribute can have multiple aliases for nested fields.
+    val attrToAliases = attributeToExtractValues.map { case (attr, nestedFields) =>

Review comment:
       We can build an `AttriuteMap[Seq[Alias]]`




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


[GitHub] [spark] AmplabJenkins commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-828951740


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42586/
   


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


[GitHub] [spark] AmplabJenkins commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-848444122


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/43475/
   


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-848445220


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/138955/
   


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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-828608152


   **[Test build #138050 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138050/testReport)** for PR 32301 at commit [`e47a441`](https://github.com/apache/spark/commit/e47a4415c6c535736310d0114f5ba836a6f6f7c2).


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


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

Posted by GitBox <gi...@apache.org>.
karenfeng commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r619491896



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -30,54 +30,61 @@ import org.apache.spark.sql.types._
  */
 object NestedColumnAliasing {
 
-  def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match {
+  def unapply(plan: LogicalPlan): Option[Map[Attribute, Seq[ExtractValue]]] = plan match {

Review comment:
       Right now, `unapply` returns a `None` instead of `Some(Map)` if all the nested fields are used. We won't fall through in this case if we only return the information needed to build a map. Let me change this around to see what would achieve the same behavior.




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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-825206531


   **[Test build #137818 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137818/testReport)** for PR 32301 at commit [`9656899`](https://github.com/apache/spark/commit/9656899c11161aceda5ce46bf764b6c052861410).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


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

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r627975154



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -17,71 +17,148 @@
 
 package org.apache.spark.sql.catalyst.optimizer
 
+import scala.collection.mutable
+
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 
 /**
- * This aims to handle a nested column aliasing pattern inside the `ColumnPruning` optimizer rule.
- * If a project or its child references to nested fields, and not all the fields
- * in a nested attribute are used, we can substitute them by alias attributes; then a project
- * of the nested fields as aliases on the children of the child will be created.
+ * This aims to handle a nested column aliasing pattern inside the [[ColumnPruning]] optimizer rule.
+ * If:
+ * - A [[Project]] or its child references nested fields
+ * - Not all of the fields in a nested attribute are used
+ * Then:
+ * - Substitute the nested field references with alias attributes
+ * - Add grandchild [[Project]]s transforming the nested fields to aliases
+ *
+ * Example 1: Project
+ * ------------------
+ * Before:
+ * +- Project [concat_ws(s#0.a, s#0.b) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [concat_ws(_gen_alias_2#2, _gen_alias_3#3) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _gen_alias_2#2, s#0.b AS _gen_alias_3#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * Example 2: Project above Filter
+ * -------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1]
+ *   +- Filter (length(s#0.b) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_gen_alias_2#2 AS s.a#1]
+ *   +- Filter (length(_gen_alias_3#3) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- Project [s#0.a AS _gen_alias_2#2, s#0.b AS _gen_alias_3#3]
+ *           +- LocalRelation <empty>, [s#0]
+ *
+ * Example 3: Nested columns in nested columns
+ * -------------------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1, s#0.a.a1 AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_gen_alias_3#3 AS s.a#1, _gen_alias_3#3.name AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _gen_alias_3#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * The schema of the datasource relation will be pruned in the [[SchemaPruning]] optimizer rule.
  */
 object NestedColumnAliasing {
 
   def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match {
     /**
      * This pattern is needed to support [[Filter]] plan cases like
-     * [[Project]]->[[Filter]]->listed plan in `canProjectPushThrough` (e.g., [[Window]]).
-     * The reason why we don't simply add [[Filter]] in `canProjectPushThrough` is that
+     * [[Project]]->[[Filter]]->listed plan in [[canProjectPushThrough]] (e.g., [[Window]]).
+     * The reason why we don't simply add [[Filter]] in [[canProjectPushThrough]] is that
      * the optimizer can hit an infinite loop during the [[PushDownPredicates]] rule.
      */
-    case Project(projectList, Filter(condition, child))
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
-      val exprCandidatesToPrune = projectList ++ Seq(condition) ++ child.expressions
-      getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases)
-      }
+    case Project(projectList, Filter(condition, child)) if
+        SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
+      rewritePlanIfSubsetFieldsUsed(
+        plan, projectList ++ Seq(condition) ++ child.expressions, child.producedAttributes.toSeq)
 
-    case Project(projectList, child)
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
-      val exprCandidatesToPrune = projectList ++ child.expressions
-      getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases)
-      }
+    case Project(projectList, child) if
+        SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
+      rewritePlanIfSubsetFieldsUsed(
+        plan, projectList ++ child.expressions, child.producedAttributes.toSeq)
 
     case p if SQLConf.get.nestedSchemaPruningEnabled && canPruneOn(p) =>
-      val exprCandidatesToPrune = p.expressions
-      getAliasSubMap(exprCandidatesToPrune, p.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(p, nestedFieldToAlias, attrToAliases)
-      }
+      rewritePlanIfSubsetFieldsUsed(
+        plan, p.expressions, p.producedAttributes.toSeq)
 
     case _ => None
   }
 
+  /**
+   * Rewrites a plan with aliases if only a subset of the nested fields are used.
+   */
+  def rewritePlanIfSubsetFieldsUsed(
+      plan: LogicalPlan,
+      exprList: Seq[Expression],
+      exclusiveAttrs: Seq[Attribute]): Option[LogicalPlan] = {
+    val attrToExtractValues = getAttributeToExtractValues(exprList, exclusiveAttrs)
+    if (attrToExtractValues.isEmpty) {
+      None
+    } else {
+      Some(rewritePlanWithAliases(plan, attrToExtractValues))
+    }
+  }
+
   /**
    * Replace nested columns to prune unused nested columns later.
    */
-  private def replaceToAliases(
+  def rewritePlanWithAliases(
       plan: LogicalPlan,
-      nestedFieldToAlias: Map[ExtractValue, Alias],
-      attrToAliases: Map[ExprId, Seq[Alias]]): LogicalPlan = plan match {
-    case Project(projectList, child) =>
-      Project(
-        getNewProjectList(projectList, nestedFieldToAlias),
-        replaceWithAliases(child, nestedFieldToAlias, attrToAliases))
-
-    // The operators reaching here was already guarded by `canPruneOn`.
-    case other =>
-      replaceWithAliases(other, nestedFieldToAlias, attrToAliases)
+      attributeToExtractValues: Map[Attribute, Seq[ExtractValue]]): LogicalPlan = {
+    // Each expression can contain multiple nested fields.
+    // Note that we keep the original names to deliver to parquet in a case-sensitive way.
+    // A new alias is created for each nested field.
+    val nestedFieldToAlias = attributeToExtractValues.flatMap { case (_, nestedFields) =>
+      nestedFields.map { f =>
+        val exprId = NamedExpression.newExprId
+        val fieldName = f match {
+          case g: GetStructField => g.extractFieldName
+          case g: GetArrayStructFields => g.field.name
+        }
+        f -> Alias(f, s"_extract_${fieldName}")(exprId, Seq.empty, None)

Review comment:
       If field name is the same, won't `s"_extract_${fieldName}"` be confusing when explaining the query?




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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-828794503


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/138050/
   


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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r620122510



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -133,83 +204,77 @@ 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
+  }
+
+  /**
+   * Creates a map from root [[Attribute]]s to non-redundant nested [[ExtractValue]]s.
+   * Nested field accessors of `exclusiveAttrs` are not considered in nested fields aliasing.
+   */
+  def getAttributeToExtractValues(
+      exprList: Seq[Expression],
+      exclusiveAttrs: Seq[Attribute]): Map[Attribute, Seq[ExtractValue]] = {
 
-    // Note that when we group by extractors with their references, we should remove
-    // cosmetic variations.
+    val nestedFieldReferences = exprList.flatMap(collectExtractValue)
+    val otherRootReferences = exprList.flatMap(collectAttributeReference)

Review comment:
       Previously we collected both the nested fields extraction and other root references at the same time, and split them later. Now we collect them separately. I think the current code is clearer but is less performant.
   
   How about we use mutable collections to implement this logic with one tree traversal?
   ```
   val nestedFieldReferences = mutable.ArrayBuffer[ExtractValue]
   val otherRootReferences = mutable.ArrayBuffer[AttributeReference]
   exprList.foreach(collectRootReferenceAndExtractValue(e, nestedFieldReferences, otherRootReferences))
   ```




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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r622728793



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -17,71 +17,148 @@
 
 package org.apache.spark.sql.catalyst.optimizer
 
+import scala.collection.mutable
+
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 
 /**
- * This aims to handle a nested column aliasing pattern inside the `ColumnPruning` optimizer rule.
- * If a project or its child references to nested fields, and not all the fields
- * in a nested attribute are used, we can substitute them by alias attributes; then a project
- * of the nested fields as aliases on the children of the child will be created.
+ * This aims to handle a nested column aliasing pattern inside the [[ColumnPruning]] optimizer rule.
+ * If:
+ * - A [[Project]] or its child references nested fields
+ * - Not all of the fields in a nested attribute are used
+ * Then:
+ * - Substitute the nested field references with alias attributes
+ * - Add grandchild [[Project]]s transforming the nested fields to aliases
+ *
+ * Example 1: Project
+ * ------------------
+ * Before:
+ * +- Project [concat_ws(s#0.a, s#0.b) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [concat_ws(_gen_alias_2#2, _gen_alias_3#3) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _gen_alias_2#2, s#0.b AS _gen_alias_3#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * Example 2: Project above Filter
+ * -------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1]
+ *   +- Filter (length(s#0.b) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_gen_alias_2#2 AS s.a#1]
+ *   +- Filter (length(_gen_alias_3#3) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- Project [s#0.a AS _gen_alias_2#2, s#0.b AS _gen_alias_3#3]
+ *           +- LocalRelation <empty>, [s#0]
+ *
+ * Example 3: Nested columns in nested columns
+ * -------------------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1, s#0.a.a1 AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_gen_alias_3#3 AS s.a#1, _gen_alias_3#3.name AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _gen_alias_3#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * The schema of the datasource relation will be pruned in the [[SchemaPruning]] optimizer rule.
  */
 object NestedColumnAliasing {
 
   def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match {
     /**
      * This pattern is needed to support [[Filter]] plan cases like
-     * [[Project]]->[[Filter]]->listed plan in `canProjectPushThrough` (e.g., [[Window]]).
-     * The reason why we don't simply add [[Filter]] in `canProjectPushThrough` is that
+     * [[Project]]->[[Filter]]->listed plan in [[canProjectPushThrough]] (e.g., [[Window]]).
+     * The reason why we don't simply add [[Filter]] in [[canProjectPushThrough]] is that
      * the optimizer can hit an infinite loop during the [[PushDownPredicates]] rule.
      */
-    case Project(projectList, Filter(condition, child))
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
-      val exprCandidatesToPrune = projectList ++ Seq(condition) ++ child.expressions
-      getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases)
-      }
+    case Project(projectList, Filter(condition, child)) if
+        SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
+      rewritePlanIfSubsetFieldsUsed(
+        plan, projectList ++ Seq(condition) ++ child.expressions, child.producedAttributes.toSeq)
 
-    case Project(projectList, child)
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
-      val exprCandidatesToPrune = projectList ++ child.expressions
-      getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases)
-      }
+    case Project(projectList, child) if
+        SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
+      rewritePlanIfSubsetFieldsUsed(
+        plan, projectList ++ child.expressions, child.producedAttributes.toSeq)
 
     case p if SQLConf.get.nestedSchemaPruningEnabled && canPruneOn(p) =>
-      val exprCandidatesToPrune = p.expressions
-      getAliasSubMap(exprCandidatesToPrune, p.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(p, nestedFieldToAlias, attrToAliases)
-      }
+      rewritePlanIfSubsetFieldsUsed(
+        plan, p.expressions, p.producedAttributes.toSeq)
 
     case _ => None
   }
 
+  /**
+   * Rewrites a plan with aliases if only a subset of the nested fields are used.
+   */
+  def rewritePlanIfSubsetFieldsUsed(
+      plan: LogicalPlan,
+      exprList: Seq[Expression],
+      exclusiveAttrs: Seq[Attribute]): Option[LogicalPlan] = {
+    val attrToExtractValues = getAttributeToExtractValues(exprList, exclusiveAttrs)
+    if (attrToExtractValues.isEmpty) {
+      None
+    } else {
+      Some(rewritePlanWithAliases(plan, attrToExtractValues))
+    }
+  }
+
   /**
    * Replace nested columns to prune unused nested columns later.
    */
-  private def replaceToAliases(
+  def rewritePlanWithAliases(
       plan: LogicalPlan,
-      nestedFieldToAlias: Map[ExtractValue, Alias],
-      attrToAliases: Map[ExprId, Seq[Alias]]): LogicalPlan = plan match {
-    case Project(projectList, child) =>
-      Project(
-        getNewProjectList(projectList, nestedFieldToAlias),
-        replaceWithAliases(child, nestedFieldToAlias, attrToAliases))
-
-    // The operators reaching here was already guarded by `canPruneOn`.
-    case other =>
-      replaceWithAliases(other, nestedFieldToAlias, attrToAliases)
+      attributeToExtractValues: Map[Attribute, Seq[ExtractValue]]): LogicalPlan = {
+    // Each expression can contain multiple nested fields.
+    // Note that we keep the original names to deliver to parquet in a case-sensitive way.
+    // A new alias is created for each nested field.
+    val nestedFieldToAlias = attributeToExtractValues.flatMap { case (_, nestedFields) =>
+      nestedFields.map { f =>
+        val exprId = NamedExpression.newExprId
+        val fieldName = f match {
+          case g: GetStructField => g.extractFieldName
+          case g: GetArrayStructFields => g.field.name
+        }
+        f -> Alias(f, s"_extract_${fieldName}")(exprId, Seq.empty, None)
+      }
+    }
+
+    // A reference attribute can have multiple aliases for nested fields.
+    val attrToAliases = attributeToExtractValues.map { case (attr, nestedFields) =>

Review comment:
       We can build an `AttriuteMap[Alias]`




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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r638909542



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -17,71 +17,155 @@
 
 package org.apache.spark.sql.catalyst.optimizer
 
+import scala.collection.mutable
+
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 
 /**
- * This aims to handle a nested column aliasing pattern inside the `ColumnPruning` optimizer rule.
- * If a project or its child references to nested fields, and not all the fields
- * in a nested attribute are used, we can substitute them by alias attributes; then a project
- * of the nested fields as aliases on the children of the child will be created.
+ * This aims to handle a nested column aliasing pattern inside the [[ColumnPruning]] optimizer rule.
+ * If:
+ * - A [[Project]] or its child references nested fields
+ * - Not all of the fields in a nested attribute are used
+ * Then:
+ * - Substitute the nested field references with alias attributes
+ * - Add grandchild [[Project]]s transforming the nested fields to aliases
+ *
+ * Example 1: Project
+ * ------------------
+ * Before:
+ * +- Project [concat_ws(s#0.a, s#0.b) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [concat_ws(_extract_a#2, _extract_b#3) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _extract_a#2, s#0.b AS _extract_b#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * Example 2: Project above Filter
+ * -------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1]
+ *   +- Filter (length(s#0.b) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_extract_a#2 AS s.a#1]
+ *   +- Filter (length(_extract_b#3) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- Project [s#0.a AS _extract_a#2, s#0.b AS _extract_b#3]
+ *           +- LocalRelation <empty>, [s#0]
+ *
+ * Example 3: Nested fields with referenced parents
+ * ------------------------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1, s#0.a.a1 AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_extract_a#3 AS s.a#1, _extract_a#3.name AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _extract_a#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * The schema of the datasource relation will be pruned in the [[SchemaPruning]] optimizer rule.
  */
 object NestedColumnAliasing {
 
   def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match {
     /**
      * This pattern is needed to support [[Filter]] plan cases like
-     * [[Project]]->[[Filter]]->listed plan in `canProjectPushThrough` (e.g., [[Window]]).
-     * The reason why we don't simply add [[Filter]] in `canProjectPushThrough` is that
+     * [[Project]]->[[Filter]]->listed plan in [[canProjectPushThrough]] (e.g., [[Window]]).
+     * The reason why we don't simply add [[Filter]] in [[canProjectPushThrough]] is that
      * the optimizer can hit an infinite loop during the [[PushDownPredicates]] rule.
      */
-    case Project(projectList, Filter(condition, child))
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>

Review comment:
       nit: I think the previous code is better that put `if` in this line.




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


[GitHub] [spark] AmplabJenkins commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-828642681


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42569/
   


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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-825081513


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/42348/
   


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


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

Posted by GitBox <gi...@apache.org>.
karenfeng commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r639357336



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -149,67 +234,52 @@ object NestedColumnAliasing {
   }
 
   /**
-   * 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.
+   * Creates a map from root [[Attribute]]s to non-redundant nested [[ExtractValue]]s.
+   * Nested field accessors of `exclusiveAttrs` are not considered in nested fields aliasing.
    */
-  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
+  def getAttributeToExtractValues(
+      exprList: Seq[Expression],
+      exclusiveAttrs: Seq[Attribute]): Map[Attribute, Seq[ExtractValue]] = {
+
+    val nestedFieldReferences = new mutable.ArrayBuffer[ExtractValue]()
+    val otherRootReferences = new mutable.ArrayBuffer[AttributeReference]()
+    exprList.foreach { e =>
+      collectRootReferenceAndExtractValue(e).foreach {
+        case ev: ExtractValue => nestedFieldReferences.append(ev)

Review comment:
       Makes sense to me!




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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-828068265


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/42531/
   


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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r638945342



##########
File path: sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala
##########
@@ -37,6 +37,8 @@ class AttributeMap[A](val baseMap: Map[ExprId, (Attribute, A)])
 
   override def get(k: Attribute): Option[A] = baseMap.get(k.exprId).map(_._2)
 
+  override def getOrElse[B1 >: A](k: Attribute, default: => B1): B1 = get(k).getOrElse(default)

Review comment:
       note: we need to make the same change in `AttributeMap` under scala-2.13.




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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-842512761


   **[Test build #138640 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138640/testReport)** for PR 32301 at commit [`351e74a`](https://github.com/apache/spark/commit/351e74ade348fddc04dd2cc86bd5bc9bdf2d538f).


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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r638933402



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -17,71 +17,155 @@
 
 package org.apache.spark.sql.catalyst.optimizer
 
+import scala.collection.mutable
+
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 
 /**
- * This aims to handle a nested column aliasing pattern inside the `ColumnPruning` optimizer rule.
- * If a project or its child references to nested fields, and not all the fields
- * in a nested attribute are used, we can substitute them by alias attributes; then a project
- * of the nested fields as aliases on the children of the child will be created.
+ * This aims to handle a nested column aliasing pattern inside the [[ColumnPruning]] optimizer rule.
+ * If:
+ * - A [[Project]] or its child references nested fields
+ * - Not all of the fields in a nested attribute are used
+ * Then:
+ * - Substitute the nested field references with alias attributes
+ * - Add grandchild [[Project]]s transforming the nested fields to aliases
+ *
+ * Example 1: Project
+ * ------------------
+ * Before:
+ * +- Project [concat_ws(s#0.a, s#0.b) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [concat_ws(_extract_a#2, _extract_b#3) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _extract_a#2, s#0.b AS _extract_b#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * Example 2: Project above Filter
+ * -------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1]
+ *   +- Filter (length(s#0.b) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_extract_a#2 AS s.a#1]
+ *   +- Filter (length(_extract_b#3) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- Project [s#0.a AS _extract_a#2, s#0.b AS _extract_b#3]
+ *           +- LocalRelation <empty>, [s#0]
+ *
+ * Example 3: Nested fields with referenced parents
+ * ------------------------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1, s#0.a.a1 AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_extract_a#3 AS s.a#1, _extract_a#3.name AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _extract_a#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * The schema of the datasource relation will be pruned in the [[SchemaPruning]] optimizer rule.
  */
 object NestedColumnAliasing {
 
   def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match {
     /**
      * This pattern is needed to support [[Filter]] plan cases like
-     * [[Project]]->[[Filter]]->listed plan in `canProjectPushThrough` (e.g., [[Window]]).
-     * The reason why we don't simply add [[Filter]] in `canProjectPushThrough` is that
+     * [[Project]]->[[Filter]]->listed plan in [[canProjectPushThrough]] (e.g., [[Window]]).
+     * The reason why we don't simply add [[Filter]] in [[canProjectPushThrough]] is that
      * the optimizer can hit an infinite loop during the [[PushDownPredicates]] rule.
      */
-    case Project(projectList, Filter(condition, child))
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
-      val exprCandidatesToPrune = projectList ++ Seq(condition) ++ child.expressions
-      getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases)
-      }
+    case Project(projectList, Filter(condition, child)) if
+        SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
+      rewritePlanIfSubsetFieldsUsed(
+        plan, projectList ++ Seq(condition) ++ child.expressions, child.producedAttributes.toSeq)
 
-    case Project(projectList, child)
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
-      val exprCandidatesToPrune = projectList ++ child.expressions
-      getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases)
-      }
+    case Project(projectList, child) if
+        SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
+      rewritePlanIfSubsetFieldsUsed(
+        plan, projectList ++ child.expressions, child.producedAttributes.toSeq)
 
     case p if SQLConf.get.nestedSchemaPruningEnabled && canPruneOn(p) =>
-      val exprCandidatesToPrune = p.expressions
-      getAliasSubMap(exprCandidatesToPrune, p.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(p, nestedFieldToAlias, attrToAliases)
-      }
+      rewritePlanIfSubsetFieldsUsed(
+        plan, p.expressions, p.producedAttributes.toSeq)
 
     case _ => None
   }
 
+  /**
+   * Rewrites a plan with aliases if only a subset of the nested fields are used.
+   */
+  def rewritePlanIfSubsetFieldsUsed(
+      plan: LogicalPlan,
+      exprList: Seq[Expression],
+      exclusiveAttrs: Seq[Attribute]): Option[LogicalPlan] = {
+    val attrToExtractValues = getAttributeToExtractValues(exprList, exclusiveAttrs)
+    if (attrToExtractValues.isEmpty) {
+      None
+    } else {
+      Some(rewritePlanWithAliases(plan, attrToExtractValues))
+    }
+  }
+
   /**
    * Replace nested columns to prune unused nested columns later.
    */
-  private def replaceToAliases(
+  def rewritePlanWithAliases(
       plan: LogicalPlan,
-      nestedFieldToAlias: Map[ExtractValue, Alias],
-      attrToAliases: Map[ExprId, Seq[Alias]]): LogicalPlan = plan match {
-    case Project(projectList, child) =>
-      Project(
-        getNewProjectList(projectList, nestedFieldToAlias),
-        replaceWithAliases(child, nestedFieldToAlias, attrToAliases))
-
-    // The operators reaching here was already guarded by `canPruneOn`.
-    case other =>
-      replaceWithAliases(other, nestedFieldToAlias, attrToAliases)
+      attributeToExtractValues: Map[Attribute, Seq[ExtractValue]]): LogicalPlan = {
+    // Each expression can contain multiple nested fields.
+    // Note that we keep the original names to deliver to parquet in a case-sensitive way.
+    // A new alias is created for each nested field.
+    // Implementation detail: we don't use mapValues, because it creates a mutable view.
+    val attributeToExtractValuesAndAliases =
+      attributeToExtractValues.map { case (attr, evSeq) =>
+        val evAliasSeq = evSeq.map { ev =>
+          val fieldName = ev match {
+            case g: GetStructField => g.extractFieldName
+            case g: GetArrayStructFields => g.field.name
+          }
+          ev -> Alias(ev, s"_extract_$fieldName")()
+        }
+
+        attr -> evAliasSeq
+      }
+
+    val nestedFieldToAlias = attributeToExtractValuesAndAliases.values.flatten.toMap
+
+    // A reference attribute can have multiple aliases for nested fields.
+    val attrToAliases = new AttributeMap(

Review comment:
       nit: we can call `AttributeMap.apply` instead of `new`
   ```
   AttributeMap(attributeToExtractValuesAndAliases.mapValues(_._2))
   ```




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


[GitHub] [spark] AmplabJenkins commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-827978228






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


[GitHub] [spark] SparkQA removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-825951124


   **[Test build #137871 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137871/testReport)** for PR 32301 at commit [`a95360a`](https://github.com/apache/spark/commit/a95360a69907fc6fbb047490f4c448dc69659321).


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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-828082267


   **[Test build #138008 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138008/testReport)** for PR 32301 at commit [`f48b376`](https://github.com/apache/spark/commit/f48b376334f415f0de95d09c62b8d3ce0b8c23ed).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


[GitHub] [spark] AmplabJenkins commented on pull request #32301: [SPARK-35194] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-825081544


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42348/
   


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-843585334


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/138689/
   


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


[GitHub] [spark] SparkQA removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-842592756


   **[Test build #138642 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138642/testReport)** for PR 32301 at commit [`c7e5c5a`](https://github.com/apache/spark/commit/c7e5c5a2228364859e47834c690b87b1fbdb7095).


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


[GitHub] [spark] SparkQA removed a comment on pull request #32301: [SPARK-35194] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-825043140


   **[Test build #137818 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137818/testReport)** for PR 32301 at commit [`9656899`](https://github.com/apache/spark/commit/9656899c11161aceda5ce46bf764b6c052861410).


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


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

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r640175084



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -17,71 +17,155 @@
 
 package org.apache.spark.sql.catalyst.optimizer
 
+import scala.collection.mutable
+
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 
 /**
- * This aims to handle a nested column aliasing pattern inside the `ColumnPruning` optimizer rule.
- * If a project or its child references to nested fields, and not all the fields
- * in a nested attribute are used, we can substitute them by alias attributes; then a project
- * of the nested fields as aliases on the children of the child will be created.
+ * This aims to handle a nested column aliasing pattern inside the [[ColumnPruning]] optimizer rule.
+ * If:
+ * - A [[Project]] or its child references nested fields
+ * - Not all of the fields in a nested attribute are used
+ * Then:
+ * - Substitute the nested field references with alias attributes
+ * - Add grandchild [[Project]]s transforming the nested fields to aliases
+ *
+ * Example 1: Project
+ * ------------------
+ * Before:
+ * +- Project [concat_ws(s#0.a, s#0.b) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [concat_ws(_extract_a#2, _extract_b#3) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _extract_a#2, s#0.b AS _extract_b#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * Example 2: Project above Filter
+ * -------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1]
+ *   +- Filter (length(s#0.b) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_extract_a#2 AS s.a#1]
+ *   +- Filter (length(_extract_b#3) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- Project [s#0.a AS _extract_a#2, s#0.b AS _extract_b#3]
+ *           +- LocalRelation <empty>, [s#0]
+ *
+ * Example 3: Nested fields with referenced parents
+ * ------------------------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1, s#0.a.a1 AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_extract_a#3 AS s.a#1, _extract_a#3.name AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _extract_a#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * The schema of the datasource relation will be pruned in the [[SchemaPruning]] optimizer rule.
  */
 object NestedColumnAliasing {
 
   def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match {
     /**
      * This pattern is needed to support [[Filter]] plan cases like
-     * [[Project]]->[[Filter]]->listed plan in `canProjectPushThrough` (e.g., [[Window]]).
-     * The reason why we don't simply add [[Filter]] in `canProjectPushThrough` is that
+     * [[Project]]->[[Filter]]->listed plan in [[canProjectPushThrough]] (e.g., [[Window]]).
+     * The reason why we don't simply add [[Filter]] in [[canProjectPushThrough]] is that

Review comment:
       `Limit` is able to be pushed through, no?




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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-842634306


   **[Test build #138643 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138643/testReport)** for PR 32301 at commit [`f49663c`](https://github.com/apache/spark/commit/f49663c11e79f19aa250c50b6f5ce787c987cad7).


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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r639418807



##########
File path: sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala
##########
@@ -23,6 +23,10 @@ package org.apache.spark.sql.catalyst.expressions
  * of the name, or the expected nullability).
  */
 object AttributeMap {
+  def apply[A](kvs: Map[Attribute, A]): AttributeMap[A] = {

Review comment:
       instead of adding a new API, can the caller side just do `AttributeMap(map.toSeq)`?




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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-828778971


   **[Test build #138050 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138050/testReport)** for PR 32301 at commit [`e47a441`](https://github.com/apache/spark/commit/e47a4415c6c535736310d0114f5ba836a6f6f7c2).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r620111033



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -23,78 +23,149 @@ import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 
 /**
- * This aims to handle a nested column aliasing pattern inside the `ColumnPruning` optimizer rule.
- * If a project or its child references to nested fields, and not all the fields
- * in a nested attribute are used, we can substitute them by alias attributes; then a project
- * of the nested fields as aliases on the children of the child will be created.
+ * This aims to handle a nested column aliasing pattern inside the [[ColumnPruning]] optimizer rule.
+ * If:
+ * - A [[Project]] or its child references nested fields
+ * - Not all of the fields in a nested attribute are used
+ * Then:
+ * - Substitute the nested field references with alias attributes
+ * - Add grandchild [[Project]]s transforming the nested fields to aliases
+ *
+ * Example 1: Project
+ * ------------------
+ * Before:
+ * +- Project [concat_ws(s#0.a, s#0.b) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [concat_ws(_gen_alias_2#2, _gen_alias_3#3) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _gen_alias_2#2, s#0.b AS _gen_alias_3#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * Example 2: Project above Filter
+ * -------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1]
+ *   +- Filter (length(s#0.b) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_gen_alias_2#2 AS s.a#1]
+ *   +- Filter (length(_gen_alias_3#3) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- Project [s#0.a AS _gen_alias_2#2, s#0.b AS _gen_alias_3#3]
+ *           +- LocalRelation <empty>, [s#0]
+ *
+ * Example 3: Nested columns in nested columns
+ * -------------------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1, s#0.a.a1 AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_gen_alias_3#3 AS s.a#1, _gen_alias_3#3.name AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _gen_alias_3#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * The schema of the datasource relation will be pruned in the [[SchemaPruning]] optimizer rule.
  */
 object NestedColumnAliasing {
 
   def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match {
     /**
      * This pattern is needed to support [[Filter]] plan cases like
-     * [[Project]]->[[Filter]]->listed plan in `canProjectPushThrough` (e.g., [[Window]]).
-     * The reason why we don't simply add [[Filter]] in `canProjectPushThrough` is that
+     * [[Project]]->[[Filter]]->listed plan in [[canProjectPushThrough]] (e.g., [[Window]]).
+     * The reason why we don't simply add [[Filter]] in [[canProjectPushThrough]] is that
      * the optimizer can hit an infinite loop during the [[PushDownPredicates]] rule.
      */
-    case Project(projectList, Filter(condition, child))
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
-      val exprCandidatesToPrune = projectList ++ Seq(condition) ++ child.expressions
-      getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases)
-      }
+    case Project(projectList, Filter(condition, child)) if
+        SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
+      rewritePlanIfSubsetFieldsUsed(
+        plan, projectList ++ Seq(condition) ++ child.expressions, child.producedAttributes.toSeq)
 
-    case Project(projectList, child)
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
-      val exprCandidatesToPrune = projectList ++ child.expressions
-      getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases)
-      }
+    case Project(projectList, child) if
+        SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
+      rewritePlanIfSubsetFieldsUsed(
+        plan, projectList ++ child.expressions, child.producedAttributes.toSeq)
 
     case p if SQLConf.get.nestedSchemaPruningEnabled && canPruneOn(p) =>
-      val exprCandidatesToPrune = p.expressions
-      getAliasSubMap(exprCandidatesToPrune, p.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(p, nestedFieldToAlias, attrToAliases)
-      }
+      rewritePlanIfSubsetFieldsUsed(
+        plan, p.expressions, p.producedAttributes.toSeq)
 
     case _ => None
   }
 
+  /**
+   * Rewrites a plan with aliases if only a subset of the nested fields are used.
+   */
+  def rewritePlanIfSubsetFieldsUsed(
+    plan: LogicalPlan,

Review comment:
       nit: 4 space indentation for function parameters.




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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r620117769



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -133,83 +204,77 @@ 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])

Review comment:
       nit: `case e: ExtractValue if isSelectedField(e) => Seq(e)`




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


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

Posted by GitBox <gi...@apache.org>.
karenfeng commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r618856755



##########
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:
       This splits `collectRootReferenceAndExtractValue` into two lists, `collectExtractValue` and `collectRootReferences`. Previously, we created a list with both types, performed a partition, and then casted the `ExtractValue`s.




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


[GitHub] [spark] karenfeng commented on pull request #32301: [SPARK-35194] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
karenfeng commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-825042480


   @viirya, this will either block/be blocked by your [ongoing PR](https://github.com/apache/spark/pull/31966)


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


[GitHub] [spark] AmplabJenkins commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-850053730


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/139035/
   


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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r620137246



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -133,83 +204,77 @@ 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
+  }
+
+  /**
+   * Creates a map from root [[Attribute]]s to non-redundant nested [[ExtractValue]]s.
+   * Nested field accessors of `exclusiveAttrs` are not considered in nested fields aliasing.
+   */
+  def getAttributeToExtractValues(
+      exprList: Seq[Expression],
+      exclusiveAttrs: Seq[Attribute]): Map[Attribute, Seq[ExtractValue]] = {
 
-    // Note that when we group by extractors with their references, we should remove
-    // cosmetic variations.
+    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
+    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 {
+          // See [[collectExtractValue]]: we only need to deal with [[GetArrayStructFields]] and
+          // [[GetStructField]]
           case e @ (_: GetStructField | _: GetArrayStructFields) =>
             val child = e.children.head
             nestedFields.forall(f => child.find(_.semanticEquals(f)).isEmpty)
           case _ => true
-        }
-
-        // Each expression can contain multiple nested fields.
-        // Note that we keep the original names to deliver to parquet in a case-sensitive way.
-        val nestedFieldToAlias = dedupNestedFields.distinct.map { f =>
-          val exprId = NamedExpression.newExprId
-          (f, Alias(f, s"_gen_alias_${exprId.id}")(exprId, Seq.empty, None))
-        }
+        }.distinct

Review comment:
       we will call `.map(_.canonicalized).distinct` later. Do we need this `.distinct` here?




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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-848441542


   Kubernetes integration test unable to build dist.
   
   exiting with code: 1
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/43475/
   


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


[GitHub] [spark] AmplabJenkins commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-829086250


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/138067/
   


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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-828934837


   **[Test build #138067 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138067/testReport)** for PR 32301 at commit [`d74b569`](https://github.com/apache/spark/commit/d74b5698616df1410fdc8cfb0e5db991ba84d321).


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


[GitHub] [spark] karenfeng commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
karenfeng commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-843511836


   @cloud-fan and @viirya, could you take another look? Thank you!


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


[GitHub] [spark] AmplabJenkins commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-849270337


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/43516/
   


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-828951740


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42586/
   


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


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

Posted by GitBox <gi...@apache.org>.
karenfeng commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r639362153



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -227,137 +297,114 @@ object NestedColumnAliasing {
 }
 
 /**
- * This prunes unnecessary nested columns from `Generate` and optional `Project` on top
- * of it.
+ * This prunes unnecessary nested columns from [[Generate]], or [[Project]] -> [[Generate]]
  */
 object GeneratorNestedColumnAliasing {
-  // Partitions `attrToAliases` based on whether the attribute is in Generator's output.
-  private def aliasesOnGeneratorOutput(
-      attrToAliases: Map[ExprId, Seq[Alias]],
-      generatorOutput: Seq[Attribute]) = {
-    val generatorOutputExprId = generatorOutput.map(_.exprId)
-    attrToAliases.partition { k =>
-      generatorOutputExprId.contains(k._1)
-    }
-  }
-
-  // Partitions `nestedFieldToAlias` based on whether the attribute of nested field extractor
-  // is in Generator's output.
-  private def nestedFieldOnGeneratorOutput(
-      nestedFieldToAlias: Map[ExtractValue, Alias],
-      generatorOutput: Seq[Attribute]) = {
-    val generatorOutputSet = AttributeSet(generatorOutput)
-    nestedFieldToAlias.partition { pair =>
-      pair._1.references.subsetOf(generatorOutputSet)
-    }
-  }
-
   def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match {
     // Either `nestedPruningOnExpressions` or `nestedSchemaPruningEnabled` is enabled, we
     // need to prune nested columns through Project and under Generate. The difference is
     // when `nestedSchemaPruningEnabled` is on, nested columns will be pruned further at
     // file format readers if it is supported.
     case Project(projectList, g: Generate) if (SQLConf.get.nestedPruningOnExpressions ||
-        SQLConf.get.nestedSchemaPruningEnabled) && canPruneGenerator(g.generator) =>
+      SQLConf.get.nestedSchemaPruningEnabled) && canPruneGenerator(g.generator) =>
       // On top on `Generate`, a `Project` that might have nested column accessors.
       // We try to get alias maps for both project list and generator's children expressions.
-      val exprsToPrune = projectList ++ g.generator.children
-      NestedColumnAliasing.getAliasSubMap(exprsToPrune).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          val (nestedFieldsOnGenerator, nestedFieldsNotOnGenerator) =
-            nestedFieldOnGeneratorOutput(nestedFieldToAlias, g.qualifiedGeneratorOutput)
-          val (attrToAliasesOnGenerator, attrToAliasesNotOnGenerator) =
-            aliasesOnGeneratorOutput(attrToAliases, g.qualifiedGeneratorOutput)
-
-          // Push nested column accessors through `Generator`.
-          // Defer updating `Generate.unrequiredChildIndex` to next round of `ColumnPruning`.
-          val newChild = NestedColumnAliasing.replaceWithAliases(g,
-            nestedFieldsNotOnGenerator, attrToAliasesNotOnGenerator)
-          val pushedThrough = Project(NestedColumnAliasing
-            .getNewProjectList(projectList, nestedFieldsNotOnGenerator), newChild)
-
-          // If the generator output is `ArrayType`, we cannot push through the extractor.
-          // It is because we don't allow field extractor on two-level array,
-          // i.e., attr.field when attr is a ArrayType(ArrayType(...)).
-          // Similarily, we also cannot push through if the child of generator is `MapType`.
-          g.generator.children.head.dataType match {
-            case _: MapType => return Some(pushedThrough)
-            case ArrayType(_: ArrayType, _) => return Some(pushedThrough)
-            case _ =>
-          }
-
-          // Pruning on `Generator`'s output. We only process single field case.
-          // For multiple field case, we cannot directly move field extractor into
-          // the generator expression. A workaround is to re-construct array of struct
-          // from multiple fields. But it will be more complicated and may not worth.
-          // TODO(SPARK-34956): support multiple fields.
-          if (nestedFieldsOnGenerator.size > 1 || nestedFieldsOnGenerator.isEmpty) {
-            pushedThrough
-          } else {
-            // Only one nested column accessor.
-            // E.g., df.select(explode($"items").as("item")).select($"item.a")
-            pushedThrough match {
-              case p @ Project(_, newG: Generate) =>
-                // Replace the child expression of `ExplodeBase` generator with
-                // nested column accessor.
-                // E.g., df.select(explode($"items").as("item")).select($"item.a") =>
-                //       df.select(explode($"items.a").as("item.a"))
-                val rewrittenG = newG.transformExpressions {
-                  case e: ExplodeBase =>
-                    val extractor = nestedFieldsOnGenerator.head._1.transformUp {
-                      case _: Attribute =>
-                        e.child
-                      case g: GetStructField =>
-                        ExtractValue(g.child, Literal(g.extractFieldName), SQLConf.get.resolver)
-                    }
-                    e.withNewChildren(Seq(extractor))
-                }
+      val attrToExtractValues = NestedColumnAliasing.getAttributeToExtractValues(
+        projectList ++ g.generator.children, Seq.empty)
+      if (attrToExtractValues.isEmpty) {
+        return None
+      }
+      val generatorOutputSet = AttributeSet(g.qualifiedGeneratorOutput)
+      val (attrToExtractValuesOnGenerator, attrToExtractValuesNotOnGenerator) =
+        attrToExtractValues.partition { case (attr, _) =>
+          attr.references.subsetOf(generatorOutputSet) }
+
+      val pushedThrough = NestedColumnAliasing.rewritePlanWithAliases(
+        plan, attrToExtractValuesNotOnGenerator)
+
+      // If the generator output is `ArrayType`, we cannot push through the extractor.
+      // It is because we don't allow field extractor on two-level array,
+      // i.e., attr.field when attr is a ArrayType(ArrayType(...)).
+      // Similarily, we also cannot push through if the child of generator is `MapType`.
+      g.generator.children.head.dataType match {
+        case _: MapType => return Some(pushedThrough)
+        case ArrayType(_: ArrayType, _) => return Some(pushedThrough)
+        case _ =>
+      }
 
-                // As we change the child of the generator, its output data type must be updated.
-                val updatedGeneratorOutput = rewrittenG.generatorOutput
-                  .zip(rewrittenG.generator.elementSchema.toAttributes)
-                  .map { case (oldAttr, newAttr) =>
-                  newAttr.withExprId(oldAttr.exprId).withName(oldAttr.name)
-                }
-                assert(updatedGeneratorOutput.length == rewrittenG.generatorOutput.length,
-                  "Updated generator output must have the same length " +
-                    "with original generator output.")
-                val updatedGenerate = rewrittenG.copy(generatorOutput = updatedGeneratorOutput)
-
-                // Replace nested column accessor with generator output.
-                p.withNewChildren(Seq(updatedGenerate)).transformExpressions {
-                  case f: ExtractValue if nestedFieldsOnGenerator.contains(f) =>
-                    updatedGenerate.output
-                      .find(a => attrToAliasesOnGenerator.contains(a.exprId))
-                      .getOrElse(f)
+      // Pruning on `Generator`'s output. We only process single field case.
+      // For multiple field case, we cannot directly move field extractor into
+      // the generator expression. A workaround is to re-construct array of struct
+      // from multiple fields. But it will be more complicated and may not worth.
+      // TODO(SPARK-34956): support multiple fields.
+      val nestedFieldsOnGenerator = attrToExtractValuesOnGenerator.values.flatten.toSet
+      if (nestedFieldsOnGenerator.size > 1 || nestedFieldsOnGenerator.isEmpty) {
+        Some(pushedThrough)
+      } else {
+        // Only one nested column accessor.

Review comment:
       Most of this is cosmetic, but at the end there is an expression ID-based lookup. The current commit explicitly uses the exprId to look up, but I can wrap more variables as an AttributeMap.




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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r622729279



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -17,71 +17,148 @@
 
 package org.apache.spark.sql.catalyst.optimizer
 
+import scala.collection.mutable
+
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 
 /**
- * This aims to handle a nested column aliasing pattern inside the `ColumnPruning` optimizer rule.
- * If a project or its child references to nested fields, and not all the fields
- * in a nested attribute are used, we can substitute them by alias attributes; then a project
- * of the nested fields as aliases on the children of the child will be created.
+ * This aims to handle a nested column aliasing pattern inside the [[ColumnPruning]] optimizer rule.
+ * If:
+ * - A [[Project]] or its child references nested fields
+ * - Not all of the fields in a nested attribute are used
+ * Then:
+ * - Substitute the nested field references with alias attributes
+ * - Add grandchild [[Project]]s transforming the nested fields to aliases
+ *
+ * Example 1: Project
+ * ------------------
+ * Before:
+ * +- Project [concat_ws(s#0.a, s#0.b) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [concat_ws(_gen_alias_2#2, _gen_alias_3#3) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _gen_alias_2#2, s#0.b AS _gen_alias_3#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * Example 2: Project above Filter
+ * -------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1]
+ *   +- Filter (length(s#0.b) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_gen_alias_2#2 AS s.a#1]
+ *   +- Filter (length(_gen_alias_3#3) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- Project [s#0.a AS _gen_alias_2#2, s#0.b AS _gen_alias_3#3]
+ *           +- LocalRelation <empty>, [s#0]
+ *
+ * Example 3: Nested columns in nested columns
+ * -------------------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1, s#0.a.a1 AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_gen_alias_3#3 AS s.a#1, _gen_alias_3#3.name AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _gen_alias_3#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * The schema of the datasource relation will be pruned in the [[SchemaPruning]] optimizer rule.
  */
 object NestedColumnAliasing {
 
   def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match {
     /**
      * This pattern is needed to support [[Filter]] plan cases like
-     * [[Project]]->[[Filter]]->listed plan in `canProjectPushThrough` (e.g., [[Window]]).
-     * The reason why we don't simply add [[Filter]] in `canProjectPushThrough` is that
+     * [[Project]]->[[Filter]]->listed plan in [[canProjectPushThrough]] (e.g., [[Window]]).
+     * The reason why we don't simply add [[Filter]] in [[canProjectPushThrough]] is that
      * the optimizer can hit an infinite loop during the [[PushDownPredicates]] rule.
      */
-    case Project(projectList, Filter(condition, child))
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
-      val exprCandidatesToPrune = projectList ++ Seq(condition) ++ child.expressions
-      getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases)
-      }
+    case Project(projectList, Filter(condition, child)) if
+        SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
+      rewritePlanIfSubsetFieldsUsed(
+        plan, projectList ++ Seq(condition) ++ child.expressions, child.producedAttributes.toSeq)
 
-    case Project(projectList, child)
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
-      val exprCandidatesToPrune = projectList ++ child.expressions
-      getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases)
-      }
+    case Project(projectList, child) if
+        SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
+      rewritePlanIfSubsetFieldsUsed(
+        plan, projectList ++ child.expressions, child.producedAttributes.toSeq)
 
     case p if SQLConf.get.nestedSchemaPruningEnabled && canPruneOn(p) =>
-      val exprCandidatesToPrune = p.expressions
-      getAliasSubMap(exprCandidatesToPrune, p.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(p, nestedFieldToAlias, attrToAliases)
-      }
+      rewritePlanIfSubsetFieldsUsed(
+        plan, p.expressions, p.producedAttributes.toSeq)
 
     case _ => None
   }
 
+  /**
+   * Rewrites a plan with aliases if only a subset of the nested fields are used.
+   */
+  def rewritePlanIfSubsetFieldsUsed(
+      plan: LogicalPlan,
+      exprList: Seq[Expression],
+      exclusiveAttrs: Seq[Attribute]): Option[LogicalPlan] = {
+    val attrToExtractValues = getAttributeToExtractValues(exprList, exclusiveAttrs)
+    if (attrToExtractValues.isEmpty) {
+      None
+    } else {
+      Some(rewritePlanWithAliases(plan, attrToExtractValues))
+    }
+  }
+
   /**
    * Replace nested columns to prune unused nested columns later.
    */
-  private def replaceToAliases(
+  def rewritePlanWithAliases(
       plan: LogicalPlan,
-      nestedFieldToAlias: Map[ExtractValue, Alias],
-      attrToAliases: Map[ExprId, Seq[Alias]]): LogicalPlan = plan match {
-    case Project(projectList, child) =>
-      Project(
-        getNewProjectList(projectList, nestedFieldToAlias),
-        replaceWithAliases(child, nestedFieldToAlias, attrToAliases))
-
-    // The operators reaching here was already guarded by `canPruneOn`.
-    case other =>
-      replaceWithAliases(other, nestedFieldToAlias, attrToAliases)
+      attributeToExtractValues: Map[Attribute, Seq[ExtractValue]]): LogicalPlan = {
+    // Each expression can contain multiple nested fields.
+    // Note that we keep the original names to deliver to parquet in a case-sensitive way.
+    // A new alias is created for each nested field.
+    val nestedFieldToAlias = attributeToExtractValues.flatMap { case (_, nestedFields) =>
+      nestedFields.map { f =>
+        val exprId = NamedExpression.newExprId
+        val fieldName = f match {
+          case g: GetStructField => g.extractFieldName
+          case g: GetArrayStructFields => g.field.name
+        }
+        f -> Alias(f, s"_extract_${fieldName}")(exprId, Seq.empty, None)
+      }
+    }
+
+    // A reference attribute can have multiple aliases for nested fields.
+    val attrToAliases = attributeToExtractValues.map { case (attr, nestedFields) =>
+      attr.exprId -> nestedFields.map(nestedFieldToAlias)

Review comment:
       this map lookup looks fragile as it use java `equals` not semantic equals. how about
   ```
   val attributeToAliasesExtractValues = AttributeMap(attributeToExtractValues.map ...)
   
   val nestedFieldToAlias = attributeToAliasesExtractValues.flatMap...
   ```




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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-825987652


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42401/
   


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


[GitHub] [spark] AmplabJenkins commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-842737668


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/138642/
   


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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r620111958



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
##########
@@ -783,7 +783,7 @@ object ColumnPruning extends Rule[LogicalPlan] {
       p.copy(child = g.copy(child = newChild, unrequiredChildIndex = unrequiredIndices))
 
     // prune unrequired nested fields from `Generate`.
-    case GeneratorNestedColumnAliasing(p) => p
+    case p @ GeneratorNestedColumnAliasing(rewrittenPlan) => rewrittenPlan

Review comment:
       do we need the `p @`?




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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-828949940






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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r622727661



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -133,83 +204,77 @@ 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
+  }
+
+  /**
+   * Creates a map from root [[Attribute]]s to non-redundant nested [[ExtractValue]]s.
+   * Nested field accessors of `exclusiveAttrs` are not considered in nested fields aliasing.
+   */
+  def getAttributeToExtractValues(
+      exprList: Seq[Expression],
+      exclusiveAttrs: Seq[Attribute]): Map[Attribute, Seq[ExtractValue]] = {
 
-    // Note that when we group by extractors with their references, we should remove
-    // cosmetic variations.
+    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
+    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 {
+          // See [[collectExtractValue]]: we only need to deal with [[GetArrayStructFields]] and
+          // [[GetStructField]]
           case e @ (_: GetStructField | _: GetArrayStructFields) =>
             val child = e.children.head
             nestedFields.forall(f => child.find(_.semanticEquals(f)).isEmpty)
           case _ => true
-        }
-
-        // Each expression can contain multiple nested fields.
-        // Note that we keep the original names to deliver to parquet in a case-sensitive way.
-        val nestedFieldToAlias = dedupNestedFields.distinct.map { f =>
-          val exprId = NamedExpression.newExprId
-          (f, Alias(f, s"_gen_alias_${exprId.id}")(exprId, Seq.empty, None))
-        }
+        }.distinct

Review comment:
       why don't we do `.map(_.canonicalized).distinct` here?




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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-842751190


   **[Test build #138643 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138643/testReport)** for PR 32301 at commit [`f49663c`](https://github.com/apache/spark/commit/f49663c11e79f19aa250c50b6f5ce787c987cad7).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


[GitHub] [spark] cloud-fan commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-849401356


   seems a legit error in `org.apache.spark.sql.hive.execution.HiveCompatibilitySuite.udf_struct`


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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r638924260



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -149,67 +234,52 @@ object NestedColumnAliasing {
   }
 
   /**
-   * 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.
+   * Creates a map from root [[Attribute]]s to non-redundant nested [[ExtractValue]]s.
+   * Nested field accessors of `exclusiveAttrs` are not considered in nested fields aliasing.
    */
-  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
+  def getAttributeToExtractValues(
+      exprList: Seq[Expression],
+      exclusiveAttrs: Seq[Attribute]): Map[Attribute, Seq[ExtractValue]] = {
+
+    val nestedFieldReferences = new mutable.ArrayBuffer[ExtractValue]()
+    val otherRootReferences = new mutable.ArrayBuffer[AttributeReference]()
+    exprList.foreach { e =>
+      collectRootReferenceAndExtractValue(e).foreach {
+        case ev: ExtractValue => nestedFieldReferences.append(ev)
+        case ar: AttributeReference => otherRootReferences.append(ar)
       }
-
-    // Note that when we group by extractors with their references, we should remove
-    // cosmetic variations.
+    }
     val exclusiveAttrSet = AttributeSet(exclusiveAttrs ++ otherRootReferences)
-    val aliasSub = nestedFieldReferences.asInstanceOf[Seq[ExtractValue]]
+
+    // Remove cosmetic variations when we group extractors by their references
+    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`.
+        // Because `a.b` requires all of the inner fields of `b`, we cannot prune `a.b.c`.
         val dedupNestedFields = nestedFields.filter {
+          // See [[collectExtractValue]]: we only need to deal with [[GetArrayStructFields]] and
+          // [[GetStructField]]
           case e @ (_: GetStructField | _: GetArrayStructFields) =>
             val child = e.children.head
             nestedFields.forall(f => child.find(_.semanticEquals(f)).isEmpty)
           case _ => true
-        }
-
-        // Each expression can contain multiple nested fields.
-        // Note that we keep the original names to deliver to parquet in a case-sensitive way.
-        val nestedFieldToAlias = dedupNestedFields.distinct.map { f =>
-          val exprId = NamedExpression.newExprId
-          (f, Alias(f, s"_gen_alias_${exprId.id}")(exprId, Seq.empty, None))
-        }
+        }.distinct

Review comment:
       I'm still a bit worried about calling distinct on uncanonicalized expressions. how about 
   ```
   .groupBy(_.canonicalized).values.map(_.head).toSeq
   ```




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


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

Posted by GitBox <gi...@apache.org>.
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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-842675855


   **[Test build #138640 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138640/testReport)** for PR 32301 at commit [`351e74a`](https://github.com/apache/spark/commit/351e74ade348fddc04dd2cc86bd5bc9bdf2d538f).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r619296928



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -30,54 +30,61 @@ import org.apache.spark.sql.types._
  */
 object NestedColumnAliasing {

Review comment:
       I think a good first step is to improve the classdoc to explain in detail what we are doing here, with examples.
   
   A good example is `DecorrelateInnerQuery`.




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


[GitHub] [spark] SparkQA removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-848427649


   **[Test build #138955 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138955/testReport)** for PR 32301 at commit [`31f8bd0`](https://github.com/apache/spark/commit/31f8bd0952f8680fd78e7c8b3579369482d4dd34).


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


[GitHub] [spark] cloud-fan commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-850413251


   thanks, merging to master!


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


[GitHub] [spark] SparkQA removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-828608152


   **[Test build #138050 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138050/testReport)** for PR 32301 at commit [`e47a441`](https://github.com/apache/spark/commit/e47a4415c6c535736310d0114f5ba836a6f6f7c2).


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-828094766






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


[GitHub] [spark] SparkQA removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-828934837


   **[Test build #138067 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138067/testReport)** for PR 32301 at commit [`d74b569`](https://github.com/apache/spark/commit/d74b5698616df1410fdc8cfb0e5db991ba84d321).


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


[GitHub] [spark] sarutak commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
sarutak commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-850470594


   This change seems to break the build with Scala 2.13 on GA.
   I'll open a PR to fix it.


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


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

Posted by GitBox <gi...@apache.org>.
karenfeng commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r639351572



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -17,71 +17,155 @@
 
 package org.apache.spark.sql.catalyst.optimizer
 
+import scala.collection.mutable
+
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 
 /**
- * This aims to handle a nested column aliasing pattern inside the `ColumnPruning` optimizer rule.
- * If a project or its child references to nested fields, and not all the fields
- * in a nested attribute are used, we can substitute them by alias attributes; then a project
- * of the nested fields as aliases on the children of the child will be created.
+ * This aims to handle a nested column aliasing pattern inside the [[ColumnPruning]] optimizer rule.
+ * If:
+ * - A [[Project]] or its child references nested fields
+ * - Not all of the fields in a nested attribute are used
+ * Then:
+ * - Substitute the nested field references with alias attributes
+ * - Add grandchild [[Project]]s transforming the nested fields to aliases
+ *
+ * Example 1: Project
+ * ------------------
+ * Before:
+ * +- Project [concat_ws(s#0.a, s#0.b) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [concat_ws(_extract_a#2, _extract_b#3) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _extract_a#2, s#0.b AS _extract_b#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * Example 2: Project above Filter
+ * -------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1]
+ *   +- Filter (length(s#0.b) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_extract_a#2 AS s.a#1]
+ *   +- Filter (length(_extract_b#3) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- Project [s#0.a AS _extract_a#2, s#0.b AS _extract_b#3]
+ *           +- LocalRelation <empty>, [s#0]
+ *
+ * Example 3: Nested fields with referenced parents
+ * ------------------------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1, s#0.a.a1 AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_extract_a#3 AS s.a#1, _extract_a#3.name AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _extract_a#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * The schema of the datasource relation will be pruned in the [[SchemaPruning]] optimizer rule.
  */
 object NestedColumnAliasing {
 
   def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match {
     /**
      * This pattern is needed to support [[Filter]] plan cases like
-     * [[Project]]->[[Filter]]->listed plan in `canProjectPushThrough` (e.g., [[Window]]).
-     * The reason why we don't simply add [[Filter]] in `canProjectPushThrough` is that
+     * [[Project]]->[[Filter]]->listed plan in [[canProjectPushThrough]] (e.g., [[Window]]).
+     * The reason why we don't simply add [[Filter]] in [[canProjectPushThrough]] is that
      * the optimizer can hit an infinite loop during the [[PushDownPredicates]] rule.
      */
-    case Project(projectList, Filter(condition, child))
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
-      val exprCandidatesToPrune = projectList ++ Seq(condition) ++ child.expressions
-      getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases)
-      }
+    case Project(projectList, Filter(condition, child)) if
+        SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
+      rewritePlanIfSubsetFieldsUsed(
+        plan, projectList ++ Seq(condition) ++ child.expressions, child.producedAttributes.toSeq)
 
-    case Project(projectList, child)
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
-      val exprCandidatesToPrune = projectList ++ child.expressions
-      getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases)
-      }
+    case Project(projectList, child) if
+        SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
+      rewritePlanIfSubsetFieldsUsed(
+        plan, projectList ++ child.expressions, child.producedAttributes.toSeq)
 
     case p if SQLConf.get.nestedSchemaPruningEnabled && canPruneOn(p) =>
-      val exprCandidatesToPrune = p.expressions
-      getAliasSubMap(exprCandidatesToPrune, p.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(p, nestedFieldToAlias, attrToAliases)
-      }
+      rewritePlanIfSubsetFieldsUsed(
+        plan, p.expressions, p.producedAttributes.toSeq)
 
     case _ => None
   }
 
+  /**
+   * Rewrites a plan with aliases if only a subset of the nested fields are used.
+   */
+  def rewritePlanIfSubsetFieldsUsed(
+      plan: LogicalPlan,
+      exprList: Seq[Expression],
+      exclusiveAttrs: Seq[Attribute]): Option[LogicalPlan] = {
+    val attrToExtractValues = getAttributeToExtractValues(exprList, exclusiveAttrs)
+    if (attrToExtractValues.isEmpty) {
+      None
+    } else {
+      Some(rewritePlanWithAliases(plan, attrToExtractValues))
+    }
+  }
+
   /**
    * Replace nested columns to prune unused nested columns later.
    */
-  private def replaceToAliases(
+  def rewritePlanWithAliases(
       plan: LogicalPlan,
-      nestedFieldToAlias: Map[ExtractValue, Alias],
-      attrToAliases: Map[ExprId, Seq[Alias]]): LogicalPlan = plan match {
-    case Project(projectList, child) =>
-      Project(
-        getNewProjectList(projectList, nestedFieldToAlias),
-        replaceWithAliases(child, nestedFieldToAlias, attrToAliases))
-
-    // The operators reaching here was already guarded by `canPruneOn`.
-    case other =>
-      replaceWithAliases(other, nestedFieldToAlias, attrToAliases)
+      attributeToExtractValues: Map[Attribute, Seq[ExtractValue]]): LogicalPlan = {
+    // Each expression can contain multiple nested fields.
+    // Note that we keep the original names to deliver to parquet in a case-sensitive way.
+    // A new alias is created for each nested field.
+    // Implementation detail: we don't use mapValues, because it creates a mutable view.
+    val attributeToExtractValuesAndAliases =
+      attributeToExtractValues.map { case (attr, evSeq) =>
+        val evAliasSeq = evSeq.map { ev =>
+          val fieldName = ev match {
+            case g: GetStructField => g.extractFieldName
+            case g: GetArrayStructFields => g.field.name
+          }
+          ev -> Alias(ev, s"_extract_$fieldName")()
+        }
+
+        attr -> evAliasSeq
+      }
+
+    val nestedFieldToAlias = attributeToExtractValuesAndAliases.values.flatten.toMap
+
+    // A reference attribute can have multiple aliases for nested fields.
+    val attrToAliases = new AttributeMap(

Review comment:
       I think we'll have to introduce a new constructor for `AttributeMap` to accept a `Map` for this to be true - I can push it into the next commit.




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


[GitHub] [spark] SparkQA removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-827944457


   **[Test build #138008 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138008/testReport)** for PR 32301 at commit [`f48b376`](https://github.com/apache/spark/commit/f48b376334f415f0de95d09c62b8d3ce0b8c23ed).


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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-828066707


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/42531/
   


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


[GitHub] [spark] SparkQA removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-849958280


   **[Test build #139035 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/139035/testReport)** for PR 32301 at commit [`8a29e94`](https://github.com/apache/spark/commit/8a29e943447808391c17f860598e3f11ae41d54d).


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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-848445019


   **[Test build #138955 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138955/testReport)** for PR 32301 at commit [`31f8bd0`](https://github.com/apache/spark/commit/31f8bd0952f8680fd78e7c8b3579369482d4dd34).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-850053105


   **[Test build #139035 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/139035/testReport)** for PR 32301 at commit [`8a29e94`](https://github.com/apache/spark/commit/8a29e943447808391c17f860598e3f11ae41d54d).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-849958280


   **[Test build #139035 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/139035/testReport)** for PR 32301 at commit [`8a29e94`](https://github.com/apache/spark/commit/8a29e943447808391c17f860598e3f11ae41d54d).


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


[GitHub] [spark] cloud-fan commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-848960183


   @viirya any more comments?


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


[GitHub] [spark] viirya commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
viirya commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-848988922


   Thanks @cloud-fan @karenfeng. I will check this again today. 


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


[GitHub] [spark] AmplabJenkins commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-842547220


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/43160/
   


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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r622728488



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -17,71 +17,148 @@
 
 package org.apache.spark.sql.catalyst.optimizer
 
+import scala.collection.mutable
+
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 
 /**
- * This aims to handle a nested column aliasing pattern inside the `ColumnPruning` optimizer rule.
- * If a project or its child references to nested fields, and not all the fields
- * in a nested attribute are used, we can substitute them by alias attributes; then a project
- * of the nested fields as aliases on the children of the child will be created.
+ * This aims to handle a nested column aliasing pattern inside the [[ColumnPruning]] optimizer rule.
+ * If:
+ * - A [[Project]] or its child references nested fields
+ * - Not all of the fields in a nested attribute are used
+ * Then:
+ * - Substitute the nested field references with alias attributes
+ * - Add grandchild [[Project]]s transforming the nested fields to aliases
+ *
+ * Example 1: Project
+ * ------------------
+ * Before:
+ * +- Project [concat_ws(s#0.a, s#0.b) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [concat_ws(_gen_alias_2#2, _gen_alias_3#3) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _gen_alias_2#2, s#0.b AS _gen_alias_3#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * Example 2: Project above Filter
+ * -------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1]
+ *   +- Filter (length(s#0.b) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_gen_alias_2#2 AS s.a#1]
+ *   +- Filter (length(_gen_alias_3#3) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- Project [s#0.a AS _gen_alias_2#2, s#0.b AS _gen_alias_3#3]
+ *           +- LocalRelation <empty>, [s#0]
+ *
+ * Example 3: Nested columns in nested columns
+ * -------------------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1, s#0.a.a1 AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_gen_alias_3#3 AS s.a#1, _gen_alias_3#3.name AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _gen_alias_3#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * The schema of the datasource relation will be pruned in the [[SchemaPruning]] optimizer rule.
  */
 object NestedColumnAliasing {
 
   def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match {
     /**
      * This pattern is needed to support [[Filter]] plan cases like
-     * [[Project]]->[[Filter]]->listed plan in `canProjectPushThrough` (e.g., [[Window]]).
-     * The reason why we don't simply add [[Filter]] in `canProjectPushThrough` is that
+     * [[Project]]->[[Filter]]->listed plan in [[canProjectPushThrough]] (e.g., [[Window]]).
+     * The reason why we don't simply add [[Filter]] in [[canProjectPushThrough]] is that
      * the optimizer can hit an infinite loop during the [[PushDownPredicates]] rule.
      */
-    case Project(projectList, Filter(condition, child))
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
-      val exprCandidatesToPrune = projectList ++ Seq(condition) ++ child.expressions
-      getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases)
-      }
+    case Project(projectList, Filter(condition, child)) if
+        SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
+      rewritePlanIfSubsetFieldsUsed(
+        plan, projectList ++ Seq(condition) ++ child.expressions, child.producedAttributes.toSeq)
 
-    case Project(projectList, child)
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
-      val exprCandidatesToPrune = projectList ++ child.expressions
-      getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases)
-      }
+    case Project(projectList, child) if
+        SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
+      rewritePlanIfSubsetFieldsUsed(
+        plan, projectList ++ child.expressions, child.producedAttributes.toSeq)
 
     case p if SQLConf.get.nestedSchemaPruningEnabled && canPruneOn(p) =>
-      val exprCandidatesToPrune = p.expressions
-      getAliasSubMap(exprCandidatesToPrune, p.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(p, nestedFieldToAlias, attrToAliases)
-      }
+      rewritePlanIfSubsetFieldsUsed(
+        plan, p.expressions, p.producedAttributes.toSeq)
 
     case _ => None
   }
 
+  /**
+   * Rewrites a plan with aliases if only a subset of the nested fields are used.
+   */
+  def rewritePlanIfSubsetFieldsUsed(
+      plan: LogicalPlan,
+      exprList: Seq[Expression],
+      exclusiveAttrs: Seq[Attribute]): Option[LogicalPlan] = {
+    val attrToExtractValues = getAttributeToExtractValues(exprList, exclusiveAttrs)
+    if (attrToExtractValues.isEmpty) {
+      None
+    } else {
+      Some(rewritePlanWithAliases(plan, attrToExtractValues))
+    }
+  }
+
   /**
    * Replace nested columns to prune unused nested columns later.
    */
-  private def replaceToAliases(
+  def rewritePlanWithAliases(
       plan: LogicalPlan,
-      nestedFieldToAlias: Map[ExtractValue, Alias],
-      attrToAliases: Map[ExprId, Seq[Alias]]): LogicalPlan = plan match {
-    case Project(projectList, child) =>
-      Project(
-        getNewProjectList(projectList, nestedFieldToAlias),
-        replaceWithAliases(child, nestedFieldToAlias, attrToAliases))
-
-    // The operators reaching here was already guarded by `canPruneOn`.
-    case other =>
-      replaceWithAliases(other, nestedFieldToAlias, attrToAliases)
+      attributeToExtractValues: Map[Attribute, Seq[ExtractValue]]): LogicalPlan = {
+    // Each expression can contain multiple nested fields.
+    // Note that we keep the original names to deliver to parquet in a case-sensitive way.
+    // A new alias is created for each nested field.
+    val nestedFieldToAlias = attributeToExtractValues.flatMap { case (_, nestedFields) =>
+      nestedFields.map { f =>
+        val exprId = NamedExpression.newExprId
+        val fieldName = f match {
+          case g: GetStructField => g.extractFieldName
+          case g: GetArrayStructFields => g.field.name
+        }
+        f -> Alias(f, s"_extract_${fieldName}")(exprId, Seq.empty, None)

Review comment:
       nit: `Alias(f, s"_extract_${fieldName}")()`. We don't need to generate the expr ID manually.




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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-829086250


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/138067/
   


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


[GitHub] [spark] viirya commented on pull request #32301: [SPARK-35194] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
viirya commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-825052954


   Thanks @karenfeng. I will take a look on 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:
users@infra.apache.org



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


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

Posted by GitBox <gi...@apache.org>.
karenfeng commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r619521322



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -30,54 +30,61 @@ import org.apache.spark.sql.types._
  */
 object NestedColumnAliasing {

Review comment:
       I wasn't sure how much to add to the classdoc, but I added some more examples.




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


[GitHub] [spark] AmplabJenkins commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-849334223


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/138997/
   


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


[GitHub] [spark] AmplabJenkins commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-849993315


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/43553/
   


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


[GitHub] [spark] AmplabJenkins commented on pull request #32301: [SPARK-35194] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-825216206


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137818/
   


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


[GitHub] [spark] AmplabJenkins commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-842759448


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/138643/
   


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


[GitHub] [spark] AmplabJenkins commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-828094766






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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r621969848



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -23,78 +23,149 @@ import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 
 /**
- * This aims to handle a nested column aliasing pattern inside the `ColumnPruning` optimizer rule.
- * If a project or its child references to nested fields, and not all the fields
- * in a nested attribute are used, we can substitute them by alias attributes; then a project
- * of the nested fields as aliases on the children of the child will be created.
+ * This aims to handle a nested column aliasing pattern inside the [[ColumnPruning]] optimizer rule.
+ * If:
+ * - A [[Project]] or its child references nested fields
+ * - Not all of the fields in a nested attribute are used
+ * Then:
+ * - Substitute the nested field references with alias attributes
+ * - Add grandchild [[Project]]s transforming the nested fields to aliases
+ *
+ * Example 1: Project
+ * ------------------
+ * Before:
+ * +- Project [concat_ws(s#0.a, s#0.b) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [concat_ws(_gen_alias_2#2, _gen_alias_3#3) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _gen_alias_2#2, s#0.b AS _gen_alias_3#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * Example 2: Project above Filter
+ * -------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1]
+ *   +- Filter (length(s#0.b) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_gen_alias_2#2 AS s.a#1]
+ *   +- Filter (length(_gen_alias_3#3) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- Project [s#0.a AS _gen_alias_2#2, s#0.b AS _gen_alias_3#3]
+ *           +- LocalRelation <empty>, [s#0]
+ *
+ * Example 3: Nested columns in nested columns
+ * -------------------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1, s#0.a.a1 AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_gen_alias_3#3 AS s.a#1, _gen_alias_3#3.name AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _gen_alias_3#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * The schema of the datasource relation will be pruned in the [[SchemaPruning]] optimizer rule.
  */
 object NestedColumnAliasing {
 
   def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match {
     /**
      * This pattern is needed to support [[Filter]] plan cases like
-     * [[Project]]->[[Filter]]->listed plan in `canProjectPushThrough` (e.g., [[Window]]).
-     * The reason why we don't simply add [[Filter]] in `canProjectPushThrough` is that
+     * [[Project]]->[[Filter]]->listed plan in [[canProjectPushThrough]] (e.g., [[Window]]).
+     * The reason why we don't simply add [[Filter]] in [[canProjectPushThrough]] is that
      * the optimizer can hit an infinite loop during the [[PushDownPredicates]] rule.
      */
-    case Project(projectList, Filter(condition, child))
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
-      val exprCandidatesToPrune = projectList ++ Seq(condition) ++ child.expressions
-      getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases)
-      }
+    case Project(projectList, Filter(condition, child)) if
+        SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
+      rewritePlanIfSubsetFieldsUsed(
+        plan, projectList ++ Seq(condition) ++ child.expressions, child.producedAttributes.toSeq)
 
-    case Project(projectList, child)
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
-      val exprCandidatesToPrune = projectList ++ child.expressions
-      getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases)
-      }
+    case Project(projectList, child) if
+        SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
+      rewritePlanIfSubsetFieldsUsed(
+        plan, projectList ++ child.expressions, child.producedAttributes.toSeq)
 
     case p if SQLConf.get.nestedSchemaPruningEnabled && canPruneOn(p) =>
-      val exprCandidatesToPrune = p.expressions
-      getAliasSubMap(exprCandidatesToPrune, p.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(p, nestedFieldToAlias, attrToAliases)
-      }
+      rewritePlanIfSubsetFieldsUsed(
+        plan, p.expressions, p.producedAttributes.toSeq)
 
     case _ => None
   }
 
+  /**
+   * Rewrites a plan with aliases if only a subset of the nested fields are used.
+   */
+  def rewritePlanIfSubsetFieldsUsed(
+    plan: LogicalPlan,
+    exprList: Seq[Expression],
+    exclusiveAttrs: Seq[Attribute]): Option[LogicalPlan] = {
+    val attrToExtractValues = getAttributeToExtractValues(exprList, exclusiveAttrs)
+    if (attrToExtractValues.isEmpty) {
+      None
+    } else {
+      Some(rewritePlanWithAliases(plan, attrToExtractValues))
+    }
+  }
+
   /**
    * Replace nested columns to prune unused nested columns later.
    */
-  private def replaceToAliases(
+  def rewritePlanWithAliases(
       plan: LogicalPlan,
-      nestedFieldToAlias: Map[ExtractValue, Alias],
-      attrToAliases: Map[ExprId, Seq[Alias]]): LogicalPlan = plan match {
-    case Project(projectList, child) =>
-      Project(
-        getNewProjectList(projectList, nestedFieldToAlias),
-        replaceWithAliases(child, nestedFieldToAlias, attrToAliases))
+      attributeToExtractValues: Map[Attribute, Seq[ExtractValue]]): LogicalPlan = {
+      // Each expression can contain multiple nested fields.
+      // Note that we keep the original names to deliver to parquet in a case-sensitive way.
+      // A new alias is created for each nested field.
+      val nestedFieldToAlias = attributeToExtractValues.flatMap { case (_, nestedFields) =>
+        nestedFields.map { f =>
+          val exprId = NamedExpression.newExprId
+          f -> Alias(f, s"_gen_alias_${exprId.id}")(exprId, Seq.empty, None)

Review comment:
       SGTM




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


[GitHub] [spark] karenfeng commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
karenfeng commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-849845128


   @viirya - in the case that the number of references is `!=1`, should we exclude the `ExtractValue` from `nestedFieldReferences`?


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


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

Posted by GitBox <gi...@apache.org>.
karenfeng commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r639953349



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -149,67 +234,52 @@ object NestedColumnAliasing {
   }
 
   /**
-   * 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.
+   * Creates a map from root [[Attribute]]s to non-redundant nested [[ExtractValue]]s.
+   * Nested field accessors of `exclusiveAttrs` are not considered in nested fields aliasing.
    */
-  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
+  def getAttributeToExtractValues(
+      exprList: Seq[Expression],
+      exclusiveAttrs: Seq[Attribute]): Map[Attribute, Seq[ExtractValue]] = {
+
+    val nestedFieldReferences = new mutable.ArrayBuffer[ExtractValue]()
+    val otherRootReferences = new mutable.ArrayBuffer[AttributeReference]()
+    exprList.foreach { e =>
+      collectRootReferenceAndExtractValue(e).foreach {
+        case ev: ExtractValue => nestedFieldReferences.append(ev)

Review comment:
       Looks like adding an assertion here actually broke `org.apache.spark.sql.hive.execution.HiveCompatibilitySuite.udf_struct`: See https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138955/testReport/org.apache.spark.sql.hive.execution/HiveCompatibilitySuite/udf_struct/.




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


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

Posted by GitBox <gi...@apache.org>.
karenfeng commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r640014333



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -149,67 +234,52 @@ object NestedColumnAliasing {
   }
 
   /**
-   * 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.
+   * Creates a map from root [[Attribute]]s to non-redundant nested [[ExtractValue]]s.
+   * Nested field accessors of `exclusiveAttrs` are not considered in nested fields aliasing.
    */
-  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
+  def getAttributeToExtractValues(
+      exprList: Seq[Expression],
+      exclusiveAttrs: Seq[Attribute]): Map[Attribute, Seq[ExtractValue]] = {
+
+    val nestedFieldReferences = new mutable.ArrayBuffer[ExtractValue]()
+    val otherRootReferences = new mutable.ArrayBuffer[AttributeReference]()
+    exprList.foreach { e =>
+      collectRootReferenceAndExtractValue(e).foreach {
+        case ev: ExtractValue => nestedFieldReferences.append(ev)

Review comment:
       @viirya, what do you recommend doing in this case?




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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-849979996


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/43553/
   


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


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

Posted by GitBox <gi...@apache.org>.
karenfeng commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r621600381



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -23,78 +23,149 @@ import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 
 /**
- * This aims to handle a nested column aliasing pattern inside the `ColumnPruning` optimizer rule.
- * If a project or its child references to nested fields, and not all the fields
- * in a nested attribute are used, we can substitute them by alias attributes; then a project
- * of the nested fields as aliases on the children of the child will be created.
+ * This aims to handle a nested column aliasing pattern inside the [[ColumnPruning]] optimizer rule.
+ * If:
+ * - A [[Project]] or its child references nested fields
+ * - Not all of the fields in a nested attribute are used
+ * Then:
+ * - Substitute the nested field references with alias attributes
+ * - Add grandchild [[Project]]s transforming the nested fields to aliases
+ *
+ * Example 1: Project
+ * ------------------
+ * Before:
+ * +- Project [concat_ws(s#0.a, s#0.b) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [concat_ws(_gen_alias_2#2, _gen_alias_3#3) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _gen_alias_2#2, s#0.b AS _gen_alias_3#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * Example 2: Project above Filter
+ * -------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1]
+ *   +- Filter (length(s#0.b) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_gen_alias_2#2 AS s.a#1]
+ *   +- Filter (length(_gen_alias_3#3) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- Project [s#0.a AS _gen_alias_2#2, s#0.b AS _gen_alias_3#3]
+ *           +- LocalRelation <empty>, [s#0]
+ *
+ * Example 3: Nested columns in nested columns
+ * -------------------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1, s#0.a.a1 AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_gen_alias_3#3 AS s.a#1, _gen_alias_3#3.name AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _gen_alias_3#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * The schema of the datasource relation will be pruned in the [[SchemaPruning]] optimizer rule.
  */
 object NestedColumnAliasing {
 
   def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match {
     /**
      * This pattern is needed to support [[Filter]] plan cases like
-     * [[Project]]->[[Filter]]->listed plan in `canProjectPushThrough` (e.g., [[Window]]).
-     * The reason why we don't simply add [[Filter]] in `canProjectPushThrough` is that
+     * [[Project]]->[[Filter]]->listed plan in [[canProjectPushThrough]] (e.g., [[Window]]).
+     * The reason why we don't simply add [[Filter]] in [[canProjectPushThrough]] is that
      * the optimizer can hit an infinite loop during the [[PushDownPredicates]] rule.
      */
-    case Project(projectList, Filter(condition, child))
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
-      val exprCandidatesToPrune = projectList ++ Seq(condition) ++ child.expressions
-      getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases)
-      }
+    case Project(projectList, Filter(condition, child)) if
+        SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
+      rewritePlanIfSubsetFieldsUsed(
+        plan, projectList ++ Seq(condition) ++ child.expressions, child.producedAttributes.toSeq)
 
-    case Project(projectList, child)
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
-      val exprCandidatesToPrune = projectList ++ child.expressions
-      getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases)
-      }
+    case Project(projectList, child) if
+        SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
+      rewritePlanIfSubsetFieldsUsed(
+        plan, projectList ++ child.expressions, child.producedAttributes.toSeq)
 
     case p if SQLConf.get.nestedSchemaPruningEnabled && canPruneOn(p) =>
-      val exprCandidatesToPrune = p.expressions
-      getAliasSubMap(exprCandidatesToPrune, p.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(p, nestedFieldToAlias, attrToAliases)
-      }
+      rewritePlanIfSubsetFieldsUsed(
+        plan, p.expressions, p.producedAttributes.toSeq)
 
     case _ => None
   }
 
+  /**
+   * Rewrites a plan with aliases if only a subset of the nested fields are used.
+   */
+  def rewritePlanIfSubsetFieldsUsed(
+    plan: LogicalPlan,
+    exprList: Seq[Expression],
+    exclusiveAttrs: Seq[Attribute]): Option[LogicalPlan] = {
+    val attrToExtractValues = getAttributeToExtractValues(exprList, exclusiveAttrs)
+    if (attrToExtractValues.isEmpty) {
+      None
+    } else {
+      Some(rewritePlanWithAliases(plan, attrToExtractValues))
+    }
+  }
+
   /**
    * Replace nested columns to prune unused nested columns later.
    */
-  private def replaceToAliases(
+  def rewritePlanWithAliases(
       plan: LogicalPlan,
-      nestedFieldToAlias: Map[ExtractValue, Alias],
-      attrToAliases: Map[ExprId, Seq[Alias]]): LogicalPlan = plan match {
-    case Project(projectList, child) =>
-      Project(
-        getNewProjectList(projectList, nestedFieldToAlias),
-        replaceWithAliases(child, nestedFieldToAlias, attrToAliases))
+      attributeToExtractValues: Map[Attribute, Seq[ExtractValue]]): LogicalPlan = {
+      // Each expression can contain multiple nested fields.
+      // Note that we keep the original names to deliver to parquet in a case-sensitive way.
+      // A new alias is created for each nested field.
+      val nestedFieldToAlias = attributeToExtractValues.flatMap { case (_, nestedFields) =>
+        nestedFields.map { f =>
+          val exprId = NamedExpression.newExprId
+          f -> Alias(f, s"_gen_alias_${exprId.id}")(exprId, Seq.empty, None)

Review comment:
       Not particularly. I discussed this with @allisonwang-db offline, and we think it may be more useful for this name to reflect the struct and field. What do you think?




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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-842512761


   **[Test build #138640 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138640/testReport)** for PR 32301 at commit [`351e74a`](https://github.com/apache/spark/commit/351e74ade348fddc04dd2cc86bd5bc9bdf2d538f).


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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-828089916


   **[Test build #138011 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138011/testReport)** for PR 32301 at commit [`69ca24e`](https://github.com/apache/spark/commit/69ca24eda8bca7216863d4c74775c342aa533b31).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


[GitHub] [spark] AmplabJenkins commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-828794503


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/138050/
   


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


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

Posted by GitBox <gi...@apache.org>.
allisonwang-db commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r618834219



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -30,54 +30,61 @@ import org.apache.spark.sql.types._
  */
 object NestedColumnAliasing {
 
-  def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match {
+  def unapply(plan: LogicalPlan): Option[Map[Attribute, Seq[ExtractValue]]] = plan match {

Review comment:
       It's a bit unnatural to return an option of map for getAttributeToExtractValues. How about using the unapply method to match the pattern and extract the information needed to build the map: 
   ```scala
   def unapply(plan: LogicalPlan): Option[(Seq[Expression], Seq[Attribute])] = plan match {
     case Project(projectList, g: Generate) if (SQLConf.get.nestedPruningOnExpressions ||
         SQLConf.get.nestedSchemaPruningEnabled) && canPruneGenerator(g.generator) =>
       Some((projectList ++ g.generator.children, g.qualifiedGeneratorOutput))
     case ...
   }
   ```
   Then build the mapping in the optimizer rule.
   ```scala
   case p @ NestedColumnAliasing((exprs, excludedAttrs)) =>
     ...
   ```

##########
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:
       This seems to be the same as the original logic?

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -30,54 +30,61 @@ import org.apache.spark.sql.types._
  */
 object NestedColumnAliasing {
 
-  def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match {
+  def unapply(plan: LogicalPlan): Option[Map[Attribute, Seq[ExtractValue]]] = plan match {
     /**
      * This pattern is needed to support [[Filter]] plan cases like
      * [[Project]]->[[Filter]]->listed plan in `canProjectPushThrough` (e.g., [[Window]]).
      * The reason why we don't simply add [[Filter]] in `canProjectPushThrough` is that
      * the optimizer can hit an infinite loop during the [[PushDownPredicates]] rule.
      */
-    case Project(projectList, Filter(condition, child))
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
-      val exprCandidatesToPrune = projectList ++ Seq(condition) ++ child.expressions
-      getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases)
-      }
+    case Project(projectList, Filter(condition, child)) if
+        SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
+      getAttributeToExtractValues(
+        projectList ++ Seq(condition) ++ child.expressions, child.producedAttributes.toSeq)
 
-    case Project(projectList, child)
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
-      val exprCandidatesToPrune = projectList ++ child.expressions
-      getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases)
-      }
+    case Project(projectList, child) if
+        SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
+      getAttributeToExtractValues(
+        projectList ++ child.expressions, child.producedAttributes.toSeq)
 
     case p if SQLConf.get.nestedSchemaPruningEnabled && canPruneOn(p) =>
-      val exprCandidatesToPrune = p.expressions
-      getAliasSubMap(exprCandidatesToPrune, p.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(p, nestedFieldToAlias, attrToAliases)
-      }
+      getAttributeToExtractValues(
+        p.expressions, p.producedAttributes.toSeq)
 
     case _ => None
   }
 
   /**
    * Replace nested columns to prune unused nested columns later.
    */
-  private def replaceToAliases(
-      plan: LogicalPlan,
-      nestedFieldToAlias: Map[ExtractValue, Alias],
-      attrToAliases: Map[ExprId, Seq[Alias]]): LogicalPlan = plan match {
-    case Project(projectList, child) =>
-      Project(
-        getNewProjectList(projectList, nestedFieldToAlias),
-        replaceWithAliases(child, nestedFieldToAlias, attrToAliases))
+  def replacePlanWithAliases(
+        plan: LogicalPlan,

Review comment:
       nit: 4 space indentation

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -227,11 +236,11 @@ object NestedColumnAliasing {
 }
 
 /**
- * This prunes unnecessary nested columns from `Generate` and optional `Project` on top
+ * This prunes unnecessary nested columns from [[Generate]] and optional [[Project]] on top
  * of it.
  */
 object GeneratorNestedColumnAliasing {
-  def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match {
+  def unapply(plan: LogicalPlan): Option[Map[Attribute, Seq[ExtractValue]]] = plan match {

Review comment:
       ditto




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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-828155775


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/138012/
   


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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r638918033



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -17,71 +17,155 @@
 
 package org.apache.spark.sql.catalyst.optimizer
 
+import scala.collection.mutable
+
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 
 /**
- * This aims to handle a nested column aliasing pattern inside the `ColumnPruning` optimizer rule.
- * If a project or its child references to nested fields, and not all the fields
- * in a nested attribute are used, we can substitute them by alias attributes; then a project
- * of the nested fields as aliases on the children of the child will be created.
+ * This aims to handle a nested column aliasing pattern inside the [[ColumnPruning]] optimizer rule.
+ * If:
+ * - A [[Project]] or its child references nested fields
+ * - Not all of the fields in a nested attribute are used
+ * Then:
+ * - Substitute the nested field references with alias attributes
+ * - Add grandchild [[Project]]s transforming the nested fields to aliases

Review comment:
       not related to this PR: we only add a new Project node as grandchild, but the scan node may still be at the bottom. Do we expect other rules to push down the Project through other nodes until it reaches the bottom?




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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32301: [SPARK-35194] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-825216206


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137818/
   


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-842547220


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/43160/
   


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-850053730


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/139035/
   


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32301: [SPARK-35194] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-825081544


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42348/
   


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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r620104642



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -23,78 +23,149 @@ import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 
 /**
- * This aims to handle a nested column aliasing pattern inside the `ColumnPruning` optimizer rule.
- * If a project or its child references to nested fields, and not all the fields
- * in a nested attribute are used, we can substitute them by alias attributes; then a project
- * of the nested fields as aliases on the children of the child will be created.
+ * This aims to handle a nested column aliasing pattern inside the [[ColumnPruning]] optimizer rule.
+ * If:
+ * - A [[Project]] or its child references nested fields
+ * - Not all of the fields in a nested attribute are used
+ * Then:
+ * - Substitute the nested field references with alias attributes
+ * - Add grandchild [[Project]]s transforming the nested fields to aliases
+ *
+ * Example 1: Project
+ * ------------------
+ * Before:
+ * +- Project [concat_ws(s#0.a, s#0.b) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [concat_ws(_gen_alias_2#2, _gen_alias_3#3) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _gen_alias_2#2, s#0.b AS _gen_alias_3#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * Example 2: Project above Filter
+ * -------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1]
+ *   +- Filter (length(s#0.b) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_gen_alias_2#2 AS s.a#1]
+ *   +- Filter (length(_gen_alias_3#3) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- Project [s#0.a AS _gen_alias_2#2, s#0.b AS _gen_alias_3#3]
+ *           +- LocalRelation <empty>, [s#0]
+ *
+ * Example 3: Nested columns in nested columns

Review comment:
       I think we do support nested column pruning with any levels. The example is more about "Nested fields with parent field being referenced".




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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r638939038



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -227,137 +297,114 @@ object NestedColumnAliasing {
 }
 
 /**
- * This prunes unnecessary nested columns from `Generate` and optional `Project` on top
- * of it.
+ * This prunes unnecessary nested columns from [[Generate]], or [[Project]] -> [[Generate]]
  */
 object GeneratorNestedColumnAliasing {
-  // Partitions `attrToAliases` based on whether the attribute is in Generator's output.
-  private def aliasesOnGeneratorOutput(
-      attrToAliases: Map[ExprId, Seq[Alias]],
-      generatorOutput: Seq[Attribute]) = {
-    val generatorOutputExprId = generatorOutput.map(_.exprId)
-    attrToAliases.partition { k =>
-      generatorOutputExprId.contains(k._1)
-    }
-  }
-
-  // Partitions `nestedFieldToAlias` based on whether the attribute of nested field extractor
-  // is in Generator's output.
-  private def nestedFieldOnGeneratorOutput(
-      nestedFieldToAlias: Map[ExtractValue, Alias],
-      generatorOutput: Seq[Attribute]) = {
-    val generatorOutputSet = AttributeSet(generatorOutput)
-    nestedFieldToAlias.partition { pair =>
-      pair._1.references.subsetOf(generatorOutputSet)
-    }
-  }
-
   def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match {
     // Either `nestedPruningOnExpressions` or `nestedSchemaPruningEnabled` is enabled, we
     // need to prune nested columns through Project and under Generate. The difference is
     // when `nestedSchemaPruningEnabled` is on, nested columns will be pruned further at
     // file format readers if it is supported.
     case Project(projectList, g: Generate) if (SQLConf.get.nestedPruningOnExpressions ||
-        SQLConf.get.nestedSchemaPruningEnabled) && canPruneGenerator(g.generator) =>

Review comment:
       nit: the previous indentation is correct




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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-842759448


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/138643/
   


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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-849246928


   **[Test build #138997 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138997/testReport)** for PR 32301 at commit [`83e2611`](https://github.com/apache/spark/commit/83e2611f47b86b1b35ba4862fc183e5ce6485a0f).


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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-842662471


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/43164/
   


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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-843568889


   **[Test build #138689 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138689/testReport)** for PR 32301 at commit [`80fa5c3`](https://github.com/apache/spark/commit/80fa5c3b46e209c59cc01a539f998969717b83f4).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-849267600


   Kubernetes integration test unable to build dist.
   
   exiting with code: 1
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/43516/
   


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


[GitHub] [spark] SparkQA removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-828051767


   **[Test build #138012 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138012/testReport)** for PR 32301 at commit [`9d41de6`](https://github.com/apache/spark/commit/9d41de6b6c793a414133f36d4113689861e47610).


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-842625011


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/43162/
   


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


[GitHub] [spark] sarutak edited a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
sarutak edited a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-850470594


   This change seems to break the build with Scala 2.13 on GA.
   https://github.com/apache/spark/runs/2694564384
   I'll open a PR to fix it.


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-849334223


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/138997/
   


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-842689666






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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r620133124



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -133,83 +204,77 @@ 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
+  }
+
+  /**
+   * Creates a map from root [[Attribute]]s to non-redundant nested [[ExtractValue]]s.
+   * Nested field accessors of `exclusiveAttrs` are not considered in nested fields aliasing.
+   */
+  def getAttributeToExtractValues(
+      exprList: Seq[Expression],
+      exclusiveAttrs: Seq[Attribute]): Map[Attribute, Seq[ExtractValue]] = {
 
-    // Note that when we group by extractors with their references, we should remove
-    // cosmetic variations.
+    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
+    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`.

Review comment:
       we can add more comments:
   ```
   ..., because `a.b` means all the inner fields of `b` are needed, and we can't prune `a.b.c`.
   ```




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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-825078215


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/42348/
   


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


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

Posted by GitBox <gi...@apache.org>.
karenfeng commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r621605209



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -133,83 +204,77 @@ 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
+  }
+
+  /**
+   * Creates a map from root [[Attribute]]s to non-redundant nested [[ExtractValue]]s.
+   * Nested field accessors of `exclusiveAttrs` are not considered in nested fields aliasing.
+   */
+  def getAttributeToExtractValues(
+      exprList: Seq[Expression],
+      exclusiveAttrs: Seq[Attribute]): Map[Attribute, Seq[ExtractValue]] = {
 
-    // Note that when we group by extractors with their references, we should remove
-    // cosmetic variations.
+    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
+    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 {
+          // See [[collectExtractValue]]: we only need to deal with [[GetArrayStructFields]] and
+          // [[GetStructField]]
           case e @ (_: GetStructField | _: GetArrayStructFields) =>
             val child = e.children.head
             nestedFields.forall(f => child.find(_.semanticEquals(f)).isEmpty)
           case _ => true
-        }
-
-        // Each expression can contain multiple nested fields.
-        // Note that we keep the original names to deliver to parquet in a case-sensitive way.
-        val nestedFieldToAlias = dedupNestedFields.distinct.map { f =>
-          val exprId = NamedExpression.newExprId
-          (f, Alias(f, s"_gen_alias_${exprId.id}")(exprId, Seq.empty, None))
-        }
+        }.distinct

Review comment:
       We use `.map(_.canonicalized).distinct` only for counting the number of nested fields. I believe we still need `.distinct` here to deduplicate the nested fields.




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


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

Posted by GitBox <gi...@apache.org>.
karenfeng commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r633820750



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -133,83 +204,77 @@ 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
+  }
+
+  /**
+   * Creates a map from root [[Attribute]]s to non-redundant nested [[ExtractValue]]s.
+   * Nested field accessors of `exclusiveAttrs` are not considered in nested fields aliasing.
+   */
+  def getAttributeToExtractValues(
+      exprList: Seq[Expression],
+      exclusiveAttrs: Seq[Attribute]): Map[Attribute, Seq[ExtractValue]] = {
 
-    // Note that when we group by extractors with their references, we should remove
-    // cosmetic variations.
+    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
+    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 {
+          // See [[collectExtractValue]]: we only need to deal with [[GetArrayStructFields]] and
+          // [[GetStructField]]
           case e @ (_: GetStructField | _: GetArrayStructFields) =>
             val child = e.children.head
             nestedFields.forall(f => child.find(_.semanticEquals(f)).isEmpty)
           case _ => true
-        }
-
-        // Each expression can contain multiple nested fields.
-        // Note that we keep the original names to deliver to parquet in a case-sensitive way.
-        val nestedFieldToAlias = dedupNestedFields.distinct.map { f =>
-          val exprId = NamedExpression.newExprId
-          (f, Alias(f, s"_gen_alias_${exprId.id}")(exprId, Seq.empty, None))
-        }
+        }.distinct

Review comment:
       When I tried this, the tests failed - turns out that canonicalization strips key information for analysis, such as the name of the AttributeReference (see [ignoreNameTypes](https://github.com/apache/spark/blob/2a335f2d7d1265cb9abd2e727f60d8eb2dfa356b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Canonicalize.scala#L44)).




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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-825043140


   **[Test build #137818 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137818/testReport)** for PR 32301 at commit [`9656899`](https://github.com/apache/spark/commit/9656899c11161aceda5ce46bf764b6c052861410).


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


[GitHub] [spark] AmplabJenkins commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-843410544


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/43210/
   


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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-843410516


   Kubernetes integration test unable to build dist.
   
   exiting with code: 1
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/43210/
   


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


[GitHub] [spark] SparkQA removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-843361599


   **[Test build #138689 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138689/testReport)** for PR 32301 at commit [`80fa5c3`](https://github.com/apache/spark/commit/80fa5c3b46e209c59cc01a539f998969717b83f4).


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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-849993303


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/43553/
   


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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r622729699



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -133,83 +204,77 @@ 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
+  }
+
+  /**
+   * Creates a map from root [[Attribute]]s to non-redundant nested [[ExtractValue]]s.
+   * Nested field accessors of `exclusiveAttrs` are not considered in nested fields aliasing.
+   */
+  def getAttributeToExtractValues(
+      exprList: Seq[Expression],
+      exclusiveAttrs: Seq[Attribute]): Map[Attribute, Seq[ExtractValue]] = {
 
-    // Note that when we group by extractors with their references, we should remove
-    // cosmetic variations.
+    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
+    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 {
+          // See [[collectExtractValue]]: we only need to deal with [[GetArrayStructFields]] and
+          // [[GetStructField]]
           case e @ (_: GetStructField | _: GetArrayStructFields) =>
             val child = e.children.head
             nestedFields.forall(f => child.find(_.semanticEquals(f)).isEmpty)
           case _ => true
-        }
-
-        // Each expression can contain multiple nested fields.
-        // Note that we keep the original names to deliver to parquet in a case-sensitive way.
-        val nestedFieldToAlias = dedupNestedFields.distinct.map { f =>
-          val exprId = NamedExpression.newExprId
-          (f, Alias(f, s"_gen_alias_${exprId.id}")(exprId, Seq.empty, None))
-        }
+        }.distinct

Review comment:
       We can also update the lookup side
   `nestedFieldToAlias.contains(f)` -> `nestedFieldToAlias.contains(f.canonicalized)`




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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r622728340



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -17,71 +17,148 @@
 
 package org.apache.spark.sql.catalyst.optimizer
 
+import scala.collection.mutable
+
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 
 /**
- * This aims to handle a nested column aliasing pattern inside the `ColumnPruning` optimizer rule.
- * If a project or its child references to nested fields, and not all the fields
- * in a nested attribute are used, we can substitute them by alias attributes; then a project
- * of the nested fields as aliases on the children of the child will be created.
+ * This aims to handle a nested column aliasing pattern inside the [[ColumnPruning]] optimizer rule.
+ * If:
+ * - A [[Project]] or its child references nested fields
+ * - Not all of the fields in a nested attribute are used
+ * Then:
+ * - Substitute the nested field references with alias attributes
+ * - Add grandchild [[Project]]s transforming the nested fields to aliases
+ *
+ * Example 1: Project
+ * ------------------
+ * Before:
+ * +- Project [concat_ws(s#0.a, s#0.b) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [concat_ws(_gen_alias_2#2, _gen_alias_3#3) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _gen_alias_2#2, s#0.b AS _gen_alias_3#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * Example 2: Project above Filter
+ * -------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1]
+ *   +- Filter (length(s#0.b) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_gen_alias_2#2 AS s.a#1]
+ *   +- Filter (length(_gen_alias_3#3) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- Project [s#0.a AS _gen_alias_2#2, s#0.b AS _gen_alias_3#3]
+ *           +- LocalRelation <empty>, [s#0]
+ *
+ * Example 3: Nested columns in nested columns
+ * -------------------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1, s#0.a.a1 AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_gen_alias_3#3 AS s.a#1, _gen_alias_3#3.name AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _gen_alias_3#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * The schema of the datasource relation will be pruned in the [[SchemaPruning]] optimizer rule.
  */
 object NestedColumnAliasing {
 
   def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match {
     /**
      * This pattern is needed to support [[Filter]] plan cases like
-     * [[Project]]->[[Filter]]->listed plan in `canProjectPushThrough` (e.g., [[Window]]).
-     * The reason why we don't simply add [[Filter]] in `canProjectPushThrough` is that
+     * [[Project]]->[[Filter]]->listed plan in [[canProjectPushThrough]] (e.g., [[Window]]).
+     * The reason why we don't simply add [[Filter]] in [[canProjectPushThrough]] is that
      * the optimizer can hit an infinite loop during the [[PushDownPredicates]] rule.
      */
-    case Project(projectList, Filter(condition, child))
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
-      val exprCandidatesToPrune = projectList ++ Seq(condition) ++ child.expressions
-      getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases)
-      }
+    case Project(projectList, Filter(condition, child)) if
+        SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
+      rewritePlanIfSubsetFieldsUsed(
+        plan, projectList ++ Seq(condition) ++ child.expressions, child.producedAttributes.toSeq)
 
-    case Project(projectList, child)
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
-      val exprCandidatesToPrune = projectList ++ child.expressions
-      getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases)
-      }
+    case Project(projectList, child) if
+        SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
+      rewritePlanIfSubsetFieldsUsed(
+        plan, projectList ++ child.expressions, child.producedAttributes.toSeq)
 
     case p if SQLConf.get.nestedSchemaPruningEnabled && canPruneOn(p) =>
-      val exprCandidatesToPrune = p.expressions
-      getAliasSubMap(exprCandidatesToPrune, p.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(p, nestedFieldToAlias, attrToAliases)
-      }
+      rewritePlanIfSubsetFieldsUsed(
+        plan, p.expressions, p.producedAttributes.toSeq)
 
     case _ => None
   }
 
+  /**
+   * Rewrites a plan with aliases if only a subset of the nested fields are used.
+   */
+  def rewritePlanIfSubsetFieldsUsed(
+      plan: LogicalPlan,
+      exprList: Seq[Expression],
+      exclusiveAttrs: Seq[Attribute]): Option[LogicalPlan] = {
+    val attrToExtractValues = getAttributeToExtractValues(exprList, exclusiveAttrs)
+    if (attrToExtractValues.isEmpty) {
+      None
+    } else {
+      Some(rewritePlanWithAliases(plan, attrToExtractValues))
+    }
+  }
+
   /**
    * Replace nested columns to prune unused nested columns later.
    */
-  private def replaceToAliases(
+  def rewritePlanWithAliases(
       plan: LogicalPlan,
-      nestedFieldToAlias: Map[ExtractValue, Alias],
-      attrToAliases: Map[ExprId, Seq[Alias]]): LogicalPlan = plan match {
-    case Project(projectList, child) =>
-      Project(
-        getNewProjectList(projectList, nestedFieldToAlias),
-        replaceWithAliases(child, nestedFieldToAlias, attrToAliases))
-
-    // The operators reaching here was already guarded by `canPruneOn`.
-    case other =>
-      replaceWithAliases(other, nestedFieldToAlias, attrToAliases)
+      attributeToExtractValues: Map[Attribute, Seq[ExtractValue]]): LogicalPlan = {
+    // Each expression can contain multiple nested fields.
+    // Note that we keep the original names to deliver to parquet in a case-sensitive way.
+    // A new alias is created for each nested field.
+    val nestedFieldToAlias = attributeToExtractValues.flatMap { case (_, nestedFields) =>
+      nestedFields.map { f =>
+        val exprId = NamedExpression.newExprId
+        val fieldName = f match {
+          case g: GetStructField => g.extractFieldName
+          case g: GetArrayStructFields => g.field.name
+        }
+        f -> Alias(f, s"_extract_${fieldName}")(exprId, Seq.empty, None)

Review comment:
       shall we update the examples in classdoc?




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


[GitHub] [spark] SparkQA commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-842592756


   **[Test build #138642 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138642/testReport)** for PR 32301 at commit [`c7e5c5a`](https://github.com/apache/spark/commit/c7e5c5a2228364859e47834c690b87b1fbdb7095).


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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r622729279



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -17,71 +17,148 @@
 
 package org.apache.spark.sql.catalyst.optimizer
 
+import scala.collection.mutable
+
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 
 /**
- * This aims to handle a nested column aliasing pattern inside the `ColumnPruning` optimizer rule.
- * If a project or its child references to nested fields, and not all the fields
- * in a nested attribute are used, we can substitute them by alias attributes; then a project
- * of the nested fields as aliases on the children of the child will be created.
+ * This aims to handle a nested column aliasing pattern inside the [[ColumnPruning]] optimizer rule.
+ * If:
+ * - A [[Project]] or its child references nested fields
+ * - Not all of the fields in a nested attribute are used
+ * Then:
+ * - Substitute the nested field references with alias attributes
+ * - Add grandchild [[Project]]s transforming the nested fields to aliases
+ *
+ * Example 1: Project
+ * ------------------
+ * Before:
+ * +- Project [concat_ws(s#0.a, s#0.b) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [concat_ws(_gen_alias_2#2, _gen_alias_3#3) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _gen_alias_2#2, s#0.b AS _gen_alias_3#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * Example 2: Project above Filter
+ * -------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1]
+ *   +- Filter (length(s#0.b) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_gen_alias_2#2 AS s.a#1]
+ *   +- Filter (length(_gen_alias_3#3) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- Project [s#0.a AS _gen_alias_2#2, s#0.b AS _gen_alias_3#3]
+ *           +- LocalRelation <empty>, [s#0]
+ *
+ * Example 3: Nested columns in nested columns
+ * -------------------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1, s#0.a.a1 AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_gen_alias_3#3 AS s.a#1, _gen_alias_3#3.name AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _gen_alias_3#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * The schema of the datasource relation will be pruned in the [[SchemaPruning]] optimizer rule.
  */
 object NestedColumnAliasing {
 
   def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match {
     /**
      * This pattern is needed to support [[Filter]] plan cases like
-     * [[Project]]->[[Filter]]->listed plan in `canProjectPushThrough` (e.g., [[Window]]).
-     * The reason why we don't simply add [[Filter]] in `canProjectPushThrough` is that
+     * [[Project]]->[[Filter]]->listed plan in [[canProjectPushThrough]] (e.g., [[Window]]).
+     * The reason why we don't simply add [[Filter]] in [[canProjectPushThrough]] is that
      * the optimizer can hit an infinite loop during the [[PushDownPredicates]] rule.
      */
-    case Project(projectList, Filter(condition, child))
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
-      val exprCandidatesToPrune = projectList ++ Seq(condition) ++ child.expressions
-      getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases)
-      }
+    case Project(projectList, Filter(condition, child)) if
+        SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
+      rewritePlanIfSubsetFieldsUsed(
+        plan, projectList ++ Seq(condition) ++ child.expressions, child.producedAttributes.toSeq)
 
-    case Project(projectList, child)
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
-      val exprCandidatesToPrune = projectList ++ child.expressions
-      getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases)
-      }
+    case Project(projectList, child) if
+        SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
+      rewritePlanIfSubsetFieldsUsed(
+        plan, projectList ++ child.expressions, child.producedAttributes.toSeq)
 
     case p if SQLConf.get.nestedSchemaPruningEnabled && canPruneOn(p) =>
-      val exprCandidatesToPrune = p.expressions
-      getAliasSubMap(exprCandidatesToPrune, p.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(p, nestedFieldToAlias, attrToAliases)
-      }
+      rewritePlanIfSubsetFieldsUsed(
+        plan, p.expressions, p.producedAttributes.toSeq)
 
     case _ => None
   }
 
+  /**
+   * Rewrites a plan with aliases if only a subset of the nested fields are used.
+   */
+  def rewritePlanIfSubsetFieldsUsed(
+      plan: LogicalPlan,
+      exprList: Seq[Expression],
+      exclusiveAttrs: Seq[Attribute]): Option[LogicalPlan] = {
+    val attrToExtractValues = getAttributeToExtractValues(exprList, exclusiveAttrs)
+    if (attrToExtractValues.isEmpty) {
+      None
+    } else {
+      Some(rewritePlanWithAliases(plan, attrToExtractValues))
+    }
+  }
+
   /**
    * Replace nested columns to prune unused nested columns later.
    */
-  private def replaceToAliases(
+  def rewritePlanWithAliases(
       plan: LogicalPlan,
-      nestedFieldToAlias: Map[ExtractValue, Alias],
-      attrToAliases: Map[ExprId, Seq[Alias]]): LogicalPlan = plan match {
-    case Project(projectList, child) =>
-      Project(
-        getNewProjectList(projectList, nestedFieldToAlias),
-        replaceWithAliases(child, nestedFieldToAlias, attrToAliases))
-
-    // The operators reaching here was already guarded by `canPruneOn`.
-    case other =>
-      replaceWithAliases(other, nestedFieldToAlias, attrToAliases)
+      attributeToExtractValues: Map[Attribute, Seq[ExtractValue]]): LogicalPlan = {
+    // Each expression can contain multiple nested fields.
+    // Note that we keep the original names to deliver to parquet in a case-sensitive way.
+    // A new alias is created for each nested field.
+    val nestedFieldToAlias = attributeToExtractValues.flatMap { case (_, nestedFields) =>
+      nestedFields.map { f =>
+        val exprId = NamedExpression.newExprId
+        val fieldName = f match {
+          case g: GetStructField => g.extractFieldName
+          case g: GetArrayStructFields => g.field.name
+        }
+        f -> Alias(f, s"_extract_${fieldName}")(exprId, Seq.empty, None)
+      }
+    }
+
+    // A reference attribute can have multiple aliases for nested fields.
+    val attrToAliases = attributeToExtractValues.map { case (attr, nestedFields) =>
+      attr.exprId -> nestedFields.map(nestedFieldToAlias)

Review comment:
       this map lookup looks fragile as it use java `equals` not semantic equals. how about
   ```
   val attributeToAliasesExtractValues = attributeToExtractValues.map ...
   
   val nestedFieldToAlias = attributeToAliasesExtractValues.flatMap...
   ```




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


[GitHub] [spark] SparkQA removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-849246928


   **[Test build #138997 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138997/testReport)** for PR 32301 at commit [`83e2611`](https://github.com/apache/spark/commit/83e2611f47b86b1b35ba4862fc183e5ce6485a0f).


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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r638911201



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -17,71 +17,155 @@
 
 package org.apache.spark.sql.catalyst.optimizer
 
+import scala.collection.mutable
+
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 
 /**
- * This aims to handle a nested column aliasing pattern inside the `ColumnPruning` optimizer rule.
- * If a project or its child references to nested fields, and not all the fields
- * in a nested attribute are used, we can substitute them by alias attributes; then a project
- * of the nested fields as aliases on the children of the child will be created.
+ * This aims to handle a nested column aliasing pattern inside the [[ColumnPruning]] optimizer rule.
+ * If:
+ * - A [[Project]] or its child references nested fields
+ * - Not all of the fields in a nested attribute are used
+ * Then:
+ * - Substitute the nested field references with alias attributes
+ * - Add grandchild [[Project]]s transforming the nested fields to aliases
+ *
+ * Example 1: Project
+ * ------------------
+ * Before:
+ * +- Project [concat_ws(s#0.a, s#0.b) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [concat_ws(_extract_a#2, _extract_b#3) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _extract_a#2, s#0.b AS _extract_b#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * Example 2: Project above Filter
+ * -------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1]
+ *   +- Filter (length(s#0.b) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_extract_a#2 AS s.a#1]
+ *   +- Filter (length(_extract_b#3) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- Project [s#0.a AS _extract_a#2, s#0.b AS _extract_b#3]
+ *           +- LocalRelation <empty>, [s#0]
+ *
+ * Example 3: Nested fields with referenced parents
+ * ------------------------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1, s#0.a.a1 AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_extract_a#3 AS s.a#1, _extract_a#3.name AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _extract_a#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * The schema of the datasource relation will be pruned in the [[SchemaPruning]] optimizer rule.
  */
 object NestedColumnAliasing {
 
   def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match {
     /**
      * This pattern is needed to support [[Filter]] plan cases like
-     * [[Project]]->[[Filter]]->listed plan in `canProjectPushThrough` (e.g., [[Window]]).
-     * The reason why we don't simply add [[Filter]] in `canProjectPushThrough` is that
+     * [[Project]]->[[Filter]]->listed plan in [[canProjectPushThrough]] (e.g., [[Window]]).
+     * The reason why we don't simply add [[Filter]] in [[canProjectPushThrough]] is that

Review comment:
       Not related to this PR: does it mean we don't support Project -> Limit -> Filter?




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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r639414608



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -17,71 +17,155 @@
 
 package org.apache.spark.sql.catalyst.optimizer
 
+import scala.collection.mutable
+
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 
 /**
- * This aims to handle a nested column aliasing pattern inside the `ColumnPruning` optimizer rule.
- * If a project or its child references to nested fields, and not all the fields
- * in a nested attribute are used, we can substitute them by alias attributes; then a project
- * of the nested fields as aliases on the children of the child will be created.
+ * This aims to handle a nested column aliasing pattern inside the [[ColumnPruning]] optimizer rule.
+ * If:
+ * - A [[Project]] or its child references nested fields
+ * - Not all of the fields in a nested attribute are used
+ * Then:
+ * - Substitute the nested field references with alias attributes
+ * - Add grandchild [[Project]]s transforming the nested fields to aliases
+ *
+ * Example 1: Project
+ * ------------------
+ * Before:
+ * +- Project [concat_ws(s#0.a, s#0.b) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [concat_ws(_extract_a#2, _extract_b#3) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _extract_a#2, s#0.b AS _extract_b#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * Example 2: Project above Filter
+ * -------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1]
+ *   +- Filter (length(s#0.b) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_extract_a#2 AS s.a#1]
+ *   +- Filter (length(_extract_b#3) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- Project [s#0.a AS _extract_a#2, s#0.b AS _extract_b#3]
+ *           +- LocalRelation <empty>, [s#0]
+ *
+ * Example 3: Nested fields with referenced parents
+ * ------------------------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1, s#0.a.a1 AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_extract_a#3 AS s.a#1, _extract_a#3.name AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _extract_a#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * The schema of the datasource relation will be pruned in the [[SchemaPruning]] optimizer rule.
  */
 object NestedColumnAliasing {
 
   def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match {
     /**
      * This pattern is needed to support [[Filter]] plan cases like
-     * [[Project]]->[[Filter]]->listed plan in `canProjectPushThrough` (e.g., [[Window]]).
-     * The reason why we don't simply add [[Filter]] in `canProjectPushThrough` is that
+     * [[Project]]->[[Filter]]->listed plan in [[canProjectPushThrough]] (e.g., [[Window]]).
+     * The reason why we don't simply add [[Filter]] in [[canProjectPushThrough]] is that

Review comment:
       cc @viirya @maropu 

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -17,71 +17,155 @@
 
 package org.apache.spark.sql.catalyst.optimizer
 
+import scala.collection.mutable
+
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 
 /**
- * This aims to handle a nested column aliasing pattern inside the `ColumnPruning` optimizer rule.
- * If a project or its child references to nested fields, and not all the fields
- * in a nested attribute are used, we can substitute them by alias attributes; then a project
- * of the nested fields as aliases on the children of the child will be created.
+ * This aims to handle a nested column aliasing pattern inside the [[ColumnPruning]] optimizer rule.
+ * If:
+ * - A [[Project]] or its child references nested fields
+ * - Not all of the fields in a nested attribute are used
+ * Then:
+ * - Substitute the nested field references with alias attributes
+ * - Add grandchild [[Project]]s transforming the nested fields to aliases

Review comment:
       cc @viirya @maropu 




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


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

Posted by GitBox <gi...@apache.org>.
karenfeng commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r639927288



##########
File path: sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala
##########
@@ -23,6 +23,10 @@ package org.apache.spark.sql.catalyst.expressions
  * of the name, or the expected nullability).
  */
 object AttributeMap {
+  def apply[A](kvs: Map[Attribute, A]): AttributeMap[A] = {

Review comment:
       That would work as well, but `AttributeMap(kvs = map.toSeq)` will end up calling `.toMap` down the line.




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


[GitHub] [spark] AmplabJenkins commented on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-843585334


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/138689/
   


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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r620139197



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -23,78 +23,149 @@ import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 
 /**
- * This aims to handle a nested column aliasing pattern inside the `ColumnPruning` optimizer rule.
- * If a project or its child references to nested fields, and not all the fields
- * in a nested attribute are used, we can substitute them by alias attributes; then a project
- * of the nested fields as aliases on the children of the child will be created.
+ * This aims to handle a nested column aliasing pattern inside the [[ColumnPruning]] optimizer rule.
+ * If:
+ * - A [[Project]] or its child references nested fields
+ * - Not all of the fields in a nested attribute are used
+ * Then:
+ * - Substitute the nested field references with alias attributes
+ * - Add grandchild [[Project]]s transforming the nested fields to aliases
+ *
+ * Example 1: Project
+ * ------------------
+ * Before:
+ * +- Project [concat_ws(s#0.a, s#0.b) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [concat_ws(_gen_alias_2#2, _gen_alias_3#3) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _gen_alias_2#2, s#0.b AS _gen_alias_3#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * Example 2: Project above Filter
+ * -------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1]
+ *   +- Filter (length(s#0.b) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_gen_alias_2#2 AS s.a#1]
+ *   +- Filter (length(_gen_alias_3#3) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- Project [s#0.a AS _gen_alias_2#2, s#0.b AS _gen_alias_3#3]
+ *           +- LocalRelation <empty>, [s#0]
+ *
+ * Example 3: Nested columns in nested columns
+ * -------------------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1, s#0.a.a1 AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_gen_alias_3#3 AS s.a#1, _gen_alias_3#3.name AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _gen_alias_3#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * The schema of the datasource relation will be pruned in the [[SchemaPruning]] optimizer rule.
  */
 object NestedColumnAliasing {
 
   def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match {
     /**
      * This pattern is needed to support [[Filter]] plan cases like
-     * [[Project]]->[[Filter]]->listed plan in `canProjectPushThrough` (e.g., [[Window]]).
-     * The reason why we don't simply add [[Filter]] in `canProjectPushThrough` is that
+     * [[Project]]->[[Filter]]->listed plan in [[canProjectPushThrough]] (e.g., [[Window]]).
+     * The reason why we don't simply add [[Filter]] in [[canProjectPushThrough]] is that
      * the optimizer can hit an infinite loop during the [[PushDownPredicates]] rule.
      */
-    case Project(projectList, Filter(condition, child))
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
-      val exprCandidatesToPrune = projectList ++ Seq(condition) ++ child.expressions
-      getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases)
-      }
+    case Project(projectList, Filter(condition, child)) if
+        SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
+      rewritePlanIfSubsetFieldsUsed(
+        plan, projectList ++ Seq(condition) ++ child.expressions, child.producedAttributes.toSeq)
 
-    case Project(projectList, child)
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
-      val exprCandidatesToPrune = projectList ++ child.expressions
-      getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases)
-      }
+    case Project(projectList, child) if
+        SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
+      rewritePlanIfSubsetFieldsUsed(
+        plan, projectList ++ child.expressions, child.producedAttributes.toSeq)
 
     case p if SQLConf.get.nestedSchemaPruningEnabled && canPruneOn(p) =>
-      val exprCandidatesToPrune = p.expressions
-      getAliasSubMap(exprCandidatesToPrune, p.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(p, nestedFieldToAlias, attrToAliases)
-      }
+      rewritePlanIfSubsetFieldsUsed(
+        plan, p.expressions, p.producedAttributes.toSeq)
 
     case _ => None
   }
 
+  /**
+   * Rewrites a plan with aliases if only a subset of the nested fields are used.
+   */
+  def rewritePlanIfSubsetFieldsUsed(
+    plan: LogicalPlan,
+    exprList: Seq[Expression],
+    exclusiveAttrs: Seq[Attribute]): Option[LogicalPlan] = {
+    val attrToExtractValues = getAttributeToExtractValues(exprList, exclusiveAttrs)
+    if (attrToExtractValues.isEmpty) {
+      None
+    } else {
+      Some(rewritePlanWithAliases(plan, attrToExtractValues))
+    }
+  }
+
   /**
    * Replace nested columns to prune unused nested columns later.
    */
-  private def replaceToAliases(
+  def rewritePlanWithAliases(
       plan: LogicalPlan,
-      nestedFieldToAlias: Map[ExtractValue, Alias],
-      attrToAliases: Map[ExprId, Seq[Alias]]): LogicalPlan = plan match {
-    case Project(projectList, child) =>
-      Project(
-        getNewProjectList(projectList, nestedFieldToAlias),
-        replaceWithAliases(child, nestedFieldToAlias, attrToAliases))
+      attributeToExtractValues: Map[Attribute, Seq[ExtractValue]]): LogicalPlan = {
+      // Each expression can contain multiple nested fields.
+      // Note that we keep the original names to deliver to parquet in a case-sensitive way.
+      // A new alias is created for each nested field.
+      val nestedFieldToAlias = attributeToExtractValues.flatMap { case (_, nestedFields) =>
+        nestedFields.map { f =>
+          val exprId = NamedExpression.newExprId
+          f -> Alias(f, s"_gen_alias_${exprId.id}")(exprId, Seq.empty, None)
+        }
+      }
+
+      // A reference attribute can have multiple aliases for nested fields.
+      val attrToAliases = attributeToExtractValues.map { case (attr, nestedFields) =>
+        attr.exprId -> nestedFields.map(nestedFieldToAlias)
+      }
 
-    // The operators reaching here was already guarded by `canPruneOn`.
-    case other =>
-      replaceWithAliases(other, nestedFieldToAlias, attrToAliases)
+      plan match {
+        case Project(projectList, child) =>
+          Project(
+            getNewProjectList(projectList, nestedFieldToAlias),
+            replaceWithAliases(child, nestedFieldToAlias, attrToAliases))
+
+        // The operators reaching here are already guarded by [[canPruneOn]].
+        case other =>
+          replaceWithAliases(other, nestedFieldToAlias, attrToAliases)
+      }
   }
 
   /**
-   * Return a replaced project list.
+   * Replace the [[ExtractValue]]s in a project list with aliased attributes.
    */
   def getNewProjectList(
-      projectList: Seq[NamedExpression],
-      nestedFieldToAlias: Map[ExtractValue, Alias]): Seq[NamedExpression] = {
+    projectList: Seq[NamedExpression],
+    nestedFieldToAlias: Map[ExtractValue, Alias]): Seq[NamedExpression] = {

Review comment:
       nit: the previous indentation is correct.




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


[GitHub] [spark] SparkQA removed a comment on pull request #32301: [SPARK-35194][SQL] Refactor nested column aliasing for readability

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32301:
URL: https://github.com/apache/spark/pull/32301#issuecomment-827948134


   **[Test build #138011 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/138011/testReport)** for PR 32301 at commit [`69ca24e`](https://github.com/apache/spark/commit/69ca24eda8bca7216863d4c74775c342aa533b31).


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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r640353935



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -17,71 +17,155 @@
 
 package org.apache.spark.sql.catalyst.optimizer
 
+import scala.collection.mutable
+
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 
 /**
- * This aims to handle a nested column aliasing pattern inside the `ColumnPruning` optimizer rule.
- * If a project or its child references to nested fields, and not all the fields
- * in a nested attribute are used, we can substitute them by alias attributes; then a project
- * of the nested fields as aliases on the children of the child will be created.
+ * This aims to handle a nested column aliasing pattern inside the [[ColumnPruning]] optimizer rule.
+ * If:
+ * - A [[Project]] or its child references nested fields
+ * - Not all of the fields in a nested attribute are used
+ * Then:
+ * - Substitute the nested field references with alias attributes
+ * - Add grandchild [[Project]]s transforming the nested fields to aliases
+ *
+ * Example 1: Project
+ * ------------------
+ * Before:
+ * +- Project [concat_ws(s#0.a, s#0.b) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [concat_ws(_extract_a#2, _extract_b#3) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _extract_a#2, s#0.b AS _extract_b#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * Example 2: Project above Filter
+ * -------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1]
+ *   +- Filter (length(s#0.b) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_extract_a#2 AS s.a#1]
+ *   +- Filter (length(_extract_b#3) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- Project [s#0.a AS _extract_a#2, s#0.b AS _extract_b#3]
+ *           +- LocalRelation <empty>, [s#0]
+ *
+ * Example 3: Nested fields with referenced parents
+ * ------------------------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1, s#0.a.a1 AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_extract_a#3 AS s.a#1, _extract_a#3.name AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _extract_a#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * The schema of the datasource relation will be pruned in the [[SchemaPruning]] optimizer rule.
  */
 object NestedColumnAliasing {
 
   def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match {
     /**
      * This pattern is needed to support [[Filter]] plan cases like
-     * [[Project]]->[[Filter]]->listed plan in `canProjectPushThrough` (e.g., [[Window]]).
-     * The reason why we don't simply add [[Filter]] in `canProjectPushThrough` is that
+     * [[Project]]->[[Filter]]->listed plan in [[canProjectPushThrough]] (e.g., [[Window]]).
+     * The reason why we don't simply add [[Filter]] in [[canProjectPushThrough]] is that

Review comment:
       Yea you are right, we will first generate a new project between Limit and Filter (the Project grandchild) to do nested column pruning, then the next batch we can match Project -> Filter and do nested column pruning further.




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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32301:
URL: https://github.com/apache/spark/pull/32301#discussion_r620138804



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala
##########
@@ -23,78 +23,149 @@ import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 
 /**
- * This aims to handle a nested column aliasing pattern inside the `ColumnPruning` optimizer rule.
- * If a project or its child references to nested fields, and not all the fields
- * in a nested attribute are used, we can substitute them by alias attributes; then a project
- * of the nested fields as aliases on the children of the child will be created.
+ * This aims to handle a nested column aliasing pattern inside the [[ColumnPruning]] optimizer rule.
+ * If:
+ * - A [[Project]] or its child references nested fields
+ * - Not all of the fields in a nested attribute are used
+ * Then:
+ * - Substitute the nested field references with alias attributes
+ * - Add grandchild [[Project]]s transforming the nested fields to aliases
+ *
+ * Example 1: Project
+ * ------------------
+ * Before:
+ * +- Project [concat_ws(s#0.a, s#0.b) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [concat_ws(_gen_alias_2#2, _gen_alias_3#3) AS concat_ws(s.a, s.b)#1]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _gen_alias_2#2, s#0.b AS _gen_alias_3#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * Example 2: Project above Filter
+ * -------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1]
+ *   +- Filter (length(s#0.b) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_gen_alias_2#2 AS s.a#1]
+ *   +- Filter (length(_gen_alias_3#3) > 2)
+ *     +- GlobalLimit 5
+ *       +- LocalLimit 5
+ *         +- Project [s#0.a AS _gen_alias_2#2, s#0.b AS _gen_alias_3#3]
+ *           +- LocalRelation <empty>, [s#0]
+ *
+ * Example 3: Nested columns in nested columns
+ * -------------------------------------------
+ * Before:
+ * +- Project [s#0.a AS s.a#1, s#0.a.a1 AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- LocalRelation <empty>, [s#0]
+ * After:
+ * +- Project [_gen_alias_3#3 AS s.a#1, _gen_alias_3#3.name AS s.a.a1#2]
+ *   +- GlobalLimit 5
+ *     +- LocalLimit 5
+ *       +- Project [s#0.a AS _gen_alias_3#3]
+ *         +- LocalRelation <empty>, [s#0]
+ *
+ * The schema of the datasource relation will be pruned in the [[SchemaPruning]] optimizer rule.
  */
 object NestedColumnAliasing {
 
   def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match {
     /**
      * This pattern is needed to support [[Filter]] plan cases like
-     * [[Project]]->[[Filter]]->listed plan in `canProjectPushThrough` (e.g., [[Window]]).
-     * The reason why we don't simply add [[Filter]] in `canProjectPushThrough` is that
+     * [[Project]]->[[Filter]]->listed plan in [[canProjectPushThrough]] (e.g., [[Window]]).
+     * The reason why we don't simply add [[Filter]] in [[canProjectPushThrough]] is that
      * the optimizer can hit an infinite loop during the [[PushDownPredicates]] rule.
      */
-    case Project(projectList, Filter(condition, child))
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
-      val exprCandidatesToPrune = projectList ++ Seq(condition) ++ child.expressions
-      getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases)
-      }
+    case Project(projectList, Filter(condition, child)) if
+        SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
+      rewritePlanIfSubsetFieldsUsed(
+        plan, projectList ++ Seq(condition) ++ child.expressions, child.producedAttributes.toSeq)
 
-    case Project(projectList, child)
-        if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
-      val exprCandidatesToPrune = projectList ++ child.expressions
-      getAliasSubMap(exprCandidatesToPrune, child.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(plan, nestedFieldToAlias, attrToAliases)
-      }
+    case Project(projectList, child) if
+        SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) =>
+      rewritePlanIfSubsetFieldsUsed(
+        plan, projectList ++ child.expressions, child.producedAttributes.toSeq)
 
     case p if SQLConf.get.nestedSchemaPruningEnabled && canPruneOn(p) =>
-      val exprCandidatesToPrune = p.expressions
-      getAliasSubMap(exprCandidatesToPrune, p.producedAttributes.toSeq).map {
-        case (nestedFieldToAlias, attrToAliases) =>
-          NestedColumnAliasing.replaceToAliases(p, nestedFieldToAlias, attrToAliases)
-      }
+      rewritePlanIfSubsetFieldsUsed(
+        plan, p.expressions, p.producedAttributes.toSeq)
 
     case _ => None
   }
 
+  /**
+   * Rewrites a plan with aliases if only a subset of the nested fields are used.
+   */
+  def rewritePlanIfSubsetFieldsUsed(
+    plan: LogicalPlan,
+    exprList: Seq[Expression],
+    exclusiveAttrs: Seq[Attribute]): Option[LogicalPlan] = {
+    val attrToExtractValues = getAttributeToExtractValues(exprList, exclusiveAttrs)
+    if (attrToExtractValues.isEmpty) {
+      None
+    } else {
+      Some(rewritePlanWithAliases(plan, attrToExtractValues))
+    }
+  }
+
   /**
    * Replace nested columns to prune unused nested columns later.
    */
-  private def replaceToAliases(
+  def rewritePlanWithAliases(
       plan: LogicalPlan,
-      nestedFieldToAlias: Map[ExtractValue, Alias],
-      attrToAliases: Map[ExprId, Seq[Alias]]): LogicalPlan = plan match {
-    case Project(projectList, child) =>
-      Project(
-        getNewProjectList(projectList, nestedFieldToAlias),
-        replaceWithAliases(child, nestedFieldToAlias, attrToAliases))
+      attributeToExtractValues: Map[Attribute, Seq[ExtractValue]]): LogicalPlan = {
+      // Each expression can contain multiple nested fields.
+      // Note that we keep the original names to deliver to parquet in a case-sensitive way.
+      // A new alias is created for each nested field.
+      val nestedFieldToAlias = attributeToExtractValues.flatMap { case (_, nestedFields) =>
+        nestedFields.map { f =>
+          val exprId = NamedExpression.newExprId
+          f -> Alias(f, s"_gen_alias_${exprId.id}")(exprId, Seq.empty, None)

Review comment:
       does the alias name matter?




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