You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2018/05/25 17:20:30 UTC

calcite git commit: [CALCITE-2323] Apply "defaultNullCollation" configuration parameter when translating ORDER BY inside OVER (John Fang)

Repository: calcite
Updated Branches:
  refs/heads/master a0928af70 -> f17c376b6


[CALCITE-2323] Apply "defaultNullCollation" configuration parameter when translating ORDER BY inside OVER (John Fang)

Close apache/calcite#697


Project: http://git-wip-us.apache.org/repos/asf/calcite/repo
Commit: http://git-wip-us.apache.org/repos/asf/calcite/commit/f17c376b
Tree: http://git-wip-us.apache.org/repos/asf/calcite/tree/f17c376b
Diff: http://git-wip-us.apache.org/repos/asf/calcite/diff/f17c376b

Branch: refs/heads/master
Commit: f17c376b6f8441d4f029a450588179341d90208f
Parents: a0928af
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Thu May 24 00:04:13 2018 +0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Thu May 24 21:40:44 2018 -0700

----------------------------------------------------------------------
 .../calcite/sql2rel/SqlToRelConverter.java      | 55 ++++++++++++++++----
 .../calcite/test/SqlToRelConverterTest.java     | 26 +++++++++
 .../apache/calcite/test/SqlToRelTestBase.java   |  5 ++
 .../calcite/test/SqlToRelConverterTest.xml      | 16 ++++++
 4 files changed, 91 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/f17c376b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
index d07927e..9da95a5 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -1878,14 +1878,16 @@ public class SqlToRelConverter {
             "Relation should have sort key for implicit ORDER BY");
       }
     }
+
     final ImmutableList.Builder<RexFieldCollation> orderKeys =
         ImmutableList.builder();
-    final Set<SqlKind> flags = EnumSet.noneOf(SqlKind.class);
     for (SqlNode order : orderList) {
-      flags.clear();
-      RexNode e = bb.convertSortExpression(order, flags);
-      orderKeys.add(new RexFieldCollation(e, flags));
+      orderKeys.add(
+          bb.convertSortExpression(order,
+              RelFieldCollation.Direction.ASCENDING,
+              RelFieldCollation.NullDirection.UNSPECIFIED));
     }
