You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by fh...@apache.org on 2018/08/21 22:24:45 UTC

[flink] 03/03: [FLINK-10172] [table] Fix Table.orderBy(String) by adding asc & desc to ExpressionParser.

This is an automated email from the ASF dual-hosted git repository.

fhueske pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 12bae25d56ca16380a724dcd15028da49c2d653e
Author: Rong Rong <wa...@hotmail.com>
AuthorDate: Mon Aug 20 09:13:56 2018 -0700

    [FLINK-10172] [table] Fix Table.orderBy(String) by adding asc & desc to ExpressionParser.
    
    * Add tests to ensure asc and desc won't be dropped in the future.
    
    This closes #6585.
---
 .../flink/table/api/scala/expressionDsl.scala      | 11 ++++
 .../flink/table/expressions/ExpressionParser.scala | 10 ++++
 .../stringexpr/SortStringExpressionTest.scala      | 60 ++++++++++++++++++++++
 3 files changed, 81 insertions(+)

diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/expressionDsl.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/expressionDsl.scala
index 8b08af6..429c37f 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/expressionDsl.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/expressionDsl.scala
@@ -236,7 +236,18 @@ trait ImplicitExpressionOperations {
     */
   def as(name: Symbol, extraNames: Symbol*) = Alias(expr, name.name, extraNames.map(_.name))
 
+  /**
+    * Specifies ascending order of an expression i.e. a field for orderBy call.
+    *
+    * @return ascend expression
+    */
   def asc = Asc(expr)
+
+  /**
+    * Specifies descending order of an expression i.e. a field for orderBy call.
+    *
+    * @return descend expression
+    */
   def desc = Desc(expr)
 
   /**
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionParser.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionParser.scala
index 4b2440c..c0a577d 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionParser.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionParser.scala
@@ -48,6 +48,8 @@ object ExpressionParser extends JavaTokenParsers with PackratParsers {
   // Keyword
   lazy val AS: Keyword = Keyword("as")
   lazy val CAST: Keyword = Keyword("cast")
+  lazy val ASC: Keyword = Keyword("asc")
+  lazy val DESC: Keyword = Keyword("desc")
   lazy val NULL: Keyword = Keyword("Null")
   lazy val IF: Keyword = Keyword("?")
   lazy val TO_DATE: Keyword = Keyword("toDate")
@@ -216,6 +218,12 @@ object ExpressionParser extends JavaTokenParsers with PackratParsers {
 
   // suffix operators
 
+  lazy val suffixAsc : PackratParser[Expression] =
+    composite <~ "." ~ ASC ~ opt("()") ^^ { e => Asc(e) }
+
+  lazy val suffixDesc : PackratParser[Expression] =
+    composite <~ "." ~ DESC ~ opt("()") ^^ { e => Desc(e) }
+
   lazy val suffixCast: PackratParser[Expression] =
     composite ~ "." ~ CAST ~ "(" ~ dataType ~ ")" ^^ {
     case e ~ _ ~ _ ~ _ ~ dt ~ _ => Cast(e, dt)
@@ -324,6 +332,8 @@ object ExpressionParser extends JavaTokenParsers with PackratParsers {
     suffixToDate |
     // expression for log
     suffixLog |
+    // expression for ordering
+    suffixAsc | suffixDesc |
     // expressions that take enumerations
     suffixCast | suffixTrim | suffixTrimWithoutArgs | suffixExtract | suffixFloor | suffixCeil |
     // expressions that take literals
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/stringexpr/SortStringExpressionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/stringexpr/SortStringExpressionTest.scala
new file mode 100644
index 0000000..204ec77
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/stringexpr/SortStringExpressionTest.scala
@@ -0,0 +1,60 @@
+/*
+ * 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.flink.table.api.batch.table.stringexpr
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.utils.TableTestBase
+import org.junit.Test
+
+class SortStringExpressionTest extends TableTestBase {
+
+  @Test
+  def testOrdering(): Unit = {
+    val util = batchTestUtil()
+    val t = util.addTable[(Int, Long, String)]("Table3")
+
+    val t1 = t.select('_1 as 'a, '_2 as 'b, '_3 as 'c).orderBy('a)
+    val t2 = t.select("_1 as a, _2 as b, _3 as c").orderBy("a")
+
+    verifyTableEquals(t1, t2)
+  }
+
+  @Test
+  def testExplicitAscendOrdering(): Unit = {
+    val util = batchTestUtil()
+    val t = util.addTable[(Int, Long, String)]("Table3")
+
+    val t1 = t.select('_1, '_2).orderBy('_1.asc)
+    val t2 = t.select("_1, _2").orderBy("_1.asc")
+
+    verifyTableEquals(t1, t2)
+  }
+
+  @Test
+  def testExplicitDescendOrdering(): Unit = {
+    val util = batchTestUtil()
+    val t = util.addTable[(Int, Long, String)]("Table3")
+
+    val t1 = t.select('_1, '_2).orderBy('_1.desc)
+    val t2 = t.select("_1, _2").orderBy("_1.desc")
+
+    verifyTableEquals(t1, t2)
+  }
+}