[CALCITE-2535] Enable SqlTester.checkFails (previously it was a no-op) (Hongze 
Zhang)

Close apache/calcite#823


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

Branch: refs/heads/master
Commit: c69e1bcb13000adf9ae4bd433e2f8c75baba1501
Parents: 337f247
Author: hongzezhang <hongzezh...@tencent.com>
Authored: Wed Sep 12 20:34:34 2018 +0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Mon Sep 17 09:39:31 2018 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/calcite/util/Bug.java  |   5 +
 .../calcite/sql/test/AbstractSqlTester.java     | 680 +++++++++++++++++++
 .../apache/calcite/sql/test/SqlAdvisorTest.java |   2 +-
 .../calcite/sql/test/SqlOperatorBaseTest.java   | 372 +++++-----
 .../calcite/sql/test/SqlRuntimeTester.java      |  88 +++
 .../apache/calcite/sql/test/SqlTesterImpl.java  | 668 ------------------
 .../org/apache/calcite/sql/test/SqlTests.java   | 224 +++++-
 .../calcite/sql/test/SqlValidatorTester.java    |  35 +
 .../sql/validate/SqlValidatorUtilTest.java      |   7 +-
 .../org/apache/calcite/test/SqlTestGen.java     |   4 +-
 .../calcite/test/SqlValidatorDynamicTest.java   |   4 +-
 .../calcite/test/SqlValidatorFeatureTest.java   |   4 +-
 .../calcite/test/SqlValidatorTestCase.java      | 210 +-----
 core/src/test/resources/sql/misc.iq             |   3 +
 core/src/test/resources/sql/spatial.iq          |   2 +
 15 files changed, 1268 insertions(+), 1040 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/c69e1bcb/core/src/main/java/org/apache/calcite/util/Bug.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Bug.java 