+
     try {
       Preconditions.checkArgument(bb.window == null,
           "already in window agg mode");
@@ -4554,19 +4556,50 @@ public class SqlToRelConverter {
     }
 
     /**
-     * Converts an item in an ORDER BY clause, extracting DESC, NULLS LAST
-     * and NULLS FIRST flags first.
+     * Converts an item in an ORDER BY clause inside a window (OVER) clause,
+     * extracting DESC, NULLS LAST and NULLS FIRST flags first.
      */
-    public RexNode convertSortExpression(SqlNode expr, Set<SqlKind> flags) {
+    public RexFieldCollation convertSortExpression(SqlNode expr,
+        RelFieldCollation.Direction direction,
+        RelFieldCollation.NullDirection nullDirection) {
       switch (expr.getKind()) {
       case DESCENDING:
+        return convertSortExpression(((SqlCall) expr).operand(0),
+            RelFieldCollation.Direction.DESCENDING, nullDirection);
       case NULLS_LAST:
+        return convertSortExpression(((SqlCall) expr).operand(0),
+            direction, RelFieldCollation.NullDirection.LAST);
       case NULLS_FIRST:
-        flags.add(expr.getKind());
-        final SqlNode operand = ((SqlCall) expr).operand(0);
-        return convertSortExpression(operand, flags);
+        return convertSortExpression(((SqlCall) expr).operand(0),
+            direction, RelFieldCollation.NullDirection.FIRST);
       default:
-        return convertExpression(expr);
+        final Set<SqlKind> flags = EnumSet.noneOf(SqlKind.class);
+        switch (direction) {
+        case DESCENDING:
+          flags.add(SqlKind.DESCENDING);
+        }
+        switch (nullDirection) {
+        case UNSPECIFIED:
+          final RelFieldCollation.NullDirection nullDefaultDirection =
+              validator.getDefaultNullCollation().last(desc(direction))
+                  ? RelFieldCollation.NullDirection.LAST
+                  : RelFieldCollation.NullDirection.FIRST;
+          if (nullDefaultDirection != direction.defaultNullDirection()) {
+            SqlKind nullDirectionSqlKind =
+                validator.getDefaultNullCollation().last(desc(direction))
+                    ? SqlKind.NULLS_LAST
+                    : SqlKind.NULLS_FIRST;
+            flags.add(nullDirectionSqlKind);
+          }
+          break;
+        case FIRST:
+          flags.add(SqlKind.NULLS_FIRST);
+          break;
+        case LAST:
+          flags.add(SqlKind.NULLS_LAST);
+          break;
+        }
+        return new RexFieldCollation(convertExpression(expr), flags);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/f17c376b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
index 844313e..bc6d6bd 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
@@ -16,6 +16,10 @@
  */
 package org.apache.calcite.test;
 
+import org.apache.calcite.config.CalciteConnectionConfigImpl;
+import org.apache.calcite.config.CalciteConnectionProperty;
+import org.apache.calcite.config.NullCollation;
+import org.apache.calcite.plan.Contexts;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.prepare.Prepare;
 import org.apache.calcite.rel.RelNode;
@@ -45,6 +49,7 @@ import java.io.PrintWriter;
 import java.io.StringWriter;
 import java.util.ArrayDeque;
 import java.util.Deque;
+import java.util.Properties;
 import java.util.Set;
 
 import static org.hamcrest.CoreMatchers.notNullValue;
@@ -2813,6 +2818,27 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
     sql(sql).ok();
   }
 
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-2323">[CALCITE-2323]
+   * Validator should allow alternative nullCollations for ORDER BY in
+   * OVER</a>. */
+  @Test public void testUserDefinedOrderByOver() {
+    String sql = "select deptno,\n"
+        + "  rank() over(partition by empno order by deptno)\n"
+        + "from emp\n"
+        + "order by row_number() over(partition by empno order by deptno)";
+    Properties properties = new Properties();
+    properties.setProperty(
+        CalciteConnectionProperty.DEFAULT_NULL_COLLATION.camelName(),
+        NullCollation.LOW.name());
+    CalciteConnectionConfigImpl connectionConfig =
+        new CalciteConnectionConfigImpl(properties);
+    TesterImpl tester = new TesterImpl(getDiffRepos(), false, false, true, false,
+        null, null, SqlToRelConverter.Config.DEFAULT,
+        SqlConformanceEnum.DEFAULT, Contexts.of(connectionConfig));
+    sql(sql).with(tester).ok();
+  }
+
   /**
    * Visitor that checks that every {@link RelNode} in a tree is valid.
    *

http://git-wip-us.apache.org/repos/asf/calcite/blob/f17c376b/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java b/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
index 5d627b8..c1b42d0 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
@@ -16,6 +16,7 @@
  */
 package org.apache.calcite.test;
 
+import org.apache.calcite.config.CalciteConnectionConfig;
 import org.apache.calcite.linq4j.tree.Expression;
 import org.apache.calcite.plan.Context;
 import org.apache.calcite.plan.Contexts;
@@ -573,6 +574,10 @@ public abstract class SqlToRelTestBase {
       final SqlValidator validator =
           createValidator(
               catalogReader, typeFactory);
+      final CalciteConnectionConfig calciteConfig = context.unwrap(CalciteConnectionConfig.class);
+      if (calciteConfig != null) {
+        validator.setDefaultNullCollation(calciteConfig.defaultNullCollation());
+      }
       if (config == SqlToRelConverter.Config.DEFAULT) {
         localConfig = SqlToRelConverter.configBuilder()
             .withTrimUnusedFields(true).withExpand(enableExpand).build();

http://git-wip-us.apache.org/repos/asf/calcite/blob/f17c376b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
index c55ad57..160792a 100644
--- a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
@@ -5192,6 +5192,22 @@ LogicalProject(EMPNO=[$0])
 ]]>
         </Resource>
     </TestCase>
+    <TestCase name="testUserDefinedOrderByOver">
+        <Resource name="sql">
+            <![CDATA[select deptno,
+  rank() over(partition by empno order by deptno)
+from emp
+order by row_number() over(partition by empno order by deptno)]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(DEPTNO=[$0], EXPR$1=[$1])
+  LogicalSort(sort0=[$2], dir0=[ASC-nulls-first])
+    LogicalProject(DEPTNO=[$7], EXPR$1=[RANK() OVER (PARTITION BY $0 ORDER BY $7 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)], EXPR$2=[ROW_NUMBER() OVER (PARTITION BY $0 ORDER BY $7 NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
     <TestCase name="testWindowOnDynamicStar">
         <Resource name="sql">
             <![CDATA[SELECT SUM(n_nationkey) OVER w