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/08/01 00:42:53 UTC

[16/50] [abbrv] incubator-calcite git commit: [CALCITE-702] Add validator test for monotonic expressions

[CALCITE-702] Add validator test for monotonic expressions

Recognize that "EXTRACT(YEAR FROM <montonic_expr>)" is monotonic.


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

Branch: refs/heads/branch-release
Commit: 91f0fca610c0ec8964e95a87547e3a941b834aab
Parents: e218cb1
Author: Julian Hyde <jh...@apache.org>
Authored: Mon Apr 27 13:02:53 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Mon Apr 27 13:02:53 2015 -0700

----------------------------------------------------------------------
 .../calcite/sql/fun/SqlExtractFunction.java     |  14 ++
 .../org/apache/calcite/sql/test/SqlTester.java  |  11 +-
 .../apache/calcite/sql/test/SqlTesterImpl.java  |  22 ++-
 .../apache/calcite/test/SqlValidatorTest.java   |  86 ++++++++++
 .../calcite/test/SqlValidatorTestCase.java      | 169 +++++++++----------
 5 files changed, 210 insertions(+), 92 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/91f0fca6/core/src/main/java/org/apache/calcite/sql/fun/SqlExtractFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlExtractFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlExtractFunction.java
index 1e870c3..86c6893 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlExtractFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlExtractFunction.java
@@ -19,10 +19,13 @@ package org.apache.calcite.sql.fun;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlFunction;
 import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlIntervalQualifier;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlWriter;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.validate.SqlMonotonicity;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
 import org.apache.calcite.util.Util;
 
 /**
@@ -64,6 +67,17 @@ public class SqlExtractFunction extends SqlFunction {
     call.operand(1).unparse(writer, 0, 0);
     writer.endFunCall(frame);
   }
+
+  @Override public SqlMonotonicity getMonotonicity(SqlCall call,
+      SqlValidatorScope scope) {
+    final SqlIntervalQualifier o = call.operand(0);
+    switch (o.timeUnitRange) {
+    case YEAR:
+      return scope.getMonotonicity(call.operand(1)).unstrict();
+    default:
+      return SqlMonotonicity.NOT_MONOTONIC;
+    }
+  }
 }
 
 // End SqlExtractFunction.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/91f0fca6/core/src/test/java/org/apache/calcite/sql/test/SqlTester.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlTester.java b/core/src/test/java/org/apache/calcite/sql/test/SqlTester.java
index 3a919fc..5c14704 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlTester.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlTester.java
@@ -22,6 +22,7 @@ import org.apache.calcite.config.Lex;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.test.SqlValidatorTestCase;
 
 import java.io.Closeable;
@@ -46,7 +47,7 @@ public interface SqlTester extends Closeable, SqlValidatorTestCase.Tester {
   /**
    * Name of a virtual machine that can potentially implement an operator.
    */
