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 2020/09/12 00:37:15 UTC

[GitHub] [spark] allisonwang-db opened a new pull request #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

allisonwang-db opened a new pull request #29734:
URL: https://github.com/apache/spark/pull/29734


   ### What changes were proposed in this pull request?
   This PR updates the `RemoveRedundantProjects` rule to make `GenerateExec` require column ordering. 
   
   ### Why are the changes needed?
   `GenerateExec` was originally considered as a node that does not require column ordering. From the implementation, however, we found that it binds its input rows directly with its `requiredChildOutput` without using the child's output schema. In this case, changing input column ordering will result in `GenerateExec` binding the wrong schema to the input columns.
   
   ### Does this PR introduce _any_ user-facing change?
   No
   
   ### How was this patch tested?
   Unit test
   


----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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






----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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


   The PR description is not clear enough. Can you revise it? It should be nicer to describe which part in `GenerateExec` has the issue. An example is better.


----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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






----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala
##########
@@ -115,9 +119,40 @@ class RemoveRedundantProjectsSuite extends QueryTest with SharedSparkSession wit
     assertProjectExec(query, 1, 2)
   }
 
-  test("generate") {
-    val query = "select a, key, explode(d) from testView where a > 10"
-    assertProjectExec(query, 0, 1)
+  test("generate should require column ordering") {
+    withTempView("testData") {
+      spark.range(0, 10, 1)
+        .selectExpr("id as key", "id * 2 as a", "id * 3 as b")
+        .createOrReplaceTempView("testData")
+
+      val data = sql("select key, a, b, count(*) from testData group by key, a, b limit 2")
+      val df = data.selectExpr("a", "b", "key", "explode(array(key, a, b)) as d").filter("d > 0")
+      df.collect()
+      val plan = df.queryExecution.executedPlan
+      val numProjects = collectWithSubqueries(plan) { case p: ProjectExec => p }.length
+
+      // Create a new plan that reverse the GenerateExec output and add a new ProjectExec between
+      // GenerateExec and its child. This is to test if the ProjectExec is removed, the output of
+      // the query will be incorrect.
+      val newPlan = stripAQEPlan(plan) transform {
+        case g @ GenerateExec(_, requiredChildOutput, _, _, child) =>
+          g.copy(requiredChildOutput = requiredChildOutput.reverse,
+            child = ProjectExec(requiredChildOutput.reverse, child))
+      }
+
+      // Re-apply remove redundant project rule.
+      val rule = RemoveRedundantProjects(spark.sessionState.conf)
+      val newExecutedPlan = rule.apply(newPlan)
+      // The manually added ProjectExec node shouldn't be removed.
+      assert(collectWithSubqueries(newExecutedPlan) {
+        case p: ProjectExec => p }.size == numProjects + 1)

Review comment:
       Updated.




----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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






----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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


   **[Test build #128675 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128675/testReport)** for PR 29734 at commit [`06ca9c1`](https://github.com/apache/spark/commit/06ca9c17da310c88c67204366bf15c1e80057719).


----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala
##########
@@ -18,17 +18,21 @@
 package org.apache.spark.sql.execution
 
 import org.apache.spark.sql.{DataFrame, QueryTest, Row}
+import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite}
 import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils}
+import org.apache.spark.sql.test.SharedSparkSession
 import org.apache.spark.sql.types.StructType
 import org.apache.spark.util.Utils
 
-class RemoveRedundantProjectsSuite extends QueryTest with SharedSparkSession with SQLTestUtils {
+abstract class RemoveRedundantProjectsSuiteBase
+  extends QueryTest
+    with SharedSparkSession
+    with AdaptiveSparkPlanHelper {

Review comment:
       This rule was also added in AQE's [query stage prep rules](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala#L88) so the test should also make sure the rule works when AQE is enabled.




----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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


   **[Test build #128724 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128724/testReport)** for PR 29734 at commit [`16ebb83`](https://github.com/apache/spark/commit/16ebb830d83c12165148545b6427d51df30deed6).


----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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






----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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






----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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






----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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


   


----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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






----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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


   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 commented on pull request #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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


   **[Test build #128675 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128675/testReport)** for PR 29734 at commit [`06ca9c1`](https://github.com/apache/spark/commit/06ca9c17da310c88c67204366bf15c1e80057719).
    * This patch **fails PySpark 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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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


   **[Test build #128724 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128724/testReport)** for PR 29734 at commit [`16ebb83`](https://github.com/apache/spark/commit/16ebb830d83c12165148545b6427d51df30deed6).
    * 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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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


   **[Test build #128585 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128585/testReport)** for PR 29734 at commit [`ed6e6d7`](https://github.com/apache/spark/commit/ed6e6d7390a677518ebd467e3b8807de40ffc1d6).
    * 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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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






----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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






----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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


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


----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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






----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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


   **[Test build #128724 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128724/testReport)** for PR 29734 at commit [`16ebb83`](https://github.com/apache/spark/commit/16ebb830d83c12165148545b6427d51df30deed6).


----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala
##########
@@ -115,9 +119,40 @@ class RemoveRedundantProjectsSuite extends QueryTest with SharedSparkSession wit
     assertProjectExec(query, 1, 2)
   }
 
-  test("generate") {
-    val query = "select a, key, explode(d) from testView where a > 10"
-    assertProjectExec(query, 0, 1)
+  test("generate should require column ordering") {
+    withTempView("testData") {
+      spark.range(0, 10, 1)
+        .selectExpr("id as key", "id * 2 as a", "id * 3 as b")
+        .createOrReplaceTempView("testData")
+
+      val data = sql("select key, a, b, count(*) from testData group by key, a, b limit 2")
+      val df = data.selectExpr("a", "b", "key", "explode(array(key, a, b)) as d").filter("d > 0")
+      df.collect()
+      val plan = df.queryExecution.executedPlan
+      val numProjects = collectWithSubqueries(plan) { case p: ProjectExec => p }.length
+
+      // Create a new plan that reverse the GenerateExec output and add a new ProjectExec between
+      // GenerateExec and its child. This is to test if the ProjectExec is removed, the output of
+      // the query will be incorrect.
+      val newPlan = stripAQEPlan(plan) transform {
+        case g @ GenerateExec(_, requiredChildOutput, _, _, child) =>
+          g.copy(requiredChildOutput = requiredChildOutput.reverse,
+            child = ProjectExec(requiredChildOutput.reverse, child))
+      }
+
+      // Re-apply remove redundant project rule.
+      val rule = RemoveRedundantProjects(spark.sessionState.conf)
+      val newExecutedPlan = rule.apply(newPlan)
+      // The manually added ProjectExec node shouldn't be removed.
+      assert(collectWithSubqueries(newExecutedPlan) {
+        case p: ProjectExec => p }.size == numProjects + 1)

Review comment:
       The style looks weird.
   
   ```scala
   assert(collectWithSubqueries(newExecutedPlan) {
     case p: ProjectExec => p
   }.size == numProjects + 1)
   
   ```




----------------------------------------------------------------
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 pull request #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

Posted by GitBox <gi...@apache.org>.
allisonwang-db commented on pull request #29734:
URL: https://github.com/apache/spark/pull/29734#issuecomment-692375135


   @viirya Good point! I've revised the description and please let me know if it's unclear.


----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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


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


----------------------------------------------------------------
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] dongjoon-hyun commented on pull request #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on pull request #29734:
URL: https://github.com/apache/spark/pull/29734#issuecomment-740354729


   Thank you so much, @cloud-fan .


----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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






----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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


   if it's a bug in 3.0, yea let's backport


----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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


   **[Test build #128585 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128585/testReport)** for PR 29734 at commit [`ed6e6d7`](https://github.com/apache/spark/commit/ed6e6d7390a677518ebd467e3b8807de40ffc1d6).


----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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






----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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


   **[Test build #128585 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128585/testReport)** for PR 29734 at commit [`ed6e6d7`](https://github.com/apache/spark/commit/ed6e6d7390a677518ebd467e3b8807de40ffc1d6).


----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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






----------------------------------------------------------------
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] maropu commented on a change in pull request #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala
##########
@@ -18,17 +18,21 @@
 package org.apache.spark.sql.execution
 
 import org.apache.spark.sql.{DataFrame, QueryTest, Row}
+import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite}
 import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils}
+import org.apache.spark.sql.test.SharedSparkSession
 import org.apache.spark.sql.types.StructType
 import org.apache.spark.util.Utils
 
-class RemoveRedundantProjectsSuite extends QueryTest with SharedSparkSession with SQLTestUtils {
+abstract class RemoveRedundantProjectsSuiteBase
+  extends QueryTest
+    with SharedSparkSession
+    with AdaptiveSparkPlanHelper {

Review comment:
       Is this PR related to AQE? (the PR description does say nothing about it though)




----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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






----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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






----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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






----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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






----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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






----------------------------------------------------------------
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] dongjoon-hyun commented on pull request #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on pull request #29734:
URL: https://github.com/apache/spark/pull/29734#issuecomment-740347431


   Hi, all.
   SPARK-32861 seems to have affected version 3.0.1.
   Do we need to backport this to branch-3.0?


----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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






----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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






----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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


   **[Test build #128675 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128675/testReport)** for PR 29734 at commit [`06ca9c1`](https://github.com/apache/spark/commit/06ca9c17da310c88c67204366bf15c1e80057719).


----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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






----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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






----------------------------------------------------------------
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 #29734: [SPARK-32861][SQL] GenerateExec should require column ordering

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






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