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/03/25 10:29:05 UTC

[GitHub] [spark] AngersZhuuuu opened a new pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

AngersZhuuuu opened a new pull request #30145:
URL: https://github.com/apache/spark/pull/30145


   ### What changes were proposed in this pull request?
   Currently, we can't support use ordinal in CUBE/ROLLUP/GROUPING SETS, 
   this pr make CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal
   
   
   ### Why are the changes needed?
   Make CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal.
   Postgres SQL and TeraData support this use case. 
   
   ### Does this PR introduce _any_ user-facing change?
   User can use ordinal in CUBE/ROLLUP/GROUPING SETS
   
   
   ### How was this patch tested?
   Added UT


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #136762 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136762/testReport)** for PR 30145 at commit [`fb159c0`](https://github.com/apache/spark/commit/fb159c0b84e512c26c7cc7a92f821888baccb27e).


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






-- 
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] AngersZhuuuu commented on a change in pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/grouping.scala
##########
@@ -144,12 +147,12 @@ case class GroupingSets(
 object GroupingSets {
   def apply(
       groupingSets: Seq[Seq[Expression]],
-      userGivenGroupByExprs: Seq[Expression]): GroupingSets = {
-    val groupingSetIndexes = GroupingSet.computeGroupingSetIndexes(groupingSets)
+      userGivenGroupByExprs: Seq[Expression]): GroupingAnalytic = {
+    val groupingSetIndexes = GroupingAnalytic.computeGroupingSetIndexes(groupingSets)
     GroupingSets(groupingSetIndexes, groupingSets.flatten, userGivenGroupByExprs)
   }
 
-  def apply(groupingSets: Seq[Seq[Expression]]): GroupingSets = {
+  def apply(groupingSets: Seq[Seq[Expression]]): GroupingAnalytic = {

Review comment:
       > we can probably do the rename in a separate PR.
   
   Done https://github.com/apache/spark/pull/32073

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/SubstituteUnresolvedOrdinals.scala
##########
@@ -42,10 +52,19 @@ object SubstituteUnresolvedOrdinals extends Rule[LogicalPlan] {
       }
       withOrigin(s.origin)(s.copy(order = newOrders))
 
-    case a: Aggregate if conf.groupByOrdinal && a.groupingExpressions.exists(isIntLiteral) =>
+    case a: Aggregate if conf.groupByOrdinal && a.groupingExpressions.exists(containIntLiteral) =>
       val newGroups = a.groupingExpressions.map {
         case ordinal @ Literal(index: Int, IntegerType) =>
           withOrigin(ordinal.origin)(UnresolvedOrdinal(index))
+        case cube @ Cube(_, children) =>
+          withOrigin(cube.origin)(cube.copy(children = children.map(substituteUnresolvedOrdinal)))
+        case rollup @ Rollup(_, children) =>
+          withOrigin(rollup.origin)(rollup.copy(
+            children = children.map(substituteUnresolvedOrdinal)))
+        case groupingSets @ GroupingSets(_, flatGroupingSets, groupByExprs) =>
+          withOrigin(groupingSets.origin)(groupingSets.copy(
+            flatGroupingSets = flatGroupingSets.map(substituteUnresolvedOrdinal),
+            groupByExprs = groupByExprs.map(substituteUnresolvedOrdinal)))

Review comment:
       > ditto, we can use `withNewChildren`
   
   Yea

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/SubstituteUnresolvedOrdinals.scala
##########
@@ -27,13 +27,23 @@ import org.apache.spark.sql.types.IntegerType
  * Replaces ordinal in 'order by' or 'group by' with UnresolvedOrdinal expression.
  */
 object SubstituteUnresolvedOrdinals extends Rule[LogicalPlan] {
-  private def isIntLiteral(e: Expression) = e match {
+  private def containIntLiteral(e: Expression): Boolean = e match {
     case Literal(_, IntegerType) => true
+    case Cube(_, groupByExprs) => groupByExprs.exists(containIntLiteral)
+    case Rollup(_, groupByExprs) => groupByExprs.exists(containIntLiteral)
+    case GroupingSets(_, flatGroupingSets, groupByExprs) =>
+      flatGroupingSets.exists(containIntLiteral) || groupByExprs.exists(containIntLiteral)

Review comment:
       > ditto, we can use `children`
   
   Yea




-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #136996 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136996/testReport)** for PR 30145 at commit [`ff6794e`](https://github.com/apache/spark/commit/ff6794eb5387b6e83bfd3875884df02b75b0fafd).


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #137044 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137044/testReport)** for PR 30145 at commit [`0e89da2`](https://github.com/apache/spark/commit/0e89da2e53a36f1ca14278c456b40571a15517c0).
    * 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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #130442 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130442/testReport)** for PR 30145 at commit [`909f36a`](https://github.com/apache/spark/commit/909f36a251038f8b9a64046497233e6e11626c41).


----------------------------------------------------------------
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] AngersZhuuuu commented on a change in pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/SubstituteUnresolvedOrdinals.scala
##########
@@ -27,13 +27,23 @@ import org.apache.spark.sql.types.IntegerType
  * Replaces ordinal in 'order by' or 'group by' with UnresolvedOrdinal expression.
  */
 object SubstituteUnresolvedOrdinals extends Rule[LogicalPlan] {
-  private def isIntLiteral(e: Expression) = e match {
+  private def containIntLiteral(e: Expression): Boolean = e match {
     case Literal(_, IntegerType) => true
+    case Cube(_, groupByExprs) => groupByExprs.exists(containIntLiteral)
+    case Rollup(_, groupByExprs) => groupByExprs.exists(containIntLiteral)
+    case GroupingSets(_, flatGroupingSets, groupByExprs) =>
+      flatGroupingSets.exists(containIntLiteral) || groupByExprs.exists(containIntLiteral)
     case _ => false
   }
 
+  private def resolveOrdinal(expression: Expression): Expression = expression match {

Review comment:
       > the name is weird as it returns `UnresolvedOrdinal`. how about `substituteUnresolvedOrdinal`?
   
   Done and updated some missed logic.




-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1788,16 +1788,30 @@ class Analyzer(override val catalogManager: CatalogManager)
       // Replace the index with the corresponding expression in aggregateExpressions. The index is
       // a 1-base position of aggregateExpressions, which is output columns (select expression)
       case Aggregate(groups, aggs, child) if aggs.forall(_.resolved) &&
-        groups.exists(_.isInstanceOf[UnresolvedOrdinal]) =>
-        val newGroups = groups.map {
-          case u @ UnresolvedOrdinal(index) if index > 0 && index <= aggs.size =>
-            aggs(index - 1)
-          case ordinal @ UnresolvedOrdinal(index) =>
-            throw QueryCompilationErrors.groupByPositionRangeError(index, aggs.size, ordinal)
-          case o => o
-        }
+        groups.exists(containUnresolvedOrdinal) =>
+        val newGroups = groups.map((resolveGroupByExpressionOrdinal(_, aggs)))

Review comment:
       `((resolveGroupByExpressionOrdinal(_, aggs)))` -> `(resolveGroupByExpressionOrdinal(_, aggs))`




-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #137002 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137002/testReport)** for PR 30145 at commit [`5cfae7a`](https://github.com/apache/spark/commit/5cfae7a4519509e4edd6300db14fbf5917d6dd24).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `case class Cube(groupingSetIndexes: Seq[Seq[Int]], children: Seq[Expression]) extends GroupingSet `


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/SubstituteUnresolvedOrdinals.scala
##########
@@ -42,10 +52,19 @@ object SubstituteUnresolvedOrdinals extends Rule[LogicalPlan] {
       }
       withOrigin(s.origin)(s.copy(order = newOrders))
 
-    case a: Aggregate if conf.groupByOrdinal && a.groupingExpressions.exists(isIntLiteral) =>
+    case a: Aggregate if conf.groupByOrdinal && a.groupingExpressions.exists(containIntLiteral) =>
       val newGroups = a.groupingExpressions.map {
         case ordinal @ Literal(index: Int, IntegerType) =>
           withOrigin(ordinal.origin)(UnresolvedOrdinal(index))
+        case cube @ Cube(_, children) =>
+          withOrigin(cube.origin)(cube.copy(children = children.map(substituteUnresolvedOrdinal)))
+        case rollup @ Rollup(_, children) =>
+          withOrigin(rollup.origin)(rollup.copy(
+            children = children.map(substituteUnresolvedOrdinal)))
+        case groupingSets @ GroupingSets(_, flatGroupingSets, groupByExprs) =>
+          withOrigin(groupingSets.origin)(groupingSets.copy(
+            flatGroupingSets = flatGroupingSets.map(substituteUnresolvedOrdinal),
+            groupByExprs = groupByExprs.map(substituteUnresolvedOrdinal)))

Review comment:
       ditto, we can use `withNewChildren`




-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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] AngersZhuuuu commented on a change in pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1788,16 +1788,30 @@ class Analyzer(override val catalogManager: CatalogManager)
       // Replace the index with the corresponding expression in aggregateExpressions. The index is
       // a 1-base position of aggregateExpressions, which is output columns (select expression)
       case Aggregate(groups, aggs, child) if aggs.forall(_.resolved) &&
-        groups.exists(_.isInstanceOf[UnresolvedOrdinal]) =>
-        val newGroups = groups.map {
-          case u @ UnresolvedOrdinal(index) if index > 0 && index <= aggs.size =>
-            aggs(index - 1)
-          case ordinal @ UnresolvedOrdinal(index) =>
-            throw QueryCompilationErrors.groupByPositionRangeError(index, aggs.size, ordinal)
-          case o => o
-        }
+        groups.exists(containUnresolvedOrdinal) =>
+        val newGroups = groups.map((resolveGroupByExpressionOrdinal(_, aggs)))
         Aggregate(newGroups, aggs, child)
     }
+
+    private def containUnresolvedOrdinal(e: Expression): Boolean = e match {
+      case _: UnresolvedOrdinal => true
+      case gs: BaseGroupingSets => gs.children.exists(containUnresolvedOrdinal)
+      case _ => false
+    }
+
+    private def resolveGroupByExpressionOrdinal(
+        expr: Expression,
+        aggs: Seq[Expression]): Expression = expr match {
+      case ordinal @ UnresolvedOrdinal(index) =>
+        if (index > 0 && index <= aggs.size) {
+          aggs(index - 1)
+        } else {
+          throw QueryCompilationErrors.groupByPositionRangeError(index, aggs.size, ordinal)

Review comment:
       > Could you add tests for this code path?
   
   Yea

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1788,16 +1788,30 @@ class Analyzer(override val catalogManager: CatalogManager)
       // Replace the index with the corresponding expression in aggregateExpressions. The index is
       // a 1-base position of aggregateExpressions, which is output columns (select expression)
       case Aggregate(groups, aggs, child) if aggs.forall(_.resolved) &&
-        groups.exists(_.isInstanceOf[UnresolvedOrdinal]) =>
-        val newGroups = groups.map {
-          case u @ UnresolvedOrdinal(index) if index > 0 && index <= aggs.size =>
-            aggs(index - 1)
-          case ordinal @ UnresolvedOrdinal(index) =>
-            throw QueryCompilationErrors.groupByPositionRangeError(index, aggs.size, ordinal)
-          case o => o
-        }
+        groups.exists(containUnresolvedOrdinal) =>
+        val newGroups = groups.map((resolveGroupByExpressionOrdinal(_, aggs)))

Review comment:
       > `((resolveGroupByExpressionOrdinal(_, aggs)))` -> `(resolveGroupByExpressionOrdinal(_, aggs))`
   
   Done




-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/grouping.scala
##########
@@ -144,12 +147,12 @@ case class GroupingSets(
 object GroupingSets {
   def apply(
       groupingSets: Seq[Seq[Expression]],
-      userGivenGroupByExprs: Seq[Expression]): GroupingSets = {
-    val groupingSetIndexes = GroupingSet.computeGroupingSetIndexes(groupingSets)
+      userGivenGroupByExprs: Seq[Expression]): GroupingAnalytic = {
+    val groupingSetIndexes = GroupingAnalytic.computeGroupingSetIndexes(groupingSets)
     GroupingSets(groupingSetIndexes, groupingSets.flatten, userGivenGroupByExprs)
   }
 
-  def apply(groupingSets: Seq[Seq[Expression]]): GroupingSets = {
+  def apply(groupingSets: Seq[Seq[Expression]]): GroupingAnalytic = {

Review comment:
       we can probably do the rename in a separate PR.




-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   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 commented on pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #130290 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130290/testReport)** for PR 30145 at commit [`feeae94`](https://github.com/apache/spark/commit/feeae94b1660ad8fd086a9d37ac2719e87db97a0).
    * 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 removed a comment on pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #130242 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130242/testReport)** for PR 30145 at commit [`fa3346c`](https://github.com/apache/spark/commit/fa3346c6203add09f1fbbc40b0f3cfd4a2d441f7).


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/34907/
   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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #130305 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130305/testReport)** for PR 30145 at commit [`5cbe261`](https://github.com/apache/spark/commit/5cbe2610422eb022039705f4c01b3d4404812879).


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -598,8 +598,8 @@ class Analyzer(override val catalogManager: CatalogManager)
       val aggForResolving = h.child match {
         // For CUBE/ROLLUP expressions, to avoid resolving repeatedly, here we delete them from
         // groupingExpressions for condition resolving.
-        case a @ Aggregate(Seq(gs: GroupingSet), _, _) =>
-          a.copy(groupingExpressions = gs.groupByExprs)
+        case a @ Aggregate(Seq(gs: GroupingAnalytic), _, _) =>
+          a.copy(groupingExpressions =gs.groupingSets, gs.groupByExprs)

Review comment:
       nit: one space after `=`




-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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] AngersZhuuuu commented on a change in pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1787,16 +1787,41 @@ class Analyzer(override val catalogManager: CatalogManager)
       // Replace the index with the corresponding expression in aggregateExpressions. The index is
       // a 1-base position of aggregateExpressions, which is output columns (select expression)
       case Aggregate(groups, aggs, child) if aggs.forall(_.resolved) &&
-        groups.exists(_.isInstanceOf[UnresolvedOrdinal]) =>
-        val newGroups = groups.map {
-          case u @ UnresolvedOrdinal(index) if index > 0 && index <= aggs.size =>
-            aggs(index - 1)
-          case ordinal @ UnresolvedOrdinal(index) =>
-            throw QueryCompilationErrors.groupByPositionRangeError(index, aggs.size, ordinal)
-          case o => o
-        }
+        groups.exists(containUnresolvedOrdinal) =>
+        val newGroups = groups.map((resolveGroupByExpressionOrdinal(_, aggs)))
         Aggregate(newGroups, aggs, child)
     }
+
+    private def containUnresolvedOrdinal(e: Expression): Boolean = e match {
+      case _: UnresolvedOrdinal => true
+      case Cube(_, groupByExprs) => groupByExprs.exists(containUnresolvedOrdinal)
+      case Rollup(_, groupByExprs) => groupByExprs.exists(containUnresolvedOrdinal)
+      case GroupingSets(_, flatGroupingSets, groupByExprs) =>
+        flatGroupingSets.exists(containUnresolvedOrdinal) ||
+          groupByExprs.exists(containUnresolvedOrdinal)

Review comment:
       > Can we simply do `case a: GroupingAnalytic a.children.exists...`?
   
   Done




-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #136765 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136765/testReport)** for PR 30145 at commit [`65fa25f`](https://github.com/apache/spark/commit/65fa25fdc779c627fa825f22620f140668f658bc).


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1787,16 +1787,36 @@ class Analyzer(override val catalogManager: CatalogManager)
       // Replace the index with the corresponding expression in aggregateExpressions. The index is
       // a 1-base position of aggregateExpressions, which is output columns (select expression)
       case Aggregate(groups, aggs, child) if aggs.forall(_.resolved) &&
-        groups.exists(_.isInstanceOf[UnresolvedOrdinal]) =>
-        val newGroups = groups.map {
-          case u @ UnresolvedOrdinal(index) if index > 0 && index <= aggs.size =>
-            aggs(index - 1)
-          case ordinal @ UnresolvedOrdinal(index) =>
-            throw QueryCompilationErrors.groupByPositionRangeError(index, aggs.size, ordinal)
-          case o => o
-        }
+        groups.exists(containUnresolvedOrdinal) =>
+        val newGroups = groups.map((resolveGroupByExpressionOrdinal(_, aggs)))
         Aggregate(newGroups, aggs, child)
     }
+
+    private def containUnresolvedOrdinal(e: Expression): Boolean = e match {
+      case _: UnresolvedOrdinal => true
+      case gs: GroupingSet => gs.children.exists(containUnresolvedOrdinal)
+      case _ => false
+    }
+
+    private def resolveGroupByExpressionOrdinal(
+        expr: Expression,
+        aggs: Seq[Expression]): Expression = expr match {
+      case ordinal @ UnresolvedOrdinal(index) =>
+        if (index > 0 && index <= aggs.size) {
+          aggs(index - 1)
+        } else {
+          throw QueryCompilationErrors.groupByPositionRangeError(index, aggs.size, ordinal)
+        }
+      case cube @ Cube(_, groupByExprs) =>
+        cube.withNewChildren(groupByExprs.map(resolveGroupByExpressionOrdinal(_, aggs)))
+      case rollup @ Rollup(_, groupByExprs) =>
+        rollup.withNewChildren(groupByExprs.map(resolveGroupByExpressionOrdinal(_, aggs)))
+      case groupingSets @ GroupingSets(_, flatGroupingSets, userGivenGroupByExprs) =>
+        groupingSets.withNewChildren(
+          flatGroupingSets.map(resolveGroupByExpressionOrdinal(_, aggs))
+            ++ userGivenGroupByExprs.map(resolveGroupByExpressionOrdinal(_, aggs)))

Review comment:
       can we unify these cases now?
   ```
   case g: GroupingSet =>
     g.withNewChildren(g.children.map...)
   ```




-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #131187 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131187/testReport)** for PR 30145 at commit [`6fa8115`](https://github.com/apache/spark/commit/6fa811589c5731aed22a45f1dd67ed8643a1d318).


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/SubstituteUnresolvedOrdinals.scala
##########
@@ -27,13 +27,23 @@ import org.apache.spark.sql.types.IntegerType
  * Replaces ordinal in 'order by' or 'group by' with UnresolvedOrdinal expression.
  */
 object SubstituteUnresolvedOrdinals extends Rule[LogicalPlan] {
-  private def isIntLiteral(e: Expression) = e match {
+  private def containIntLiteral(e: Expression): Boolean = e match {
     case Literal(_, IntegerType) => true
+    case Cube(_, groupByExprs) => groupByExprs.exists(containIntLiteral)
+    case Rollup(_, groupByExprs) => groupByExprs.exists(containIntLiteral)
+    case GroupingSets(_, flatGroupingSets, groupByExprs) =>
+      flatGroupingSets.exists(containIntLiteral) || groupByExprs.exists(containIntLiteral)
     case _ => false
   }
 
+  private def resolveOrdinal(expression: Expression): Expression = expression match {

Review comment:
       the name is weird as it returns `UnresolvedOrdinal`. how about `substituteUnresolvedOrdinal`?




-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #137009 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137009/testReport)** for PR 30145 at commit [`391fda3`](https://github.com/apache/spark/commit/391fda3b311cd1cfa2cc9abf7b560d6f365568d1).


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #130305 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130305/testReport)** for PR 30145 at commit [`5cbe261`](https://github.com/apache/spark/commit/5cbe2610422eb022039705f4c01b3d4404812879).


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1788,16 +1788,30 @@ class Analyzer(override val catalogManager: CatalogManager)
       // Replace the index with the corresponding expression in aggregateExpressions. The index is
       // a 1-base position of aggregateExpressions, which is output columns (select expression)
       case Aggregate(groups, aggs, child) if aggs.forall(_.resolved) &&
-        groups.exists(_.isInstanceOf[UnresolvedOrdinal]) =>
-        val newGroups = groups.map {
-          case u @ UnresolvedOrdinal(index) if index > 0 && index <= aggs.size =>
-            aggs(index - 1)
-          case ordinal @ UnresolvedOrdinal(index) =>
-            throw QueryCompilationErrors.groupByPositionRangeError(index, aggs.size, ordinal)
-          case o => o
-        }
+        groups.exists(containUnresolvedOrdinal) =>
+        val newGroups = groups.map((resolveGroupByExpressionOrdinal(_, aggs)))
         Aggregate(newGroups, aggs, child)
     }
+
+    private def containUnresolvedOrdinal(e: Expression): Boolean = e match {
+      case _: UnresolvedOrdinal => true
+      case gs: BaseGroupingSets => gs.children.exists(containUnresolvedOrdinal)
+      case _ => false
+    }
+
+    private def resolveGroupByExpressionOrdinal(
+        expr: Expression,
+        aggs: Seq[Expression]): Expression = expr match {
+      case ordinal @ UnresolvedOrdinal(index) =>
+        if (index > 0 && index <= aggs.size) {
+          aggs(index - 1)
+        } else {
+          throw QueryCompilationErrors.groupByPositionRangeError(index, aggs.size, ordinal)

Review comment:
       Could you add tests for this code path?




-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/130435/
   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] maropu commented on a change in pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -244,6 +244,7 @@ class Analyzer(
       ExtractGenerator ::
       ResolveGenerate ::
       ResolveFunctions ::
+      new SubstituteUnresolvedOrdinals(conf) ::

Review comment:
       Ah, I got it. Thanks. But, it looks a bit weird for the resolution batch to have the substitution rule. Instead, we cannot support CUBE(...) and ROLLUP(...) in the parser level so as to avoid `UnresolvedFunction('cube(1,2)')`? cc: @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 commented on pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #130229 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130229/testReport)** for PR 30145 at commit [`fa3346c`](https://github.com/apache/spark/commit/fa3346c6203add09f1fbbc40b0f3cfd4a2d441f7).


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
##########
@@ -3691,6 +3691,22 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark
       checkAnswer(sql("SELECT id FROM t WHERE (SELECT true)"), Row(0L))
     }
   }
+
+  test("SPARK-33233: CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal") {

Review comment:
       Please add tests in `SQLQueryTestSuite`.




----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -244,6 +244,7 @@ class Analyzer(
       ExtractGenerator ::
       ResolveGenerate ::
       ResolveFunctions ::
+      new SubstituteUnresolvedOrdinals(conf) ::

Review comment:
       Please use `SQLConf.get` instead? https://github.com/apache/spark/pull/30097#issuecomment-716490495




----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #136996 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136996/testReport)** for PR 30145 at commit [`ff6794e`](https://github.com/apache/spark/commit/ff6794eb5387b6e83bfd3875884df02b75b0fafd).


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #136765 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136765/testReport)** for PR 30145 at commit [`65fa25f`](https://github.com/apache/spark/commit/65fa25fdc779c627fa825f22620f140668f658bc).
    * 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] AmplabJenkins removed a comment on pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #136772 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136772/testReport)** for PR 30145 at commit [`7ab346e`](https://github.com/apache/spark/commit/7ab346e898cb4c4ec4db9a28c48b66f1108b5f61).


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #136808 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136808/testReport)** for PR 30145 at commit [`c01ff72`](https://github.com/apache/spark/commit/c01ff729f4c53bce1fee7a662f9901ebb550ab22).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `trait GroupingAnalytic extends Expression with CodegenFallback `
     * `case class Cube(`


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
##########
@@ -3691,6 +3691,22 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark
       checkAnswer(sql("SELECT id FROM t WHERE (SELECT true)"), Row(0L))
     }
   }
+
+  test("SPARK-33233: CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal") {
+    withTable("t") {
+      sql("CREATE TABLE t USING PARQUET AS SELECT id AS a, id AS b, id AS c FROM range(1)")

Review comment:
       Since `group-by-ordinal.sql` looks enough for end-2-end tests, I think we don't need to update this file.




----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #136989 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136989/testReport)** for PR 30145 at commit [`897890c`](https://github.com/apache/spark/commit/897890c1e54be1e211373d25e96d8d8376b038cd).


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/35034/
   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] AngersZhuuuu commented on a change in pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1950,16 +1950,39 @@ class Analyzer(override val catalogManager: CatalogManager)
       // Replace the index with the corresponding expression in aggregateExpressions. The index is
       // a 1-base position of aggregateExpressions, which is output columns (select expression)
       case Aggregate(groups, aggs, child) if aggs.forall(_.resolved) &&
-        groups.exists(_.isInstanceOf[UnresolvedOrdinal]) =>
-        val newGroups = groups.map {
-          case u @ UnresolvedOrdinal(index) if index > 0 && index <= aggs.size =>
-            aggs(index - 1)
-          case ordinal @ UnresolvedOrdinal(index) =>
-            throw QueryCompilationErrors.groupByPositionRangeError(index, aggs.size, ordinal)
-          case o => o
-        }
+        groups.exists(containUnresolvedOrdinal) =>
+        val newGroups = groups.map((resolveGroupByExpressionOrdinal(_, aggs)))
         Aggregate(newGroups, aggs, child)
     }
+
+    private def containUnresolvedOrdinal(e: Expression): Boolean = e match {
+      case _: UnresolvedOrdinal => true
+      case Cube(_, groupByExprs) => groupByExprs.exists(containUnresolvedOrdinal)

Review comment:
       > how about `g: GroupingSet => g.groupByExprs.exists(containUnresolvedOrdinal)`
   
   If we remove the strange grammer `group by 1, 2 grouping sets((a, b), ())`, I think we can do like this since
   
   




-- 
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] AngersZhuuuu commented on pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   Gentle ping @cloud-fan @maropu We can start review this too


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #136518 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136518/testReport)** for PR 30145 at commit [`6fa8115`](https://github.com/apache/spark/commit/6fa811589c5731aed22a45f1dd67ed8643a1d318).


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #130242 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130242/testReport)** for PR 30145 at commit [`fa3346c`](https://github.com/apache/spark/commit/fa3346c6203add09f1fbbc40b0f3cfd4a2d441f7).


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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] AngersZhuuuu commented on pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   retest this please


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #131187 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131187/testReport)** for PR 30145 at commit [`6fa8115`](https://github.com/apache/spark/commit/6fa811589c5731aed22a45f1dd67ed8643a1d318).
    * 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 removed a comment on pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1775,17 +1776,29 @@ class Analyzer(
       // Replace the index with the corresponding expression in aggregateExpressions. The index is
       // a 1-base position of aggregateExpressions, which is output columns (select expression)
       case Aggregate(groups, aggs, child) if aggs.forall(_.resolved) &&
-        groups.exists(_.isInstanceOf[UnresolvedOrdinal]) =>
-        val newGroups = groups.map {
-          case u @ UnresolvedOrdinal(index) if index > 0 && index <= aggs.size =>
-            aggs(index - 1)
-          case ordinal @ UnresolvedOrdinal(index) =>
-            ordinal.failAnalysis(
-              s"GROUP BY position $index is not in select list " +
-                s"(valid range is [1, ${aggs.size}])")
-          case o => o
-        }
+        groups.exists(_.find(e => e.isInstanceOf[UnresolvedOrdinal]).isDefined) =>

Review comment:
       nit: `e => e.isInstanceOf` -> `_.isInstanceOf`




----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #137044 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137044/testReport)** for PR 30145 at commit [`0e89da2`](https://github.com/apache/spark/commit/0e89da2e53a36f1ca14278c456b40571a15517c0).


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #136829 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136829/testReport)** for PR 30145 at commit [`ff7971b`](https://github.com/apache/spark/commit/ff7971b2817b46c45b0584dfdfdda999bfd2b96d).


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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 pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   > Sure, Postgres SQL support this
   > Tera DB support too.
   
   Could you add this info in the PR description?


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #137002 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137002/testReport)** for PR 30145 at commit [`5cfae7a`](https://github.com/apache/spark/commit/5cfae7a4519509e4edd6300db14fbf5917d6dd24).


-- 
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] AngersZhuuuu commented on pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   retest this please


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #136989 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136989/testReport)** for PR 30145 at commit [`897890c`](https://github.com/apache/spark/commit/897890c1e54be1e211373d25e96d8d8376b038cd).


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


----------------------------------------------------------------
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] AngersZhuuuu commented on a change in pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -244,6 +244,7 @@ class Analyzer(
       ExtractGenerator ::
       ResolveGenerate ::
       ResolveFunctions ::
+      new SubstituteUnresolvedOrdinals(conf) ::

Review comment:
       > Still not sure about why we cannot transform `cube(1, 2)` -> `cube(unresolvedordinal(1), unresolvedordinal(2))` in the substitution batch, then resolve them in the resolution batch. Any technical difficulty?
   
   since when we execute substitution batch, `cube(1,2)` still is UnresolvedFunction('cube(1,2)').  We need to solve it to `Cube(unresolvedordinal(1), unresolvedordinal(2))` after `ResolveFunction`




----------------------------------------------------------------
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] AngersZhuuuu commented on a change in pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -244,6 +244,7 @@ class Analyzer(
       ExtractGenerator ::
       ResolveGenerate ::
       ResolveFunctions ::
+      new SubstituteUnresolvedOrdinals(conf) ::

Review comment:
       > okay, thanks for the check. Could you try it in a separate PR first?
   
   Working 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] github-actions[bot] commented on pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #30145:
URL: https://github.com/apache/spark/pull/30145#issuecomment-786329822


   We're closing this PR because it hasn't been updated in a while. This isn't a judgement on the merit of the PR in any way. It's just a way of keeping the PR queue manageable.
   If you'd like to revive this PR, please reopen it and ask a committer to remove the Stale tag!


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #136765 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136765/testReport)** for PR 30145 at commit [`65fa25f`](https://github.com/apache/spark/commit/65fa25fdc779c627fa825f22620f140668f658bc).


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #136815 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136815/testReport)** for PR 30145 at commit [`ff7971b`](https://github.com/apache/spark/commit/ff7971b2817b46c45b0584dfdfdda999bfd2b96d).
    * 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] maropu commented on a change in pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -244,6 +244,7 @@ class Analyzer(
       ExtractGenerator ::
       ResolveGenerate ::
       ResolveFunctions ::
+      new SubstituteUnresolvedOrdinals(conf) ::

Review comment:
       Why we need to add this rule in the resolution batch?




----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/35029/
   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] AngersZhuuuu commented on a change in pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1787,16 +1787,41 @@ class Analyzer(override val catalogManager: CatalogManager)
       // Replace the index with the corresponding expression in aggregateExpressions. The index is
       // a 1-base position of aggregateExpressions, which is output columns (select expression)
       case Aggregate(groups, aggs, child) if aggs.forall(_.resolved) &&
-        groups.exists(_.isInstanceOf[UnresolvedOrdinal]) =>
-        val newGroups = groups.map {
-          case u @ UnresolvedOrdinal(index) if index > 0 && index <= aggs.size =>
-            aggs(index - 1)
-          case ordinal @ UnresolvedOrdinal(index) =>
-            throw QueryCompilationErrors.groupByPositionRangeError(index, aggs.size, ordinal)
-          case o => o
-        }
+        groups.exists(containUnresolvedOrdinal) =>
+        val newGroups = groups.map((resolveGroupByExpressionOrdinal(_, aggs)))
         Aggregate(newGroups, aggs, child)
     }
+
+    private def containUnresolvedOrdinal(e: Expression): Boolean = e match {
+      case _: UnresolvedOrdinal => true
+      case Cube(_, groupByExprs) => groupByExprs.exists(containUnresolvedOrdinal)
+      case Rollup(_, groupByExprs) => groupByExprs.exists(containUnresolvedOrdinal)
+      case GroupingSets(_, flatGroupingSets, groupByExprs) =>
+        flatGroupingSets.exists(containUnresolvedOrdinal) ||
+          groupByExprs.exists(containUnresolvedOrdinal)
+      case _ => false
+    }
+
+    private def resolveGroupByExpressionOrdinal(
+        expr: Expression,
+        aggs: Seq[Expression]): Expression = expr match {
+      case ordinal @ UnresolvedOrdinal(index) =>
+        if (index > 0 && index <= aggs.size) {
+          aggs(index - 1)
+        } else {
+          throw QueryCompilationErrors.groupByPositionRangeError(index, aggs.size, ordinal)
+        }
+      case cube @ Cube(_, groupByExprs) =>

Review comment:
       > how about using `expr.withNewChildren`>
   
   Done




-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #136829 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136829/testReport)** for PR 30145 at commit [`ff7971b`](https://github.com/apache/spark/commit/ff7971b2817b46c45b0584dfdfdda999bfd2b96d).
    * 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] AngersZhuuuu commented on a change in pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
##########
@@ -3691,6 +3691,22 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark
       checkAnswer(sql("SELECT id FROM t WHERE (SELECT true)"), Row(0L))
     }
   }
+
+  test("SPARK-33233: CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal") {
+    withTable("t") {
+      sql("CREATE TABLE t USING PARQUET AS SELECT id AS a, id AS b, id AS c FROM range(1)")

Review comment:
       > Since `group-by-ordinal.sql` looks enough for end-2-end tests, I think we don't need to update this file.
   
   Yea.




----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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] AngersZhuuuu commented on a change in pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -244,6 +244,7 @@ class Analyzer(
       ExtractGenerator ::
       ResolveGenerate ::
       ResolveFunctions ::
+      new SubstituteUnresolvedOrdinals(conf) ::

Review comment:
       > Ah, I got it. Thanks. But, it looks a bit weird for the resolution batch to have the substitution rule. Instead, we cannot support CUBE(...) and ROLLUP(...) in the parser level so as to avoid `UnresolvedFunction('cube(1,2)')`? cc: @cloud-fan
   
   These days. I am thinking about this too. Since as you mentioned in https://github.com/apache/spark/pull/30144#issuecomment-716353950,  `grouping sets` can be a expression like `cube` , `rollup`.
   
   Current parser level can't support it smoothly. Maybe we can reconstruct 
   ```
   
   aggregationClause
       : GROUP BY groupingExpressions+=expression (',' groupingExpressions+=expression)* (
         WITH kind=ROLLUP
       | WITH kind=CUBE
       | kind=GROUPING SETS '(' groupingSet (',' groupingSet)* ')')?
       | GROUP BY kind=GROUPING SETS '(' groupingSet (',' groupingSet)* ')'
       ;
   
   groupingSet
       : '(' (expression (',' expression)*)? ')'
       | 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] maropu commented on pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   Please fill the description? https://issues.apache.org/jira/browse/SPARK-33233?focusedCommentId=17220648&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17220648


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #130290 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130290/testReport)** for PR 30145 at commit [`feeae94`](https://github.com/apache/spark/commit/feeae94b1660ad8fd086a9d37ac2719e87db97a0).


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #130229 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130229/testReport)** for PR 30145 at commit [`fa3346c`](https://github.com/apache/spark/commit/fa3346c6203add09f1fbbc40b0f3cfd4a2d441f7).
    * This patch **fails due to an unknown error code, -9**.
    * 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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1787,16 +1787,41 @@ class Analyzer(override val catalogManager: CatalogManager)
       // Replace the index with the corresponding expression in aggregateExpressions. The index is
       // a 1-base position of aggregateExpressions, which is output columns (select expression)
       case Aggregate(groups, aggs, child) if aggs.forall(_.resolved) &&
-        groups.exists(_.isInstanceOf[UnresolvedOrdinal]) =>
-        val newGroups = groups.map {
-          case u @ UnresolvedOrdinal(index) if index > 0 && index <= aggs.size =>
-            aggs(index - 1)
-          case ordinal @ UnresolvedOrdinal(index) =>
-            throw QueryCompilationErrors.groupByPositionRangeError(index, aggs.size, ordinal)
-          case o => o
-        }
+        groups.exists(containUnresolvedOrdinal) =>
+        val newGroups = groups.map((resolveGroupByExpressionOrdinal(_, aggs)))
         Aggregate(newGroups, aggs, child)
     }
+
+    private def containUnresolvedOrdinal(e: Expression): Boolean = e match {
+      case _: UnresolvedOrdinal => true
+      case Cube(_, groupByExprs) => groupByExprs.exists(containUnresolvedOrdinal)
+      case Rollup(_, groupByExprs) => groupByExprs.exists(containUnresolvedOrdinal)
+      case GroupingSets(_, flatGroupingSets, groupByExprs) =>
+        flatGroupingSets.exists(containUnresolvedOrdinal) ||
+          groupByExprs.exists(containUnresolvedOrdinal)
+      case _ => false
+    }
+
+    private def resolveGroupByExpressionOrdinal(
+        expr: Expression,
+        aggs: Seq[Expression]): Expression = expr match {
+      case ordinal @ UnresolvedOrdinal(index) =>
+        if (index > 0 && index <= aggs.size) {
+          aggs(index - 1)
+        } else {
+          throw QueryCompilationErrors.groupByPositionRangeError(index, aggs.size, ordinal)
+        }
+      case cube @ Cube(_, groupByExprs) =>

Review comment:
       how about using `expr.withNewChildren`>




-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   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] AngersZhuuuu commented on pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   > > Sure, Postgres SQL support this
   > > Tera DB support too.
   > 
   > Could you add this info in the PR description?
   
   Yea, later for. a more detail. desc


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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] AngersZhuuuu commented on pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   retest this please


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #131171 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131171/testReport)** for PR 30145 at commit [`6fa8115`](https://github.com/apache/spark/commit/6fa811589c5731aed22a45f1dd67ed8643a1d318).


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #137044 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137044/testReport)** for PR 30145 at commit [`0e89da2`](https://github.com/apache/spark/commit/0e89da2e53a36f1ca14278c456b40571a15517c0).


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   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 commented on pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/35030/
   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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #136994 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136994/testReport)** for PR 30145 at commit [`a013120`](https://github.com/apache/spark/commit/a013120c8e9f0bdfb6eac91b3ed881059577d855).


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -658,14 +658,15 @@ class Analyzer(override val catalogManager: CatalogManager)
     // CUBE/ROLLUP/GROUPING SETS. This also replace grouping()/grouping_id() in resolved
     // Filter/Sort.
     def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperatorsDown {
-      case h @ UnresolvedHaving(_, agg @ Aggregate(Seq(gs: GroupingSet), aggregateExpressions, _))
+      case h @ UnresolvedHaving(
+      _, agg @ Aggregate(Seq(gs: GroupingAnalytic), aggregateExpressions, _))

Review comment:
       nit: to avoid the line break, how about renaming `aggregateExpressions` -> `aggExprs`?

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1950,16 +1951,39 @@ class Analyzer(override val catalogManager: CatalogManager)
       // Replace the index with the corresponding expression in aggregateExpressions. The index is
       // a 1-base position of aggregateExpressions, which is output columns (select expression)
       case Aggregate(groups, aggs, child) if aggs.forall(_.resolved) &&
-        groups.exists(_.isInstanceOf[UnresolvedOrdinal]) =>
-        val newGroups = groups.map {
-          case u @ UnresolvedOrdinal(index) if index > 0 && index <= aggs.size =>
-            aggs(index - 1)
-          case ordinal @ UnresolvedOrdinal(index) =>
-            throw QueryCompilationErrors.groupByPositionRangeError(index, aggs.size, ordinal)
-          case o => o
-        }
+        groups.exists(containUnresolvedOrdinal) =>
+        val newGroups = groups.map((resolveGroupByExpressionOrdinal(_, aggs)))
         Aggregate(newGroups, aggs, child)
     }
+
+    private def containUnresolvedOrdinal(e: Expression): Boolean = e match {
+      case _: UnresolvedOrdinal => true
+      case Cube(_, groupByExprs) => groupByExprs.exists(containUnresolvedOrdinal)
+      case Rollup(_, groupByExprs) => groupByExprs.exists(containUnresolvedOrdinal)
+      case GroupingSets(_, flatGroupingSets, groupByExprs) =>
+        flatGroupingSets.exists(containUnresolvedOrdinal) ||
+          groupByExprs.exists(containUnresolvedOrdinal)
+      case _ => false
+    }
+
+    private def resolveGroupByExpressionOrdinal(
+        expr: Expression,
+        aggs: Seq[Expression]): Expression = expr match {
+      case u @ UnresolvedOrdinal(index) if index > 0 && index <= aggs.size =>

Review comment:
       `u` not used.

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1950,16 +1951,39 @@ class Analyzer(override val catalogManager: CatalogManager)
       // Replace the index with the corresponding expression in aggregateExpressions. The index is
       // a 1-base position of aggregateExpressions, which is output columns (select expression)
       case Aggregate(groups, aggs, child) if aggs.forall(_.resolved) &&
-        groups.exists(_.isInstanceOf[UnresolvedOrdinal]) =>
-        val newGroups = groups.map {
-          case u @ UnresolvedOrdinal(index) if index > 0 && index <= aggs.size =>
-            aggs(index - 1)
-          case ordinal @ UnresolvedOrdinal(index) =>
-            throw QueryCompilationErrors.groupByPositionRangeError(index, aggs.size, ordinal)
-          case o => o
-        }
+        groups.exists(containUnresolvedOrdinal) =>
+        val newGroups = groups.map((resolveGroupByExpressionOrdinal(_, aggs)))
         Aggregate(newGroups, aggs, child)
     }
+
+    private def containUnresolvedOrdinal(e: Expression): Boolean = e match {
+      case _: UnresolvedOrdinal => true
+      case Cube(_, groupByExprs) => groupByExprs.exists(containUnresolvedOrdinal)
+      case Rollup(_, groupByExprs) => groupByExprs.exists(containUnresolvedOrdinal)
+      case GroupingSets(_, flatGroupingSets, groupByExprs) =>
+        flatGroupingSets.exists(containUnresolvedOrdinal) ||
+          groupByExprs.exists(containUnresolvedOrdinal)
+      case _ => false
+    }
+
+    private def resolveGroupByExpressionOrdinal(
+        expr: Expression,
+        aggs: Seq[Expression]): Expression = expr match {
+      case u @ UnresolvedOrdinal(index) if index > 0 && index <= aggs.size =>
+        aggs(index - 1)
+      case ordinal @ UnresolvedOrdinal(index) =>
+        throw QueryCompilationErrors.groupByPositionRangeError(index, aggs.size, ordinal)
+      case cube @ Cube(_, groupByExprs) =>
+        cube.copy(children = groupByExprs.map(resolveGroupByExpressionOrdinal(_, aggs)))
+      case rollup @ Rollup(_, groupByExprs) =>
+        rollup.copy(children = groupByExprs.map(resolveGroupByExpressionOrdinal(_, aggs)))
+      case groupingSets @ GroupingSets(_, flatGroupingSets, groupByExprs) =>
+        groupingSets.copy(
+          flatGroupingSets = flatGroupingSets.map(resolveGroupByExpressionOrdinal(_, aggs)),
+          groupByExprs = groupByExprs.map(resolveGroupByExpressionOrdinal(_, aggs))
+        )

Review comment:
       We can merge them into a single entry.

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1950,16 +1951,39 @@ class Analyzer(override val catalogManager: CatalogManager)
       // Replace the index with the corresponding expression in aggregateExpressions. The index is
       // a 1-base position of aggregateExpressions, which is output columns (select expression)
       case Aggregate(groups, aggs, child) if aggs.forall(_.resolved) &&
-        groups.exists(_.isInstanceOf[UnresolvedOrdinal]) =>
-        val newGroups = groups.map {
-          case u @ UnresolvedOrdinal(index) if index > 0 && index <= aggs.size =>
-            aggs(index - 1)
-          case ordinal @ UnresolvedOrdinal(index) =>
-            throw QueryCompilationErrors.groupByPositionRangeError(index, aggs.size, ordinal)
-          case o => o
-        }
+        groups.exists(containUnresolvedOrdinal) =>
+        val newGroups = groups.map((resolveGroupByExpressionOrdinal(_, aggs)))
         Aggregate(newGroups, aggs, child)
     }
+
+    private def containUnresolvedOrdinal(e: Expression): Boolean = e match {
+      case _: UnresolvedOrdinal => true
+      case Cube(_, groupByExprs) => groupByExprs.exists(containUnresolvedOrdinal)
+      case Rollup(_, groupByExprs) => groupByExprs.exists(containUnresolvedOrdinal)
+      case GroupingSets(_, flatGroupingSets, groupByExprs) =>
+        flatGroupingSets.exists(containUnresolvedOrdinal) ||
+          groupByExprs.exists(containUnresolvedOrdinal)
+      case _ => false
+    }
+
+    private def resolveGroupByExpressionOrdinal(
+        expr: Expression,
+        aggs: Seq[Expression]): Expression = expr match {
+      case u @ UnresolvedOrdinal(index) if index > 0 && index <= aggs.size =>

Review comment:
       nit: how about this?
   ```
         case ordinal @ UnresolvedOrdinal(index) =>
           if (index > 0 && index <= aggs.size) {
             aggs(index - 1)
           } else {
             throw QueryCompilationErrors.groupByPositionRangeError(index, aggs.size, ordinal)
           }
   
   ```




-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -244,6 +244,7 @@ class Analyzer(
       ExtractGenerator ::
       ResolveGenerate ::
       ResolveFunctions ::
+      new SubstituteUnresolvedOrdinals(conf) ::

Review comment:
       Still not sure about why we cannot transform `cube(1, 2)` -> `cube(unresolvedordinal(1), unresolvedordinal(2))` in the substitution batch, then resolve them in the resolution batch. Any technical difficulty?




----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #137060 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137060/testReport)** for PR 30145 at commit [`8d9a392`](https://github.com/apache/spark/commit/8d9a3928aaee57aeafb979f41dba4b0fe6c85d40).
    * 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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1787,16 +1787,41 @@ class Analyzer(override val catalogManager: CatalogManager)
       // Replace the index with the corresponding expression in aggregateExpressions. The index is
       // a 1-base position of aggregateExpressions, which is output columns (select expression)
       case Aggregate(groups, aggs, child) if aggs.forall(_.resolved) &&
-        groups.exists(_.isInstanceOf[UnresolvedOrdinal]) =>
-        val newGroups = groups.map {
-          case u @ UnresolvedOrdinal(index) if index > 0 && index <= aggs.size =>
-            aggs(index - 1)
-          case ordinal @ UnresolvedOrdinal(index) =>
-            throw QueryCompilationErrors.groupByPositionRangeError(index, aggs.size, ordinal)
-          case o => o
-        }
+        groups.exists(containUnresolvedOrdinal) =>
+        val newGroups = groups.map((resolveGroupByExpressionOrdinal(_, aggs)))
         Aggregate(newGroups, aggs, child)
     }
+
+    private def containUnresolvedOrdinal(e: Expression): Boolean = e match {
+      case _: UnresolvedOrdinal => true
+      case Cube(_, groupByExprs) => groupByExprs.exists(containUnresolvedOrdinal)
+      case Rollup(_, groupByExprs) => groupByExprs.exists(containUnresolvedOrdinal)
+      case GroupingSets(_, flatGroupingSets, groupByExprs) =>
+        flatGroupingSets.exists(containUnresolvedOrdinal) ||
+          groupByExprs.exists(containUnresolvedOrdinal)

Review comment:
       Can we simply do `case a: GroupingAnalytic a.children.exists...`?




-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #136518 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136518/testReport)** for PR 30145 at commit [`6fa8115`](https://github.com/apache/spark/commit/6fa811589c5731aed22a45f1dd67ed8643a1d318).


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #130293 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130293/testReport)** for PR 30145 at commit [`82e901f`](https://github.com/apache/spark/commit/82e901f95316efa31d5d55867bf0152e99d94b70).


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   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] AmplabJenkins commented on pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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] github-actions[bot] closed pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

Posted by GitBox <gi...@apache.org>.
github-actions[bot] closed pull request #30145:
URL: https://github.com/apache/spark/pull/30145


   


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1950,16 +1950,39 @@ class Analyzer(override val catalogManager: CatalogManager)
       // Replace the index with the corresponding expression in aggregateExpressions. The index is
       // a 1-base position of aggregateExpressions, which is output columns (select expression)
       case Aggregate(groups, aggs, child) if aggs.forall(_.resolved) &&
-        groups.exists(_.isInstanceOf[UnresolvedOrdinal]) =>
-        val newGroups = groups.map {
-          case u @ UnresolvedOrdinal(index) if index > 0 && index <= aggs.size =>
-            aggs(index - 1)
-          case ordinal @ UnresolvedOrdinal(index) =>
-            throw QueryCompilationErrors.groupByPositionRangeError(index, aggs.size, ordinal)
-          case o => o
-        }
+        groups.exists(containUnresolvedOrdinal) =>
+        val newGroups = groups.map((resolveGroupByExpressionOrdinal(_, aggs)))
         Aggregate(newGroups, aggs, child)
     }
+
+    private def containUnresolvedOrdinal(e: Expression): Boolean = e match {
+      case _: UnresolvedOrdinal => true
+      case Cube(_, groupByExprs) => groupByExprs.exists(containUnresolvedOrdinal)

Review comment:
       BTW we should refine the name. `GroupingSets extends GroupingSet` looks weird.




-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #130290 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130290/testReport)** for PR 30145 at commit [`feeae94`](https://github.com/apache/spark/commit/feeae94b1660ad8fd086a9d37ac2719e87db97a0).


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #131171 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131171/testReport)** for PR 30145 at commit [`6fa8115`](https://github.com/apache/spark/commit/6fa811589c5731aed22a45f1dd67ed8643a1d318).
    * 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] AmplabJenkins commented on pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/35047/
   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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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] AngersZhuuuu commented on a change in pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1950,16 +1951,39 @@ class Analyzer(override val catalogManager: CatalogManager)
       // Replace the index with the corresponding expression in aggregateExpressions. The index is
       // a 1-base position of aggregateExpressions, which is output columns (select expression)
       case Aggregate(groups, aggs, child) if aggs.forall(_.resolved) &&
-        groups.exists(_.isInstanceOf[UnresolvedOrdinal]) =>
-        val newGroups = groups.map {
-          case u @ UnresolvedOrdinal(index) if index > 0 && index <= aggs.size =>
-            aggs(index - 1)
-          case ordinal @ UnresolvedOrdinal(index) =>
-            throw QueryCompilationErrors.groupByPositionRangeError(index, aggs.size, ordinal)
-          case o => o
-        }
+        groups.exists(containUnresolvedOrdinal) =>
+        val newGroups = groups.map((resolveGroupByExpressionOrdinal(_, aggs)))
         Aggregate(newGroups, aggs, child)
     }
+
+    private def containUnresolvedOrdinal(e: Expression): Boolean = e match {
+      case _: UnresolvedOrdinal => true
+      case Cube(_, groupByExprs) => groupByExprs.exists(containUnresolvedOrdinal)
+      case Rollup(_, groupByExprs) => groupByExprs.exists(containUnresolvedOrdinal)
+      case GroupingSets(_, flatGroupingSets, groupByExprs) =>
+        flatGroupingSets.exists(containUnresolvedOrdinal) ||
+          groupByExprs.exists(containUnresolvedOrdinal)
+      case _ => false
+    }
+
+    private def resolveGroupByExpressionOrdinal(
+        expr: Expression,
+        aggs: Seq[Expression]): Expression = expr match {
+      case u @ UnresolvedOrdinal(index) if index > 0 && index <= aggs.size =>
+        aggs(index - 1)
+      case ordinal @ UnresolvedOrdinal(index) =>
+        throw QueryCompilationErrors.groupByPositionRangeError(index, aggs.size, ordinal)
+      case cube @ Cube(_, groupByExprs) =>
+        cube.copy(children = groupByExprs.map(resolveGroupByExpressionOrdinal(_, aggs)))
+      case rollup @ Rollup(_, groupByExprs) =>
+        rollup.copy(children = groupByExprs.map(resolveGroupByExpressionOrdinal(_, aggs)))
+      case groupingSets @ GroupingSets(_, flatGroupingSets, groupByExprs) =>
+        groupingSets.copy(
+          flatGroupingSets = flatGroupingSets.map(resolveGroupByExpressionOrdinal(_, aggs)),
+          groupByExprs = groupByExprs.map(resolveGroupByExpressionOrdinal(_, aggs))
+        )

Review comment:
       > We can merge them into a single entry.
   
   DOne

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1950,16 +1951,39 @@ class Analyzer(override val catalogManager: CatalogManager)
       // Replace the index with the corresponding expression in aggregateExpressions. The index is
       // a 1-base position of aggregateExpressions, which is output columns (select expression)
       case Aggregate(groups, aggs, child) if aggs.forall(_.resolved) &&
-        groups.exists(_.isInstanceOf[UnresolvedOrdinal]) =>
-        val newGroups = groups.map {
-          case u @ UnresolvedOrdinal(index) if index > 0 && index <= aggs.size =>
-            aggs(index - 1)
-          case ordinal @ UnresolvedOrdinal(index) =>
-            throw QueryCompilationErrors.groupByPositionRangeError(index, aggs.size, ordinal)
-          case o => o
-        }
+        groups.exists(containUnresolvedOrdinal) =>
+        val newGroups = groups.map((resolveGroupByExpressionOrdinal(_, aggs)))
         Aggregate(newGroups, aggs, child)
     }
+
+    private def containUnresolvedOrdinal(e: Expression): Boolean = e match {
+      case _: UnresolvedOrdinal => true
+      case Cube(_, groupByExprs) => groupByExprs.exists(containUnresolvedOrdinal)
+      case Rollup(_, groupByExprs) => groupByExprs.exists(containUnresolvedOrdinal)
+      case GroupingSets(_, flatGroupingSets, groupByExprs) =>
+        flatGroupingSets.exists(containUnresolvedOrdinal) ||
+          groupByExprs.exists(containUnresolvedOrdinal)
+      case _ => false
+    }
+
+    private def resolveGroupByExpressionOrdinal(
+        expr: Expression,
+        aggs: Seq[Expression]): Expression = expr match {
+      case u @ UnresolvedOrdinal(index) if index > 0 && index <= aggs.size =>

Review comment:
       Done

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -658,14 +658,15 @@ class Analyzer(override val catalogManager: CatalogManager)
     // CUBE/ROLLUP/GROUPING SETS. This also replace grouping()/grouping_id() in resolved
     // Filter/Sort.
     def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperatorsDown {
-      case h @ UnresolvedHaving(_, agg @ Aggregate(Seq(gs: GroupingSet), aggregateExpressions, _))
+      case h @ UnresolvedHaving(
+      _, agg @ Aggregate(Seq(gs: GroupingAnalytic), aggregateExpressions, _))

Review comment:
       > nit: to avoid the line break, how about renaming `aggregateExpressions` -> `aggExprs`?
   
   Done




-- 
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] AngersZhuuuu commented on a change in pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/core/src/test/resources/sql-tests/inputs/group-by-ordinal.sql
##########
@@ -54,6 +54,26 @@ select count(a), a from (select 1 as a) tmp group by 2 having a > 0;
 -- mixed cases: group-by ordinals and aliases
 select a, a AS k, count(b) from data group by k, 1;
 
+-- can use ordinal in CUBE
+select a, b, count(1) from data group by cube(1, 2);

Review comment:
       > could you add tests for the cases: `with rollup`/`with cube`?
   
   Updated.
   
   
   and 
   ```
   select a, b, count(1) from data group by cube(1, 2) with cube;
   ```
    with throw 
   ```
            > select a, b, c from x group by cube(a, b, c) with cube;
   20/10/30 11:16:24 ERROR SparkSQLDriver: Failed in [select a, b, c from x group by cube(a, b, c) with cube]
   java.lang.UnsupportedOperationException
   	at org.apache.spark.sql.catalyst.expressions.GroupingSet.dataType(grouping.scala:36)
   	at org.apache.spark.sql.catalyst.expressions.GroupingSet.dataType$(grouping.scala:36)
   ```
   
   I will rase a pr to catch  similar exception and throw an exception that is easy for the user to understand. Such as 
   ```
   we can't use `WITH CUBE ` and `cube func` together
   ```




----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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] AngersZhuuuu commented on pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   ping @cloud-fan @maropu Any more suggestion?


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #130429 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130429/testReport)** for PR 30145 at commit [`76fdf6b`](https://github.com/apache/spark/commit/76fdf6b42e96663f2ed45263aa905d8ae77e9223).


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #137009 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137009/testReport)** for PR 30145 at commit [`391fda3`](https://github.com/apache/spark/commit/391fda3b311cd1cfa2cc9abf7b560d6f365568d1).
    * 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 removed a comment on pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #130229 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130229/testReport)** for PR 30145 at commit [`fa3346c`](https://github.com/apache/spark/commit/fa3346c6203add09f1fbbc40b0f3cfd4a2d441f7).


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #130435 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130435/testReport)** for PR 30145 at commit [`909f36a`](https://github.com/apache/spark/commit/909f36a251038f8b9a64046497233e6e11626c41).
    * 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] AngersZhuuuu commented on a change in pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1950,16 +1950,39 @@ class Analyzer(override val catalogManager: CatalogManager)
       // Replace the index with the corresponding expression in aggregateExpressions. The index is
       // a 1-base position of aggregateExpressions, which is output columns (select expression)
       case Aggregate(groups, aggs, child) if aggs.forall(_.resolved) &&
-        groups.exists(_.isInstanceOf[UnresolvedOrdinal]) =>
-        val newGroups = groups.map {
-          case u @ UnresolvedOrdinal(index) if index > 0 && index <= aggs.size =>
-            aggs(index - 1)
-          case ordinal @ UnresolvedOrdinal(index) =>
-            throw QueryCompilationErrors.groupByPositionRangeError(index, aggs.size, ordinal)
-          case o => o
-        }
+        groups.exists(containUnresolvedOrdinal) =>
+        val newGroups = groups.map((resolveGroupByExpressionOrdinal(_, aggs)))
         Aggregate(newGroups, aggs, child)
     }
+
+    private def containUnresolvedOrdinal(e: Expression): Boolean = e match {
+      case _: UnresolvedOrdinal => true
+      case Cube(_, groupByExprs) => groupByExprs.exists(containUnresolvedOrdinal)

Review comment:
       > BTW we should refine the name. `GroupingSets extends GroupingSet` looks weird.
   
   Renaming to `GroupingAnalytic`




-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #136766 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136766/testReport)** for PR 30145 at commit [`fa75aa1`](https://github.com/apache/spark/commit/fa75aa139e4703414620c1ce8386ab64fd53d67b).


-- 
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] AngersZhuuuu commented on a change in pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -598,8 +598,8 @@ class Analyzer(override val catalogManager: CatalogManager)
       val aggForResolving = h.child match {
         // For CUBE/ROLLUP expressions, to avoid resolving repeatedly, here we delete them from
         // groupingExpressions for condition resolving.
-        case a @ Aggregate(Seq(gs: GroupingSet), _, _) =>
-          a.copy(groupingExpressions = gs.groupByExprs)
+        case a @ Aggregate(Seq(gs: GroupingAnalytic), _, _) =>
+          a.copy(groupingExpressions =gs.groupingSets, gs.groupByExprs)

Review comment:
       > nit: one space after `=`
   
   Mistake when merge code, done




-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #136772 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136772/testReport)** for PR 30145 at commit [`7ab346e`](https://github.com/apache/spark/commit/7ab346e898cb4c4ec4db9a28c48b66f1108b5f61).


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #136772 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136772/testReport)** for PR 30145 at commit [`7ab346e`](https://github.com/apache/spark/commit/7ab346e898cb4c4ec4db9a28c48b66f1108b5f61).
    * 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] maropu commented on a change in pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -244,6 +244,7 @@ class Analyzer(
       ExtractGenerator ::
       ResolveGenerate ::
       ResolveFunctions ::
+      new SubstituteUnresolvedOrdinals(conf) ::

Review comment:
       okay, thanks for the check. Could you try it in a separate PR first?




----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #136762 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136762/testReport)** for PR 30145 at commit [`fb159c0`](https://github.com/apache/spark/commit/fb159c0b84e512c26c7cc7a92f821888baccb27e).


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #131171 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131171/testReport)** for PR 30145 at commit [`6fa8115`](https://github.com/apache/spark/commit/6fa811589c5731aed22a45f1dd67ed8643a1d318).


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #130293 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130293/testReport)** for PR 30145 at commit [`82e901f`](https://github.com/apache/spark/commit/82e901f95316efa31d5d55867bf0152e99d94b70).
    * 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] AngersZhuuuu commented on a change in pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -244,6 +244,7 @@ class Analyzer(
       ExtractGenerator ::
       ResolveGenerate ::
       ResolveFunctions ::
+      new SubstituteUnresolvedOrdinals(conf) ::

Review comment:
       > Why we need to add this rule in the resolution batch?
   
   Since `cube/rollup` can be a function. 




----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #131187 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131187/testReport)** for PR 30145 at commit [`6fa8115`](https://github.com/apache/spark/commit/6fa811589c5731aed22a45f1dd67ed8643a1d318).


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #136994 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136994/testReport)** for PR 30145 at commit [`a013120`](https://github.com/apache/spark/commit/a013120c8e9f0bdfb6eac91b3ed881059577d855).
    * This patch **fails to build**.
    * 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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #136994 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136994/testReport)** for PR 30145 at commit [`a013120`](https://github.com/apache/spark/commit/a013120c8e9f0bdfb6eac91b3ed881059577d855).


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #130435 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130435/testReport)** for PR 30145 at commit [`909f36a`](https://github.com/apache/spark/commit/909f36a251038f8b9a64046497233e6e11626c41).


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1950,16 +1950,39 @@ class Analyzer(override val catalogManager: CatalogManager)
       // Replace the index with the corresponding expression in aggregateExpressions. The index is
       // a 1-base position of aggregateExpressions, which is output columns (select expression)
       case Aggregate(groups, aggs, child) if aggs.forall(_.resolved) &&
-        groups.exists(_.isInstanceOf[UnresolvedOrdinal]) =>
-        val newGroups = groups.map {
-          case u @ UnresolvedOrdinal(index) if index > 0 && index <= aggs.size =>
-            aggs(index - 1)
-          case ordinal @ UnresolvedOrdinal(index) =>
-            throw QueryCompilationErrors.groupByPositionRangeError(index, aggs.size, ordinal)
-          case o => o
-        }
+        groups.exists(containUnresolvedOrdinal) =>
+        val newGroups = groups.map((resolveGroupByExpressionOrdinal(_, aggs)))
         Aggregate(newGroups, aggs, child)
     }
+
+    private def containUnresolvedOrdinal(e: Expression): Boolean = e match {
+      case _: UnresolvedOrdinal => true
+      case Cube(_, groupByExprs) => groupByExprs.exists(containUnresolvedOrdinal)

Review comment:
       how about `g: GroupingSet => g.groupByExprs.exists(containUnresolvedOrdinal)`




-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/131171/
   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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #136762 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136762/testReport)** for PR 30145 at commit [`fb159c0`](https://github.com/apache/spark/commit/fb159c0b84e512c26c7cc7a92f821888baccb27e).
    * This patch **fails to build**.
    * 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 removed a comment on pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/core/src/test/resources/sql-tests/inputs/group-by-ordinal.sql
##########
@@ -54,6 +54,26 @@ select count(a), a from (select 1 as a) tmp group by 2 having a > 0;
 -- mixed cases: group-by ordinals and aliases
 select a, a AS k, count(b) from data group by k, 1;
 
+-- can use ordinal in CUBE
+select a, b, count(1) from data group by cube(1, 2);

Review comment:
       could you add tests for the cases: `with rollup`/`with cube`?




----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #136518 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136518/testReport)** for PR 30145 at commit [`6fa8115`](https://github.com/apache/spark/commit/6fa811589c5731aed22a45f1dd67ed8643a1d318).
    * This patch **fails PySpark unit tests**.
    * This patch **does not merge 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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #130305 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130305/testReport)** for PR 30145 at commit [`5cbe261`](https://github.com/apache/spark/commit/5cbe2610422eb022039705f4c01b3d4404812879).
    * 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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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] AngersZhuuuu commented on pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   retest this please


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/130229/
   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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #136766 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136766/testReport)** for PR 30145 at commit [`fa75aa1`](https://github.com/apache/spark/commit/fa75aa139e4703414620c1ce8386ab64fd53d67b).
    * 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] AmplabJenkins removed a comment on pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #136815 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136815/testReport)** for PR 30145 at commit [`ff7971b`](https://github.com/apache/spark/commit/ff7971b2817b46c45b0584dfdfdda999bfd2b96d).


-- 
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] AngersZhuuuu commented on a change in pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1787,16 +1787,36 @@ class Analyzer(override val catalogManager: CatalogManager)
       // Replace the index with the corresponding expression in aggregateExpressions. The index is
       // a 1-base position of aggregateExpressions, which is output columns (select expression)
       case Aggregate(groups, aggs, child) if aggs.forall(_.resolved) &&
-        groups.exists(_.isInstanceOf[UnresolvedOrdinal]) =>
-        val newGroups = groups.map {
-          case u @ UnresolvedOrdinal(index) if index > 0 && index <= aggs.size =>
-            aggs(index - 1)
-          case ordinal @ UnresolvedOrdinal(index) =>
-            throw QueryCompilationErrors.groupByPositionRangeError(index, aggs.size, ordinal)
-          case o => o
-        }
+        groups.exists(containUnresolvedOrdinal) =>
+        val newGroups = groups.map((resolveGroupByExpressionOrdinal(_, aggs)))
         Aggregate(newGroups, aggs, child)
     }
+
+    private def containUnresolvedOrdinal(e: Expression): Boolean = e match {
+      case _: UnresolvedOrdinal => true
+      case gs: GroupingSet => gs.children.exists(containUnresolvedOrdinal)
+      case _ => false
+    }
+
+    private def resolveGroupByExpressionOrdinal(
+        expr: Expression,
+        aggs: Seq[Expression]): Expression = expr match {
+      case ordinal @ UnresolvedOrdinal(index) =>
+        if (index > 0 && index <= aggs.size) {
+          aggs(index - 1)
+        } else {
+          throw QueryCompilationErrors.groupByPositionRangeError(index, aggs.size, ordinal)
+        }
+      case cube @ Cube(_, groupByExprs) =>
+        cube.withNewChildren(groupByExprs.map(resolveGroupByExpressionOrdinal(_, aggs)))
+      case rollup @ Rollup(_, groupByExprs) =>
+        rollup.withNewChildren(groupByExprs.map(resolveGroupByExpressionOrdinal(_, aggs)))
+      case groupingSets @ GroupingSets(_, flatGroupingSets, userGivenGroupByExprs) =>
+        groupingSets.withNewChildren(
+          flatGroupingSets.map(resolveGroupByExpressionOrdinal(_, aggs))
+            ++ userGivenGroupByExprs.map(resolveGroupByExpressionOrdinal(_, aggs)))

Review comment:
       > can we unify these cases now?
   > 
   > ```
   > case g: GroupingSet =>
   >   g.withNewChildren(g.children.map...)
   > ```
   
   Done. Not familiar about this api..==




-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #136996 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136996/testReport)** for PR 30145 at commit [`ff6794e`](https://github.com/apache/spark/commit/ff6794eb5387b6e83bfd3875884df02b75b0fafd).
    * This patch **fails to build**.
    * 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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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] AngersZhuuuu commented on a change in pull request #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1775,17 +1776,29 @@ class Analyzer(
       // Replace the index with the corresponding expression in aggregateExpressions. The index is
       // a 1-base position of aggregateExpressions, which is output columns (select expression)
       case Aggregate(groups, aggs, child) if aggs.forall(_.resolved) &&
-        groups.exists(_.isInstanceOf[UnresolvedOrdinal]) =>
-        val newGroups = groups.map {
-          case u @ UnresolvedOrdinal(index) if index > 0 && index <= aggs.size =>
-            aggs(index - 1)
-          case ordinal @ UnresolvedOrdinal(index) =>
-            ordinal.failAnalysis(
-              s"GROUP BY position $index is not in select list " +
-                s"(valid range is [1, ${aggs.size}])")
-          case o => o
-        }
+        groups.exists(_.find(e => e.isInstanceOf[UnresolvedOrdinal]).isDefined) =>

Review comment:
       > nit: `e => e.isInstanceOf` -> `_.isInstanceOf`
   
   Done




----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


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


----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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






----------------------------------------------------------------
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #136808 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136808/testReport)** for PR 30145 at commit [`c01ff72`](https://github.com/apache/spark/commit/c01ff729f4c53bce1fee7a662f9901ebb550ab22).


-- 
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 #30145: [SPARK-33233][SQL]CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

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


   **[Test build #137009 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137009/testReport)** for PR 30145 at commit [`391fda3`](https://github.com/apache/spark/commit/391fda3b311cd1cfa2cc9abf7b560d6f365568d1).


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