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 2015/06/13 02:38:09 UTC

[5/7] incubator-calcite git commit: [CALCITE-754] Validator error when resolving OVER clause of JOIN query

[CALCITE-754] Validator error when resolving OVER clause of JOIN query


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

Branch: refs/heads/master
Commit: 87acda3179d6a5536f8af1506d5048ef06aeb591
Parents: ebce955
Author: Julian Hyde <jh...@apache.org>
Authored: Thu Jun 11 22:52:47 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Thu Jun 11 23:46:07 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/calcite/sql/SqlNode.java    |  5 +-
 .../java/org/apache/calcite/sql/SqlWindow.java  | 32 ++-------
 .../apache/calcite/sql/fun/SqlCastFunction.java | 72 ++++++--------------
 .../calcite/sql/validate/SelectNamespace.java   |  2 +-
 .../calcite/sql/validate/SqlValidatorImpl.java  | 18 +----
 .../calcite/sql/validate/SqlValidatorUtil.java  | 39 +++++++++++
 .../calcite/sql2rel/SqlToRelConverter.java      |  1 +
 .../apache/calcite/test/SqlValidatorTest.java   |  1 -
 8 files changed, 72 insertions(+), 98 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/87acda31/core/src/main/java/org/apache/calcite/sql/SqlNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlNode.java b/core/src/main/java/org/apache/calcite/sql/SqlNode.java
index 6708ba5..6531114 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlNode.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlNode.java
@@ -105,6 +105,7 @@ public abstract class SqlNode implements Cloneable {
     return getKind().belongsTo(category);
   }
 
