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 2022/12/21 00:36:19 UTC

[GitHub] [spark] rxin opened a new pull request, #39144: [SPARK-41637][SQL] ORDER BY ALL

rxin opened a new pull request, #39144:
URL: https://github.com/apache/spark/pull/39144

   ### What changes were proposed in this pull request?
   This patch adds ORDER BY ALL support to SQL. ORDER BY ALL is a syntactic sugar to sort the output by all the fields, from left to right. It also allows specifying asc/desc as well as null ordering.
   
   ### Why are the changes needed?
   It's a good convenience sugar added initially by DuckDB to avoid repeating the fields.
   
   ### Does this PR introduce _any_ user-facing change?
   Yes. See above.
   
   ### How was this patch tested?
   Added SQL tests.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@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 diff in pull request #39144: [SPARK-41637][SQL] ORDER BY ALL

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #39144:
URL: https://github.com/apache/spark/pull/39144#discussion_r1056029662


##########
sql/core/src/test/resources/sql-tests/inputs/order-by-all.sql:
##########
@@ -0,0 +1,44 @@
+create temporary view data as select * from values

Review Comment:
   isn't it to better to test different features in test golden files?



##########
sql/core/src/test/resources/sql-tests/inputs/order-by-all.sql:
##########
@@ -0,0 +1,44 @@
+create temporary view data as select * from values

Review Comment:
   isn't it better to test different features in test golden files?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@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 #39144: [SPARK-41637][SQL] ORDER BY ALL

Posted by GitBox <gi...@apache.org>.
cloud-fan closed pull request #39144: [SPARK-41637][SQL] ORDER BY ALL
URL: https://github.com/apache/spark/pull/39144


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] rxin commented on a diff in pull request #39144: [SPARK-41637][SQL] ORDER BY ALL

Posted by GitBox <gi...@apache.org>.
rxin commented on code in PR #39144:
URL: https://github.com/apache/spark/pull/39144#discussion_r1053874730


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveOrderByAll.scala:
##########
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import org.apache.spark.sql.catalyst.expressions.SortOrder
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Sort}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SORT, UNRESOLVED_ATTRIBUTE}
+
+/**
+ * Resolve "order by all" in the following SQL pattern:
+ *  `select col1, col2 from table order by all`.
+ *
+ * It orders the query result by all columns, from left to right. The query above becomes:
+ *
+ *  `select col1, col2 from table order by col1, col2`
+ *
+ * This should also support specifying asc/desc, and nulls first/last.
+ */
+object ResolveOrderByAll extends Rule[LogicalPlan] {
+
+  val ALL = "ALL"
+
+  /**
+   * An extractor to pull out the SortOrder field in the ORDER BY ALL clause. We pull out that
+   * SortOrder object so we can pass its direction and null ordering.
+   */
+  object OrderByAll {
+    def unapply(s: Sort): Option[SortOrder] = {
+      // This only applies to global ordering.
+      if (!s.global) {
+        return None
+      }
+      // Don't do this if we have more than one order field. That means it's not order by all.
+      if (s.order.size != 1) {
+        return None
+      }
+      // Don't do this if there's a child field called ALL. That should take precedence.
+      if (s.child.output.exists(_.name.toUpperCase() == ALL)) {
+        return None
+      }
+
+      s.order.find { so =>
+        so.child match {
+          case a: UnresolvedAttribute => a.name.toUpperCase() == ALL
+          case _ => false
+        }
+      }
+    }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUpWithPruning(
+    _.containsAllPatterns(UNRESOLVED_ATTRIBUTE, SORT), ruleId) {
+    // This only makes sense if the children is resolved.

Review Comment:
   nit: child, not children



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] gengliangwang commented on a diff in pull request #39144: [SPARK-41637][SQL] ORDER BY ALL

Posted by GitBox <gi...@apache.org>.
gengliangwang commented on code in PR #39144:
URL: https://github.com/apache/spark/pull/39144#discussion_r1055890337


##########
sql/core/src/test/resources/sql-tests/inputs/order-by-all.sql:
##########
@@ -0,0 +1,44 @@
+create temporary view data as select * from values

Review Comment:
   nit: we can have a test case combining `group by all` and `order by all`. It should work, just for covering a testing scenario.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@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 diff in pull request #39144: [SPARK-41637][SQL] ORDER BY ALL

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #39144:
URL: https://github.com/apache/spark/pull/39144#discussion_r1056029662


##########
sql/core/src/test/resources/sql-tests/inputs/order-by-all.sql:
##########
@@ -0,0 +1,44 @@
+create temporary view data as select * from values

Review Comment:
   isn't it better to test different features in different golden files?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@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 diff in pull request #39144: [SPARK-41637][SQL] ORDER BY ALL

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #39144:
URL: https://github.com/apache/spark/pull/39144#discussion_r1056037162


##########
sql/core/src/test/resources/sql-tests/inputs/order-by-all.sql:
##########
@@ -0,0 +1,44 @@
+create temporary view data as select * from values

Review Comment:
   Oh you mean test both feature in one query? That's a good idea, we can add later.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] rxin commented on pull request #39144: [SPARK-41637][SQL] ORDER BY ALL

Posted by GitBox <gi...@apache.org>.
rxin commented on PR #39144:
URL: https://github.com/apache/spark/pull/39144#issuecomment-1363223263

   cc @cloud-fan this one 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@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 #39144: [SPARK-41637][SQL] ORDER BY ALL

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

   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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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