b/core/src/main/java/org/apache/calcite/util/Bug.java
index 26f1bbc..7eeb208 100644
--- a/core/src/main/java/org/apache/calcite/util/Bug.java
+++ b/core/src/main/java/org/apache/calcite/util/Bug.java
@@ -180,6 +180,11 @@ public abstract class Bug {
    */
   public static final boolean CALCITE_2401_FIXED = false;
 
+  /** Whether
+   * <a 
href="https://issues.apache.org/jira/browse/CALCITE-2539";>[CALCITE-2539]
+   * Several test case not passed in CalciteSqlOperatorTest.java</a> is fixed. 
*/
+  public static final boolean CALCITE_2539_FIXED = false;
+
   /**
    * Use this to flag temporary code.
    */

http://git-wip-us.apache.org/repos/asf/calcite/blob/c69e1bcb/core/src/test/java/org/apache/calcite/sql/test/AbstractSqlTester.java
----------------------------------------------------------------------
diff --git 
a/core/src/test/java/org/apache/calcite/sql/test/AbstractSqlTester.java 
b/core/src/test/java/org/apache/calcite/sql/test/AbstractSqlTester.java
new file mode 100644
index 0000000..f9f252e
--- /dev/null
+++ b/core/src/test/java/org/apache/calcite/sql/test/AbstractSqlTester.java
@@ -0,0 +1,680 @@
+/*
+ * 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.calcite.sql.test;
+
+import org.apache.calcite.avatica.util.Casing;
+import org.apache.calcite.avatica.util.Quoting;
+import org.apache.calcite.config.CalciteConnectionProperty;
+import org.apache.calcite.config.Lex;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.runtime.Utilities;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCollation;
+import org.apache.calcite.sql.SqlIntervalLiteral;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.SqlSelect;
+import org.apache.calcite.sql.dialect.AnsiSqlDialect;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.parser.SqlParserUtil;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.util.SqlShuttle;
+import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.sql.validate.SqlConformanceEnum;
+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.CalciteAssert;
+import org.apache.calcite.test.SqlValidatorTestCase;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.TestUtil;
+import org.apache.calcite.util.Util;
+
+import com.google.common.collect.ImmutableList;
+
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.List;
+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;
+
+/**
+ * Abstract implementation of
+ * {@link org.apache.calcite.test.SqlValidatorTestCase.Tester}
+ * that talks to a mock catalog.
+ *
+ * <p>This is to implement the default behavior: testing is only against the
+ * {@link SqlValidator}.
+ */
+public abstract class AbstractSqlTester implements SqlTester, AutoCloseable {
+  protected final SqlTestFactory factory;
+
+  public AbstractSqlTester(SqlTestFactory factory) {
+    this.factory = factory;
+  }
+
+  public final SqlTestFactory getFactory() {
+    return factory;
+  }
+
+  /**
+   * {@inheritDoc}
+   *
+   * <p>This default implementation does nothing.
+   */
+  public void close() {
+    // no resources to release
+  }
+
+  public final SqlConformance getConformance() {
+    return (SqlConformance) factory.get("conformance");
+  }
+
+  public final SqlValidator getValidator() {
+    return factory.getValidator();
+  }
+
+  public void assertExceptionIsThrown(String sql, String expectedMsgPattern) {
+    final SqlValidator validator;
+    final SqlNode sqlNode;
+    final SqlParserUtil.StringAndPos sap = SqlParserUtil.findPos(sql);
+    try {
+      sqlNode = parseQuery(sap.sql);
+      validator = getValidator();
+    } catch (Throwable e) {
+      checkParseEx(e, expectedMsgPattern, sap.sql);
+      return;
+    }
+
+    Throwable thrown = null;
+    try {
+      validator.validate(sqlNode);
+    } catch (Throwable ex) {
+      thrown = ex;
+    }
+
+    SqlValidatorTestCase.checkEx(thrown, expectedMsgPattern, sap);
+  }
+
+  protected void checkParseEx(Throwable e, String expectedMsgPattern, String 
sql) {
+    try {
+      throw e;
+    } catch (SqlParseException spe) {
+      String errMessage = spe.getMessage();
+      if (expectedMsgPattern == null) {
+        throw new RuntimeException("Error while parsing query:" + sql, spe);
+      } else if (errMessage == null
+          || !errMessage.matches(expectedMsgPattern)) {
+        throw new RuntimeException("Error did not match expected ["
+            + expectedMsgPattern + "] while parsing query ["
+            + sql + "]", spe);
+      }
+    } catch (Throwable t) {
+      throw new RuntimeException("Error while parsing query: " + sql, t);
+    }
+  }
+
+  public RelDataType getColumnType(String sql) {
+    RelDataType rowType = getResultType(sql);
+    final List<RelDataTypeField> fields = rowType.getFieldList();
+    assertEquals("expected query to return 1 field", 1, fields.size());
+    return fields.get(0).getType();
+  }
+
+  public RelDataType getResultType(String sql) {
+    SqlValidator validator = getValidator();
+    SqlNode n = parseAndValidate(validator, sql);
+
+    return validator.getValidatedNodeType(n);
+  }
+
+  public SqlNode parseAndValidate(SqlValidator validator, String sql) {
+    if (validator == null) {
+      validator = getValidator();
+    }
+    SqlNode sqlNode;
+    try {
+      sqlNode = parseQuery(sql);
+    } catch (Throwable e) {
+      throw new RuntimeException("Error while parsing query: " + sql, e);
+    }
+    return validator.validate(sqlNode);
+  }
+
+  public SqlNode parseQuery(String sql) throws SqlParseException {
+    SqlParser parser = factory.createParser(sql);
+    return parser.parseQuery();
+  }
+
+  public void checkColumnType(String sql, String expected) {
+    RelDataType actualType = getColumnType(sql);
+    String actual = SqlTests.getTypeString(actualType);
+    assertEquals(expected, actual);
+  }
+
+  public void checkFieldOrigin(String sql, String fieldOriginList) {
+    SqlValidator validator = getValidator();
+    SqlNode n = parseAndValidate(validator, sql);
+    final List<List<String>> list = validator.getFieldOrigins(n);
+    final StringBuilder buf = new StringBuilder("{");
+    int i = 0;
+    for (List<String> strings : list) {
+      if (i++ > 0) {
+        buf.append(", ");
+      }
+      if (strings == null) {
+        buf.append("null");
+      } else {
+        int j = 0;
+        for (String s : strings) {
+          if (j++ > 0) {
+            buf.append('.');
+          }
+          buf.append(s);
+        }
+      }
+    }
+    buf.append("}");
+    assertEquals(fieldOriginList, buf.toString());
+  }
+
+  public void checkResultType(String sql, String expected) {
+    RelDataType actualType = getResultType(sql);
+    String actual = SqlTests.getTypeString(actualType);
+    assertEquals(expected, actual);
+  }
+
+  public void checkIntervalConv(String sql, String expected) {
+    SqlValidator validator = getValidator();
+    final SqlCall n = (SqlCall) parseAndValidate(validator, sql);
+
+    SqlNode node = null;
+    for (int i = 0; i < n.operandCount(); i++) {
+      node = stripAs(n.operand(i));
+      if (node instanceof SqlCall) {
+        node = ((SqlCall) node).operand(0);
+        break;
+      }
+    }
+
+    assertNotNull(node);
+    SqlIntervalLiteral intervalLiteral = (SqlIntervalLiteral) node;
+    SqlIntervalLiteral.IntervalValue interval =
+        (SqlIntervalLiteral.IntervalValue) intervalLiteral.getValue();
+    long l =
+        interval.getIntervalQualifier().isYearMonth()
+            ? SqlParserUtil.intervalToMonths(interval)
+            : SqlParserUtil.intervalToMillis(interval);
+    String actual = l + "";
+    assertEquals(expected, actual);
+  }
+
+  public void checkType(String expression, String type) {
+    for (String sql : buildQueries(expression)) {
+      checkColumnType(sql, type);
+    }
+  }
+
+  public void checkCollation(
+      String expression,
+      String expectedCollationName,
+      SqlCollation.Coercibility expectedCoercibility) {
+    for (String sql : buildQueries(expression)) {
+      RelDataType actualType = getColumnType(sql);
+      SqlCollation collation = actualType.getCollation();
+
+      assertEquals(
+          expectedCollationName, collation.getCollationName());
+      assertEquals(expectedCoercibility, collation.getCoercibility());
+    }
+  }
+
+  public void checkCharset(
+      String expression,
+      Charset expectedCharset) {
+    for (String sql : buildQueries(expression)) {
+      RelDataType actualType = getColumnType(sql);
+      Charset actualCharset = actualType.getCharset();
+
+      if (!expectedCharset.equals(actualCharset)) {
+        fail("\n"
+            + "Expected=" + expectedCharset.name() + "\n"
+            + "  actual=" + actualCharset.name());
+      }
+    }
+  }
+
+  public SqlTester withQuoting(Quoting quoting) {
+    return with("quoting", quoting);
+  }
+
+  public SqlTester withQuotedCasing(Casing casing) {
+    return with("quotedCasing", casing);
+  }
+
+  public SqlTester withUnquotedCasing(Casing casing) {
+    return with("unquotedCasing", casing);
+  }
+
+  public SqlTester withCaseSensitive(boolean sensitive) {
+    return with("caseSensitive", sensitive);
+  }
+
+  public SqlTester withLex(Lex lex) {
+    return withQuoting(lex.quoting)
+        .withCaseSensitive(lex.caseSensitive)
+        .withQuotedCasing(lex.quotedCasing)
+        .withUnquotedCasing(lex.unquotedCasing);
+  }
+
+  public SqlTester withConformance(SqlConformance conformance) {
+    if (conformance == null) {
+      conformance = SqlConformanceEnum.DEFAULT;
+    }
+    final SqlTester tester = with("conformance", conformance);
+    if (conformance instanceof SqlConformanceEnum) {
+      return tester
+          .withConnectionFactory(
+              CalciteAssert.EMPTY_CONNECTION_FACTORY
+                  .with(CalciteConnectionProperty.CONFORMANCE, conformance));
+    } else {
+      return tester;
+    }
+  }
+
+  public SqlTester withOperatorTable(SqlOperatorTable operatorTable) {
+    return with("operatorTable", operatorTable);
+  }
+
+  public SqlTester withConnectionFactory(
+      CalciteAssert.ConnectionFactory connectionFactory) {
+    return with("connectionFactory", connectionFactory);
+  }
+
+  protected final SqlTester with(final String name, final Object value) {
+    return with(factory.with(name, value));
+  }
+
+  protected abstract SqlTester with(SqlTestFactory factory);
+
+  // SqlTester methods
+
+  public void setFor(
+      SqlOperator operator,
+      VmName... unimplementedVmNames) {
+    // do nothing
+  }
+
+  public void checkAgg(
+      String expr,
+      String[] inputValues,
+      Object result,
+      double delta) {
+    String query =
+        SqlTests.generateAggQuery(expr, inputValues);
+    check(query, SqlTests.ANY_TYPE_CHECKER, result, delta);
+  }
+
+  public void checkWinAgg(
+      String expr,
+      String[] inputValues,
+      String windowSpec,
+      String type,
+      Object result,
+      double delta) {
+    String query =
+        SqlTests.generateWinAggQuery(
+            expr, windowSpec, inputValues);
+    check(query, SqlTests.ANY_TYPE_CHECKER, result, delta);
+  }
+
+  public void checkScalar(
+      String expression,
+      Object result,
+      String resultType) {
+    checkType(expression, resultType);
+    for (String sql : buildQueries(expression)) {
+      check(sql, SqlTests.ANY_TYPE_CHECKER, result, 0);
+    }
+  }
+
+  public void checkScalarExact(
+      String expression,
+      String result) {
+    for (String sql : buildQueries(expression)) {
+      check(sql, SqlTests.INTEGER_TYPE_CHECKER, result, 0);
+    }
+  }
+
+  public void checkScalarExact(
+      String expression,
+      String expectedType,
+      String result) {
+    for (String sql : buildQueries(expression)) {
+      TypeChecker typeChecker =
+          new SqlTests.StringTypeChecker(expectedType);
+      check(sql, typeChecker, result, 0);
+    }
+  }
+
+  public void checkScalarApprox(
+      String expression,
+      String expectedType,
+      double expectedResult,
+      double delta) {
+    for (String sql : buildQueries(expression)) {
+      TypeChecker typeChecker =
+          new SqlTests.StringTypeChecker(expectedType);
+      check(sql, typeChecker, expectedResult, delta);
+    }
+  }
+
+  public void checkBoolean(
+      String expression,
+      Boolean result) {
+    for (String sql : buildQueries(expression)) {
+      if (null == result) {
+        checkNull(expression);
+      } else {
+        check(
+            sql,
+            SqlTests.BOOLEAN_TYPE_CHECKER,
+            result.toString(),
+            0);
+      }
+    }
+  }
+
+  public void checkString(
+      String expression,
+      String result,
+      String expectedType) {
+    for (String sql : buildQueries(expression)) {
+      TypeChecker typeChecker =
+          new SqlTests.StringTypeChecker(expectedType);
+      check(sql, typeChecker, result, 0);
+    }
+  }
+
+  public void checkNull(String expression) {
+    for (String sql : buildQueries(expression)) {
+      check(sql, SqlTests.ANY_TYPE_CHECKER, null, 0);
+    }
+  }
+
+  public final void check(
+      String query,
+      TypeChecker typeChecker,
+      Object result,
+      double delta) {
+    check(query, typeChecker, SqlTests.ANY_PARAMETER_CHECKER,
+        SqlTests.createChecker(result, delta));
+  }
+
+  public void check(String query, TypeChecker typeChecker,
+      ParameterChecker parameterChecker, ResultChecker resultChecker) {
+    // This implementation does NOT check the result!
+    // All it does is check the return type.
+
+    if (typeChecker == null) {
+      // Parse and validate. There should be no errors.
+      Util.discard(getResultType(query));
+    } else {
+      // Parse and validate. There should be no errors.
+      // There must be 1 column. Get its type.
+      RelDataType actualType = getColumnType(query);
+
+      // Check result type.
+      typeChecker.checkType(actualType);
+    }
+
+    SqlValidator validator = getValidator();
+    SqlNode n = parseAndValidate(validator, query);
+    final RelDataType parameterRowType = validator.getParameterRowType(n);
+    parameterChecker.checkParameters(parameterRowType);
+  }
+
+  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,
+      String expectedRewrite) {
+    SqlNode rewrittenNode = parseAndValidate(validator, query);
+    String actualRewrite =
+        rewrittenNode.toSqlString(AnsiSqlDialect.DEFAULT, false).getSql();
+    TestUtil.assertEqualsVerbose(expectedRewrite, Util.toLinux(actualRewrite));
+  }
+
+  public void checkFails(
+      String expression,
+      String expectedError,
+      boolean runtime) {
+    if (runtime) {
+      // We need to test that the expression fails at runtime.
+      // Ironically, that means that it must succeed at prepare time.
+      SqlValidator validator = getValidator();
+      final String sql = buildQuery(expression);
+      SqlNode n = parseAndValidate(validator, sql);
+      assertNotNull(n);
+    } else {
+      checkQueryFails(buildQuery(expression), expectedError);
+    }
+  }
+
+  public void checkQueryFails(String sql, String expectedError) {
+    assertExceptionIsThrown(sql, expectedError);
+  }
+
+  public void checkQuery(String sql) {
+    assertExceptionIsThrown(sql, null);
+  }
+
+  public SqlMonotonicity getMonotonicity(String sql) {
+    final SqlValidator validator = getValidator();
+    final SqlNode node = parseAndValidate(validator, sql);
+    final SqlSelect select = (SqlSelect) node;
+    final SqlNode selectItem0 = select.getSelectList().get(0);
+    final SqlValidatorScope scope = validator.getSelectScope(select);
+    return selectItem0.getMonotonicity(scope);
+  }
+
+  public static String buildQuery(String expression) {
+    return "values (" + expression + ")";
+  }
+
+  public static String buildQueryAgg(String expression) {
+    return "select " + expression + " from (values (1)) as t(x) group by x";
+  }
+
+  /**
+   * Builds a query that extracts all literals as columns in an underlying
+   * select.
+   *
+   * <p>For example,</p>
+   *
+   * <blockquote>{@code 1 < 5}</blockquote>
+   *
+   * <p>becomes</p>
+   *
+   * <blockquote>{@code SELECT p0 < p1
+   * FROM (VALUES (1, 5)) AS t(p0, p1)}</blockquote>
+   *
+   * <p>Null literals don't have enough type information to be extracted.
+   * We push down {@code CAST(NULL AS type)} but raw nulls such as
+   * {@code CASE 1 WHEN 2 THEN 'a' ELSE NULL END} are left as is.</p>
+   *
+   * @param expression Scalar expression
+   * @return Query that evaluates a scalar expression
+   */
+  protected String buildQuery2(String expression) {
+    // "values (1 < 5)"
+    // becomes
+    // "select p0 < p1 from (values (1, 5)) as t(p0, p1)"
+    SqlNode x;
+    final String sql = "values (" + expression + ")";
+    try {
+      x = parseQuery(sql);
+    } catch (SqlParseException e) {
+      throw new RuntimeException(e);
+    }
+    final Collection<SqlNode> literalSet = new LinkedHashSet<>();
+    x.accept(
+        new SqlShuttle() {
+          private final List<SqlOperator> ops =
+              ImmutableList.of(
+                  SqlStdOperatorTable.LITERAL_CHAIN,
+                  SqlStdOperatorTable.LOCALTIME,
+                  SqlStdOperatorTable.LOCALTIMESTAMP,
+                  SqlStdOperatorTable.CURRENT_TIME,
+                  SqlStdOperatorTable.CURRENT_TIMESTAMP);
+
+          @Override public SqlNode visit(SqlLiteral literal) {
+            if (!isNull(literal)
+                && literal.getTypeName() != SqlTypeName.SYMBOL) {
+              literalSet.add(literal);
+            }
+            return literal;
+          }
+
+          @Override public SqlNode visit(SqlCall call) {
+            final SqlOperator operator = call.getOperator();
+            if (operator == SqlStdOperatorTable.CAST
+                && isNull(call.operand(0))) {
+              literalSet.add(call);
+              return call;
+            } else if (ops.contains(operator)) {
+              // "Argument to function 'LOCALTIME' must be a
+              // literal"
+              return call;
+            } else {
+              return super.visit(call);
+            }
+          }
+
+          private boolean isNull(SqlNode sqlNode) {
+            return sqlNode instanceof SqlLiteral
+                && ((SqlLiteral) sqlNode).getTypeName()
+                == SqlTypeName.NULL;
+          }
+        });
+    final List<SqlNode> nodes = new ArrayList<>(literalSet);
+    nodes.sort((o1, o2) -> {
+      final SqlParserPos pos0 = o1.getParserPosition();
+      final SqlParserPos pos1 = o2.getParserPosition();
+      int c = -Utilities.compare(pos0.getLineNum(), pos1.getLineNum());
+      if (c != 0) {
+        return c;
+      }
+      return -Utilities.compare(pos0.getColumnNum(), pos1.getColumnNum());
+    });
+    String sql2 = sql;
+    final List<Pair<String, String>> values = new ArrayList<>();
+    int p = 0;
+    for (SqlNode literal : nodes) {
+      final SqlParserPos pos = literal.getParserPosition();
+      final int start =
+          SqlParserUtil.lineColToIndex(
+              sql, pos.getLineNum(), pos.getColumnNum());
+      final int end =
+          SqlParserUtil.lineColToIndex(
+              sql,
+              pos.getEndLineNum(),
+              pos.getEndColumnNum()) + 1;
+      String param = "p" + (p++);
+      values.add(Pair.of(sql2.substring(start, end), param));
+      sql2 = sql2.substring(0, start)
+          + param
+          + sql2.substring(end);
+    }
+    if (values.isEmpty()) {
+      values.add(Pair.of("1", "p0"));
+    }
+    return "select "
+        + sql2.substring("values (".length(), sql2.length() - 1)
+        + " from (values ("
+        + Util.commaList(Pair.left(values))
+        + ")) as t("
+        + Util.commaList(Pair.right(values))
+        + ")";
+  }
+
+  /**
+   * Converts a scalar expression into a list of SQL queries that
+   * evaluate it.
+   *
+   * @param expression Scalar expression
+   * @return List of queries that evaluate an expression
+   */
+  private Iterable<String> buildQueries(final String expression) {
+    // Why an explicit iterable rather than a list? If there is
+    // a syntax error in the expression, the calling code discovers it
+    // before we try to parse it to do substitutions on the parse tree.
+    return () -> new Iterator<String>() {
+      int i = 0;
+
+      public void remove() {
+        throw new UnsupportedOperationException();
+      }
+
+      public String next() {
+        switch (i++) {
+        case 0:
+          return buildQuery(expression);
+        case 1:
+          return buildQuery2(expression);
+        default:
+          throw new NoSuchElementException();
+        }
+      }
+
+      public boolean hasNext() {
+        return i < 2;
+      }
+    };
+  }
+
+}
+
+// End AbstractSqlTester.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/c69e1bcb/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java 
b/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
index ebee877..5b57d82 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
@@ -576,7 +576,7 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
   }
 
   @Override public SqlTester getTester() {
-    return new SqlTesterImpl(ADVISOR_TEST_FACTORY);
+    return new SqlValidatorTester(ADVISOR_TEST_FACTORY);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/c69e1bcb/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
----------------------------------------------------------------------
diff --git 
a/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java 
b/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
index 903504c..aac111c 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
@@ -531,9 +531,11 @@ public abstract class SqlOperatorBaseTest {
         "cast(.48 as varchar(10))",
         ".48",
         "VARCHAR(10) NOT NULL");
-    tester.checkFails(
-        "cast(2.523 as char(2))", STRING_TRUNC_MESSAGE,
-        true);
+    if (Bug.CALCITE_2539_FIXED) {
+      tester.checkFails(
+          "cast(2.523 as char(2))", STRING_TRUNC_MESSAGE,
+          true);
+    }
 
     tester.checkString(
         "cast(-0.29 as varchar(10))",
@@ -563,13 +565,14 @@ public abstract class SqlOperatorBaseTest {
     if (TODO) {
       checkCastToString("cast(-0.1 as real)", "CHAR(5)", "-1E-1");
     }
-
-    tester.checkFails(
-        "cast(1.3243232e0 as varchar(4))", STRING_TRUNC_MESSAGE,
-        true);
-    tester.checkFails(
-        "cast(1.9e5 as char(4))", STRING_TRUNC_MESSAGE,
-        true);
+    if (Bug.CALCITE_2539_FIXED) {
+      tester.checkFails(
+          "cast(1.3243232e0 as varchar(4))", STRING_TRUNC_MESSAGE,
+          true);
+      tester.checkFails(
+          "cast(1.9e5 as char(4))", STRING_TRUNC_MESSAGE,
+          true);
+    }
 
     // string
     checkCastToString("'abc'", "CHAR(1)", "a");
@@ -634,18 +637,21 @@ public abstract class SqlOperatorBaseTest {
     checkCastToString("True", "CHAR(6)", "TRUE  ");
     checkCastToString("True", "VARCHAR(6)", "TRUE");
     checkCastToString("False", "CHAR(5)", "FALSE");
-    tester.checkFails(
-        "cast(true as char(3))", INVALID_CHAR_MESSAGE,
-        true);
-    tester.checkFails(
-        "cast(false as char(4))", INVALID_CHAR_MESSAGE,
-        true);
-    tester.checkFails(
-        "cast(true as varchar(3))", INVALID_CHAR_MESSAGE,
-        true);
-    tester.checkFails(
-        "cast(false as varchar(4))", INVALID_CHAR_MESSAGE,
-        true);
+
+    if (Bug.CALCITE_2539_FIXED) {
+      tester.checkFails(
+          "cast(true as char(3))", INVALID_CHAR_MESSAGE,
+          true);
+      tester.checkFails(
+          "cast(false as char(4))", INVALID_CHAR_MESSAGE,
+          true);
+      tester.checkFails(
+          "cast(true as varchar(3))", INVALID_CHAR_MESSAGE,
+          true);
+      tester.checkFails(
+          "cast(false as varchar(4))", INVALID_CHAR_MESSAGE,
+          true);
+    }
   }
 
   @Test public void testCastExactNumericLimits() {
@@ -678,14 +684,16 @@ public abstract class SqlOperatorBaseTest {
             type, LITERAL_OUT_OF_RANGE_MESSAGE,
             false);
       } else {
-        checkCastFails(
-            MAX_OVERFLOW_NUMERIC_STRINGS[i],
-            type, OUT_OF_RANGE_MESSAGE,
-            true);
-        checkCastFails(
-            MIN_OVERFLOW_NUMERIC_STRINGS[i],
-            type, OUT_OF_RANGE_MESSAGE,
-            true);
+        if (Bug.CALCITE_2539_FIXED) {
+          checkCastFails(
+              MAX_OVERFLOW_NUMERIC_STRINGS[i],
+              type, OUT_OF_RANGE_MESSAGE,
+              true);
+          checkCastFails(
+              MIN_OVERFLOW_NUMERIC_STRINGS[i],
+              type, OUT_OF_RANGE_MESSAGE,
+              true);
+        }
       }
 
       // Convert from string to type
@@ -698,15 +706,17 @@ public abstract class SqlOperatorBaseTest {
           type,
           MIN_NUMERIC_STRINGS[i]);
 
-      checkCastFails(
-          "'" + MAX_OVERFLOW_NUMERIC_STRINGS[i] + "'",
-          type, OUT_OF_RANGE_MESSAGE,
-          true);
-      checkCastFails(
-          "'" + MIN_OVERFLOW_NUMERIC_STRINGS[i] + "'",
-          type,
-          OUT_OF_RANGE_MESSAGE,
-          true);
+      if (Bug.CALCITE_2539_FIXED) {
+        checkCastFails(
+            "'" + MAX_OVERFLOW_NUMERIC_STRINGS[i] + "'",
+            type, OUT_OF_RANGE_MESSAGE,
+            true);
+        checkCastFails(
+            "'" + MIN_OVERFLOW_NUMERIC_STRINGS[i] + "'",
+            type,
+            OUT_OF_RANGE_MESSAGE,
+            true);
+      }
 
       // Convert from type to string
       checkCastToString(MAX_NUMERIC_STRINGS[i], null, null);
@@ -715,7 +725,9 @@ public abstract class SqlOperatorBaseTest {
       checkCastToString(MIN_NUMERIC_STRINGS[i], null, null);
       checkCastToString(MIN_NUMERIC_STRINGS[i], type, null);
 
-      checkCastFails("'notnumeric'", type, INVALID_CHAR_MESSAGE, true);
+      if (Bug.CALCITE_2539_FIXED) {
+        checkCastFails("'notnumeric'", type, INVALID_CHAR_MESSAGE, true);
+      }
     }
   }
 
@@ -1215,14 +1227,16 @@ public abstract class SqlOperatorBaseTest {
     // generate Java constants that throw when the class is loaded, thus
     // ExceptionInInitializerError.
     tester.checkScalarExact("cast('15' as integer)", "INTEGER NOT NULL", "15");
-    tester.checkFails("cast('15.4' as integer)", "xxx", true);
-    tester.checkFails("cast('15.6' as integer)", "xxx", true);
-    tester.checkFails("cast('ue' as boolean)", "xxx", true);
-    tester.checkFails("cast('' as boolean)", "xxx", true);
-    tester.checkFails("cast('' as integer)", "xxx", true);
-    tester.checkFails("cast('' as real)", "xxx", true);
-    tester.checkFails("cast('' as double)", "xxx", true);
-    tester.checkFails("cast('' as smallint)", "xxx", true);
+    if (Bug.CALCITE_2539_FIXED) {
+      tester.checkFails("cast('15.4' as integer)", "xxx", true);
+      tester.checkFails("cast('15.6' as integer)", "xxx", true);
+      tester.checkFails("cast('ue' as boolean)", "xxx", true);
+      tester.checkFails("cast('' as boolean)", "xxx", true);
+      tester.checkFails("cast('' as integer)", "xxx", true);
+      tester.checkFails("cast('' as real)", "xxx", true);
+      tester.checkFails("cast('' as double)", "xxx", true);
+      tester.checkFails("cast('' as smallint)", "xxx", true);
+    }
   }
 
   @Test public void testCastDateTime() {
@@ -1342,21 +1356,23 @@ public abstract class SqlOperatorBaseTest {
     tester.checkFails(
         "cast('nottime' as TIME)", BAD_DATETIME_MESSAGE,
         true);
-    tester.checkFails(
-        "cast('1241241' as TIME)", BAD_DATETIME_MESSAGE,
-        true);
-    tester.checkFails(
-        "cast('12:54:78' as TIME)", BAD_DATETIME_MESSAGE,
-        true);
-    tester.checkFails(
-        "cast('12:34:5' as TIME)", BAD_DATETIME_MESSAGE,
-        true);
-    tester.checkFails(
-        "cast('12:3:45' as TIME)", BAD_DATETIME_MESSAGE,
-        true);
-    tester.checkFails(
-        "cast('1:23:45' as TIME)", BAD_DATETIME_MESSAGE,
-        true);
+    if (Bug.CALCITE_2539_FIXED) {
+      tester.checkFails(
+          "cast('1241241' as TIME)", BAD_DATETIME_MESSAGE,
+          true);
+      tester.checkFails(
+          "cast('12:54:78' as TIME)", BAD_DATETIME_MESSAGE,
+          true);
+      tester.checkFails(
+          "cast('12:34:5' as TIME)", BAD_DATETIME_MESSAGE,
+          true);
+      tester.checkFails(
+          "cast('12:3:45' as TIME)", BAD_DATETIME_MESSAGE,
+          true);
+      tester.checkFails(
+          "cast('1:23:45' as TIME)", BAD_DATETIME_MESSAGE,
+          true);
+    }
 
     // timestamp <-> string
     checkCastToString(
@@ -1406,18 +1422,21 @@ public abstract class SqlOperatorBaseTest {
     tester.checkFails(
         "cast('nottime' as TIMESTAMP)", BAD_DATETIME_MESSAGE,
         true);
-    tester.checkFails(
-        "cast('1241241' as TIMESTAMP)", BAD_DATETIME_MESSAGE,
-        true);
-    tester.checkFails(
-        "cast('1945-20-24 12:42:25.34' as TIMESTAMP)", BAD_DATETIME_MESSAGE,
-        true);
-    tester.checkFails(
-        "cast('1945-01-24 25:42:25.34' as TIMESTAMP)", BAD_DATETIME_MESSAGE,
-        true);
-    tester.checkFails(
-        "cast('1945-1-24 12:23:34.454' as TIMESTAMP)", BAD_DATETIME_MESSAGE,
-        true);
+
+    if (Bug.CALCITE_2539_FIXED) {
+      tester.checkFails(
+          "cast('1241241' as TIMESTAMP)", BAD_DATETIME_MESSAGE,
+          true);
+      tester.checkFails(
+          "cast('1945-20-24 12:42:25.34' as TIMESTAMP)", BAD_DATETIME_MESSAGE,
+          true);
+      tester.checkFails(
+          "cast('1945-01-24 25:42:25.34' as TIMESTAMP)", BAD_DATETIME_MESSAGE,
+          true);
+      tester.checkFails(
+          "cast('1945-1-24 12:23:34.454' as TIMESTAMP)", BAD_DATETIME_MESSAGE,
+          true);
+    }
 
     // date <-> string
     checkCastToString("DATE '1945-02-24'", null, "1945-02-24");
@@ -1438,12 +1457,15 @@ public abstract class SqlOperatorBaseTest {
     tester.checkFails(
         "cast('notdate' as DATE)", BAD_DATETIME_MESSAGE,
         true);
-    tester.checkFails(
-        "cast('52534253' as DATE)", BAD_DATETIME_MESSAGE,
-        true);
-    tester.checkFails(
-        "cast('1945-30-24' as DATE)", BAD_DATETIME_MESSAGE,
-        true);
+
+    if (Bug.CALCITE_2539_FIXED) {
+      tester.checkFails(
+          "cast('52534253' as DATE)", BAD_DATETIME_MESSAGE,
+          true);
+      tester.checkFails(
+          "cast('1945-30-24' as DATE)", BAD_DATETIME_MESSAGE,
+          true);
+    }
 
     // cast null
     tester.checkNull("cast(null as date)");
@@ -1896,12 +1918,14 @@ public abstract class SqlOperatorBaseTest {
     }
     tester.checkScalar("{fn DAYOFMONTH(DATE '2014-12-10')}", 10,
         "BIGINT NOT NULL");
-    tester.checkFails("{fn DAYOFWEEK(DATE '2014-12-10')}",
-        "cannot translate call EXTRACT.*",
-        true);
-    tester.checkFails("{fn DAYOFYEAR(DATE '2014-12-10')}",
-        "cannot translate call EXTRACT.*",
-        true);
+    if (Bug.CALCITE_2539_FIXED) {
+      tester.checkFails("{fn DAYOFWEEK(DATE '2014-12-10')}",
+          "cannot translate call EXTRACT.*",
+          true);
+      tester.checkFails("{fn DAYOFYEAR(DATE '2014-12-10')}",
+          "cannot translate call EXTRACT.*",
+          true);
+    }
     tester.checkScalar("{fn HOUR(TIMESTAMP '2014-12-10 12:34:56')}", 12,
         "BIGINT NOT NULL");
     tester.checkScalar("{fn MINUTE(TIMESTAMP '2014-12-10 12:34:56')}", 34,
@@ -1921,9 +1945,12 @@ public abstract class SqlOperatorBaseTest {
     tester.checkScalar("{fn TIMESTAMPDIFF(HOUR,"
         + " TIMESTAMP '2014-03-29 12:34:56',"
         + " TIMESTAMP '2014-03-29 12:34:56')}", "0", "INTEGER NOT NULL");
-    tester.checkFails("{fn WEEK(DATE '2014-12-10')}",
-        "cannot translate call EXTRACT.*",
-        true);
+
+    if (Bug.CALCITE_2539_FIXED) {
+      tester.checkFails("{fn WEEK(DATE '2014-12-10')}",
+          "cannot translate call EXTRACT.*",
+          true);
+    }
     tester.checkScalar("{fn YEAR(DATE '2014-12-10')}", 2014, "BIGINT NOT 
NULL");
 
     // System Functions
@@ -2192,9 +2219,11 @@ public abstract class SqlOperatorBaseTest {
     }
     tester.checkNull("1e1 / cast(null as float)");
 
-    tester.checkFails(
-        "100.1 / 0.00000000000000001", OUT_OF_RANGE_MESSAGE,
-        true);
+    if (Bug.CALCITE_2539_FIXED) {
+      tester.checkFails(
+          "100.1 / 0.00000000000000001", OUT_OF_RANGE_MESSAGE,
+          true);
+    }
   }
 
   @Test public void testDivideOperatorIntervals() {
@@ -3963,15 +3992,17 @@ public abstract class SqlOperatorBaseTest {
             + "                    \\^",
         true);
 
-    tester.checkFails(
-        "'cd' similar to '[(a-e)]d' ",
-        "Invalid regular expression: \\[\\(a-e\\)\\]d at 1",
-        true);
+    if (Bug.CALCITE_2539_FIXED) {
+      tester.checkFails(
+          "'cd' similar to '[(a-e)]d' ",
+          "Invalid regular expression: \\[\\(a-e\\)\\]d at 1",
+          true);
 
-    tester.checkFails(
-        "'yd' similar to '[(a-e)]d' ",
-        "Invalid regular expression: \\[\\(a-e\\)\\]d at 1",
-        true);
+      tester.checkFails(
+          "'yd' similar to '[(a-e)]d' ",
+          "Invalid regular expression: \\[\\(a-e\\)\\]d at 1",
+          true);
+    }
 
     // all the following tests wrong results due to missing functionality
     // or defect (FRG-375, 377).
@@ -5643,15 +5674,17 @@ public abstract class SqlOperatorBaseTest {
         SqlStdOperatorTable.WEEK,
         VM_FENNEL,
         VM_JAVA);
-    // TODO: Not implemented in operator test execution code
-    tester.checkFails(
-        "week(date '2008-1-23')",
-        "cannot translate call EXTRACT.*",
-        true);
-    tester.checkFails(
-        "week(cast(null as date))",
-        "cannot translate call EXTRACT.*",
-        true);
+    if (Bug.CALCITE_2539_FIXED) {
+      // TODO: Not implemented in operator test execution code
+      tester.checkFails(
+          "week(date '2008-1-23')",
+          "cannot translate call EXTRACT.*",
+          true);
+      tester.checkFails(
+          "week(cast(null as date))",
+          "cannot translate call EXTRACT.*",
+          true);
+    }
   }
 
   @Test public void testDayOfYear() {
@@ -5659,15 +5692,17 @@ public abstract class SqlOperatorBaseTest {
         SqlStdOperatorTable.DAYOFYEAR,
         VM_FENNEL,
         VM_JAVA);
-    // TODO: Not implemented in operator test execution code
-    tester.checkFails(
-        "dayofyear(date '2008-1-23')",
-        "cannot translate call EXTRACT.*",
-        true);
-    tester.checkFails(
-        "dayofyear(cast(null as date))",
-        "cannot translate call EXTRACT.*",
-        true);
+    if (Bug.CALCITE_2539_FIXED) {
+      // TODO: Not implemented in operator test execution code
+      tester.checkFails(
+          "dayofyear(date '2008-1-23')",
+          "cannot translate call EXTRACT.*",
+          true);
+      tester.checkFails(
+          "dayofyear(cast(null as date))",
+          "cannot translate call EXTRACT.*",
+          true);
+    }
   }
 
   @Test public void testDayOfMonth() {
@@ -5687,14 +5722,16 @@ public abstract class SqlOperatorBaseTest {
         SqlStdOperatorTable.DAYOFWEEK,
         VM_FENNEL,
         VM_JAVA);
-    // TODO: Not implemented in operator test execution code
-    tester.checkFails(
-        "dayofweek(date '2008-1-23')",
-        "cannot translate call EXTRACT.*",
-        true);
-    tester.checkFails("dayofweek(cast(null as date))",
-        "cannot translate call EXTRACT.*",
-        true);
+    if (Bug.CALCITE_2539_FIXED) {
+      // TODO: Not implemented in operator test execution code
+      tester.checkFails(
+          "dayofweek(date '2008-1-23')",
+          "cannot translate call EXTRACT.*",
+          true);
+      tester.checkFails("dayofweek(cast(null as date))",
+          "cannot translate call EXTRACT.*",
+          true);
+    }
   }
 
   @Test public void testHour() {
@@ -5874,14 +5911,16 @@ public abstract class SqlOperatorBaseTest {
 
     // Postgres doesn't support DOW, ISODOW, DOY and WEEK on INTERVAL DAY TIME 
type.
     // SQL standard doesn't have extract units for DOW, ISODOW, DOY and WEEK.
-    tester.checkFails("extract(doy from interval '2 3:4:5.678' day to second)",
-        INVALID_EXTRACT_UNIT_CONVERTLET_ERROR, true);
-    tester.checkFails("extract(dow from interval '2 3:4:5.678' day to second)",
-        INVALID_EXTRACT_UNIT_CONVERTLET_ERROR, true);
-    tester.checkFails("extract(week from interval '2 3:4:5.678' day to 
second)",
-        INVALID_EXTRACT_UNIT_CONVERTLET_ERROR, true);
-    tester.checkFails("extract(isodow from interval '2 3:4:5.678' day to 
second)",
-        INVALID_EXTRACT_UNIT_CONVERTLET_ERROR, true);
+    if (Bug.CALCITE_2539_FIXED) {
+      tester.checkFails("extract(doy from interval '2 3:4:5.678' day to 
second)",
+          INVALID_EXTRACT_UNIT_CONVERTLET_ERROR, true);
+      tester.checkFails("extract(dow from interval '2 3:4:5.678' day to 
second)",
+          INVALID_EXTRACT_UNIT_CONVERTLET_ERROR, true);
+      tester.checkFails("extract(week from interval '2 3:4:5.678' day to 
second)",
+          INVALID_EXTRACT_UNIT_CONVERTLET_ERROR, true);
+      tester.checkFails("extract(isodow from interval '2 3:4:5.678' day to 
second)",
+          INVALID_EXTRACT_UNIT_CONVERTLET_ERROR, true);
+    }
 
     tester.checkFails(
         "^extract(month from interval '2 3:4:5.678' day to second)^",
@@ -6109,23 +6148,25 @@ public abstract class SqlOperatorBaseTest {
         "2008",
         "BIGINT NOT NULL");
 
-    // TODO: Not implemented in operator test execution code
-    tester.checkFails(
-        "extract(doy from timestamp '2008-2-23 12:34:56')",
-        "cannot translate call EXTRACT.*",
-        true);
+    if (Bug.CALCITE_2539_FIXED) {
+      // TODO: Not implemented in operator test execution code
+      tester.checkFails(
+          "extract(doy from timestamp '2008-2-23 12:34:56')",
+          "cannot translate call EXTRACT.*",
+          true);
 
-    // TODO: Not implemented in operator test execution code
-    tester.checkFails(
-        "extract(dow from timestamp '2008-2-23 12:34:56')",
-        "cannot translate call EXTRACT.*",
-        true);
+      // TODO: Not implemented in operator test execution code
+      tester.checkFails(
+          "extract(dow from timestamp '2008-2-23 12:34:56')",
+          "cannot translate call EXTRACT.*",
+          true);
 
-    // TODO: Not implemented in operator test execution code
-    tester.checkFails(
-        "extract(week from timestamp '2008-2-23 12:34:56')",
-        "cannot translate call EXTRACT.*",
-        true);
+      // TODO: Not implemented in operator test execution code
+      tester.checkFails(
+          "extract(week from timestamp '2008-2-23 12:34:56')",
+          "cannot translate call EXTRACT.*",
+          true);
+    }
 
     tester.checkScalar(
         "extract(decade from timestamp '2008-2-23 12:34:56')",
@@ -6817,7 +6858,7 @@ public abstract class SqlOperatorBaseTest {
    * {@code SELECT sum(1) FROM emp GROUP BY deptno} has type "INTEGER NOT 
NULL",
    */
   protected void checkAggType(SqlTester tester, String expr, String type) {
-    tester.checkColumnType(SqlTesterImpl.buildQueryAgg(expr), type);
+    tester.checkColumnType(AbstractSqlTester.buildQueryAgg(expr), type);
   }
 
   @Test public void testAvgFunc() {
@@ -7322,6 +7363,7 @@ public abstract class SqlOperatorBaseTest {
         "0",
         0d);
   }
+
   /**
    * Tests that CAST fails when given a value just outside the valid range for
    * that type. For example,
@@ -7414,15 +7456,17 @@ public abstract class SqlOperatorBaseTest {
           // Casting overlarge string/binary values do not fail -
           // they are truncated. See testCastTruncates().
         } else {
-          // Value outside legal bound should fail at runtime (not
-          // validate time).
-          //
-          // NOTE: Because Java and Fennel calcs give
-          // different errors, the pattern hedges its bets.
-          tester.checkFails(
-              "CAST(" + literalString + " AS " + type + ")",
-              "(?s).*(Overflow during calculation or cast\\.|Code=22003).*",
-              true);
+          if (Bug.CALCITE_2539_FIXED) {
+            // Value outside legal bound should fail at runtime (not
+            // validate time).
+            //
+            // NOTE: Because Java and Fennel calcs give
+            // different errors, the pattern hedges its bets.
+            tester.checkFails(
+                "CAST(" + literalString + " AS " + type + ")",
+                "(?s).*(Overflow during calculation or cast\\.|Code=22003).*",
+                true);
+          }
         }
       }
     }
@@ -7533,7 +7577,7 @@ public abstract class SqlOperatorBaseTest {
                   query = "SELECT " + s + " FROM (VALUES (1))";
                 }
               } else {
-                query = SqlTesterImpl.buildQuery(s);
+                query = AbstractSqlTester.buildQuery(s);
               }
               tester.check(query, SqlTests.ANY_TYPE_CHECKER,
                   SqlTests.ANY_PARAMETER_CHECKER, result -> { });
@@ -7648,7 +7692,7 @@ public abstract class SqlOperatorBaseTest {
    * Implementation of {@link org.apache.calcite.sql.test.SqlTester} based on a
    * JDBC connection.
    */
-  protected static class TesterImpl extends SqlTesterImpl {
+  protected static class TesterImpl extends SqlRuntimeTester {
     public TesterImpl(SqlTestFactory testFactory) {
       super(testFactory);
     }
@@ -7672,8 +7716,8 @@ public abstract class SqlOperatorBaseTest {
       }
     }
 
-    @Override protected TesterImpl with(final String name, final Object value) 
{
-      return new TesterImpl(factory.with(name, value));
+    @Override protected SqlTester with(SqlTestFactory factory) {
+      return new TesterImpl(factory);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/c69e1bcb/core/src/test/java/org/apache/calcite/sql/test/SqlRuntimeTester.java
----------------------------------------------------------------------
diff --git 
a/core/src/test/java/org/apache/calcite/sql/test/SqlRuntimeTester.java 
b/core/src/test/java/org/apache/calcite/sql/test/SqlRuntimeTester.java
new file mode 100644
index 0000000..25f5285
--- /dev/null
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlRuntimeTester.java
@@ -0,0 +1,88 @@
+/*
+ * 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.calcite.sql.test;
+
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.parser.SqlParserUtil;
+import org.apache.calcite.sql.validate.SqlValidator;
+
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * Tester of {@link SqlValidator} and runtime execution of the input SQL.
+ */
+public class SqlRuntimeTester extends AbstractSqlTester {
+  public SqlRuntimeTester(SqlTestFactory factory) {
+    super(factory);
+  }
+
+  @Override protected SqlTester with(SqlTestFactory factory) {
+    return new SqlRuntimeTester(factory);
+  }
+
+  @Override public void checkFails(String expression, String expectedError,
+      boolean runtime) {
+    final String sql =
+        runtime ? buildQuery2(expression) : buildQuery(expression);
+    assertExceptionIsThrown(sql, expectedError, runtime);
+  }
+
+  public void assertExceptionIsThrown(
+      String sql,
+      String expectedMsgPattern) {
+    assertExceptionIsThrown(sql, expectedMsgPattern, false);
+  }
+
+  public void assertExceptionIsThrown(String sql, String expectedMsgPattern,
+      boolean runtime) {
+    final SqlNode sqlNode;
+    final SqlParserUtil.StringAndPos sap = SqlParserUtil.findPos(sql);
+    try {
+      sqlNode = parseQuery(sap.sql);
+    } catch (Throwable e) {
+      checkParseEx(e, expectedMsgPattern, sap.sql);
+      return;
+    }
+
+    Throwable thrown = null;
+    final SqlTests.Stage stage;
+    final SqlValidator validator = getValidator();
+    if (runtime) {
+      stage = SqlTests.Stage.RUNTIME;
+      SqlNode validated = validator.validate(sqlNode);
+      assertNotNull(validated);
+      try {
+        check(sap.sql, SqlTests.ANY_TYPE_CHECKER,
+            SqlTests.ANY_PARAMETER_CHECKER, SqlTests.ANY_RESULT_CHECKER);
+      } catch (Throwable ex) {
+        // get the real exception in runtime check
+        thrown = ex;
+      }
+    } else {
+      stage = SqlTests.Stage.VALIDATE;
+      try {
+        validator.validate(sqlNode);
+      } catch (Throwable ex) {
+        thrown = ex;
+      }
+    }
+
+    SqlTests.checkEx(thrown, expectedMsgPattern, sap, stage);
+  }
+}
+
+// End SqlRuntimeTester.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/c69e1bcb/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
deleted file mode 100644
index 533cb99..0000000
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlTesterImpl.java
+++ /dev/null
@@ -1,668 +0,0 @@
-/*
- * 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.calcite.sql.test;
-
-import org.apache.calcite.avatica.util.Casing;
-import org.apache.calcite.avatica.util.Quoting;
-import org.apache.calcite.config.CalciteConnectionProperty;
-import org.apache.calcite.config.Lex;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.calcite.runtime.Utilities;
-import org.apache.calcite.sql.SqlCall;
-import org.apache.calcite.sql.SqlCollation;
-import org.apache.calcite.sql.SqlIntervalLiteral;
-import org.apache.calcite.sql.SqlLiteral;
-import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.SqlOperator;
-import org.apache.calcite.sql.SqlOperatorTable;
-import org.apache.calcite.sql.SqlSelect;
-import org.apache.calcite.sql.dialect.AnsiSqlDialect;
-import org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.calcite.sql.parser.SqlParseException;
-import org.apache.calcite.sql.parser.SqlParser;
-import org.apache.calcite.sql.parser.SqlParserPos;
-import org.apache.calcite.sql.parser.SqlParserUtil;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.sql.util.SqlShuttle;
-import org.apache.calcite.sql.validate.SqlConformance;
-import org.apache.calcite.sql.validate.SqlConformanceEnum;
-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.CalciteAssert;
-import org.apache.calcite.test.SqlValidatorTestCase;
-import org.apache.calcite.util.Pair;
-import org.apache.calcite.util.TestUtil;
-import org.apache.calcite.util.Util;
-
-import com.google.common.collect.ImmutableList;
-
-import java.nio.charset.Charset;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.LinkedHashSet;
-import java.util.List;
-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;
-
-/**
- * Implementation of {@link 
org.apache.calcite.test.SqlValidatorTestCase.Tester}
- * that talks to a mock catalog.
- */
-public class SqlTesterImpl implements SqlTester, AutoCloseable {
-  protected final SqlTestFactory factory;
-
-  public SqlTesterImpl(SqlTestFactory factory) {
-    this.factory = factory;
-  }
-
-  public final SqlTestFactory getFactory() {
-    return factory;
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * <p>This default implementation does nothing.
-   */
-  public void close() {
-    // no resources to release
-  }
-
-  public final SqlConformance getConformance() {
-    return (SqlConformance) factory.get("conformance");
-  }
-
-  public final SqlValidator getValidator() {
-    return factory.getValidator();
-  }
-
-  public void assertExceptionIsThrown(
-      String sql,
-      String expectedMsgPattern) {
-    SqlValidator validator;
-    SqlNode sqlNode;
-    SqlParserUtil.StringAndPos sap = SqlParserUtil.findPos(sql);
-    try {
-      sqlNode = parseQuery(sap.sql);
-      validator = getValidator();
-    } catch (SqlParseException e) {
-      String errMessage = e.getMessage();
-      if (expectedMsgPattern == null) {
-        throw new RuntimeException("Error while parsing query:" + sap.sql, e);
-      } else if (errMessage == null
-          || !errMessage.matches(expectedMsgPattern)) {
-        throw new RuntimeException("Error did not match expected ["
-            + expectedMsgPattern + "] while parsing query ["
-            + sap.sql + "]", e);
-      }
-      return;
-    } catch (Throwable e) {
-      throw new RuntimeException("Error while parsing query: " + sap.sql, e);
-    }
-
-    Throwable thrown = null;
-    try {
-      validator.validate(sqlNode);
-    } catch (Throwable ex) {
-      thrown = ex;
-    }
-
-    SqlValidatorTestCase.checkEx(thrown, expectedMsgPattern, sap);
-  }
-
-  public RelDataType getColumnType(String sql) {
-    RelDataType rowType = getResultType(sql);
-    final List<RelDataTypeField> fields = rowType.getFieldList();
-    assertEquals("expected query to return 1 field", 1, fields.size());
-    return fields.get(0).getType();
-  }
-
-  public RelDataType getResultType(String sql) {
-    SqlValidator validator = getValidator();
-    SqlNode n = parseAndValidate(validator, sql);
-
-    return validator.getValidatedNodeType(n);
-  }
-
-  public SqlNode parseAndValidate(SqlValidator validator, String sql) {
-    if (validator == null) {
-      validator = getValidator();
-    }
-    SqlNode sqlNode;
-    try {
-      sqlNode = parseQuery(sql);
-    } catch (Throwable e) {
-      throw new RuntimeException("Error while parsing query: " + sql, e);
-    }
-    return validator.validate(sqlNode);
-  }
-
-  public SqlNode parseQuery(String sql) throws SqlParseException {
-    SqlParser parser = factory.createParser(sql);
-    return parser.parseQuery();
-  }
-
-  public void checkColumnType(String sql, String expected) {
-    RelDataType actualType = getColumnType(sql);
-    String actual = SqlTests.getTypeString(actualType);
-    assertEquals(expected, actual);
-  }
-
-  public void checkFieldOrigin(String sql, String fieldOriginList) {
-    SqlValidator validator = getValidator();
-    SqlNode n = parseAndValidate(validator, sql);
-    final List<List<String>> list = validator.getFieldOrigins(n);
-    final StringBuilder buf = new StringBuilder("{");
-    int i = 0;
-    for (List<String> strings : list) {
-      if (i++ > 0) {
-        buf.append(", ");
-      }
-      if (strings == null) {
-        buf.append("null");
-      } else {
-        int j = 0;
-        for (String s : strings) {
-          if (j++ > 0) {
-            buf.append('.');
-          }
-          buf.append(s);
-        }
-      }
-    }
-    buf.append("}");
-    assertEquals(fieldOriginList, buf.toString());
-  }
-
-  public void checkResultType(String sql, String expected) {
-    RelDataType actualType = getResultType(sql);
-    String actual = SqlTests.getTypeString(actualType);
-    assertEquals(expected, actual);
-  }
-
-  public void checkIntervalConv(String sql, String expected) {
-    SqlValidator validator = getValidator();
-    final SqlCall n = (SqlCall) parseAndValidate(validator, sql);
-
-    SqlNode node = null;
-    for (int i = 0; i < n.operandCount(); i++) {
-      node = stripAs(n.operand(i));
-      if (node instanceof SqlCall) {
-        node = ((SqlCall) node).operand(0);
-        break;
-      }
-    }
-
-    assertNotNull(node);
-    SqlIntervalLiteral intervalLiteral = (SqlIntervalLiteral) node;
-    SqlIntervalLiteral.IntervalValue interval =
-        (SqlIntervalLiteral.IntervalValue) intervalLiteral.getValue();
-    long l =
-        interval.getIntervalQualifier().isYearMonth()
-            ? SqlParserUtil.intervalToMonths(interval)
-            : SqlParserUtil.intervalToMillis(interval);
-    String actual = l + "";
-    assertEquals(expected, actual);
-  }
-
-  public void checkType(String expression, String type) {
-    for (String sql : buildQueries(expression)) {
-      checkColumnType(sql, type);
-    }
-  }
-
-  public void checkCollation(
-      String expression,
-      String expectedCollationName,
-      SqlCollation.Coercibility expectedCoercibility) {
-    for (String sql : buildQueries(expression)) {
-      RelDataType actualType = getColumnType(sql);
-      SqlCollation collation = actualType.getCollation();
-
-      assertEquals(
-          expectedCollationName, collation.getCollationName());
-      assertEquals(expectedCoercibility, collation.getCoercibility());
-    }
-  }
-
-  public void checkCharset(
-      String expression,
-      Charset expectedCharset) {
-    for (String sql : buildQueries(expression)) {
-      RelDataType actualType = getColumnType(sql);
-      Charset actualCharset = actualType.getCharset();
-
-      if (!expectedCharset.equals(actualCharset)) {
-        fail("\n"
-            + "Expected=" + expectedCharset.name() + "\n"
-            + "  actual=" + actualCharset.name());
-      }
-    }
-  }
-
-  public SqlTesterImpl withQuoting(Quoting quoting) {
-    return with("quoting", quoting);
-  }
-
-  public SqlTester withQuotedCasing(Casing casing) {
-    return with("quotedCasing", casing);
-  }
-
-  public SqlTester withUnquotedCasing(Casing casing) {
-    return with("unquotedCasing", casing);
-  }
-
-  public SqlTester withCaseSensitive(boolean sensitive) {
-    return with("caseSensitive", sensitive);
-  }
-
-  public SqlTester withLex(Lex lex) {
-    return withQuoting(lex.quoting)
-        .withCaseSensitive(lex.caseSensitive)
-        .withQuotedCasing(lex.quotedCasing)
-        .withUnquotedCasing(lex.unquotedCasing);
-  }
-
-  public SqlTesterImpl withConformance(SqlConformance conformance) {
-    if (conformance == null) {
-      conformance = SqlConformanceEnum.DEFAULT;
-    }
-    final SqlTesterImpl tester = with("conformance", conformance);
-    if (conformance instanceof SqlConformanceEnum) {
-      return tester
-          .withConnectionFactory(
-              CalciteAssert.EMPTY_CONNECTION_FACTORY
-                  .with(CalciteConnectionProperty.CONFORMANCE, conformance));
-    } else {
-      return tester;
-    }
-  }
-
-  public SqlTester withOperatorTable(SqlOperatorTable operatorTable) {
-    return with("operatorTable", operatorTable);
-  }
-
-  public SqlTesterImpl withConnectionFactory(
-      CalciteAssert.ConnectionFactory connectionFactory) {
-    return with("connectionFactory", connectionFactory);
-  }
-
-  protected SqlTesterImpl with(final String name, final Object value) {
-    return new SqlTesterImpl(factory.with(name, value));
-  }
-
-  // SqlTester methods
-
-  public void setFor(
-      SqlOperator operator,
-      VmName... unimplementedVmNames) {
-    // do nothing
-  }
-
-  public void checkAgg(
-      String expr,
-      String[] inputValues,
-      Object result,
-      double delta) {
-    String query =
-        SqlTests.generateAggQuery(expr, inputValues);
-    check(query, SqlTests.ANY_TYPE_CHECKER, result, delta);
-  }
-
-  public void checkWinAgg(
-      String expr,
-      String[] inputValues,
-      String windowSpec,
-      String type,
-      Object result,
-      double delta) {
-    String query =
-        SqlTests.generateWinAggQuery(
-            expr, windowSpec, inputValues);
-    check(query, SqlTests.ANY_TYPE_CHECKER, result, delta);
-  }
-
-  public void checkScalar(
-      String expression,
-      Object result,
-      String resultType) {
-    checkType(expression, resultType);
-    for (String sql : buildQueries(expression)) {
-      check(sql, SqlTests.ANY_TYPE_CHECKER, result, 0);
-    }
-  }
-
-  public void checkScalarExact(
-      String expression,
-      String result) {
-    for (String sql : buildQueries(expression)) {
-      check(sql, SqlTests.INTEGER_TYPE_CHECKER, result, 0);
-    }
-  }
-
-  public void checkScalarExact(
-      String expression,
-      String expectedType,
-      String result) {
-    for (String sql : buildQueries(expression)) {
-      TypeChecker typeChecker =
-          new SqlTests.StringTypeChecker(expectedType);
-      check(sql, typeChecker, result, 0);
-    }
-  }
-
-  public void checkScalarApprox(
-      String expression,
-      String expectedType,
-      double expectedResult,
-      double delta) {
-    for (String sql : buildQueries(expression)) {
-      TypeChecker typeChecker =
-          new SqlTests.StringTypeChecker(expectedType);
-      check(sql, typeChecker, expectedResult, delta);
-    }
-  }
-
-  public void checkBoolean(
-      String expression,
-      Boolean result) {
-    for (String sql : buildQueries(expression)) {
-      if (null == result) {
-        checkNull(expression);
-      } else {
-        check(
-            sql,
-            SqlTests.BOOLEAN_TYPE_CHECKER,
-            result.toString(),
-            0);
-      }
-    }
-  }
-
-  public void checkString(
-      String expression,
-      String result,
-      String expectedType) {
-    for (String sql : buildQueries(expression)) {
-      TypeChecker typeChecker =
-          new SqlTests.StringTypeChecker(expectedType);
-      check(sql, typeChecker, result, 0);
-    }
-  }
-
-  public void checkNull(String expression) {
-    for (String sql : buildQueries(expression)) {
-      check(sql, SqlTests.ANY_TYPE_CHECKER, null, 0);
-    }
-  }
-
-  public final void check(
-      String query,
-      TypeChecker typeChecker,
-      Object result,
-      double delta) {
-    check(query, typeChecker, SqlTests.ANY_PARAMETER_CHECKER,
-        SqlTests.createChecker(result, delta));
-  }
-
-  public void check(String query, TypeChecker typeChecker,
-      ParameterChecker parameterChecker, ResultChecker resultChecker) {
-    // This implementation does NOT check the result!
-    // All it does is check the return type.
-
-    if (typeChecker == null) {
-      // Parse and validate. There should be no errors.
-      Util.discard(getResultType(query));
-    } else {
-      // Parse and validate. There should be no errors.
-      // There must be 1 column. Get its type.
-      RelDataType actualType = getColumnType(query);
-
-      // Check result type.
-      typeChecker.checkType(actualType);
-    }
-
-    SqlValidator validator = getValidator();
-    SqlNode n = parseAndValidate(validator, query);
-    final RelDataType parameterRowType = validator.getParameterRowType(n);
-    parameterChecker.checkParameters(parameterRowType);
-  }
-
-  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,
-      String expectedRewrite) {
-    SqlNode rewrittenNode = parseAndValidate(validator, query);
-    String actualRewrite =
-        rewrittenNode.toSqlString(AnsiSqlDialect.DEFAULT, false).getSql();
-    TestUtil.assertEqualsVerbose(expectedRewrite, Util.toLinux(actualRewrite));
-  }
-
-  public void checkFails(
-      String expression,
-      String expectedError,
-      boolean runtime) {
-    if (runtime) {
-      // We need to test that the expression fails at runtime.
-      // Ironically, that means that it must succeed at prepare time.
-      SqlValidator validator = getValidator();
-      final String sql = buildQuery(expression);
-      SqlNode n = parseAndValidate(validator, sql);
-      assertNotNull(n);
-    } else {
-      checkQueryFails(buildQuery(expression), expectedError);
-    }
-  }
-
-  public void checkQueryFails(String sql, String expectedError) {
-    assertExceptionIsThrown(sql, expectedError);
-  }
-
-  public void checkQuery(String sql) {
-    assertExceptionIsThrown(sql, null);
-  }
-
-  public SqlMonotonicity getMonotonicity(String sql) {
-    final SqlValidator validator = getValidator();
-    final SqlNode node = parseAndValidate(validator, sql);
-    final SqlSelect select = (SqlSelect) node;
-    final SqlNode selectItem0 = select.getSelectList().get(0);
-    final SqlValidatorScope scope = validator.getSelectScope(select);
-    return selectItem0.getMonotonicity(scope);
-  }
-
-  public static String buildQuery(String expression) {
-    return "values (" + expression + ")";
-  }
-
-  public static String buildQueryAgg(String expression) {
-    return "select " + expression + " from (values (1)) as t(x) group by x";
-  }
-
-  /**
-   * Builds a query that extracts all literals as columns in an underlying
-   * select.
-   *
-   * <p>For example,</p>
-   *
-   * <blockquote>{@code 1 < 5}</blockquote>
-   *
-   * <p>becomes</p>
-   *
-   * <blockquote>{@code SELECT p0 < p1
-   * FROM (VALUES (1, 5)) AS t(p0, p1)}</blockquote>
-   *
-   * <p>Null literals don't have enough type information to be extracted.
-   * We push down {@code CAST(NULL AS type)} but raw nulls such as
-   * {@code CASE 1 WHEN 2 THEN 'a' ELSE NULL END} are left as is.</p>
-   *
-   * @param expression Scalar expression
-   * @return Query that evaluates a scalar expression
-   */
-  private String buildQuery2(String expression) {
-    // "values (1 < 5)"
-    // becomes
-    // "select p0 < p1 from (values (1, 5)) as t(p0, p1)"
-    SqlNode x;
-    final String sql = "values (" + expression + ")";
-    try {
-      x = parseQuery(sql);
-    } catch (SqlParseException e) {
-      throw new RuntimeException(e);
-    }
-    final Collection<SqlNode> literalSet = new LinkedHashSet<>();
-    x.accept(
-        new SqlShuttle() {
-          private final List<SqlOperator> ops =
-              ImmutableList.of(
-                  SqlStdOperatorTable.LITERAL_CHAIN,
-                  SqlStdOperatorTable.LOCALTIME,
-                  SqlStdOperatorTable.LOCALTIMESTAMP,
-                  SqlStdOperatorTable.CURRENT_TIME,
-                  SqlStdOperatorTable.CURRENT_TIMESTAMP);
-
-          @Override public SqlNode visit(SqlLiteral literal) {
-            if (!isNull(literal)
-                && literal.getTypeName() != SqlTypeName.SYMBOL) {
-              literalSet.add(literal);
-            }
-            return literal;
-          }
-
-          @Override public SqlNode visit(SqlCall call) {
-            final SqlOperator operator = call.getOperator();
-            if (operator == SqlStdOperatorTable.CAST
-                && isNull(call.operand(0))) {
-              literalSet.add(call);
-              return call;
-            } else if (ops.contains(operator)) {
-              // "Argument to function 'LOCALTIME' must be a
-              // literal"
-              return call;
-            } else {
-              return super.visit(call);
-            }
-          }
-
-          private boolean isNull(SqlNode sqlNode) {
-            return sqlNode instanceof SqlLiteral
-                && ((SqlLiteral) sqlNode).getTypeName()
-                == SqlTypeName.NULL;
-          }
-        });
-    final List<SqlNode> nodes = new ArrayList<>(literalSet);
-    nodes.sort((o1, o2) -> {
-      final SqlParserPos pos0 = o1.getParserPosition();
-      final SqlParserPos pos1 = o2.getParserPosition();
-      int c = -Utilities.compare(pos0.getLineNum(), pos1.getLineNum());
-      if (c != 0) {
-        return c;
-      }
-      return -Utilities.compare(pos0.getColumnNum(), pos1.getColumnNum());
-    });
-    String sql2 = sql;
-    final List<Pair<String, String>> values = new ArrayList<>();
-    int p = 0;
-    for (SqlNode literal : nodes) {
-      final SqlParserPos pos = literal.getParserPosition();
-      final int start =
-          SqlParserUtil.lineColToIndex(
-              sql, pos.getLineNum(), pos.getColumnNum());
-      final int end =
-          SqlParserUtil.lineColToIndex(
-              sql,
-              pos.getEndLineNum(),
-              pos.getEndColumnNum()) + 1;
-      String param = "p" + (p++);
-      values.add(Pair.of(sql2.substring(start, end), param));
-      sql2 = sql2.substring(0, start)
-          + param
-          + sql2.substring(end);
-    }
-    if (values.isEmpty()) {
-      values.add(Pair.of("1", "p0"));
-    }
-    return "select "
-        + sql2.substring("values (".length(), sql2.length() - 1)
-        + " from (values ("
-        + Util.commaList(Pair.left(values))
-        + ")) as t("
-        + Util.commaList(Pair.right(values))
-        + ")";
-  }
-
-  /**
-   * Converts a scalar expression into a list of SQL queries that
-   * evaluate it.
-   *
-   * @param expression Scalar expression
-   * @return List of queries that evaluate an expression
-   */
-  private Iterable<String> buildQueries(final String expression) {
-    // Why an explicit iterable rather than a list? If there is
-    // a syntax error in the expression, the calling code discovers it
-    // before we try to parse it to do substitutions on the parse tree.
-    return () -> new Iterator<String>() {
-      int i = 0;
-
-      public void remove() {
-        throw new UnsupportedOperationException();
-      }
-
-      public String next() {
-        switch (i++) {
-        case 0:
-          return buildQuery(expression);
-        case 1:
-          return buildQuery2(expression);
-        default:
-          throw new NoSuchElementException();
-        }
-      }
-
-      public boolean hasNext() {
-        return i < 2;
-      }
-    };
-  }
-
-}
-
-// End SqlTesterImpl.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/c69e1bcb/core/src/test/java/org/apache/calcite/sql/test/SqlTests.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlTests.java 
b/core/src/test/java/org/apache/calcite/sql/test/SqlTests.java
index e0c4bb1..22e3d95 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlTests.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlTests.java
@@ -18,13 +18,19 @@ package org.apache.calcite.sql.test;
 
 import org.apache.calcite.avatica.ColumnMetaData;
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.runtime.CalciteContextException;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.parser.SqlParserUtil;
 import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.TestUtil;
+import org.apache.calcite.util.Util;
 
 import java.sql.ResultSet;
 import java.sql.Types;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.Set;
+import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import static org.apache.calcite.sql.test.SqlTester.ParameterChecker;
@@ -32,6 +38,7 @@ import static 
org.apache.calcite.sql.test.SqlTester.ResultChecker;
 import static org.apache.calcite.sql.test.SqlTester.TypeChecker;
 
 import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
@@ -62,6 +69,24 @@ public abstract class SqlTests {
   };
 
   /**
+   * Checker that allows any result.
+   */
+  public static final ResultChecker ANY_RESULT_CHECKER = result -> {
+    while (true) {
+      if (!result.next()) {
+        break;
+      }
+    }
+  };
+
+  private static final Pattern LINE_COL_PATTERN =
+      Pattern.compile("At line ([0-9]+), column ([0-9]+)");
+
+  private static final Pattern LINE_COL_TWICE_PATTERN =
+      Pattern.compile(
+          "(?s)From line ([0-9]+), column ([0-9]+) to line ([0-9]+), column 
([0-9]+): (.*)");
+
+  /**
    * Helper function to get the string representation of a RelDataType
    * (include precision/scale but no charset or collation)
    *
@@ -265,6 +290,199 @@ public abstract class SqlTests {
     }
   }
 
+  /**
+   * Checks whether an exception matches the expected pattern. If
+   * <code>sap</code> contains an error location, checks this too.
+   *
+   * @param ex                 Exception thrown
+   * @param expectedMsgPattern Expected pattern
+   * @param sap                Query and (optional) position in query
+   * @param stage              Query processing stage
+   */
+  public static void checkEx(Throwable ex,
+      String expectedMsgPattern,
+      SqlParserUtil.StringAndPos sap,
+      Stage stage) {
+    if (null == ex) {
+      if (expectedMsgPattern == null) {
+        // No error expected, and no error happened.
+        return;
+      } else {
+        throw new AssertionError("Expected query to throw exception, "
+            + "but it did not; query [" + sap.sql
+            + "]; expected [" + expectedMsgPattern + "]");
+      }
+    }
+    Throwable actualException = ex;
+    String actualMessage = actualException.getMessage();
+    int actualLine = -1;
+    int actualColumn = -1;
+    int actualEndLine = 100;
+    int actualEndColumn = 99;
+
+    // Search for an CalciteContextException somewhere in the stack.
+    CalciteContextException ece = null;
+    for (Throwable x = ex; x != null; x = x.getCause()) {
+      if (x instanceof CalciteContextException) {
+        ece = (CalciteContextException) x;
+        break;
+      }
+      if (x.getCause() == x) {
+        break;
+      }
+    }
+
+    // Search for a SqlParseException -- with its position set -- somewhere
+    // in the stack.
+    SqlParseException spe = null;
+    for (Throwable x = ex; x != null; x = x.getCause()) {
+      if ((x instanceof SqlParseException)
+          && (((SqlParseException) x).getPos() != null)) {
+        spe = (SqlParseException) x;
+        break;
+      }
+      if (x.getCause() == x) {
+        break;
+      }
+    }
+
+    if (ece != null) {
+      actualLine = ece.getPosLine();
+      actualColumn = ece.getPosColumn();
+      actualEndLine = ece.getEndPosLine();
+      actualEndColumn = ece.getEndPosColumn();
+      if (ece.getCause() != null) {
+        actualException = ece.getCause();
+        actualMessage = actualException.getMessage();
+      }
+    } else if (spe != null) {
+      actualLine = spe.getPos().getLineNum();
+      actualColumn = spe.getPos().getColumnNum();
+      actualEndLine = spe.getPos().getEndLineNum();
+      actualEndColumn = spe.getPos().getEndColumnNum();
+      if (spe.getCause() != null) {
+        actualException = spe.getCause();
+        actualMessage = actualException.getMessage();
+      }
+    } else {
+      final String message = ex.getMessage();
+      if (message != null) {
+        Matcher matcher = LINE_COL_TWICE_PATTERN.matcher(message);
+        if (matcher.matches()) {
+          actualLine = Integer.parseInt(matcher.group(1));
+          actualColumn = Integer.parseInt(matcher.group(2));
+          actualEndLine = Integer.parseInt(matcher.group(3));
+          actualEndColumn = Integer.parseInt(matcher.group(4));
+          actualMessage = matcher.group(5);
+        } else {
+          matcher = LINE_COL_PATTERN.matcher(message);
+          if (matcher.matches()) {
+            actualLine = Integer.parseInt(matcher.group(1));
+            actualColumn = Integer.parseInt(matcher.group(2));
+          } else {
+            if (expectedMsgPattern != null
+                && actualMessage.matches(expectedMsgPattern)) {
+              return;
+            }
+          }
+        }
+      }
+    }
+
+    if (null == expectedMsgPattern) {
+      actualException.printStackTrace();
+      fail(stage.componentName + " 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) {
+        throw new AssertionError("Expected error to have position,"
+            + " but actual error did not: "
+            + " actual pos [line " + actualLine
+            + " col " + actualColumn
+            + " thru line " + actualEndLine + " col "
+            + actualEndColumn + "]", actualException);
+      }
+      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(stage.componentName + " 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(stage.componentName + " threw expected "
+          + "exception [" + actualMessage
+          + "];\nbut at pos [line " + actualLine
+          + " col " + actualColumn
+          + " thru line " + actualEndLine
+          + " col " + actualEndColumn
+          + "];\nsql [" + sqlWithCarets + "]");
+    }
+  }
+
+  /** Stage of query processing */
+  public enum Stage {
+    PARSE("Parser"),
+    VALIDATE("Validator"),
+    RUNTIME("Executor");
+
+    public final String componentName;
+
+    Stage(String componentName) {
+      this.componentName = componentName;
+    }
+  }
+
   //~ Inner Classes ----------------------------------------------------------
 
   /**
@@ -279,9 +497,7 @@ public abstract class SqlTests {
     }
 
     public void checkType(RelDataType type) {
-      assertEquals(
-          typeName.toString(),
-          type.toString());
+      assertThat(type.toString(), is(typeName.toString()));
     }
   }
 
@@ -308,7 +524,7 @@ public abstract class SqlTests {
 
     public void checkType(RelDataType type) {
       String actual = getTypeString(type);
-      assertEquals(expected, actual);
+      assertThat(actual, is(expected));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/c69e1bcb/core/src/test/java/org/apache/calcite/sql/test/SqlValidatorTester.java
----------------------------------------------------------------------
diff --git 
a/core/src/test/java/org/apache/calcite/sql/test/SqlValidatorTester.java 
b/core/src/test/java/org/apache/calcite/sql/test/SqlValidatorTester.java
new file mode 100644
index 0000000..e38d0ed
--- /dev/null
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlValidatorTester.java
@@ -0,0 +1,35 @@
+/*
+ * 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.calcite.sql.test;
+
+import org.apache.calcite.sql.validate.SqlValidator;
+
+/**
+ * Tester of {@link SqlValidator}.
+ */
+public class SqlValidatorTester extends AbstractSqlTester {
+
+  public SqlValidatorTester(SqlTestFactory factory) {
+    super(factory);
+  }
+
+  @Override protected SqlTester with(SqlTestFactory factory) {
+    return new SqlValidatorTester(factory);
+  }
+}
+
+// End SqlValidatorTester.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/c69e1bcb/core/src/test/java/org/apache/calcite/sql/validate/SqlValidatorUtilTest.java
----------------------------------------------------------------------
diff --git 
a/core/src/test/java/org/apache/calcite/sql/validate/SqlValidatorUtilTest.java 
b/core/src/test/java/org/apache/calcite/sql/validate/SqlValidatorUtilTest.java
index 0405308..71f6786 100644
--- 
a/core/src/test/java/org/apache/calcite/sql/validate/SqlValidatorUtilTest.java
+++ 
b/core/src/test/java/org/apache/calcite/sql/validate/SqlValidatorUtilTest.java
@@ -21,7 +21,8 @@ import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.test.SqlTestFactory;
-import org.apache.calcite.sql.test.SqlTesterImpl;
+import org.apache.calcite.sql.test.SqlTester;
+import org.apache.calcite.sql.test.SqlValidatorTester;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
@@ -124,8 +125,8 @@ public class SqlValidatorUtilTest {
     final List<SqlNode> newList = new ArrayList<>(2);
     newList.add(new SqlIdentifier(Arrays.asList("f0", "c0"), 
SqlParserPos.ZERO));
     newList.add(new SqlIdentifier(Arrays.asList("f0", "c0"), 
SqlParserPos.ZERO));
-    final SqlTesterImpl tester =
-        new SqlTesterImpl(SqlTestFactory.INSTANCE);
+    final SqlTester tester =
+        new SqlValidatorTester(SqlTestFactory.INSTANCE);
     final SqlValidatorImpl validator =
         (SqlValidatorImpl) tester.getValidator();
     try {

http://git-wip-us.apache.org/repos/asf/calcite/blob/c69e1bcb/core/src/test/java/org/apache/calcite/test/SqlTestGen.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlTestGen.java 
b/core/src/test/java/org/apache/calcite/test/SqlTestGen.java
index ceac204..9a4e263 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlTestGen.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlTestGen.java
@@ -19,7 +19,7 @@ package org.apache.calcite.test;
 import org.apache.calcite.sql.SqlCollation;
 import org.apache.calcite.sql.test.SqlTestFactory;
 import org.apache.calcite.sql.test.SqlTester;
-import org.apache.calcite.sql.test.SqlTesterImpl;
+import org.apache.calcite.sql.test.SqlValidatorTester;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.util.BarfingInvocationHandler;
 import org.apache.calcite.util.Util;
@@ -102,7 +102,7 @@ public class SqlTestGen {
     }
 
     public SqlTester getTester() {
-      return new SqlTesterImpl(SPOOLER_VALIDATOR) {
+      return new SqlValidatorTester(SPOOLER_VALIDATOR) {
         public void assertExceptionIsThrown(
             String sql,
             String expectedMsgPattern) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/c69e1bcb/core/src/test/java/org/apache/calcite/test/SqlValidatorDynamicTest.java
----------------------------------------------------------------------
diff --git 
a/core/src/test/java/org/apache/calcite/test/SqlValidatorDynamicTest.java 
b/core/src/test/java/org/apache/calcite/test/SqlValidatorDynamicTest.java
index 1863157..1bca24b 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorDynamicTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorDynamicTest.java
@@ -18,7 +18,7 @@ package org.apache.calcite.test;
 
 import org.apache.calcite.sql.test.SqlTestFactory;
 import org.apache.calcite.sql.test.SqlTester;
-import org.apache.calcite.sql.test.SqlTesterImpl;
+import org.apache.calcite.sql.test.SqlValidatorTester;
 import org.apache.calcite.test.catalog.MockCatalogReaderDynamic;
 
 import org.junit.BeforeClass;
@@ -96,7 +96,7 @@ public class SqlValidatorDynamicTest extends 
SqlValidatorTestCase {
   @Override public SqlTester getTester() {
     // Dymamic schema should not be reused since it is mutable, so
     // we create new SqlTestFactory for each test
-    return new SqlTesterImpl(SqlTestFactory.INSTANCE
+    return new SqlValidatorTester(SqlTestFactory.INSTANCE
         .withCatalogReader(MockCatalogReaderDynamic::new));
   }
 //~ Methods ----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/calcite/blob/c69e1bcb/core/src/test/java/org/apache/calcite/test/SqlValidatorFeatureTest.java
----------------------------------------------------------------------
diff --git 
a/core/src/test/java/org/apache/calcite/test/SqlValidatorFeatureTest.java 
b/core/src/test/java/org/apache/calcite/test/SqlValidatorFeatureTest.java
index 48cbf0f..b44318f 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorFeatureTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorFeatureTest.java
@@ -24,7 +24,7 @@ import org.apache.calcite.sql.SqlOperatorTable;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.test.SqlTestFactory;
 import org.apache.calcite.sql.test.SqlTester;
-import org.apache.calcite.sql.test.SqlTesterImpl;
+import org.apache.calcite.sql.test.SqlValidatorTester;
 import org.apache.calcite.sql.validate.SqlConformance;
 import org.apache.calcite.sql.validate.SqlValidatorCatalogReader;
 import org.apache.calcite.sql.validate.SqlValidatorImpl;
@@ -55,7 +55,7 @@ public class SqlValidatorFeatureTest extends 
SqlValidatorTestCase {
   //~ Methods ----------------------------------------------------------------
 
   @Override public SqlTester getTester() {
-    return new 
SqlTesterImpl(SqlTestFactory.INSTANCE.withValidator(FeatureValidator::new));
+    return new 
SqlValidatorTester(SqlTestFactory.INSTANCE.withValidator(FeatureValidator::new));
   }
 
   @Test public void testDistinct() {

Reply via email to