+  @Deprecated // to be removed before 2.0
   public static SqlNode[] cloneArray(SqlNode[] nodes) {
     SqlNode[] clones = nodes.clone();
     for (int i = 0; i < clones.length; i++) {
@@ -279,11 +280,13 @@ public abstract class SqlNode implements Cloneable {
 
   /**
    * Returns whether expression is always ascending, descending or constant.
-   * This property is useful because it allows to safely aggregte infinite
+   * This property is useful because it allows to safely aggregate infinite
    * streams of values.
    *
    * <p>The default implementation returns
    * {@link SqlMonotonicity#NOT_MONOTONIC}.
+   *
+   * @param scope Scope
    */
   public SqlMonotonicity getMonotonicity(SqlValidatorScope scope) {
     return SqlMonotonicity.NOT_MONOTONIC;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/87acda31/core/src/main/java/org/apache/calcite/sql/SqlWindow.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlWindow.java b/core/src/main/java/org/apache/calcite/sql/SqlWindow.java
index afc6b45..a8ae456 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlWindow.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlWindow.java
@@ -24,16 +24,14 @@ import org.apache.calcite.sql.type.ReturnTypes;
 import org.apache.calcite.sql.type.SqlTypeFamily;
 import org.apache.calcite.sql.util.SqlBasicVisitor;
 import org.apache.calcite.sql.util.SqlVisitor;
-import org.apache.calcite.sql.validate.SqlMoniker;
-import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.util.ImmutableNullableList;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 
-import java.util.ArrayList;
 import java.util.List;
 
 import static org.apache.calcite.util.Static.RESOURCE;
@@ -329,28 +327,6 @@ public class SqlWindow extends SqlCall {
     }
   }
 
-  /**
-   * This method retrieves the list of columns for the current table then
-   * walks through the list looking for a column that is monotonic (sorted)
-   */
-  static boolean isTableSorted(SqlValidatorScope scope) {
-    List<SqlMoniker> columnNames = new ArrayList<SqlMoniker>();
-
-    // REVIEW: jhyde, 2007/11/7: This is the only use of
-    // findAllColumnNames. Find a better way to detect monotonicity, then
-    // remove that method.
-    scope.findAllColumnNames(columnNames);
-    for (SqlMoniker columnName : columnNames) {
-      SqlIdentifier columnId = columnName.toIdentifier();
-      final SqlMonotonicity monotonicity =
-          scope.getMonotonicity(columnId);
-      if (monotonicity != SqlMonotonicity.NOT_MONOTONIC) {
-        return true;
-      }
-    }
-    return false;
-  }
-
   public static SqlNode createCurrentRow(SqlParserPos pos) {
     return Bound.CURRENT_ROW.symbol(pos);
   }
@@ -564,7 +540,7 @@ public class SqlWindow extends SqlCall {
 
     // 6.10 rule 6a Function RANK & DENSE_RANK require ORDER BY clause
     if (orderList.size() == 0
-        && !SqlWindow.isTableSorted(scope)
+        && !SqlValidatorUtil.containsMonotonic(scope)
         && windowCall != null
         && windowCall.getOperator().requiresOrder()) {
       throw validator.newValidationError(this, RESOURCE.funcNeedsOrderBy());
@@ -596,7 +572,7 @@ public class SqlWindow extends SqlCall {
         // requires an ORDER BY clause if frame is logical(RANGE)
         // We relax this requirement if the table appears to be
         // sorted already
-        if (!isRows() && !SqlWindow.isTableSorted(scope)) {
+        if (!isRows() && !SqlValidatorUtil.containsMonotonic(scope)) {
           throw validator.newValidationError(this,
               RESOURCE.overMissingOrderBy());
         }
@@ -619,7 +595,7 @@ public class SqlWindow extends SqlCall {
       // Validate across boundaries. 7.10 Rule 8 a-d
       checkSpecialLiterals(this, validator);
     } else if (orderList.size() == 0
-        && !SqlWindow.isTableSorted(scope)
+        && !SqlValidatorUtil.containsMonotonic(scope)
         && windowCall != null
         && windowCall.getOperator().requiresOrder()) {
       throw validator.newValidationError(this, RESOURCE.overMissingOrderBy());

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/87acda31/core/src/main/java/org/apache/calcite/sql/fun/SqlCastFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlCastFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlCastFunction.java
index 7c7c7c4..08bd3b3 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlCastFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlCastFunction.java
@@ -39,7 +39,8 @@ import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
 
-import java.util.HashSet;
+import com.google.common.collect.ImmutableSet;
+
 import java.util.Set;
 
 import static org.apache.calcite.util.Static.RESOURCE;
@@ -74,56 +75,25 @@ public class SqlCastFunction extends SqlFunction {
    * List all casts that do not preserve monotonicity.
    */
   private Set<TypeFamilyCast> createNonMonotonicPreservingCasts() {
-    Set<TypeFamilyCast> result = new HashSet<TypeFamilyCast>();
-    result.add(
-        new TypeFamilyCast(
-            SqlTypeFamily.EXACT_NUMERIC,
-            SqlTypeFamily.CHARACTER));
-    result.add(
-        new TypeFamilyCast(
-            SqlTypeFamily.NUMERIC,
-            SqlTypeFamily.CHARACTER));
-    result.add(
-        new TypeFamilyCast(
-            SqlTypeFamily.APPROXIMATE_NUMERIC,
-            SqlTypeFamily.CHARACTER));
-    result.add(
-        new TypeFamilyCast(
-            SqlTypeFamily.DATETIME_INTERVAL,
-            SqlTypeFamily.CHARACTER));
-    result.add(
-        new TypeFamilyCast(
-            SqlTypeFamily.CHARACTER,
-            SqlTypeFamily.EXACT_NUMERIC));
-    result.add(
-        new TypeFamilyCast(
-            SqlTypeFamily.CHARACTER,
-            SqlTypeFamily.NUMERIC));
-    result.add(
-        new TypeFamilyCast(
-            SqlTypeFamily.CHARACTER,
-            SqlTypeFamily.APPROXIMATE_NUMERIC));
-    result.add(
-        new TypeFamilyCast(
-            SqlTypeFamily.CHARACTER,
-            SqlTypeFamily.DATETIME_INTERVAL));
-    result.add(
-        new TypeFamilyCast(
-            SqlTypeFamily.DATETIME,
-            SqlTypeFamily.TIME));
-    result.add(
-        new TypeFamilyCast(
-            SqlTypeFamily.TIMESTAMP,
-            SqlTypeFamily.TIME));
-    result.add(
-        new TypeFamilyCast(
-            SqlTypeFamily.TIME,
-            SqlTypeFamily.DATETIME));
-    result.add(
-        new TypeFamilyCast(
-            SqlTypeFamily.TIME,
-            SqlTypeFamily.TIMESTAMP));
-    return result;
+    ImmutableSet.Builder<TypeFamilyCast> builder = ImmutableSet.builder();
+    add(builder, SqlTypeFamily.EXACT_NUMERIC, SqlTypeFamily.CHARACTER);
+    add(builder, SqlTypeFamily.NUMERIC, SqlTypeFamily.CHARACTER);
+    add(builder, SqlTypeFamily.APPROXIMATE_NUMERIC, SqlTypeFamily.CHARACTER);
+    add(builder, SqlTypeFamily.DATETIME_INTERVAL, SqlTypeFamily.CHARACTER);
+    add(builder, SqlTypeFamily.CHARACTER, SqlTypeFamily.EXACT_NUMERIC);
+    add(builder, SqlTypeFamily.CHARACTER, SqlTypeFamily.NUMERIC);
+    add(builder, SqlTypeFamily.CHARACTER, SqlTypeFamily.APPROXIMATE_NUMERIC);
+    add(builder, SqlTypeFamily.CHARACTER, SqlTypeFamily.DATETIME_INTERVAL);
+    add(builder, SqlTypeFamily.DATETIME, SqlTypeFamily.TIME);
+    add(builder, SqlTypeFamily.TIMESTAMP, SqlTypeFamily.TIME);
+    add(builder, SqlTypeFamily.TIME, SqlTypeFamily.DATETIME);
+    add(builder, SqlTypeFamily.TIME, SqlTypeFamily.TIMESTAMP);
+    return builder.build();
+  }
+
+  private void add(ImmutableSet.Builder<TypeFamilyCast> result,
+      SqlTypeFamily from, SqlTypeFamily to) {
+    result.add(new TypeFamilyCast(from, to));
   }
 
   private boolean isMonotonicPreservingCast(

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/87acda31/core/src/main/java/org/apache/calcite/sql/validate/SelectNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SelectNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/SelectNamespace.java
index a0e0ad2..48a45c1 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SelectNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SelectNamespace.java
@@ -52,7 +52,7 @@ public class SelectNamespace extends AbstractNamespace {
   //~ Methods ----------------------------------------------------------------
 
   // implement SqlValidatorNamespace, overriding return type
-  public SqlSelect getNode() {
+  @Override public SqlSelect getNode() {
     return select;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/87acda31/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
index 4a47c78..f210749 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
@@ -3033,7 +3033,8 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       switch (modality) {
       case STREAM:
         SqlNodeList groupList = select.getGroup();
-        if (groupList == null || !containsMonotonic(scope, groupList)) {
+        if (groupList == null
+            || !SqlValidatorUtil.containsMonotonic(scope, groupList)) {
           if (fail) {
             throw newValidationError(aggregateNode,
                 Static.RESOURCE.streamMustGroupByMonotonic());
@@ -3087,21 +3088,6 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     }
   }
 
-  private static boolean containsMonotonic(SelectScope scope,
-      SqlNodeList nodes) {
-    for (SqlNode node : nodes) {
-      final SqlMonotonicity monotonicity = scope.getMonotonicity(node);
-      switch (monotonicity) {
-      case CONSTANT:
-      case NOT_MONOTONIC:
-        break;
-      default:
-        return true;
-      }
-    }
-    return false;
-  }
-
   protected void validateWindowClause(SqlSelect select) {
     final SqlNodeList windowList = select.getWindowList();
     if ((windowList == null) || (windowList.size() == 0)) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/87acda31/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
index 768852e..3ddeed4 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
@@ -571,6 +571,45 @@ public class SqlValidatorUtil {
     return ImmutableList.copyOf(flattenedBitSets);
   }
 
+  /**
+   * Returns whether there are any input columns that are sorted.
+   *
+   * <p>If so, it can be the default ORDER BY clause for a WINDOW specification.
+   * (This is an extension to the SQL standard for streaming.)
+   */
+  public static boolean containsMonotonic(SqlValidatorScope scope) {
+    for (SqlValidatorNamespace ns : children(scope)) {
+      ns = ns.resolve();
+      for (String field : ns.getRowType().getFieldNames()) {
+        if (!ns.getMonotonicity(field).mayRepeat()) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
+  private static List<SqlValidatorNamespace> children(SqlValidatorScope scope) {
+    return scope instanceof ListScope
+        ? ((ListScope) scope).getChildren()
+        : ImmutableList.<SqlValidatorNamespace>of();
+  }
+
+  /**
+   * Returns whether any of the given expressions are sorted.
+   *
+   * <p>If so, it can be the default ORDER BY clause for a WINDOW specification.
+   * (This is an extension to the SQL standard for streaming.)
+   */
+  static boolean containsMonotonic(SelectScope scope, SqlNodeList nodes) {
+    for (SqlNode node : nodes) {
+      if (!scope.getMonotonicity(node).mayRepeat()) {
+        return true;
+      }
+    }
+    return false;
+  }
+
   //~ Inner Classes ----------------------------------------------------------
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/87acda31/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 a190159..ec36ad8 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -3483,6 +3483,7 @@ public class SqlToRelConverter {
     for (int i = 0; i < joinList.size(); i++) {
       Object o = joinList.get(i);
       if (o instanceof List) {
+        @SuppressWarnings("unchecked")
         List<SqlNode> projectList = (List<SqlNode>) o;
         final List<RexNode> selectList = new ArrayList<>();
         final List<String> fieldNameList = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/87acda31/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
index 4b7c1f8..af82649 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -4262,7 +4262,6 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
   /** Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-754">[CALCITE-754]
    * Validator error when resolving OVER clause of JOIN query</a>. */
-  @Ignore
   @Test public void testPartitionByColumnInJoinAlias() {
     sql("select sum(1) over(partition by t1.ename) \n"
             + "from emp t1, emp t2")