-  public enum VmName {
+  enum VmName {
     FENNEL, JAVA, EXPAND
   }
 
@@ -269,6 +270,14 @@ public interface SqlTester extends Closeable, SqlValidatorTestCase.Tester {
       ResultChecker resultChecker);
 
   /**
+   * Tests that the first column of a SQL query has a given monotonicity.
+   *
+   * @param expectedMonotonicity Expected monotonicity
+   * @param query                SQL query
+   */
+  void checkMonotonic(String query, SqlMonotonicity expectedMonotonicity);
+
+  /**
    * Declares that this test is for a given operator. So we can check that all
    * operators are tested.
    *

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/91f0fca6/core/src/test/java/org/apache/calcite/sql/test/SqlTesterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlTesterImpl.java b/core/src/test/java/org/apache/calcite/sql/test/SqlTesterImpl.java
index 36c1228..98741f4 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlTesterImpl.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlTesterImpl.java
@@ -40,6 +40,7 @@ import org.apache.calcite.sql.util.SqlShuttle;
 import org.apache.calcite.sql.validate.SqlConformance;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorNamespace;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
 import org.apache.calcite.test.SqlValidatorTestCase;
 import org.apache.calcite.util.Pair;
@@ -60,8 +61,10 @@ import java.util.NoSuchElementException;
 
 import static org.apache.calcite.sql.SqlUtil.stripAs;
 
+import static org.hamcrest.CoreMatchers.equalTo;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThat;
 import static org.junit.Assert.fail;
 
 /**
@@ -447,6 +450,18 @@ public class SqlTesterImpl implements SqlTester {
     typeChecker.checkType(actualType);
   }
 
+  public void checkMonotonic(String query,
+      SqlMonotonicity expectedMonotonicity) {
+    SqlValidator validator = getValidator();
+    SqlNode n = parseAndValidate(validator, query);
+    final RelDataType rowType = validator.getValidatedNodeType(n);
+    final SqlValidatorNamespace selectNamespace = validator.getNamespace(n);
+    final String field0 = rowType.getFieldList().get(0).getName();
+    final SqlMonotonicity monotonicity =
+        selectNamespace.getMonotonicity(field0);
+    assertThat(monotonicity, equalTo(expectedMonotonicity));
+  }
+
   public void checkRewrite(
       SqlValidator validator,
       String query,
@@ -529,7 +544,7 @@ public class SqlTesterImpl implements SqlTester {
     } catch (SqlParseException e) {
       throw new RuntimeException(e);
     }
-    final Collection<SqlNode> literalSet = new LinkedHashSet<SqlNode>();
+    final Collection<SqlNode> literalSet = new LinkedHashSet<>();
     x.accept(
         new SqlShuttle() {
           private final List<SqlOperator> ops =
@@ -569,7 +584,7 @@ public class SqlTesterImpl implements SqlTester {
                 == SqlTypeName.NULL;
           }
         });
-    final List<SqlNode> nodes = new ArrayList<SqlNode>(literalSet);
+    final List<SqlNode> nodes = new ArrayList<>(literalSet);
     Collections.sort(
         nodes,
         new Comparator<SqlNode>() {
@@ -586,8 +601,7 @@ public class SqlTesterImpl implements SqlTester {
           }
         });
     String sql2 = sql;
-    final List<Pair<String, String>> values =
-        new ArrayList<Pair<String, String>>();
+    final List<Pair<String, String>> values = new ArrayList<>();
     int p = 0;
     for (SqlNode literal : nodes) {
       final SqlParserPos pos = literal.getParserPosition();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/91f0fca6/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 a24e16a..aa097d9 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -27,6 +27,7 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.test.SqlTester;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.util.Bug;
@@ -7118,6 +7119,91 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         .fails(STR_AGG_REQUIRES_MONO);
   }
 
+  /** Tests that various expressions are monotonic. */
+  @Test public void testMonotonic() {
+    sql("select stream floor(rowtime to hour) from orders")
+        .monotonic(SqlMonotonicity.INCREASING);
+    sql("select stream ceil(rowtime to minute) from orders")
+        .monotonic(SqlMonotonicity.INCREASING);
+    sql("select stream extract(minute from rowtime) from orders")
+        .monotonic(SqlMonotonicity.NOT_MONOTONIC);
+    sql("select stream (rowtime - timestamp '1970-01-01 00:00:00') hour from orders")
+        .monotonic(SqlMonotonicity.INCREASING);
+    sql("select stream\n"
+        + "cast((rowtime - timestamp '1970-01-01 00:00:00') hour as integer)\n"
+        + "from orders")
+        .monotonic(SqlMonotonicity.INCREASING);
+    sql("select stream\n"
+        + "cast((rowtime - timestamp '1970-01-01 00:00:00') hour as integer) / 15\n"
+        + "from orders")
+        .monotonic(SqlMonotonicity.INCREASING);
+    sql("select stream\n"
+        + "mod(cast((rowtime - timestamp '1970-01-01 00:00:00') hour as integer), 15)\n"
+        + "from orders")
+        .monotonic(SqlMonotonicity.NOT_MONOTONIC);
+
+    // constant
+    sql("select stream 1 - 2 from orders")
+        .monotonic(SqlMonotonicity.CONSTANT);
+    sql("select stream 1 + 2 from orders")
+        .monotonic(SqlMonotonicity.CONSTANT);
+
+    // extract(YEAR) is monotonic, extract(other time unit) is not
+    sql("select stream extract(year from rowtime) from orders")
+        .monotonic(SqlMonotonicity.INCREASING);
+    sql("select stream extract(month from rowtime) from orders")
+        .monotonic(SqlMonotonicity.NOT_MONOTONIC);
+
+    // <monotonic> - constant
+    sql("select stream extract(year from rowtime) - 3 from orders")
+        .monotonic(SqlMonotonicity.INCREASING);
+    sql("select stream extract(year from rowtime) * 5 from orders")
+        .monotonic(SqlMonotonicity.INCREASING);
+    sql("select stream extract(year from rowtime) * -5 from orders")
+        .monotonic(SqlMonotonicity.DECREASING);
+
+    // <monotonic> / constant
+    sql("select stream extract(year from rowtime) / -5 from orders")
+        .monotonic(SqlMonotonicity.DECREASING);
+    sql("select stream extract(year from rowtime) / 5 from orders")
+        .monotonic(SqlMonotonicity.INCREASING);
+    sql("select stream extract(year from rowtime) / 0 from orders")
+        .monotonic(SqlMonotonicity.CONSTANT); // +inf is constant!
+
+    // constant / <monotonic> is not monotonic (we don't know whether sign of
+    // expression ever changes)
+    sql("select stream 5 / extract(year from rowtime) from orders")
+        .monotonic(SqlMonotonicity.NOT_MONOTONIC);
+
+    // <monotonic> * constant
+    sql("select stream extract(year from rowtime) * -5 from orders")
+        .monotonic(SqlMonotonicity.DECREASING);
+    sql("select stream extract(year from rowtime) * 5 from orders")
+        .monotonic(SqlMonotonicity.INCREASING);
+    sql("select stream extract(year from rowtime) * 0 from orders")
+        .monotonic(SqlMonotonicity.CONSTANT); // 0 is constant!
+
+    // constant * <monotonic>
+    sql("select stream -5 * extract(year from rowtime) from orders")
+        .monotonic(SqlMonotonicity.DECREASING);
+    sql("select stream 5 * extract(year from rowtime) from orders")
+        .monotonic(SqlMonotonicity.INCREASING);
+    sql("select stream 0 * extract(year from rowtime) from orders")
+        .monotonic(SqlMonotonicity.CONSTANT);
+
+    // <monotonic> - <monotonic>
+    sql("select stream\n"
+        + "extract(year from rowtime) - extract(year from rowtime)\n"
+        + "from orders")
+        .monotonic(SqlMonotonicity.NOT_MONOTONIC);
+
+    // <monotonic> + <monotonic>
+    sql("select stream\n"
+        + "extract(year from rowtime) + extract(year from rowtime)\n"
+        + "from orders")
+        .monotonic(SqlMonotonicity.INCREASING);
+  }
+
   @Test public void testStreamUnionAll() {
     sql("select orderId\n"
         + "from ^orders^\n"

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/91f0fca6/core/src/test/java/org/apache/calcite/test/SqlValidatorTestCase.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlValidatorTestCase.java b/core/src/test/java/org/apache/calcite/test/SqlValidatorTestCase.java
index 039cfe8..ff87245 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTestCase.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTestCase.java
@@ -314,97 +314,87 @@ public class SqlValidatorTestCase {
     }
 
     if (null == expectedMsgPattern) {
-      if (null != actualException) {
-        actualException.printStackTrace();
-        fail(
-            "Validator threw unexpected exception"
-            + "; query [" + sap.sql
-            + "]; exception [" + actualMessage
-            + "]; class [" + actualException.getClass()
-            + "]; pos [line " + actualLine
-            + " col " + actualColumn
-            + " thru line " + actualLine
-            + " col " + actualColumn + "]");
+      actualException.printStackTrace();
+      fail("Validator threw unexpected exception"
+          + "; query [" + sap.sql
+          + "]; exception [" + actualMessage
+          + "]; class [" + actualException.getClass()
+          + "]; pos [line " + actualLine
+          + " col " + actualColumn
+          + " thru line " + actualLine
+          + " col " + actualColumn + "]");
+    }
+
+    String sqlWithCarets;
+    if (actualColumn <= 0
+        || actualLine <= 0
+        || actualEndColumn <= 0
+        || actualEndLine <= 0) {
+      if (sap.pos != null) {
+        AssertionError e =
+            new AssertionError("Expected error to have position,"
+                + " but actual error did not: "
+                + " actual pos [line " + actualLine
+                + " col " + actualColumn
+                + " thru line " + actualEndLine + " col "
+                + actualEndColumn + "]");
+        e.initCause(actualException);
+        throw e;
       }
+      sqlWithCarets = sap.sql;
     } else {
-      if (null == actualException) {
-        fail(
-            "Expected validator to throw "
-            + "exception, but it did not; query [" + sap.sql
-            + "]; expected [" + expectedMsgPattern + "]");
-      } else {
-        String sqlWithCarets;
-        if ((actualColumn <= 0)
-            || (actualLine <= 0)
-            || (actualEndColumn <= 0)
-            || (actualEndLine <= 0)) {
-          if (sap.pos != null) {
-            AssertionError e =
-                new AssertionError("Expected error to have position,"
-                    + " but actual error did not: "
-                    + " actual pos [line " + actualLine
-                    + " col " + actualColumn
-                    + " thru line " + actualEndLine + " col "
-                    + actualEndColumn + "]");
-            e.initCause(actualException);
-            throw e;
-          }
-          sqlWithCarets = sap.sql;
-        } else {
-          sqlWithCarets =
-              SqlParserUtil.addCarets(
-                  sap.sql,
-                  actualLine,
-                  actualColumn,
-                  actualEndLine,
-                  actualEndColumn + 1);
-          if (sap.pos == null) {
-            throw new AssertionError(
-                "Actual error had a position, but expected error"
-                + " did not. Add error position carets to sql:\n"
-                + sqlWithCarets);
-          }
-        }
-        if (actualMessage != null) {
-          actualMessage = Util.toLinux(actualMessage);
-        }
-        if ((actualMessage == null)
-            || !actualMessage.matches(expectedMsgPattern)) {
-          actualException.printStackTrace();
-          final String actualJavaRegexp =
-              (actualMessage == null) ? "null"
-                  : TestUtil.quoteForJava(
-                      TestUtil.quotePattern(actualMessage));
-          fail("Validator threw different "
-              + "exception than expected; query [" + sap.sql
-              + "];\n"
-              + " expected pattern [" + expectedMsgPattern
-              + "];\n"
-              + " actual [" + actualMessage
-              + "];\n"
-              + " actual as java regexp [" + actualJavaRegexp
-              + "]; pos [" + actualLine
-              + " col " + actualColumn
-              + " thru line " + actualEndLine
-              + " col " + actualEndColumn
-              + "]; sql [" + sqlWithCarets + "]");
-        } else if (
-            (sap.pos != null)
-                && ((actualLine != sap.pos.getLineNum())
-                || (actualColumn != sap.pos.getColumnNum())
-                || (actualEndLine != sap.pos.getEndLineNum())
-                || (actualEndColumn != sap.pos.getEndColumnNum()))) {
-          fail(
-              "Validator threw expected "
-              + "exception [" + actualMessage
-              + "];\nbut at pos [line " + actualLine
-              + " col " + actualColumn
-              + " thru line " + actualEndLine
-              + " col " + actualEndColumn
-              + "];\nsql [" + sqlWithCarets + "]");
-        }
+      sqlWithCarets =
+          SqlParserUtil.addCarets(
+              sap.sql,
+              actualLine,
+              actualColumn,
+              actualEndLine,
+              actualEndColumn + 1);
+      if (sap.pos == null) {
+        throw new AssertionError("Actual error had a position, but expected "
+            + "error did not. Add error position carets to sql:\n"
+            + sqlWithCarets);
       }
     }
+
+    if (actualMessage != null) {
+      actualMessage = Util.toLinux(actualMessage);
+    }
+
+    if (actualMessage == null
+        || !actualMessage.matches(expectedMsgPattern)) {
+      actualException.printStackTrace();
+      final String actualJavaRegexp =
+          (actualMessage == null)
+              ? "null"
+              : TestUtil.quoteForJava(
+                  TestUtil.quotePattern(actualMessage));
+      fail("Validator threw different "
+          + "exception than expected; query [" + sap.sql
+          + "];\n"
+          + " expected pattern [" + expectedMsgPattern
+          + "];\n"
+          + " actual [" + actualMessage
+          + "];\n"
+          + " actual as java regexp [" + actualJavaRegexp
+          + "]; pos [" + actualLine
+          + " col " + actualColumn
+          + " thru line " + actualEndLine
+          + " col " + actualEndColumn
+          + "]; sql [" + sqlWithCarets + "]");
+    } else if (sap.pos != null
+        && (actualLine != sap.pos.getLineNum()
+            || actualColumn != sap.pos.getColumnNum()
+            || actualEndLine != sap.pos.getEndLineNum()
+            || actualEndColumn != sap.pos.getEndColumnNum())) {
+      fail("Validator threw expected "
+          + "exception [" + actualMessage
+          + "];\nbut at pos [line " + actualLine
+          + " col " + actualColumn
+          + " thru line " + actualEndLine
+          + " col " + actualEndColumn
+          + "];\nsql [" + sqlWithCarets + "]");
+    }
   }
 
   //~ Inner Interfaces -------------------------------------------------------
@@ -565,6 +555,11 @@ public class SqlValidatorTestCase {
       tester.checkResultType(sql, expectedType);
       return this;
     }
+
+    public Sql monotonic(SqlMonotonicity expectedMonotonicity) {
+      tester.checkMonotonic(sql, expectedMonotonicity);
+      return this;
+    }
   }
 }