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 2017/10/02 21:00:01 UTC

[01/15] calcite git commit: [CALCITE-1976] linq4j: support List and Map literals

Repository: calcite
Updated Branches:
  refs/heads/master 980625481 -> 914b5cfbf


[CALCITE-1976] linq4j: support List and Map literals

Lists are printed as Collections.EMPTY_LIST or Arrays.asList();
maps are printed as common.collect.ImmutableMap of(..) or
builder().add(..).build().

Close apache/calcite#534


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

Branch: refs/heads/master
Commit: 41687f3bd7316acd931e81c7d2b061681fefc30b
Parents: 9806254
Author: Vladimir Sitnikov <si...@gmail.com>
Authored: Fri Sep 8 14:57:18 2017 +0300
Committer: Julian Hyde <jh...@apache.org>
Committed: Mon Oct 2 11:13:40 2017 -0700

----------------------------------------------------------------------
 .../calcite/linq4j/tree/ConstantExpression.java | 40 ++++++++++++
 .../calcite/linq4j/test/ExpressionTest.java     | 65 ++++++++++++++++++++
 2 files changed, 105 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/41687f3b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConstantExpression.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConstantExpression.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConstantExpression.java
index 3fc112b..3437a42 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConstantExpression.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ConstantExpression.java
@@ -26,6 +26,7 @@ import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.util.Arrays;
 import java.util.List;
+import java.util.Map;
 import java.util.Objects;
 
 /**
@@ -175,6 +176,17 @@ public class ConstantExpression extends Expression {
       list(writer, Primitive.asList(value), "[] {\n", ",\n", "}");
       return writer;
     }
+    if (value instanceof List) {
+      if (((List) value).isEmpty()) {
+        writer.append("java.util.Collections.EMPTY_LIST");
+        return writer;
+      }
+      list(writer, (List) value, "java.util.Arrays.asList(", ",\n", ")");
+      return writer;
+    }
+    if (value instanceof Map) {
+      return writeMap(writer, (Map) value);
+    }
     Constructor constructor = matchingConstructor(value);
     if (constructor != null) {
       writer.append("new ").append(value.getClass());
@@ -208,6 +220,34 @@ public class ConstantExpression extends Expression {
     writer.end(end);
   }
 
+  private static ExpressionWriter writeMap(ExpressionWriter writer, Map map) {
+    writer.append("com.google.common.collect.ImmutableMap.");
+    if (map.isEmpty()) {
+      return writer.append("of()");
+    }
+    if (map.size() < 5) {
+      return map(writer, map, "of(", ",\n", ")");
+    }
+    return map(writer, map, "builder().put(", ")\n.put(", ").build()");
+  }
+
+  private static ExpressionWriter map(ExpressionWriter writer, Map map,
+      String begin, String entrySep, String end) {
+    writer.append(begin);
+    boolean comma = false;
+    for (Object o : map.entrySet()) {
+      Map.Entry entry = (Map.Entry) o;
+      if (comma) {
+        writer.append(entrySep).indent();
+      }
+      write(writer, entry.getKey(), null);
+      writer.append(", ");
+      write(writer, entry.getValue(), null);
+      comma = true;
+    }
+    return writer.append(end);
+  }
+
   private static Constructor matchingConstructor(Object value) {
     final Field[] fields = value.getClass().getFields();
     for (Constructor<?> constructor : value.getClass().getConstructors()) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/41687f3b/linq4j/src/test/java/org/apache/calcite/linq4j/test/ExpressionTest.java
----------------------------------------------------------------------
diff --git a/linq4j/src/test/java/org/apache/calcite/linq4j/test/ExpressionTest.java b/linq4j/src/test/java/org/apache/calcite/linq4j/test/ExpressionTest.java
index a8c1053..9d094b3 100644
--- a/linq4j/src/test/java/org/apache/calcite/linq4j/test/ExpressionTest.java
+++ b/linq4j/src/test/java/org/apache/calcite/linq4j/test/ExpressionTest.java
@@ -34,6 +34,8 @@ import org.apache.calcite.linq4j.tree.ParameterExpression;
 import org.apache.calcite.linq4j.tree.Shuttle;
 import org.apache.calcite.linq4j.tree.Types;
 
+import com.google.common.collect.ImmutableMap;
+
 import org.junit.Test;
 
 import java.lang.reflect.Modifier;
@@ -44,7 +46,10 @@ import java.util.AbstractList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.TreeSet;
 
 import static org.hamcrest.core.Is.is;
@@ -1199,6 +1204,66 @@ public class ExpressionTest {
         Expressions.toString(builder.toBlock()));
   }
 
+  @Test public void testEmptyListLiteral() throws Exception {
+    assertEquals("java.util.Collections.EMPTY_LIST",
+        Expressions.toString(Expressions.constant(Arrays.asList())));
+  }
+
+  @Test public void testEneElementListLiteral() throws Exception {
+    assertEquals("java.util.Arrays.asList(1)",
+        Expressions.toString(Expressions.constant(Arrays.asList(1))));
+  }
+
+  @Test public void testTwoElementListLiteral() throws Exception {
+    assertEquals("java.util.Arrays.asList(1,\n"
+            + "  2)",
+        Expressions.toString(Expressions.constant(Arrays.asList(1, 2))));
+  }
+
+  @Test public void testNestedListsLiteral() throws Exception {
+    assertEquals("java.util.Arrays.asList(java.util.Arrays.asList(1,\n"
+            + "    2),\n"
+            + "  java.util.Arrays.asList(3,\n"
+            + "    4))",
+        Expressions.toString(
+            Expressions.constant(
+                Arrays.asList(Arrays.asList(1, 2), Arrays.asList(3, 4)))));
+  }
+
+  @Test public void testEmptyMapLiteral() throws Exception {
+    assertEquals("com.google.common.collect.ImmutableMap.of()",
+        Expressions.toString(Expressions.constant(new HashMap())));
+  }
+
+  @Test public void testOneElementMapLiteral() throws Exception {
+    assertEquals("com.google.common.collect.ImmutableMap.of(\"abc\", 42)",
+        Expressions.toString(Expressions.constant(Collections.singletonMap("abc", 42))));
+  }
+
+  @Test public void testTwoElementMapLiteral() throws Exception {
+    assertEquals("com.google.common.collect.ImmutableMap.of(\"abc\", 42,\n"
+            + "\"def\", 43)",
+        Expressions.toString(Expressions.constant(ImmutableMap.of("abc", 42, "def", 43))));
+  }
+
+  @Test public void testTenElementMapLiteral() throws Exception {
+    Map<String, String> map = new LinkedHashMap<>(); // for consistent output
+    for (int i = 0; i < 10; i++) {
+      map.put("key_" + i, "value_" + i);
+    }
+    assertEquals("com.google.common.collect.ImmutableMap.builder().put(\"key_0\", \"value_0\")\n"
+            + ".put(\"key_1\", \"value_1\")\n"
+            + ".put(\"key_2\", \"value_2\")\n"
+            + ".put(\"key_3\", \"value_3\")\n"
+            + ".put(\"key_4\", \"value_4\")\n"
+            + ".put(\"key_5\", \"value_5\")\n"
+            + ".put(\"key_6\", \"value_6\")\n"
+            + ".put(\"key_7\", \"value_7\")\n"
+            + ".put(\"key_8\", \"value_8\")\n"
+            + ".put(\"key_9\", \"value_9\").build()",
+        Expressions.toString(Expressions.constant(map)));
+  }
+
   /** An enum. */
   enum MyEnum {
     X,


[09/15] calcite git commit: [CALCITE-1987] Implement EXTRACT for JDBC (Pavel Gubin)

Posted by jh...@apache.org.
[CALCITE-1987] Implement EXTRACT for JDBC (Pavel Gubin)

Remove EXTRACT_DATE function (it's all just EXTRACTi now),
and revise Druid adapter.

Close apache/calcite#539


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

Branch: refs/heads/master
Commit: 796a28f9a9d49a16971135074f315d536eb929cf
Parents: 8441e79
Author: pavelgubin <pa...@contiamo.com>
Authored: Thu Sep 14 14:53:41 2017 +0200
Committer: Julian Hyde <jh...@apache.org>
Committed: Mon Oct 2 11:13:43 2017 -0700

----------------------------------------------------------------------
 .../calcite/adapter/enumerable/RexImpTable.java | 163 ++++++++++++++++++-
 .../calcite/sql/fun/SqlStdOperatorTable.java    |   5 -
 .../sql2rel/StandardConvertletTable.java        | 154 +-----------------
 .../calcite/rel/rules/DateRangeRulesTest.java   | 118 ++++++--------
 .../java/org/apache/calcite/test/JdbcTest.java  |  24 +++
 .../apache/calcite/test/SqlValidatorTest.java   |   1 -
 .../org/apache/calcite/test/RelOptRulesTest.xml |  20 +--
 .../org/apache/calcite/test/DruidAdapterIT.java |  64 ++++----
 .../calcite/test/DruidDateRangeRulesTest.java   |  24 +--
 9 files changed, 283 insertions(+), 290 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/796a28f9/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
index 0b3aa84..3f99182 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
@@ -125,7 +125,7 @@ import static org.apache.calcite.sql.fun.SqlStdOperatorTable.DIVIDE_INTEGER;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.ELEMENT;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.EQUALS;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.EXP;
-import static org.apache.calcite.sql.fun.SqlStdOperatorTable.EXTRACT_DATE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.EXTRACT;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.FIRST_VALUE;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.FLOOR;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.GREATER_THAN;
@@ -323,8 +323,8 @@ public class RexImpTable {
         new DatetimeArithmeticImplementor(), false);
     defineImplementor(MINUS_DATE, NullPolicy.STRICT,
         new DatetimeArithmeticImplementor(), false);
-    defineMethod(EXTRACT_DATE, BuiltInMethod.UNIX_DATE_EXTRACT.method,
-        NullPolicy.STRICT);
+    defineImplementor(EXTRACT, NullPolicy.STRICT,
+        new ExtractImplementor(), false);
     defineImplementor(FLOOR, NullPolicy.STRICT,
         new FloorImplementor(BuiltInMethod.FLOOR.method.getName(),
             BuiltInMethod.UNIX_TIMESTAMP_FLOOR.method,
@@ -1889,6 +1889,163 @@ public class RexImpTable {
     }
   }
 
+  /** Implementor for the {@code EXTRACT(unit FROM datetime)} function. */
+  private static class ExtractImplementor implements NotNullImplementor {
+    public Expression implement(RexToLixTranslator translator, RexCall call,
+        List<Expression> translatedOperands) {
+      final TimeUnitRange timeUnitRange =
+          (TimeUnitRange) ((ConstantExpression) translatedOperands.get(0)).value;
+      final TimeUnit unit = timeUnitRange.startUnit;
+      Expression operand = translatedOperands.get(1);
+      final SqlTypeName sqlTypeName =
+          call.operands.get(1).getType().getSqlTypeName();
+      switch (unit) {
+      case MILLENNIUM:
+      case CENTURY:
+      case YEAR:
+      case QUARTER:
+      case MONTH:
+      case DAY:
+      case DOW:
+      case DOY:
+      case WEEK:
+        switch (sqlTypeName) {
+        case INTERVAL_YEAR:
+        case INTERVAL_YEAR_MONTH:
+        case INTERVAL_MONTH:
+        case INTERVAL_DAY:
+        case INTERVAL_DAY_HOUR:
+        case INTERVAL_DAY_MINUTE:
+        case INTERVAL_DAY_SECOND:
+        case INTERVAL_HOUR:
+        case INTERVAL_HOUR_MINUTE:
+        case INTERVAL_HOUR_SECOND:
+        case INTERVAL_MINUTE:
+        case INTERVAL_MINUTE_SECOND:
+        case INTERVAL_SECOND:
+          break;
+        case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+          operand = Expressions.call(
+              BuiltInMethod.TIMESTAMP_WITH_LOCAL_TIME_ZONE_TO_TIMESTAMP.method,
+              operand,
+              Expressions.call(BuiltInMethod.TIME_ZONE.method, translator.getRoot()));
+          // fall through
+        case TIMESTAMP:
+          operand = Expressions.divide(operand,
+              Expressions.constant(TimeUnit.DAY.multiplier.longValue()));
+          // fall through
+        case DATE:
+          return Expressions.call(BuiltInMethod.UNIX_DATE_EXTRACT.method,
+              translatedOperands.get(0), operand);
+        default:
+          throw new AssertionError("unexpected " + sqlTypeName);
+        }
+        break;
+      case DECADE:
+        switch (sqlTypeName) {
+        case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+          operand = Expressions.call(
+              BuiltInMethod.TIMESTAMP_WITH_LOCAL_TIME_ZONE_TO_TIMESTAMP.method,
+              operand,
+              Expressions.call(BuiltInMethod.TIME_ZONE.method, translator.getRoot()));
+          // fall through
+        case TIMESTAMP:
+          operand = Expressions.divide(operand,
+              Expressions.constant(TimeUnit.DAY.multiplier.longValue()));
+          // fall through
+        case DATE:
+          operand = Expressions.call(
+              BuiltInMethod.UNIX_DATE_EXTRACT.method,
+              Expressions.constant(TimeUnitRange.YEAR), operand);
+          return Expressions.divide(operand,
+              Expressions.constant(
+                  unit.multiplier.divideToIntegralValue(TimeUnit.YEAR.multiplier)
+                      .longValue()));
+        }
+        break;
+      case EPOCH:
+        switch (sqlTypeName) {
+        case DATE:
+          // convert to milliseconds
+          operand = Expressions.multiply(operand,
+              Expressions.constant(TimeUnit.DAY.multiplier.longValue()));
+          // fall through
+        case TIMESTAMP:
+          // convert to seconds
+          return Expressions.divide(operand,
+              Expressions.constant(TimeUnit.SECOND.multiplier.longValue()));
+        case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+          operand = Expressions.call(
+              BuiltInMethod.TIMESTAMP_WITH_LOCAL_TIME_ZONE_TO_TIMESTAMP.method,
+              operand,
+              Expressions.call(BuiltInMethod.TIME_ZONE.method, translator.getRoot()));
+          return Expressions.divide(operand,
+              Expressions.constant(TimeUnit.SECOND.multiplier.longValue()));
+        case INTERVAL_YEAR:
+        case INTERVAL_YEAR_MONTH:
+        case INTERVAL_MONTH:
+        case INTERVAL_DAY:
+        case INTERVAL_DAY_HOUR:
+        case INTERVAL_DAY_MINUTE:
+        case INTERVAL_DAY_SECOND:
+        case INTERVAL_HOUR:
+        case INTERVAL_HOUR_MINUTE:
+        case INTERVAL_HOUR_SECOND:
+        case INTERVAL_MINUTE:
+        case INTERVAL_MINUTE_SECOND:
+        case INTERVAL_SECOND:
+          // no convertlet conversion, pass it as extract
+          throw new AssertionError("unexpected " + sqlTypeName);
+        }
+        break;
+      }
+
+      operand = mod(operand, getFactor(unit));
+      if (unit == TimeUnit.QUARTER) {
+        operand = Expressions.subtract(operand, Expressions.constant(1L));
+      }
+      operand = Expressions.divide(operand,
+          Expressions.constant(unit.multiplier.longValue()));
+      if (unit == TimeUnit.QUARTER) {
+        operand = Expressions.add(operand, Expressions.constant(1L));
+      }
+      return operand;
+    }
+
+  }
+
+  private static Expression mod(Expression operand, long factor) {
+    if (factor == 1L) {
+      return operand;
+    } else {
+      return Expressions.modulo(operand, Expressions.constant(factor));
+    }
+  }
+
+  private static long getFactor(TimeUnit unit) {
+    switch (unit) {
+    case DAY:
+      return 1L;
+    case HOUR:
+      return TimeUnit.DAY.multiplier.longValue();
+    case MINUTE:
+      return TimeUnit.HOUR.multiplier.longValue();
+    case SECOND:
+      return TimeUnit.MINUTE.multiplier.longValue();
+    case MONTH:
+      return TimeUnit.YEAR.multiplier.longValue();
+    case QUARTER:
+      return TimeUnit.YEAR.multiplier.longValue();
+    case YEAR:
+    case DECADE:
+    case CENTURY:
+    case MILLENNIUM:
+      return 1L;
+    default:
+      throw Util.unexpected(unit);
+    }
+  }
+
   /** Implementor for the SQL {@code CASE} operator. */
   private static class CaseImplementor implements CallImplementor {
     public Expression implement(RexToLixTranslator translator, RexCall call,

http://git-wip-us.apache.org/repos/asf/calcite/blob/796a28f9/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
index 7b6b771..c8adf5e 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
@@ -1235,11 +1235,6 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
         }
       };
 
-  /** Internal operator that extracts time periods (year, month, date) from a
-   * date in internal format (number of days since epoch). */
-  public static final SqlSpecialOperator EXTRACT_DATE =
-      new SqlSpecialOperator("EXTRACT_DATE", SqlKind.EXTRACT);
-
   //-------------------------------------------------------------
   //                   FUNCTIONS
   //-------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/calcite/blob/796a28f9/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java b/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
index 14bff94..4c45716 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
@@ -18,7 +18,6 @@ package org.apache.calcite.sql2rel;
 
 import org.apache.calcite.avatica.util.DateTimeUtils;
 import org.apache.calcite.avatica.util.TimeUnit;
-import org.apache.calcite.avatica.util.TimeUnitRange;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.type.RelDataType;
@@ -635,158 +634,7 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
       SqlRexContext cx,
       SqlExtractFunction op,
       SqlCall call) {
-    final RexBuilder rexBuilder = cx.getRexBuilder();
-    final List<SqlNode> operands = call.getOperandList();
-    final List<RexNode> exprs = convertExpressionList(cx, operands,
-        SqlOperandTypeChecker.Consistency.NONE);
-
-    // TODO: Will need to use decimal type for seconds with precision
-    RelDataType resType =
-        cx.getTypeFactory().createSqlType(SqlTypeName.BIGINT);
-    resType =
-        cx.getTypeFactory().createTypeWithNullability(
-            resType,
-            exprs.get(1).getType().isNullable());
-    RexNode res = rexBuilder.makeReinterpretCast(
-        resType, exprs.get(1), rexBuilder.makeLiteral(false));
-
-    final TimeUnit unit =
-        ((SqlIntervalQualifier) operands.get(0)).getStartUnit();
-    final SqlTypeName sqlTypeName = exprs.get(1).getType().getSqlTypeName();
-    switch (unit) {
-    case MILLENNIUM:
-    case CENTURY:
-    case YEAR:
-    case QUARTER:
-    case MONTH:
-    case DAY:
-    case DOW:
-    case DOY:
-    case WEEK:
-      switch (sqlTypeName) {
-      case INTERVAL_YEAR:
-      case INTERVAL_YEAR_MONTH:
-      case INTERVAL_MONTH:
-      case INTERVAL_DAY:
-      case INTERVAL_DAY_HOUR:
-      case INTERVAL_DAY_MINUTE:
-      case INTERVAL_DAY_SECOND:
-      case INTERVAL_HOUR:
-      case INTERVAL_HOUR_MINUTE:
-      case INTERVAL_HOUR_SECOND:
-      case INTERVAL_MINUTE:
-      case INTERVAL_MINUTE_SECOND:
-      case INTERVAL_SECOND:
-        break;
-      case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
-        RelDataType type =
-            cx.getTypeFactory().createSqlType(SqlTypeName.TIMESTAMP);
-        type = cx.getTypeFactory().createTypeWithNullability(
-            type,
-            exprs.get(1).getType().isNullable());
-        res = rexBuilder.makeCast(type, res);
-        // fall through
-      case TIMESTAMP:
-        res = divide(rexBuilder, res, TimeUnit.DAY.multiplier);
-        // fall through
-      case DATE:
-        return rexBuilder.makeCall(resType, SqlStdOperatorTable.EXTRACT_DATE,
-            ImmutableList.of(exprs.get(0), res));
-      default:
-        throw new AssertionError("unexpected " + sqlTypeName);
-      }
-      break;
-    case DECADE:
-      switch (sqlTypeName) {
-      case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
-        RelDataType type =
-            cx.getTypeFactory().createSqlType(SqlTypeName.TIMESTAMP);
-        type = cx.getTypeFactory().createTypeWithNullability(
-            type,
-            exprs.get(1).getType().isNullable());
-        res = rexBuilder.makeCast(type, res);
-        // fall through
-      case TIMESTAMP:
-        res = divide(rexBuilder, res, TimeUnit.DAY.multiplier);
-        // fall through
-      case DATE:
-        res = rexBuilder.makeCall(resType, SqlStdOperatorTable.EXTRACT_DATE,
-            ImmutableList.of(rexBuilder.makeFlag(TimeUnitRange.YEAR), res));
-        return divide(rexBuilder, res, unit.multiplier.divide(TimeUnit.YEAR.multiplier));
-      }
-      break;
-    case EPOCH:
-      switch (sqlTypeName) {
-      case DATE:
-        // convert to milliseconds
-        res = rexBuilder.makeCall(resType, SqlStdOperatorTable.MULTIPLY,
-            ImmutableList.of(res, rexBuilder.makeExactLiteral(TimeUnit.DAY.multiplier)));
-        // fall through
-      case TIMESTAMP:
-        // convert to seconds
-        return divide(rexBuilder, res, TimeUnit.SECOND.multiplier);
-      case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
-        RelDataType type =
-            cx.getTypeFactory().createSqlType(SqlTypeName.TIMESTAMP);
-        type = cx.getTypeFactory().createTypeWithNullability(
-            type,
-            exprs.get(1).getType().isNullable());
-        return divide(
-            rexBuilder,
-            rexBuilder.makeCast(type, res),
-            TimeUnit.SECOND.multiplier);
-      case INTERVAL_YEAR:
-      case INTERVAL_YEAR_MONTH:
-      case INTERVAL_MONTH:
-      case INTERVAL_DAY:
-      case INTERVAL_DAY_HOUR:
-      case INTERVAL_DAY_MINUTE:
-      case INTERVAL_DAY_SECOND:
-      case INTERVAL_HOUR:
-      case INTERVAL_HOUR_MINUTE:
-      case INTERVAL_HOUR_SECOND:
-      case INTERVAL_MINUTE:
-      case INTERVAL_MINUTE_SECOND:
-      case INTERVAL_SECOND:
-        // no convertlet conversion, pass it as extract
-        return convertFunction(cx, (SqlFunction) call.getOperator(), call);
-      }
-      break;
-    }
-
-    res = mod(rexBuilder, resType, res, getFactor(unit));
-    if (unit == TimeUnit.QUARTER) {
-      res = minus(rexBuilder, res, rexBuilder.makeExactLiteral(BigDecimal.ONE));
-    }
-    res = divide(rexBuilder, res, unit.multiplier);
-    if (unit == TimeUnit.QUARTER) {
-      res = plus(rexBuilder, res, rexBuilder.makeExactLiteral(BigDecimal.ONE));
-    }
-    return res;
-  }
-
-  private static BigDecimal getFactor(TimeUnit unit) {
-    switch (unit) {
-    case DAY:
-      return BigDecimal.ONE;
-    case HOUR:
-      return TimeUnit.DAY.multiplier;
-    case MINUTE:
-      return TimeUnit.HOUR.multiplier;
-    case SECOND:
-      return TimeUnit.MINUTE.multiplier;
-    case MONTH:
-      return TimeUnit.YEAR.multiplier;
-    case QUARTER:
-      return TimeUnit.YEAR.multiplier;
-    case YEAR:
-    case DECADE:
-    case CENTURY:
-    case MILLENNIUM:
-      return BigDecimal.ONE;
-    default:
-      throw Util.unexpected(unit);
-    }
+    return convertFunction(cx, (SqlFunction) call.getOperator(), call);
   }
 
   private RexNode mod(RexBuilder rexBuilder, RelDataType resType, RexNode res,

http://git-wip-us.apache.org/repos/asf/calcite/blob/796a28f9/core/src/test/java/org/apache/calcite/rel/rules/DateRangeRulesTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/rel/rules/DateRangeRulesTest.java b/core/src/test/java/org/apache/calcite/rel/rules/DateRangeRulesTest.java
index ea7b58b..5fd746d 100644
--- a/core/src/test/java/org/apache/calcite/rel/rules/DateRangeRulesTest.java
+++ b/core/src/test/java/org/apache/calcite/rel/rules/DateRangeRulesTest.java
@@ -53,34 +53,34 @@ public class DateRangeRulesTest {
     assertThat(DateRangeRules.extractTimeUnits(f.literal(1)), is(set()));
 
     // extract YEAR from a DATE column
-    checkDateRange(f, e, is("AND(>=($8, 2014-01-01), <($8, 2015-01-01))"));
+    checkDateRange(f, e, is("AND(>=($9, 2014-01-01), <($9, 2015-01-01))"));
     checkDateRange(f, f.eq(f.exYear, f.literal(2014)),
-        is("AND(>=($8, 2014-01-01), <($8, 2015-01-01))"));
+        is("AND(>=($9, 2014-01-01), <($9, 2015-01-01))"));
     checkDateRange(f, f.ge(f.exYear, f.literal(2014)),
-        is(">=($8, 2014-01-01)"));
+        is(">=($9, 2014-01-01)"));
     checkDateRange(f, f.gt(f.exYear, f.literal(2014)),
-        is(">=($8, 2015-01-01)"));
+        is(">=($9, 2015-01-01)"));
     checkDateRange(f, f.lt(f.exYear, f.literal(2014)),
-        is("<($8, 2014-01-01)"));
+        is("<($9, 2014-01-01)"));
     checkDateRange(f, f.le(f.exYear, f.literal(2014)),
-        is("<($8, 2015-01-01)"));
+        is("<($9, 2015-01-01)"));
     checkDateRange(f, f.ne(f.exYear, f.literal(2014)),
-        is("<>(EXTRACT_DATE(FLAG(YEAR), $8), 2014)"));
+        is("<>(EXTRACT(FLAG(YEAR), $9), 2014)"));
   }
 
   @Test public void testExtractYearFromTimestampColumn() {
     final Fixture2 f = new Fixture2();
-    checkDateRange(f, f.eq(f.exYearTs, f.literal(2014)),
+    checkDateRange(f, f.eq(f.exYear, f.literal(2014)),
         is("AND(>=($9, 2014-01-01), <($9, 2015-01-01))"));
-    checkDateRange(f, f.ge(f.exYearTs, f.literal(2014)),
+    checkDateRange(f, f.ge(f.exYear, f.literal(2014)),
         is(">=($9, 2014-01-01)"));
-    checkDateRange(f, f.gt(f.exYearTs, f.literal(2014)),
+    checkDateRange(f, f.gt(f.exYear, f.literal(2014)),
         is(">=($9, 2015-01-01)"));
-    checkDateRange(f, f.lt(f.exYearTs, f.literal(2014)),
+    checkDateRange(f, f.lt(f.exYear, f.literal(2014)),
         is("<($9, 2014-01-01)"));
-    checkDateRange(f, f.le(f.exYearTs, f.literal(2014)),
+    checkDateRange(f, f.le(f.exYear, f.literal(2014)),
         is("<($9, 2015-01-01)"));
-    checkDateRange(f, f.ne(f.exYearTs, f.literal(2014)),
+    checkDateRange(f, f.ne(f.exYear, f.literal(2014)),
         is("<>(EXTRACT(FLAG(YEAR), $9), 2014)"));
   }
 
@@ -88,10 +88,10 @@ public class DateRangeRulesTest {
     final Fixture2 f = new Fixture2();
     checkDateRange(f,
         f.and(f.eq(f.exYear, f.literal(2014)), f.eq(f.exMonth, f.literal(6))),
-        is("AND(AND(>=($8, 2014-01-01), <($8, 2015-01-01)),"
-            + " AND(>=($8, 2014-06-01), <($8, 2014-07-01)))"),
-        is("AND(>=($8, 2014-01-01), <($8, 2015-01-01),"
-            + " >=($8, 2014-06-01), <($8, 2014-07-01))"));
+        is("AND(AND(>=($9, 2014-01-01), <($9, 2015-01-01)),"
+            + " AND(>=($9, 2014-06-01), <($9, 2014-07-01)))"),
+        is("AND(>=($9, 2014-01-01), <($9, 2015-01-01),"
+            + " >=($9, 2014-06-01), <($9, 2014-07-01))"));
   }
 
   /** Test case for
@@ -100,16 +100,16 @@ public class DateRangeRulesTest {
   @Test public void testExtractYearAndMonthFromDateColumn2() {
     final Fixture2 f = new Fixture2();
     final String s1 = "AND("
-        + "AND(>=($8, 2000-01-01), <($8, 2001-01-01)),"
+        + "AND(>=($9, 2000-01-01), <($9, 2001-01-01)),"
         + " OR("
-        + "AND(>=($8, 2000-02-01), <($8, 2000-03-01)), "
-        + "AND(>=($8, 2000-03-01), <($8, 2000-04-01)), "
-        + "AND(>=($8, 2000-05-01), <($8, 2000-06-01))))";
-    final String s2 = "AND(>=($8, 2000-01-01), <($8, 2001-01-01),"
+        + "AND(>=($9, 2000-02-01), <($9, 2000-03-01)), "
+        + "AND(>=($9, 2000-03-01), <($9, 2000-04-01)), "
+        + "AND(>=($9, 2000-05-01), <($9, 2000-06-01))))";
+    final String s2 = "AND(>=($9, 2000-01-01), <($9, 2001-01-01),"
         + " OR("
-        + "AND(>=($8, 2000-02-01), <($8, 2000-03-01)), "
-        + "AND(>=($8, 2000-03-01), <($8, 2000-04-01)), "
-        + "AND(>=($8, 2000-05-01), <($8, 2000-06-01))))";
+        + "AND(>=($9, 2000-02-01), <($9, 2000-03-01)), "
+        + "AND(>=($9, 2000-03-01), <($9, 2000-04-01)), "
+        + "AND(>=($9, 2000-05-01), <($9, 2000-06-01))))";
     final RexNode e =
         f.and(f.eq(f.exYear, f.literal(2000)),
             f.or(f.eq(f.exMonth, f.literal(2)),
@@ -122,14 +122,14 @@ public class DateRangeRulesTest {
     final Fixture2 f = new Fixture2();
     checkDateRange(f,
         f.and(f.eq(f.exYear, f.literal(2010)), f.eq(f.exDay, f.literal(31))),
-        is("AND(AND(>=($8, 2010-01-01), <($8, 2011-01-01)),"
-            + " OR(AND(>=($8, 2010-01-31), <($8, 2010-02-01)),"
-            + " AND(>=($8, 2010-03-31), <($8, 2010-04-01)),"
-            + " AND(>=($8, 2010-05-31), <($8, 2010-06-01)),"
-            + " AND(>=($8, 2010-07-31), <($8, 2010-08-01)),"
-            + " AND(>=($8, 2010-08-31), <($8, 2010-09-01)),"
-            + " AND(>=($8, 2010-10-31), <($8, 2010-11-01)),"
-            + " AND(>=($8, 2010-12-31), <($8, 2011-01-01))))"));
+        is("AND(AND(>=($9, 2010-01-01), <($9, 2011-01-01)),"
+            + " OR(AND(>=($9, 2010-01-31), <($9, 2010-02-01)),"
+            + " AND(>=($9, 2010-03-31), <($9, 2010-04-01)),"
+            + " AND(>=($9, 2010-05-31), <($9, 2010-06-01)),"
+            + " AND(>=($9, 2010-07-31), <($9, 2010-08-01)),"
+            + " AND(>=($9, 2010-08-31), <($9, 2010-09-01)),"
+            + " AND(>=($9, 2010-10-31), <($9, 2010-11-01)),"
+            + " AND(>=($9, 2010-12-31), <($9, 2011-01-01))))"));
 
   }
 
@@ -139,32 +139,32 @@ public class DateRangeRulesTest {
     // namely 29th February 2012 and 2016.
     //
     // Currently there are redundant conditions, e.g.
-    // "AND(>=($8, 2011-01-01), <($8, 2020-01-01))". We should remove them by
+    // "AND(>=($9, 2011-01-01), <($9, 2020-01-01))". We should remove them by
     // folding intervals.
     checkDateRange(f,
         f.and(f.gt(f.exYear, f.literal(2010)), f.lt(f.exYear, f.literal(2020)),
             f.eq(f.exMonth, f.literal(2)), f.eq(f.exDay, f.literal(29))),
-        is("AND(>=($8, 2011-01-01),"
-            + " AND(>=($8, 2011-01-01), <($8, 2020-01-01)),"
-            + " OR(AND(>=($8, 2011-02-01), <($8, 2011-03-01)),"
-            + " AND(>=($8, 2012-02-01), <($8, 2012-03-01)),"
-            + " AND(>=($8, 2013-02-01), <($8, 2013-03-01)),"
-            + " AND(>=($8, 2014-02-01), <($8, 2014-03-01)),"
-            + " AND(>=($8, 2015-02-01), <($8, 2015-03-01)),"
-            + " AND(>=($8, 2016-02-01), <($8, 2016-03-01)),"
-            + " AND(>=($8, 2017-02-01), <($8, 2017-03-01)),"
-            + " AND(>=($8, 2018-02-01), <($8, 2018-03-01)),"
-            + " AND(>=($8, 2019-02-01), <($8, 2019-03-01))),"
-            + " OR(AND(>=($8, 2012-02-29), <($8, 2012-03-01)),"
-            + " AND(>=($8, 2016-02-29), <($8, 2016-03-01))))"));
+        is("AND(>=($9, 2011-01-01),"
+            + " AND(>=($9, 2011-01-01), <($9, 2020-01-01)),"
+            + " OR(AND(>=($9, 2011-02-01), <($9, 2011-03-01)),"
+            + " AND(>=($9, 2012-02-01), <($9, 2012-03-01)),"
+            + " AND(>=($9, 2013-02-01), <($9, 2013-03-01)),"
+            + " AND(>=($9, 2014-02-01), <($9, 2014-03-01)),"
+            + " AND(>=($9, 2015-02-01), <($9, 2015-03-01)),"
+            + " AND(>=($9, 2016-02-01), <($9, 2016-03-01)),"
+            + " AND(>=($9, 2017-02-01), <($9, 2017-03-01)),"
+            + " AND(>=($9, 2018-02-01), <($9, 2018-03-01)),"
+            + " AND(>=($9, 2019-02-01), <($9, 2019-03-01))),"
+            + " OR(AND(>=($9, 2012-02-29), <($9, 2012-03-01)),"
+            + " AND(>=($9, 2016-02-29), <($9, 2016-03-01))))"));
   }
 
   @Test public void testExtractYearMonthDayFromTimestampColumn() {
     final Fixture2 f = new Fixture2();
     checkDateRange(f,
-        f.and(f.gt(f.exYearTs, f.literal(2010)),
-            f.lt(f.exYearTs, f.literal(2020)),
-            f.eq(f.exMonthTs, f.literal(2)), f.eq(f.exDayTs, f.literal(29))),
+        f.and(f.gt(f.exYear, f.literal(2010)),
+            f.lt(f.exYear, f.literal(2020)),
+            f.eq(f.exMonth, f.literal(2)), f.eq(f.exDay, f.literal(29))),
         is("AND(>=($9, 2011-01-01),"
             + " AND(>=($9, 2011-01-01), <($9, 2020-01-01)),"
             + " OR(AND(>=($9, 2011-02-01), <($9, 2011-03-01)),"
@@ -212,26 +212,14 @@ public class DateRangeRulesTest {
     private final RexNode exYear;
     private final RexNode exMonth;
     private final RexNode exDay;
-    private final RexNode exYearTs;
-    private final RexNode exMonthTs;
-    private final RexNode exDayTs;
 
     Fixture2() {
-      exYear = rexBuilder.makeCall(intRelDataType,
-          SqlStdOperatorTable.EXTRACT_DATE,
-          ImmutableList.of(rexBuilder.makeFlag(TimeUnitRange.YEAR), dt));
-      exMonth = rexBuilder.makeCall(intRelDataType,
-          SqlStdOperatorTable.EXTRACT_DATE,
-          ImmutableList.of(rexBuilder.makeFlag(TimeUnitRange.MONTH), dt));
-      exDay = rexBuilder.makeCall(intRelDataType,
-          SqlStdOperatorTable.EXTRACT_DATE,
-          ImmutableList.of(rexBuilder.makeFlag(TimeUnitRange.DAY), dt));
-      exYearTs = rexBuilder.makeCall(SqlStdOperatorTable.EXTRACT,
+      exYear = rexBuilder.makeCall(SqlStdOperatorTable.EXTRACT,
           ImmutableList.of(rexBuilder.makeFlag(TimeUnitRange.YEAR), ts));
-      exMonthTs = rexBuilder.makeCall(intRelDataType,
+      exMonth = rexBuilder.makeCall(intRelDataType,
           SqlStdOperatorTable.EXTRACT,
           ImmutableList.of(rexBuilder.makeFlag(TimeUnitRange.MONTH), ts));
-      exDayTs = rexBuilder.makeCall(intRelDataType,
+      exDay = rexBuilder.makeCall(intRelDataType,
           SqlStdOperatorTable.EXTRACT,
           ImmutableList.of(rexBuilder.makeFlag(TimeUnitRange.DAY), ts));
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/796a28f9/core/src/test/java/org/apache/calcite/test/JdbcTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcTest.java b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
index 2282037..7d722cb 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
@@ -1516,6 +1516,30 @@ public class JdbcTest {
             });
   }
 
+  @Test public void testExtractMonthFromTimestamp() {
+    CalciteAssert.that()
+        .with(CalciteAssert.Config.JDBC_FOODMART)
+        .query("select extract(month from \"birth_date\") as c \n"
+            + "from \"foodmart\".\"employee\" where \"employee_id\"=1")
+        .returns("C=8\n");
+  }
+
+  @Test public void testExtractYearFromTimestamp() {
+    CalciteAssert.that()
+        .with(CalciteAssert.Config.JDBC_FOODMART)
+        .query("select extract(year from \"birth_date\") as c \n"
+            + "from \"foodmart\".\"employee\" where \"employee_id\"=1")
+        .returns("C=1961\n");
+  }
+
+  @Test public void testExtractFromInterval() {
+    CalciteAssert.that()
+        .with(CalciteAssert.Config.JDBC_FOODMART)
+        .query("select extract(month from interval '2-3' year to month) as c \n"
+            + "from \"foodmart\".\"employee\" where \"employee_id\"=1")
+        .returns("C=3\n");
+  }
+
   @Test public void testFloorDate() {
     CalciteAssert.that()
         .with(CalciteAssert.Config.JDBC_FOODMART)

http://git-wip-us.apache.org/repos/asf/calcite/blob/796a28f9/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 95c47fc..570c7a0 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -8575,7 +8575,6 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         + "UNION ALL left\n"
         + "\n"
         + "$throw -\n"
-        + "EXTRACT_DATE -\n"
         + "FILTER left\n"
         + "Reinterpret -\n"
         + "TABLE pre\n"

http://git-wip-us.apache.org/repos/asf/calcite/blob/796a28f9/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
index f575271..ecf1e46 100644
--- a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
@@ -18,11 +18,9 @@ limitations under the License.
 <Root>
     <TestCase name="testReduceNot">
         <Resource name="sql">
-            <![CDATA[select sal
-from emp
-where case when (sal = 1000) then
-(case when sal = 1000 then null else 1 end is null) else
-(case when sal = 2000 then null else 1 end is null) end is true]]>
+            <![CDATA[select *
+from (select (case when sal > 1000 then null else false end) as caseCol from emp)
+where NOT(caseCol)]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
@@ -656,7 +654,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[$SUM0($2)], EXPR$2=[SUM($1)])
     </TestCase>
     <TestCase name="testDistinctNonDistinctTwoAggregatesWithGrouping">
         <Resource name="sql">
-            <![CDATA[SELECT deptno, sum(comm), min(comm), SUM(DISTINCT sal)
+            <![CDATA[SELECT deptno, SUM(comm), MIN(comm), SUM(DISTINCT sal)
 FROM emp
 GROUP BY deptno]]>
         </Resource>
@@ -678,7 +676,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[SUM($2)], EXPR$2=[MIN($3)], EXPR$3=[SUM($1
     </TestCase>
     <TestCase name="testDistinctWithGrouping">
         <Resource name="sql">
-            <![CDATA[SELECT sal, sum(comm), min(comm), SUM(DISTINCT sal)
+            <![CDATA[SELECT sal, SUM(comm), MIN(comm), SUM(DISTINCT sal)
 FROM emp
 GROUP BY sal]]>
         </Resource>
@@ -2331,14 +2329,14 @@ where extract(year from birthdate) = 2014and extract(month from birthdate) = 4]]
         <Resource name="planBefore">
             <![CDATA[
 LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], BIRTHDATE=[$9])
-  LogicalFilter(condition=[AND(=(EXTRACT_DATE(FLAG(YEAR), Reinterpret($9)), 2014), =(EXTRACT_DATE(FLAG(MONTH), Reinterpret($9)), 4))])
+  LogicalFilter(condition=[AND(=(EXTRACT(FLAG(YEAR), $9), 2014), =(EXTRACT(FLAG(MONTH), $9), 4))])
     LogicalTableScan(table=[[CATALOG, SALES, EMP_B]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
 LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], BIRTHDATE=[$9])
-  LogicalFilter(condition=[AND(>=(Reinterpret($9), 2014-01-01), <(Reinterpret($9), 2015-01-01), >=(Reinterpret($9), 2014-04-01), <(Reinterpret($9), 2014-05-01))])
+  LogicalFilter(condition=[AND(>=($9, 2014-04-01), <($9, 2014-05-01))])
     LogicalTableScan(table=[[CATALOG, SALES, EMP_B]])
 ]]>
         </Resource>
@@ -2352,14 +2350,14 @@ where extract(year from birthdate) = 2014]]>
         <Resource name="planBefore">
             <![CDATA[
 LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], BIRTHDATE=[$9])
-  LogicalFilter(condition=[=(EXTRACT_DATE(FLAG(YEAR), Reinterpret($9)), 2014)])
+  LogicalFilter(condition=[=(EXTRACT(FLAG(YEAR), $9), 2014)])
     LogicalTableScan(table=[[CATALOG, SALES, EMP_B]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
 LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], BIRTHDATE=[$9])
-  LogicalFilter(condition=[AND(>=(Reinterpret($9), 2014-01-01), <(Reinterpret($9), 2015-01-01))])
+  LogicalFilter(condition=[AND(>=($9, 2014-01-01), <($9, 2015-01-01))])
     LogicalTableScan(table=[[CATALOG, SALES, EMP_B]])
 ]]>
         </Resource>

http://git-wip-us.apache.org/repos/asf/calcite/blob/796a28f9/druid/src/test/java/org/apache/calcite/test/DruidAdapterIT.java
----------------------------------------------------------------------
diff --git a/druid/src/test/java/org/apache/calcite/test/DruidAdapterIT.java b/druid/src/test/java/org/apache/calcite/test/DruidAdapterIT.java
index 1a0d3d3..c167e06 100644
--- a/druid/src/test/java/org/apache/calcite/test/DruidAdapterIT.java
+++ b/druid/src/test/java/org/apache/calcite/test/DruidAdapterIT.java
@@ -1378,10 +1378,9 @@ public class DruidAdapterIT {
         + "where extract(year from \"timestamp\") = 1997\n"
         + "and extract(month from \"timestamp\") in (4, 6)\n";
     final String explain = "DruidQuery(table=[[foodmart, foodmart]], "
-        + "intervals=[[1900-01-09T00:00:00.000/2992-01-10T00:00:00.000]], filter=[AND(="
-        + "(EXTRACT_DATE(FLAG(YEAR), /INT(CAST(Reinterpret($0)):TIMESTAMP(0), 86400000)), 1997), OR(=(EXTRACT_DATE"
-        + "(FLAG(MONTH), /INT(CAST(Reinterpret($0)):TIMESTAMP(0), 86400000)), 4), =(EXTRACT_DATE(FLAG(MONTH), "
-        + "/INT(CAST(Reinterpret($0)):TIMESTAMP(0), 86400000)), 6)))], groups=[{}], aggs=[[COUNT()]])";
+        + "intervals=[[1900-01-09T00:00:00.000/2992-01-10T00:00:00.000]], "
+        + "filter=[AND(=(EXTRACT(FLAG(YEAR), $0), 1997), OR(=(EXTRACT(FLAG(MONTH), $0), 4), "
+        + "=(EXTRACT(FLAG(MONTH), $0), 6)))], groups=[{}], aggs=[[COUNT()]])";
     sql(sql)
         .explainContains(explain)
         .returnsUnordered("C=13500");
@@ -1586,10 +1585,9 @@ public class DruidAdapterIT {
                     + "'timeZone':'UTC','locale':'en-US'}}"))
         .explainContains("PLAN=EnumerableInterpreter\n"
             + "  DruidQuery(table=[[foodmart, foodmart]], "
-            + "intervals=[[1997-01-01T00:00:00.001/1997-01-20T00:00:00.000]], filter=[=($1, 1016)"
-            + "], projects=[[EXTRACT_DATE(FLAG(DAY), /INT(CAST(Reinterpret($0)):TIMESTAMP(0), 86400000)), "
-            + "EXTRACT_DATE(FLAG(MONTH), /INT(CAST(Reinterpret($0)):TIMESTAMP(0), 86400000)), EXTRACT_DATE(FLAG"
-            + "(YEAR), /INT(CAST(Reinterpret($0)):TIMESTAMP(0), 86400000)), $1]], groups=[{0, 1, 2, 3}], aggs=[[]])\n")
+            + "intervals=[[1997-01-01T00:00:00.001/1997-01-20T00:00:00.000]], "
+            + "filter=[=($1, 1016)], projects=[[EXTRACT(FLAG(DAY), $0), EXTRACT(FLAG(MONTH), $0), "
+            + "EXTRACT(FLAG(YEAR), $0), $1]], groups=[{0, 1, 2, 3}], aggs=[[]])\n")
         .returnsUnordered("day=2; month=1; year=1997; product_id=1016",
             "day=10; month=1; year=1997; product_id=1016",
             "day=13; month=1; year=1997; product_id=1016",
@@ -1620,10 +1618,9 @@ public class DruidAdapterIT {
                     + "'timeZone':'UTC','locale':'en-US'}}"))
         .explainContains("PLAN=EnumerableInterpreter\n"
             + "  DruidQuery(table=[[foodmart, foodmart]], "
-            + "intervals=[[1997-01-01T00:00:00.001/1997-01-20T00:00:00.000]], filter=[=($1, 1016)"
-            + "], projects=[[EXTRACT_DATE(FLAG(DAY), /INT(CAST(Reinterpret($0)):TIMESTAMP(0), 86400000)), "
-            + "EXTRACT_DATE(FLAG(MONTH), /INT(CAST(Reinterpret($0)):TIMESTAMP(0), 86400000)), EXTRACT_DATE(FLAG"
-            + "(YEAR), /INT(CAST(Reinterpret($0)):TIMESTAMP(0), 86400000)), $1]], groups=[{0, 1, 2, 3}], aggs=[[]])\n")
+            + "intervals=[[1997-01-01T00:00:00.001/1997-01-20T00:00:00.000]], "
+            + "filter=[=($1, 1016)], projects=[[EXTRACT(FLAG(DAY), $0), EXTRACT(FLAG(MONTH), $0), "
+            + "EXTRACT(FLAG(YEAR), $0), $1]], groups=[{0, 1, 2, 3}], aggs=[[]])\n")
         .returnsUnordered("EXPR$0=2; EXPR$1=1; EXPR$2=1997; product_id=1016",
             "EXPR$0=10; EXPR$1=1; EXPR$2=1997; product_id=1016",
             "EXPR$0=13; EXPR$1=1; EXPR$2=1997; product_id=1016",
@@ -1647,8 +1644,8 @@ public class DruidAdapterIT {
                     + "'timeZone':'UTC','locale':'en-US'}}"))
         .explainContains("PLAN=EnumerableInterpreter\n"
             + "  DruidQuery(table=[[foodmart, foodmart]], "
-            + "intervals=[[1997-01-01T00:00:00.001/1997-01-20T00:00:00.000]], filter=[=($1, 1016)], "
-            + "projects=[[EXTRACT_DATE(FLAG(DAY), /INT(CAST(Reinterpret($0)):TIMESTAMP(0), 86400000)), $1]], "
+            + "intervals=[[1997-01-01T00:00:00.001/1997-01-20T00:00:00.000]], "
+            + "filter=[=($1, 1016)], projects=[[EXTRACT(FLAG(DAY), $0), $1]], "
             + "groups=[{0, 1}], aggs=[[]])\n")
         .returnsUnordered("EXPR$0=2; dayOfMonth=1016", "EXPR$0=10; dayOfMonth=1016",
             "EXPR$0=13; dayOfMonth=1016", "EXPR$0=16; dayOfMonth=1016");
@@ -1675,10 +1672,10 @@ public class DruidAdapterIT {
     sql(sql)
         .explainContains("PLAN=EnumerableInterpreter\n"
             + "  DruidQuery(table=[[foodmart, foodmart]], "
-            + "intervals=[[1900-01-09T00:00:00.000/2992-01-10T00:00:00.000]], filter=[AND(>=(CAST"
-            + "($11):BIGINT, 8), <=(CAST($11):BIGINT, 10), <(CAST($10):BIGINT, 15), =(EXTRACT_DATE"
-            + "(FLAG(YEAR), /INT(CAST(Reinterpret($0)):TIMESTAMP(0), 86400000)), 1997))], groups=[{}], "
-            + "aggs=[[SUM($90)]])")
+            + "intervals=[[1900-01-09T00:00:00.000/2992-01-10T00:00:00.000]], "
+            + "filter=[AND(>=(CAST($11):BIGINT, 8), <=(CAST($11):BIGINT, 10), "
+            + "<(CAST($10):BIGINT, 15), =(EXTRACT(FLAG(YEAR), $0), 1997))], groups=[{}], "
+            + "aggs=[[SUM($90)]])\n")
         .queryContains(druidChecker(druidQuery))
         .returnsUnordered("EXPR$0=75364.09998679161");
   }
@@ -1824,10 +1821,10 @@ public class DruidAdapterIT {
             + "'intervals':['1900-01-09T00:00:00.000/2992-01-10T00:00:00.000']}"))
         .explainContains("PLAN=EnumerableInterpreter\n"
             + "  DruidQuery(table=[[foodmart, foodmart]], "
-            + "intervals=[[1900-01-09T00:00:00.000/2992-01-10T00:00:00.000]], filter=[>=(CAST($1)"
-            + ":BIGINT, 1558)], projects=[[EXTRACT_DATE(FLAG(MONTH), /INT(CAST(Reinterpret($0)):TIMESTAMP(0), "
-            + "86400000)), $1, $89]], groups=[{0, 1}], aggs=[[SUM($2)]], sort0=[0], sort1=[2], "
-            + "sort2=[1], dir0=[ASC], dir1=[ASC], dir2=[ASC])");
+            + "intervals=[[1900-01-09T00:00:00.000/2992-01-10T00:00:00.000]], "
+            + "filter=[>=(CAST($1):BIGINT, 1558)], projects=[[EXTRACT(FLAG(MONTH), $0), $1, $89]], "
+            + "groups=[{0, 1}], aggs=[[SUM($2)]], sort0=[0], sort1=[2], sort2=[1], "
+            + "dir0=[ASC], dir1=[ASC], dir2=[ASC])");
   }
 
 
@@ -1870,10 +1867,10 @@ public class DruidAdapterIT {
         + " by y DESC, m ASC, s DESC, \"product_id\" LIMIT 3";
     final String expectedPlan = "PLAN=EnumerableInterpreter\n"
         + "  DruidQuery(table=[[foodmart, foodmart]], "
-        + "intervals=[[1900-01-09T00:00:00.000/2992-01-10T00:00:00.000]], filter=[>=(CAST($1)"
-        + ":BIGINT, 1558)], projects=[[EXTRACT_DATE(FLAG(YEAR), /INT(CAST(Reinterpret($0)):TIMESTAMP(0), 86400000)),"
-        + " EXTRACT_DATE(FLAG(MONTH), /INT(CAST(Reinterpret($0)):TIMESTAMP(0), 86400000)), $1, $89]], groups=[{0, 1,"
-        + " 2}], aggs=[[SUM($3)]], sort0=[0], sort1=[1], sort2=[3], sort3=[2], dir0=[DESC], "
+        + "intervals=[[1900-01-09T00:00:00.000/2992-01-10T00:00:00.000]], "
+        + "filter=[>=(CAST($1):BIGINT, 1558)], projects=[[EXTRACT(FLAG(YEAR), $0), "
+        + "EXTRACT(FLAG(MONTH), $0), $1, $89]], groups=[{0, 1, 2}], aggs=[[SUM($3)]], sort0=[0], "
+        + "sort1=[1], sort2=[3], sort3=[2], dir0=[DESC], "
         + "dir1=[ASC], dir2=[DESC], dir3=[ASC], fetch=[3])";
     final String expectedDruidQuery = "{'queryType':'groupBy','dataSource':'foodmart',"
         + "'granularity':'all','dimensions':[{'type':'extraction',"
@@ -1906,11 +1903,10 @@ public class DruidAdapterIT {
         + " by s DESC, m DESC, \"product_id\" LIMIT 3";
     final String expectedPlan = "PLAN=EnumerableInterpreter\n"
         + "  DruidQuery(table=[[foodmart, foodmart]], "
-        + "intervals=[[1900-01-09T00:00:00.000/2992-01-10T00:00:00.000]], filter=[>=(CAST($1)"
-        + ":BIGINT, 1558)], projects=[[EXTRACT_DATE(FLAG(YEAR), /INT(CAST(Reinterpret($0)):TIMESTAMP(0), 86400000)),"
-        + " EXTRACT_DATE(FLAG(MONTH), /INT(CAST(Reinterpret($0)):TIMESTAMP(0), 86400000)), $1, $89]], groups=[{0, 1,"
-        + " 2}], aggs=[[SUM($3)]], sort0=[3], sort1=[1], sort2=[2], dir0=[DESC], dir1=[DESC], "
-        + "dir2=[ASC], fetch=[3])";
+        + "intervals=[[1900-01-09T00:00:00.000/2992-01-10T00:00:00.000]], "
+        + "filter=[>=(CAST($1):BIGINT, 1558)], projects=[[EXTRACT(FLAG(YEAR), $0), "
+        + "EXTRACT(FLAG(MONTH), $0), $1, $89]], groups=[{0, 1, 2}], aggs=[[SUM($3)]], "
+        + "sort0=[3], sort1=[1], sort2=[2], dir0=[DESC], dir1=[DESC], dir2=[ASC], fetch=[3])";
     final String expectedDruidQuery = "{'queryType':'groupBy','dataSource':'foodmart',"
         + "'granularity':'all','dimensions':[{'type':'extraction',"
         + "'dimension':'__time','outputName':'extract_year',"
@@ -2040,10 +2036,10 @@ public class DruidAdapterIT {
         + "\"product_id\" = 1558 group by extract(CENTURY from \"timestamp\")";
     final String plan = "PLAN=EnumerableInterpreter\n"
         + "  BindableAggregate(group=[{0}])\n"
-        + "    BindableProject(EXPR$0=[EXTRACT_DATE(FLAG(CENTURY), /INT(CAST(Reinterpret($0)):TIMESTAMP(0), 86400000))])\n"
+        + "    BindableProject(EXPR$0=[EXTRACT(FLAG(CENTURY), $0)])\n"
         + "      DruidQuery(table=[[foodmart, foodmart]], "
-        + "intervals=[[1900-01-09T00:00:00.000/2992-01-10T00:00:00.000]], filter=[=($1, 1558)], "
-        + "projects=[[$0]])";
+        + "intervals=[[1900-01-09T00:00:00.000/2992-01-10T00:00:00.000]], "
+        + "filter=[=($1, 1558)], projects=[[$0]])\n";
     sql(sql).explainContains(plan).queryContains(druidChecker("'queryType':'select'"))
         .returnsUnordered("EXPR$0=20");
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/796a28f9/druid/src/test/java/org/apache/calcite/test/DruidDateRangeRulesTest.java
----------------------------------------------------------------------
diff --git a/druid/src/test/java/org/apache/calcite/test/DruidDateRangeRulesTest.java b/druid/src/test/java/org/apache/calcite/test/DruidDateRangeRulesTest.java
index 0e1948d..f993a53 100644
--- a/druid/src/test/java/org/apache/calcite/test/DruidDateRangeRulesTest.java
+++ b/druid/src/test/java/org/apache/calcite/test/DruidDateRangeRulesTest.java
@@ -112,9 +112,9 @@ public class DruidDateRangeRulesTest {
     //     OR(AND(>=($9, 2012-02-29), <($9, 2012-03-01)),"
     //        AND(>=($9, 2016-02-29), <($9, 2016-03-01))))
     checkDateRange(f,
-        f.and(f.gt(f.exYearTs, f.literal(2010)),
-            f.lt(f.exYearTs, f.literal(2020)),
-            f.eq(f.exMonthTs, f.literal(2)), f.eq(f.exDayTs, f.literal(29))),
+        f.and(f.gt(f.exYear, f.literal(2010)),
+            f.lt(f.exYear, f.literal(2020)),
+            f.eq(f.exMonth, f.literal(2)), f.eq(f.exDay, f.literal(29))),
         is("[2012-02-29T00:00:00.000/2012-03-01T00:00:00.000, "
             + "2016-02-29T00:00:00.000/2016-03-01T00:00:00.000]"));
   }
@@ -190,26 +190,14 @@ public class DruidDateRangeRulesTest {
     private final RexNode exYear;
     private final RexNode exMonth;
     private final RexNode exDay;
-    private final RexNode exYearTs;
-    private final RexNode exMonthTs;
-    private final RexNode exDayTs;
 
     Fixture2() {
-      exYear = rexBuilder.makeCall(intRelDataType,
-          SqlStdOperatorTable.EXTRACT_DATE,
-          ImmutableList.of(rexBuilder.makeFlag(TimeUnitRange.YEAR), dt));
-      exMonth = rexBuilder.makeCall(intRelDataType,
-          SqlStdOperatorTable.EXTRACT_DATE,
-          ImmutableList.of(rexBuilder.makeFlag(TimeUnitRange.MONTH), dt));
-      exDay = rexBuilder.makeCall(intRelDataType,
-          SqlStdOperatorTable.EXTRACT_DATE,
-          ImmutableList.of(rexBuilder.makeFlag(TimeUnitRange.DAY), dt));
-      exYearTs = rexBuilder.makeCall(SqlStdOperatorTable.EXTRACT,
+      exYear = rexBuilder.makeCall(SqlStdOperatorTable.EXTRACT,
           ImmutableList.of(rexBuilder.makeFlag(TimeUnitRange.YEAR), ts));
-      exMonthTs = rexBuilder.makeCall(intRelDataType,
+      exMonth = rexBuilder.makeCall(intRelDataType,
           SqlStdOperatorTable.EXTRACT,
           ImmutableList.of(rexBuilder.makeFlag(TimeUnitRange.MONTH), ts));
-      exDayTs = rexBuilder.makeCall(intRelDataType,
+      exDay = rexBuilder.makeCall(intRelDataType,
           SqlStdOperatorTable.EXTRACT,
           ImmutableList.of(rexBuilder.makeFlag(TimeUnitRange.DAY), ts));
     }


[08/15] calcite git commit: [CALCITE-1988] Various code quality issues

Posted by jh...@apache.org.
[CALCITE-1988] Various code quality issues

* Fix error that SqlNode.clone does not call super.clone; deprecate
  it, and add SqlNode.clone(SqlNode)
* Fix error that NlsString.clone does not call super.clone
* Make AggregateNode.AccumulatorList static, because non-static inner
  classes must not implement Serializable
* Change XmlOutput.content to not use LineNumberReader.readLine, which
  is susceptible to a DoS attack


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

Branch: refs/heads/master
Commit: 8441e79cdde01a901de894d44b597cf582a259d1
Parents: d173640
Author: Julian Hyde <jh...@apache.org>
Authored: Mon Sep 18 11:56:35 2017 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Mon Oct 2 11:13:43 2017 -0700

----------------------------------------------------------------------
 .../calcite/interpreter/AggregateNode.java      |  2 +-
 .../calcite/sql/SqlBinaryStringLiteral.java     |  2 +-
 .../calcite/sql/SqlCharStringLiteral.java       |  2 +-
 .../org/apache/calcite/sql/SqlDateLiteral.java  |  2 +-
 .../apache/calcite/sql/SqlIntervalLiteral.java  |  7 ++--
 .../java/org/apache/calcite/sql/SqlLiteral.java |  2 +-
 .../java/org/apache/calcite/sql/SqlNode.java    | 12 ++++++-
 .../apache/calcite/sql/SqlNumericLiteral.java   | 10 ++----
 .../org/apache/calcite/sql/SqlTimeLiteral.java  |  2 +-
 .../apache/calcite/sql/SqlTimestampLiteral.java |  2 +-
 .../calcite/sql/fun/SqlCoalesceFunction.java    | 11 +++---
 .../calcite/sql/fun/SqlNullifFunction.java      |  5 ++-
 .../calcite/sql/validate/SqlValidatorImpl.java  | 11 +++---
 .../calcite/sql/validate/SqlValidatorUtil.java  |  8 ++---
 .../java/org/apache/calcite/util/NlsString.java | 10 ++++--
 .../java/org/apache/calcite/util/XmlOutput.java | 28 +++++++++------
 .../java/org/apache/calcite/util/UtilTest.java  | 37 ++++++++++++++++++++
 17 files changed, 98 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/8441e79c/core/src/main/java/org/apache/calcite/interpreter/AggregateNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/interpreter/AggregateNode.java b/core/src/main/java/org/apache/calcite/interpreter/AggregateNode.java
index b25a268..47b933b 100644
--- a/core/src/main/java/org/apache/calcite/interpreter/AggregateNode.java
+++ b/core/src/main/java/org/apache/calcite/interpreter/AggregateNode.java
@@ -356,7 +356,7 @@ public class AggregateNode extends AbstractSingleNode<Aggregate> {
   /**
    * A list of accumulators used during grouping.
    */
-  private class AccumulatorList extends ArrayList<Accumulator> {
+  private static class AccumulatorList extends ArrayList<Accumulator> {
     public void send(Row row) {
       for (Accumulator a : this) {
         a.send(row);

http://git-wip-us.apache.org/repos/asf/calcite/blob/8441e79c/core/src/main/java/org/apache/calcite/sql/SqlBinaryStringLiteral.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlBinaryStringLiteral.java b/core/src/main/java/org/apache/calcite/sql/SqlBinaryStringLiteral.java
index 01ad109..71380e1 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlBinaryStringLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlBinaryStringLiteral.java
@@ -56,7 +56,7 @@ public class SqlBinaryStringLiteral extends SqlAbstractStringLiteral {
     return (BitString) value;
   }
 
-  public SqlNode clone(SqlParserPos pos) {
+  @Override public SqlBinaryStringLiteral clone(SqlParserPos pos) {
     return new SqlBinaryStringLiteral((BitString) value, pos);
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/8441e79c/core/src/main/java/org/apache/calcite/sql/SqlCharStringLiteral.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlCharStringLiteral.java b/core/src/main/java/org/apache/calcite/sql/SqlCharStringLiteral.java
index 9f15a22..b74176c 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlCharStringLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlCharStringLiteral.java
@@ -63,7 +63,7 @@ public class SqlCharStringLiteral extends SqlAbstractStringLiteral {
     return getNlsString().getCollation();
   }
 
-  public SqlNode clone(SqlParserPos pos) {
+  @Override public SqlCharStringLiteral clone(SqlParserPos pos) {
     return new SqlCharStringLiteral((NlsString) value, pos);
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/8441e79c/core/src/main/java/org/apache/calcite/sql/SqlDateLiteral.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlDateLiteral.java b/core/src/main/java/org/apache/calcite/sql/SqlDateLiteral.java
index 1a757f6..27cc8b4 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDateLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDateLiteral.java
@@ -42,7 +42,7 @@ public class SqlDateLiteral extends SqlAbstractDateTimeLiteral {
     return (DateString) value;
   }
 
-  public SqlNode clone(SqlParserPos pos) {
+  @Override public SqlDateLiteral clone(SqlParserPos pos) {
     return new SqlDateLiteral((DateString) value, pos);
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/8441e79c/core/src/main/java/org/apache/calcite/sql/SqlIntervalLiteral.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlIntervalLiteral.java b/core/src/main/java/org/apache/calcite/sql/SqlIntervalLiteral.java
index bd2969c..246e673 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlIntervalLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlIntervalLiteral.java
@@ -64,11 +64,8 @@ public class SqlIntervalLiteral extends SqlLiteral {
 
   //~ Methods ----------------------------------------------------------------
 
-  public SqlNode clone(SqlParserPos pos) {
-    return new SqlIntervalLiteral(
-        (IntervalValue) value,
-        getTypeName(),
-        pos);
+  @Override public SqlIntervalLiteral clone(SqlParserPos pos) {
+    return new SqlIntervalLiteral((IntervalValue) value, getTypeName(), pos);
   }
 
   public void unparse(

http://git-wip-us.apache.org/repos/asf/calcite/blob/8441e79c/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java b/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
index 4fcafb9..4cf88d7 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
@@ -234,7 +234,7 @@ public class SqlLiteral extends SqlNode {
     }
   }
 
-  public SqlNode clone(SqlParserPos pos) {
+  public SqlLiteral clone(SqlParserPos pos) {
     return new SqlLiteral(value, typeName, pos);
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/8441e79c/core/src/main/java/org/apache/calcite/sql/SqlNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlNode.java b/core/src/main/java/org/apache/calcite/sql/SqlNode.java
index e7c77fb..1d22f96 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlNode.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlNode.java
@@ -62,10 +62,20 @@ public abstract class SqlNode implements Cloneable {
 
   //~ Methods ----------------------------------------------------------------
 
+  /** @deprecated Please use {@link #clone(SqlNode)}; this method brings
+   * along too much baggage from early versions of Java */
+  @Deprecated
+  @SuppressWarnings("MethodDoesntCallSuperMethod")
   public Object clone() {
     return clone(getParserPosition());
   }
 
+  /** Creates a copy of a SqlNode. */
+  public static <E extends SqlNode> E clone(E e) {
+    //noinspection unchecked
+    return (E) e.clone(e.pos);
+  }
+
   /**
    * Clones a SqlNode with a different position.
    */
@@ -104,7 +114,7 @@ public abstract class SqlNode implements Cloneable {
     for (int i = 0; i < clones.length; i++) {
       SqlNode node = clones[i];
       if (node != null) {
-        clones[i] = (SqlNode) node.clone();
+        clones[i] = SqlNode.clone(node);
       }
     }
     return clones;

http://git-wip-us.apache.org/repos/asf/calcite/blob/8441e79c/core/src/main/java/org/apache/calcite/sql/SqlNumericLiteral.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlNumericLiteral.java b/core/src/main/java/org/apache/calcite/sql/SqlNumericLiteral.java
index b754a36..45f9dfb 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlNumericLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlNumericLiteral.java
@@ -65,13 +65,9 @@ public class SqlNumericLiteral extends SqlLiteral {
     return isExact;
   }
 
-  public SqlNode clone(SqlParserPos pos) {
-    return new SqlNumericLiteral(
-        (BigDecimal) value,
-        getPrec(),
-        getScale(),
-        isExact,
-        pos);
+  @Override public SqlNumericLiteral clone(SqlParserPos pos) {
+    return new SqlNumericLiteral((BigDecimal) value, getPrec(), getScale(),
+        isExact, pos);
   }
 
   public void unparse(

http://git-wip-us.apache.org/repos/asf/calcite/blob/8441e79c/core/src/main/java/org/apache/calcite/sql/SqlTimeLiteral.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlTimeLiteral.java b/core/src/main/java/org/apache/calcite/sql/SqlTimeLiteral.java
index 4327e2a..be3a04d 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlTimeLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlTimeLiteral.java
@@ -44,7 +44,7 @@ public class SqlTimeLiteral extends SqlAbstractDateTimeLiteral {
     return (TimeString) value;
   }
 
-  public SqlNode clone(SqlParserPos pos) {
+  @Override public SqlTimeLiteral clone(SqlParserPos pos) {
     return new SqlTimeLiteral((TimeString) value, precision, hasTimeZone, pos);
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/8441e79c/core/src/main/java/org/apache/calcite/sql/SqlTimestampLiteral.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlTimestampLiteral.java b/core/src/main/java/org/apache/calcite/sql/SqlTimestampLiteral.java
index cc659d5..41d9fe4 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlTimestampLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlTimestampLiteral.java
@@ -39,7 +39,7 @@ public class SqlTimestampLiteral extends SqlAbstractDateTimeLiteral {
 
   //~ Methods ----------------------------------------------------------------
 
-  public SqlNode clone(SqlParserPos pos) {
+  @Override public SqlTimestampLiteral clone(SqlParserPos pos) {
     return new SqlTimestampLiteral((TimestampString) value, precision,
         hasTimeZone, pos);
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/8441e79c/core/src/main/java/org/apache/calcite/sql/fun/SqlCoalesceFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlCoalesceFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlCoalesceFunction.java
index fc8d3c8..e2efc47 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlCoalesceFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlCoalesceFunction.java
@@ -70,17 +70,14 @@ public class SqlCoalesceFunction extends SqlFunction {
 
     // todo: optimize when know operand is not null.
 
-    for (int i = 0; (i + 1) < operands.size(); ++i) {
+    for (SqlNode operand : Util.skipLast(operands)) {
       whenList.add(
-          SqlStdOperatorTable.IS_NOT_NULL.createCall(
-              pos,
-              operands.get(i)));
-      thenList.add(operands.get(i).clone(operands.get(i).getParserPosition()));
+          SqlStdOperatorTable.IS_NOT_NULL.createCall(pos, operand));
+      thenList.add(SqlNode.clone(operand));
     }
     SqlNode elseExpr = Util.last(operands);
     assert call.getFunctionQuantifier() == null;
-    return SqlCase.createSwitched(
-        pos, null, whenList, thenList, elseExpr);
+    return SqlCase.createSwitched(pos, null, whenList, thenList, elseExpr);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/8441e79c/core/src/main/java/org/apache/calcite/sql/fun/SqlNullifFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlNullifFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlNullifFunction.java
index 0aa310e..63cc774 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlNullifFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlNullifFunction.java
@@ -67,9 +67,8 @@ public class SqlNullifFunction extends SqlFunction {
     SqlNodeList thenList = new SqlNodeList(pos);
     whenList.add(operands.get(1));
     thenList.add(SqlLiteral.createNull(SqlParserPos.ZERO));
-    return SqlCase.createSwitched(
-        pos, operands.get(0), whenList, thenList, operands.get(0).clone(
-        operands.get(0).getParserPosition()));
+    return SqlCase.createSwitched(pos, operands.get(0), whenList, thenList,
+        SqlNode.clone(operands.get(0)));
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/8441e79c/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
index cd83186..90bdb88 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
@@ -1191,8 +1191,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       selectList.add(SqlIdentifier.star(SqlParserPos.ZERO));
       final SqlNodeList orderList;
       if (getInnerSelect(node) != null && isAggregate(getInnerSelect(node))) {
-        orderList =
-            orderBy.orderList.clone(orderBy.orderList.getParserPosition());
+        orderList = SqlNode.clone(orderBy.orderList);
         // We assume that ORDER BY item does not have ASC etc.
         // We assume that ORDER BY item is present in SELECT list.
         for (int i = 0; i < orderList.size(); i++) {
@@ -1281,9 +1280,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       // from the update statement's source since it's the same as
       // what we want for the select list of the merge source -- '*'
       // followed by the update set expressions
-      selectList =
-          (SqlNodeList) updateStmt.getSourceSelect().getSelectList()
-              .clone();
+      selectList = SqlNode.clone(updateStmt.getSourceSelect().getSelectList());
     } else {
       // otherwise, just use select *
       selectList = new SqlNodeList(SqlParserPos.ZERO);
@@ -1305,7 +1302,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     SqlNode sourceTableRef = call.getSourceTableRef();
     SqlInsert insertCall = call.getInsertCall();
     JoinType joinType = (insertCall == null) ? JoinType.INNER : JoinType.LEFT;
-    SqlNode leftJoinTerm = (SqlNode) sourceTableRef.clone();
+    final SqlNode leftJoinTerm = SqlNode.clone(sourceTableRef);
     SqlNode outerJoin =
         new SqlJoin(SqlParserPos.ZERO,
             leftJoinTerm,
@@ -1331,7 +1328,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
           new SqlNodeList(
               rowCall.getOperandList(),
               SqlParserPos.ZERO);
-      SqlNode insertSource = (SqlNode) sourceTableRef.clone();
+      final SqlNode insertSource = SqlNode.clone(sourceTableRef);
       select =
           new SqlSelect(SqlParserPos.ZERO, null, selectList, insertSource, null,
               null, null, null, null, null, null);

http://git-wip-us.apache.org/repos/asf/calcite/blob/8441e79c/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
index c56c0ec..2200e01 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
@@ -1005,7 +1005,7 @@ public class SqlValidatorUtil {
     }
 
     public SqlNode visit(SqlLiteral literal) {
-      return (SqlNode) literal.clone();
+      return SqlNode.clone(literal);
     }
 
     public SqlNode visit(SqlIdentifier id) {
@@ -1020,15 +1020,15 @@ public class SqlValidatorUtil {
     }
 
     public SqlNode visit(SqlDataTypeSpec type) {
-      return (SqlNode) type.clone();
+      return SqlNode.clone(type);
     }
 
     public SqlNode visit(SqlDynamicParam param) {
-      return (SqlNode) param.clone();
+      return SqlNode.clone(param);
     }
 
     public SqlNode visit(SqlIntervalQualifier intervalQualifier) {
-      return (SqlNode) intervalQualifier.clone();
+      return SqlNode.clone(intervalQualifier);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/8441e79c/core/src/main/java/org/apache/calcite/util/NlsString.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/NlsString.java b/core/src/main/java/org/apache/calcite/util/NlsString.java
index eee7884..fbb9386 100644
--- a/core/src/main/java/org/apache/calcite/util/NlsString.java
+++ b/core/src/main/java/org/apache/calcite/util/NlsString.java
@@ -36,7 +36,7 @@ import static org.apache.calcite.util.Static.RESOURCE;
  * A string, optionally with {@link Charset character set} and
  * {@link SqlCollation}. It is immutable.
  */
-public class NlsString implements Comparable<NlsString> {
+public class NlsString implements Comparable<NlsString>, Cloneable {
   //~ Instance fields --------------------------------------------------------
 
   private final String charsetName;
@@ -47,7 +47,7 @@ public class NlsString implements Comparable<NlsString> {
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a string in a specfied character set.
+   * Creates a string in a specified character set.
    *
    * @param value       String constant, must not be null
    * @param charsetName Name of the character set, may be null
@@ -91,7 +91,11 @@ public class NlsString implements Comparable<NlsString> {
   //~ Methods ----------------------------------------------------------------
 
   public Object clone() {
-    return new NlsString(value, charsetName, collation);
+    try {
+      return super.clone();
+    } catch (CloneNotSupportedException e) {
+      throw new AssertionError();
+    }
   }
 
   public int hashCode() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/8441e79c/core/src/main/java/org/apache/calcite/util/XmlOutput.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/XmlOutput.java b/core/src/main/java/org/apache/calcite/util/XmlOutput.java
index ba1e5c8..e58cae0 100644
--- a/core/src/main/java/org/apache/calcite/util/XmlOutput.java
+++ b/core/src/main/java/org/apache/calcite/util/XmlOutput.java
@@ -18,10 +18,7 @@ package org.apache.calcite.util;
 
 import com.google.common.collect.Lists;
 
-import java.io.IOException;
-import java.io.LineNumberReader;
 import java.io.PrintWriter;
-import java.io.StringReader;
 import java.io.Writer;
 import java.util.ArrayDeque;
 import java.util.ArrayList;
@@ -397,19 +394,28 @@ public class XmlOutput {
    * Writes content.
    */
   public void content(String content) {
+    // This method previously used a LineNumberReader, but that class is
+    // susceptible to a form of DoS attack. It uses lots of memory and CPU if a
+    // malicious client gives it input with very long lines.
     if (content != null) {
       indent++;
-      LineNumberReader
-          in = new LineNumberReader(new StringReader(content));
-      try {
-        String line;
-        while ((line = in.readLine()) != null) {
+      final char[] chars = content.toCharArray();
+      int prev = 0;
+      for (int i = 0; i < chars.length; i++) {
+        if (chars[i] == '\n'
+            || chars[i] == '\r'
+            && i + 1 < chars.length
+            && chars[i + 1] == '\n') {
           displayIndent(out, indent);
-          out.println(line);
+          out.println(content.substring(prev, i));
+          if (chars[i] == '\r') {
+            ++i;
+          }
+          prev = i + 1;
         }
-      } catch (IOException ex) {
-        throw new AssertionError(ex);
       }
+      displayIndent(out, indent);
+      out.println(content.substring(prev, chars.length));
       indent--;
       out.flush();
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/8441e79c/core/src/test/java/org/apache/calcite/util/UtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/util/UtilTest.java b/core/src/test/java/org/apache/calcite/util/UtilTest.java
index 080ad4c..3711654 100644
--- a/core/src/test/java/org/apache/calcite/util/UtilTest.java
+++ b/core/src/test/java/org/apache/calcite/util/UtilTest.java
@@ -30,6 +30,7 @@ import org.apache.calcite.runtime.FlatLists;
 import org.apache.calcite.runtime.Resources;
 import org.apache.calcite.runtime.SqlFunctions;
 import org.apache.calcite.runtime.Utilities;
+import org.apache.calcite.sql.SqlCollation;
 import org.apache.calcite.sql.SqlDialect;
 import org.apache.calcite.sql.util.SqlBuilder;
 import org.apache.calcite.sql.util.SqlString;
@@ -2007,6 +2008,42 @@ public class UtilTest {
     assertThat(map.range("BAZ", true).size(), is(0));
     assertThat(map.range("Baz", true).size(), is(1));
   }
+
+  @Test public void testNlsStringClone() {
+    final NlsString s = new NlsString("foo", "LATIN1", SqlCollation.IMPLICIT);
+    assertThat(s.toString(), is("_LATIN1'foo'"));
+    final Object s2 = s.clone();
+    assertThat(s2, instanceOf(NlsString.class));
+    assertThat(s2, not(sameInstance((Object) s)));
+    assertThat(s2.toString(), is(s.toString()));
+  }
+
+  @Test public void testXmlOutput() {
+    final StringWriter w = new StringWriter();
+    final XmlOutput o = new XmlOutput(w);
+    o.beginBeginTag("root");
+    o.attribute("a1", "v1");
+    o.attribute("a2", null);
+    o.endBeginTag("root");
+    o.beginTag("someText", null);
+    o.content("line 1 followed by empty line\n"
+        + "\n"
+        + "line 3 with windows line ending\r\n"
+        + "line 4 with no ending");
+    o.endTag("someText");
+    o.endTag("root");
+    final String s = w.toString();
+    final String expected = ""
+        + "<root a1=\"v1\">\n"
+        + "\t<someText>\n"
+        + "\t\t\tline 1 followed by empty line\n"
+        + "\t\t\t\n"
+        + "\t\t\tline 3 with windows line ending\n"
+        + "\t\t\tline 4 with no ending\n"
+        + "\t</someText>\n"
+        + "</root>\n";
+    assertThat(s, is(expected));
+  }
 }
 
 // End UtilTest.java


[15/15] calcite git commit: [CALCITE-1913] Replace usages of DatabaseProduct with dialect methods, and introduce a configurable SqlDialectFactory (Christian Beikov)

Posted by jh...@apache.org.
[CALCITE-1913] Replace usages of DatabaseProduct with dialect methods, and introduce a configurable SqlDialectFactory (Christian Beikov)

Create class SqlDialect.Context to hold all arguments to the
constructor of a SqlDialect, including allowing future expansion.
(Julian Hyde)

Back out Christian's deprecation of DatabaseProduct. In my opinion,
deprecation is a step too far. People can use it if they like, but
there are usually better alternatives. (Julian Hyde)

Close apache/calcite#540


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

Branch: refs/heads/master
Commit: 914b5cfbf978e796afeaff7b780e268ed39d8ec5
Parents: 1a71a66
Author: Christian Beikov <ch...@gmail.com>
Authored: Thu Sep 14 07:27:57 2017 +0200
Committer: Julian Hyde <jh...@apache.org>
Committed: Mon Oct 2 12:28:20 2017 -0700

----------------------------------------------------------------------
 .../adapter/cassandra/CassandraSchema.java      |   4 +-
 .../apache/calcite/adapter/jdbc/JdbcRules.java  |  23 +-
 .../apache/calcite/adapter/jdbc/JdbcSchema.java |  44 +-
 .../adapter/jdbc/JdbcToEnumerableConverter.java |  26 +-
 .../apache/calcite/adapter/jdbc/JdbcUtils.java  |  25 +-
 .../apache/calcite/model/JsonJdbcSchema.java    |   7 +
 .../org/apache/calcite/model/ModelHandler.java  |  16 +-
 .../calcite/prepare/CalcitePrepareImpl.java     |   1 -
 .../org/apache/calcite/prepare/Prepare.java     |   3 -
 .../calcite/rel/rel2sql/RelToSqlConverter.java  |  19 +-
 .../calcite/rel/rel2sql/SqlImplementor.java     | 146 +------
 .../calcite/sql/SqlAbstractDateTimeLiteral.java |   2 +-
 .../org/apache/calcite/sql/SqlDateLiteral.java  |   9 +-
 .../java/org/apache/calcite/sql/SqlDialect.java | 411 +++++++++++--------
 .../apache/calcite/sql/SqlDialectFactory.java   |  42 ++
 .../calcite/sql/SqlDialectFactoryImpl.java      | 218 ++++++++++
 .../java/org/apache/calcite/sql/SqlNode.java    |   3 +-
 .../org/apache/calcite/sql/SqlNodeList.java     |  23 ++
 .../org/apache/calcite/sql/SqlSampleSpec.java   |   4 +-
 .../org/apache/calcite/sql/SqlTimeLiteral.java  |   7 +
 .../apache/calcite/sql/SqlTimestampLiteral.java |   9 +-
 .../calcite/sql/dialect/AccessSqlDialect.java   |  36 ++
 .../calcite/sql/dialect/AnsiSqlDialect.java     |  41 ++
 .../calcite/sql/dialect/CalciteSqlDialect.java  |  43 ++
 .../calcite/sql/dialect/Db2SqlDialect.java      |  43 ++
 .../calcite/sql/dialect/DerbySqlDialect.java    |  35 ++
 .../calcite/sql/dialect/FirebirdSqlDialect.java |  35 ++
 .../calcite/sql/dialect/H2SqlDialect.java       |  40 ++
 .../calcite/sql/dialect/HiveSqlDialect.java     |  43 ++
 .../calcite/sql/dialect/HsqldbHandler.java      |  82 ----
 .../calcite/sql/dialect/HsqldbSqlDialect.java   | 126 ++++++
 .../sql/dialect/InfobrightSqlDialect.java       |  36 ++
 .../calcite/sql/dialect/InformixSqlDialect.java |  35 ++
 .../calcite/sql/dialect/IngresSqlDialect.java   |  35 ++
 .../sql/dialect/InterbaseSqlDialect.java        |  35 ++
 .../calcite/sql/dialect/LucidDbSqlDialect.java  |  36 ++
 .../calcite/sql/dialect/MssqlHandler.java       | 128 ------
 .../calcite/sql/dialect/MssqlSqlDialect.java    | 139 +++++++
 .../calcite/sql/dialect/MysqlHandler.java       | 108 -----
 .../calcite/sql/dialect/MysqlSqlDialect.java    | 215 ++++++++++
 .../calcite/sql/dialect/NeoviewSqlDialect.java  |  35 ++
 .../calcite/sql/dialect/NetezzaSqlDialect.java  |  36 ++
 .../calcite/sql/dialect/OracleHandler.java      |  67 ---
 .../calcite/sql/dialect/OracleSqlDialect.java   |  78 ++++
 .../calcite/sql/dialect/ParaccelSqlDialect.java |  36 ++
 .../calcite/sql/dialect/PhoenixSqlDialect.java  |  40 ++
 .../calcite/sql/dialect/PostgresqlHandler.java  |  59 ---
 .../sql/dialect/PostgresqlSqlDialect.java       |  75 ++++
 .../calcite/sql/dialect/RedshiftSqlDialect.java |  40 ++
 .../calcite/sql/dialect/SybaseSqlDialect.java   |  35 ++
 .../calcite/sql/dialect/TeradataSqlDialect.java |  36 ++
 .../calcite/sql/dialect/VerticaSqlDialect.java  |  40 ++
 .../calcite/sql/pretty/SqlPrettyWriter.java     |   3 +-
 .../calcite/sql/type/IntervalSqlType.java       |   3 +-
 .../rel/rel2sql/RelToSqlConverterTest.java      |   3 +-
 .../calcite/sql/parser/SqlParserTest.java       |  20 +-
 .../calcite/sql/test/SqlOperatorBaseTest.java   |   9 +-
 .../calcite/sql/test/SqlPrettyWriterTest.java   |  26 +-
 .../apache/calcite/sql/test/SqlTesterImpl.java  |   4 +-
 .../org/apache/calcite/test/SqlLimitsTest.java  |   4 +-
 .../apache/calcite/tools/FrameworksTest.java    |   4 +-
 .../java/org/apache/calcite/util/Smalls.java    |   8 +-
 .../java/org/apache/calcite/util/UtilTest.java  |   6 +-
 63 files changed, 2114 insertions(+), 886 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchema.java
----------------------------------------------------------------------
diff --git a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchema.java b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchema.java
index 00a7a40..709c7b4 100644
--- a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchema.java
+++ b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchema.java
@@ -29,9 +29,9 @@ import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.Table;
 import org.apache.calcite.schema.impl.AbstractSchema;
 import org.apache.calcite.schema.impl.MaterializedViewTable;
-import org.apache.calcite.sql.SqlDialect;
 import org.apache.calcite.sql.SqlSelect;
 import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.dialect.CalciteSqlDialect;
 import org.apache.calcite.sql.parser.SqlParseException;
 import org.apache.calcite.sql.parser.SqlParser;
 import org.apache.calcite.sql.pretty.SqlPrettyWriter;
@@ -266,7 +266,7 @@ public class CassandraSchema extends AbstractSchema {
 
       StringWriter stringWriter = new StringWriter(query.length());
       PrintWriter printWriter = new PrintWriter(stringWriter);
-      SqlWriter writer = new SqlPrettyWriter(SqlDialect.CALCITE, true, printWriter);
+      SqlWriter writer = new SqlPrettyWriter(CalciteSqlDialect.DEFAULT, true, printWriter);
       parsedQuery.unparse(writer, 0, 0);
       query = stringWriter.toString();
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
index 736c78d..c4e4c82 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
@@ -69,7 +69,6 @@ import org.apache.calcite.rex.RexProgram;
 import org.apache.calcite.schema.ModifiableTable;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlDialect;
-import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.trace.CalciteTrace;
@@ -109,21 +108,6 @@ public class JdbcRules {
         new JdbcValuesRule(out));
   }
 
-  static final ImmutableList<SqlKind> AGG_FUNCS;
-  static final ImmutableList<SqlKind> MYSQL_AGG_FUNCS;
-
-  static {
-    ImmutableList.Builder<SqlKind> builder = ImmutableList.builder();
-    builder.add(SqlKind.COUNT);
-    builder.add(SqlKind.SUM);
-    builder.add(SqlKind.SUM0);
-    builder.add(SqlKind.MIN);
-    builder.add(SqlKind.MAX);
-    AGG_FUNCS = builder.build();
-    builder.add(SqlKind.SINGLE_VALUE);
-    MYSQL_AGG_FUNCS = builder.build();
-  }
-
   /** Abstract base class for rule that converts to JDBC. */
   abstract static class JdbcConverterRule extends ConverterRule {
     protected final JdbcConvention out;
@@ -477,12 +461,7 @@ public class JdbcRules {
   /** Returns whether this JDBC data source can implement a given aggregate
    * function. */
   private static boolean canImplement(SqlAggFunction aggregation, SqlDialect sqlDialect) {
-    switch (sqlDialect.getDatabaseProduct()) {
-    case MYSQL:
-      return MYSQL_AGG_FUNCS.contains(aggregation.getKind());
-    default:
-      return AGG_FUNCS.contains(aggregation.getKind());
-    }
+    return sqlDialect.supportsAggregateFunction(aggregation.getKind());
   }
 
   /** Aggregate operator implemented in JDBC convention. */

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcSchema.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcSchema.java
index 31e262e..8e7ecdf 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcSchema.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcSchema.java
@@ -32,6 +32,8 @@ import org.apache.calcite.schema.SchemaVersion;
 import org.apache.calcite.schema.Schemas;
 import org.apache.calcite.schema.Table;
 import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.SqlDialectFactory;
+import org.apache.calcite.sql.SqlDialectFactoryImpl;
 import org.apache.calcite.sql.type.SqlTypeFactoryImpl;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.Util;
@@ -102,9 +104,19 @@ public class JdbcSchema implements Schema {
       DataSource dataSource,
       String catalog,
       String schema) {
+    return create(parentSchema, name, dataSource, new SqlDialectFactoryImpl(), catalog, schema);
+  }
+
+  public static JdbcSchema create(
+      SchemaPlus parentSchema,
+      String name,
+      DataSource dataSource,
+      SqlDialectFactory dialectFactory,
+      String catalog,
+      String schema) {
     final Expression expression =
         Schemas.subSchemaExpression(parentSchema, name, JdbcSchema.class);
-    final SqlDialect dialect = createDialect(dataSource);
+    final SqlDialect dialect = createDialect(dialectFactory, dataSource);
     final JdbcConvention convention =
         JdbcConvention.of(dialect, expression, name);
     return new JdbcSchema(dataSource, dialect, convention, catalog, schema);
@@ -140,13 +152,35 @@ public class JdbcSchema implements Schema {
     }
     String jdbcCatalog = (String) operand.get("jdbcCatalog");
     String jdbcSchema = (String) operand.get("jdbcSchema");
-    return JdbcSchema.create(
-        parentSchema, name, dataSource, jdbcCatalog, jdbcSchema);
+    String sqlDialectFactory = (String) operand.get("sqlDialectFactory");
+
+    if (sqlDialectFactory == null || sqlDialectFactory.isEmpty()) {
+      return JdbcSchema.create(
+          parentSchema, name, dataSource, jdbcCatalog, jdbcSchema);
+    } else {
+      SqlDialectFactory factory = AvaticaUtils.instantiatePlugin(
+          SqlDialectFactory.class, sqlDialectFactory);
+      return JdbcSchema.create(
+          parentSchema, name, dataSource, factory, jdbcCatalog, jdbcSchema);
+    }
   }
 
-  /** Returns a suitable SQL dialect for the given data source. */
+  /**
+   * Returns a suitable SQL dialect for the given data source.
+   *
+   * @param dataSource The data source
+   *
+   * @deprecated Use {@link #createDialect(SqlDialectFactory, DataSource)} instead
+   */
+  @Deprecated // to be removed before 2.0
   public static SqlDialect createDialect(DataSource dataSource) {
-    return JdbcUtils.DialectPool.INSTANCE.get(dataSource);
+    return createDialect(new SqlDialectFactoryImpl(), dataSource);
+  }
+
+  /** Returns a suitable SQL dialect for the given data source. */
+  public static SqlDialect createDialect(SqlDialectFactory dialectFactory,
+      DataSource dataSource) {
+    return JdbcUtils.DialectPool.INSTANCE.get(dialectFactory, dataSource);
   }
 
   /** Creates a JDBC data source with the given specification. */

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcToEnumerableConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcToEnumerableConverter.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcToEnumerableConverter.java
index c74e741..61bf750 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcToEnumerableConverter.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcToEnumerableConverter.java
@@ -100,7 +100,8 @@ public class JdbcToEnumerableConverter
     final ParameterExpression resultSet_ =
         Expressions.parameter(Modifier.FINAL, ResultSet.class,
             builder.newName("resultSet"));
-    CalendarPolicy calendarPolicy = CalendarPolicy.of(jdbcConvention.dialect);
+    final SqlDialect.CalendarPolicy calendarPolicy =
+        jdbcConvention.dialect.getCalendarPolicy();
     final Expression calendar_;
     switch (calendarPolicy) {
     case LOCAL:
@@ -179,7 +180,7 @@ public class JdbcToEnumerableConverter
   private void generateGet(EnumerableRelImplementor implementor,
       PhysType physType, BlockBuilder builder, ParameterExpression resultSet_,
       int i, Expression target, Expression calendar_,
-      CalendarPolicy calendarPolicy) {
+      SqlDialect.CalendarPolicy calendarPolicy) {
     final Primitive primitive = Primitive.ofBoxOr(physType.fieldClass(i));
     final RelDataType fieldType =
         physType.getRowType().getFieldList().get(i).getType();
@@ -299,27 +300,6 @@ public class JdbcToEnumerableConverter
         jdbcImplementor.visitChild(0, getInput());
     return result.asStatement().toSqlString(dialect).getSql();
   }
-
-  /** Whether this JDBC driver needs you to pass a Calendar object to methods
-   * such as {@link ResultSet#getTimestamp(int, java.util.Calendar)}. */
-  private enum CalendarPolicy {
-    NONE,
-    NULL,
-    LOCAL,
-    DIRECT,
-    SHIFT;
-
-    static CalendarPolicy of(SqlDialect dialect) {
-      switch (dialect.getDatabaseProduct()) {
-      case MYSQL:
-        return SHIFT;
-      case HSQLDB:
-      default:
-        // NULL works for hsqldb-2.3; nothing worked for hsqldb-1.8.
-        return NULL;
-      }
-    }
-  }
 }
 
 // End JdbcToEnumerableConverter.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcUtils.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcUtils.java
index bb8e558..0fe1761 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcUtils.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcUtils.java
@@ -21,6 +21,7 @@ import org.apache.calcite.avatica.util.DateTimeUtils;
 import org.apache.calcite.linq4j.function.Function0;
 import org.apache.calcite.linq4j.function.Function1;
 import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.SqlDialectFactory;
 import org.apache.calcite.util.ImmutableNullableList;
 import org.apache.calcite.util.Pair;
 
@@ -58,15 +59,19 @@ final class JdbcUtils {
 
   /** Pool of dialects. */
   static class DialectPool {
-    final Map<DataSource, SqlDialect> map0 = new IdentityHashMap<>();
+    final Map<DataSource, Map<SqlDialectFactory, SqlDialect>> map0 = new IdentityHashMap<>();
     final Map<List, SqlDialect> map = new HashMap<>();
 
     public static final DialectPool INSTANCE = new DialectPool();
 
-    SqlDialect get(DataSource dataSource) {
-      final SqlDialect sqlDialect = map0.get(dataSource);
-      if (sqlDialect != null) {
-        return sqlDialect;
+    // TODO: Discuss why we need a pool. If we do, I'd like to improve performance
+    synchronized SqlDialect get(SqlDialectFactory dialectFactory, DataSource dataSource) {
+      Map<SqlDialectFactory, SqlDialect> dialectMap = map0.get(dataSource);
+      if (dialectMap != null) {
+        final SqlDialect sqlDialect = dialectMap.get(dialectFactory);
+        if (sqlDialect != null) {
+          return sqlDialect;
+        }
       }
       Connection connection = null;
       try {
@@ -74,12 +79,16 @@ final class JdbcUtils {
         DatabaseMetaData metaData = connection.getMetaData();
         String productName = metaData.getDatabaseProductName();
         String productVersion = metaData.getDatabaseProductVersion();
-        List key = ImmutableList.of(productName, productVersion);
+        List key = ImmutableList.of(productName, productVersion, dialectFactory);
         SqlDialect dialect = map.get(key);
         if (dialect == null) {
-          dialect = SqlDialect.create(metaData);
+          dialect = dialectFactory.create(metaData);
           map.put(key, dialect);
-          map0.put(dataSource, dialect);
+          if (dialectMap == null) {
+            dialectMap = new IdentityHashMap<>();
+            map0.put(dataSource, dialectMap);
+          }
+          dialectMap.put(dialectFactory, dialect);
         }
         connection.close();
         connection = null;

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/model/JsonJdbcSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/model/JsonJdbcSchema.java b/core/src/main/java/org/apache/calcite/model/JsonJdbcSchema.java
index 761dab5..4c8aaf5 100644
--- a/core/src/main/java/org/apache/calcite/model/JsonJdbcSchema.java
+++ b/core/src/main/java/org/apache/calcite/model/JsonJdbcSchema.java
@@ -32,6 +32,13 @@ public class JsonJdbcSchema extends JsonSchema {
    */
   public String jdbcDriver;
 
+  /** The FQN of the {@link org.apache.calcite.sql.SqlDialectFactory} implementation.
+   *
+   * <p>Optional. If not specified, uses whichever class the JDBC
+   * {@link java.sql.DriverManager} chooses.
+   */
+  public String sqlDialectFactory;
+
   /** JDBC connect string, for example "jdbc:mysql://localhost/foodmart".
    *
    * <p>Optional.

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/model/ModelHandler.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/model/ModelHandler.java b/core/src/main/java/org/apache/calcite/model/ModelHandler.java
index 6af4395..2f2419d 100644
--- a/core/src/main/java/org/apache/calcite/model/ModelHandler.java
+++ b/core/src/main/java/org/apache/calcite/model/ModelHandler.java
@@ -37,6 +37,7 @@ import org.apache.calcite.schema.impl.ScalarFunctionImpl;
 import org.apache.calcite.schema.impl.TableFunctionImpl;
 import org.apache.calcite.schema.impl.TableMacroImpl;
 import org.apache.calcite.schema.impl.ViewTable;
+import org.apache.calcite.sql.SqlDialectFactory;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
@@ -320,9 +321,18 @@ public class ModelHandler {
             jsonSchema.jdbcDriver,
             jsonSchema.jdbcUser,
             jsonSchema.jdbcPassword);
-    JdbcSchema schema =
-        JdbcSchema.create(parentSchema, jsonSchema.name, dataSource,
-            jsonSchema.jdbcCatalog, jsonSchema.jdbcSchema);
+    final JdbcSchema schema;
+    if (jsonSchema.sqlDialectFactory == null || jsonSchema.sqlDialectFactory.isEmpty()) {
+      schema =
+          JdbcSchema.create(parentSchema, jsonSchema.name, dataSource,
+              jsonSchema.jdbcCatalog, jsonSchema.jdbcSchema);
+    } else {
+      SqlDialectFactory factory = AvaticaUtils.instantiatePlugin(
+          SqlDialectFactory.class, jsonSchema.sqlDialectFactory);
+      schema =
+          JdbcSchema.create(parentSchema, jsonSchema.name, dataSource,
+              factory, jsonSchema.jdbcCatalog, jsonSchema.jdbcSchema);
+    }
     final SchemaPlus schemaPlus = parentSchema.add(jsonSchema.name, schema);
     populateSchema(jsonSchema, schemaPlus);
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
index ec57a8d..74b62d5 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
@@ -1120,7 +1120,6 @@ public class CalcitePrepareImpl implements CalcitePrepare {
 
     private PreparedResult prepare_(Supplier<RelNode> fn,
         RelDataType resultType) {
-      queryString = null;
       Class runtimeContextClass = Object.class;
       init(runtimeContextClass);
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/prepare/Prepare.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/Prepare.java b/core/src/main/java/org/apache/calcite/prepare/Prepare.java
index 8cb09c3..42d8cdf 100644
--- a/core/src/main/java/org/apache/calcite/prepare/Prepare.java
+++ b/core/src/main/java/org/apache/calcite/prepare/Prepare.java
@@ -87,7 +87,6 @@ public abstract class Prepare {
 
   protected final CalcitePrepare.Context context;
   protected final CatalogReader catalogReader;
-  protected String queryString = null;
   /**
    * Convention via which results should be returned by execution.
    */
@@ -241,8 +240,6 @@ public abstract class Prepare {
       Class runtimeContextClass,
       SqlValidator validator,
       boolean needsValidation) {
-    queryString = sqlQuery.toString();
-
     init(runtimeContextClass);
 
     final SqlToRelConverter.ConfigBuilder builder =

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/rel/rel2sql/RelToSqlConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rel2sql/RelToSqlConverter.java b/core/src/main/java/org/apache/calcite/rel/rel2sql/RelToSqlConverter.java
index b4e1a08..45ec2f7 100644
--- a/core/src/main/java/org/apache/calcite/rel/rel2sql/RelToSqlConverter.java
+++ b/core/src/main/java/org/apache/calcite/rel/rel2sql/RelToSqlConverter.java
@@ -198,8 +198,8 @@ public class RelToSqlConverter extends SqlImplementor
     for (AggregateCall aggCall : e.getAggCallList()) {
       SqlNode aggCallSqlNode = builder.context.toSql(aggCall);
       if (aggCall.getAggregation() instanceof SqlSingleValueAggFunction) {
-        aggCallSqlNode =
-            rewriteSingleValueExpr(aggCallSqlNode, dialect);
+        aggCallSqlNode = dialect.
+            rewriteSingleValueExpr(aggCallSqlNode);
       }
       addSelect(selectList, aggCallSqlNode, e.getRowType());
     }
@@ -397,12 +397,15 @@ public class RelToSqlConverter extends SqlImplementor
     final List<SqlNode> orderBySqlList = new ArrayList<>();
     if (e.getOrderKeys() != null) {
       for (RelFieldCollation fc : e.getOrderKeys().getFieldCollations()) {
-        if (fc.nullDirection != RelFieldCollation.NullDirection.UNSPECIFIED
-            && dialect.getDatabaseProduct() == SqlDialect.DatabaseProduct.MYSQL) {
-          orderBySqlList.add(
-              ISNULL_FUNCTION.createCall(POS, context.field(fc.getFieldIndex())));
-          fc = new RelFieldCollation(fc.getFieldIndex(), fc.getDirection(),
-              RelFieldCollation.NullDirection.UNSPECIFIED);
+        if (fc.nullDirection != RelFieldCollation.NullDirection.UNSPECIFIED) {
+          boolean first = fc.nullDirection == RelFieldCollation.NullDirection.FIRST;
+          SqlNode nullDirectionNode = dialect.emulateNullDirection(
+              context.field(fc.getFieldIndex()), first);
+          if (nullDirectionNode != null) {
+            orderBySqlList.add(nullDirectionNode);
+            fc = new RelFieldCollation(fc.getFieldIndex(), fc.getDirection(),
+                RelFieldCollation.NullDirection.UNSPECIFIED);
+          }
         }
         orderBySqlList.add(context.toSql(fc));
       }

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java b/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java
index 3a4b649..0304743 100644
--- a/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java
+++ b/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java
@@ -46,11 +46,8 @@ import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlBasicCall;
 import org.apache.calcite.sql.SqlBinaryOperator;
 import org.apache.calcite.sql.SqlCall;
-import org.apache.calcite.sql.SqlDataTypeSpec;
 import org.apache.calcite.sql.SqlDialect;
 import org.apache.calcite.sql.SqlDynamicParam;
-import org.apache.calcite.sql.SqlFunction;
-import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlJoin;
 import org.apache.calcite.sql.SqlKind;
@@ -67,13 +64,8 @@ import org.apache.calcite.sql.fun.SqlCase;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.fun.SqlSumEmptyIsZeroAggFunction;
 import org.apache.calcite.sql.parser.SqlParserPos;
-import org.apache.calcite.sql.type.BasicSqlType;
-import org.apache.calcite.sql.type.InferTypes;
-import org.apache.calcite.sql.type.OperandTypes;
-import org.apache.calcite.sql.type.ReturnTypes;
 import org.apache.calcite.sql.type.SqlTypeFamily;
 import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.util.DateString;
 import org.apache.calcite.util.TimeString;
@@ -86,9 +78,6 @@ import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import java.math.BigDecimal;
 import java.util.AbstractList;
 import java.util.ArrayList;
@@ -107,17 +96,9 @@ import java.util.Set;
  * State for generating a SQL statement.
  */
 public abstract class SqlImplementor {
-  private static final Logger LOGGER =
-      LoggerFactory.getLogger(SqlImplementor.class);
 
   public static final SqlParserPos POS = SqlParserPos.ZERO;
 
-  /** MySQL specific function. */
-  public static final SqlFunction ISNULL_FUNCTION =
-      new SqlFunction("ISNULL", SqlKind.OTHER_FUNCTION,
-          ReturnTypes.BOOLEAN, InferTypes.FIRST_KNOWN,
-          OperandTypes.ANY, SqlFunctionCategory.SYSTEM);
-
   public final SqlDialect dialect;
   protected final Set<String> aliasSet = new LinkedHashSet<>();
   protected final Map<String, SqlNode> ordinalMap = new HashMap<>();
@@ -130,82 +111,6 @@ public abstract class SqlImplementor {
 
   public abstract Result visitChild(int i, RelNode e);
 
-  /** Rewrite SINGLE_VALUE into expression based on database variants
-   *  E.g. HSQLDB, MYSQL, ORACLE, etc
-   */
-  public static SqlNode rewriteSingleValueExpr(SqlNode aggCall,
-      SqlDialect sqlDialect) {
-    final SqlNode operand = ((SqlBasicCall) aggCall).operand(0);
-    final SqlNode caseOperand;
-    final SqlNode elseExpr;
-    final SqlNode countCall =
-        SqlStdOperatorTable.COUNT.createCall(POS, operand);
-
-    final SqlLiteral nullLiteral = SqlLiteral.createNull(POS);
-    final SqlNode wrappedOperand;
-    switch (sqlDialect.getDatabaseProduct()) {
-    case MYSQL:
-    case HSQLDB:
-      // For MySQL, generate
-      //   CASE COUNT(*)
-      //   WHEN 0 THEN NULL
-      //   WHEN 1 THEN <result>
-      //   ELSE (SELECT NULL UNION ALL SELECT NULL)
-      //   END
-      //
-      // For hsqldb, generate
-      //   CASE COUNT(*)
-      //   WHEN 0 THEN NULL
-      //   WHEN 1 THEN MIN(<result>)
-      //   ELSE (VALUES 1 UNION ALL VALUES 1)
-      //   END
-      caseOperand = countCall;
-
-      final SqlNodeList selectList = new SqlNodeList(POS);
-      selectList.add(nullLiteral);
-      final SqlNode unionOperand;
-      switch (sqlDialect.getDatabaseProduct()) {
-      case MYSQL:
-        wrappedOperand = operand;
-        unionOperand = new SqlSelect(POS, SqlNodeList.EMPTY, selectList,
-            null, null, null, null, SqlNodeList.EMPTY, null, null, null);
-        break;
-      default:
-        wrappedOperand = SqlStdOperatorTable.MIN.createCall(POS, operand);
-        unionOperand = SqlStdOperatorTable.VALUES.createCall(POS,
-            SqlLiteral.createApproxNumeric("0", POS));
-      }
-
-      SqlCall unionAll = SqlStdOperatorTable.UNION_ALL
-          .createCall(POS, unionOperand, unionOperand);
-
-      final SqlNodeList selectList2 = new SqlNodeList(POS);
-      selectList2.add(nullLiteral);
-      elseExpr = SqlStdOperatorTable.SCALAR_QUERY.createCall(POS, unionAll);
-      break;
-
-    default:
-      LOGGER.debug("SINGLE_VALUE rewrite not supported for {}",
-          sqlDialect.getDatabaseProduct());
-      return aggCall;
-    }
-
-    final SqlNodeList whenList = new SqlNodeList(POS);
-    whenList.add(SqlLiteral.createExactNumeric("0", POS));
-    whenList.add(SqlLiteral.createExactNumeric("1", POS));
-
-    final SqlNodeList thenList = new SqlNodeList(POS);
-    thenList.add(nullLiteral);
-    thenList.add(wrappedOperand);
-
-    SqlNode caseExpr =
-        new SqlCase(POS, caseOperand, whenList, thenList, elseExpr);
-
-    LOGGER.debug("SINGLE_VALUE rewritten into [{}]", caseExpr);
-
-    return caseExpr;
-  }
-
   public void addSelect(List<SqlNode> selectList, SqlNode node,
       RelDataType rowType) {
     String name = rowType.getFieldNames().get(selectList.size());
@@ -680,7 +585,7 @@ public abstract class SqlImplementor {
             assert nodeList.size() == 1;
             return nodeList.get(0);
           } else {
-            nodeList.add(toSql(call.getType()));
+            nodeList.add(dialect.getCastSpec(call.getType()));
           }
         }
         if (op instanceof SqlBinaryOperator && nodeList.size() > 2) {
@@ -788,35 +693,6 @@ public abstract class SqlImplementor {
       return op.createCall(new SqlNodeList(ImmutableList.of(call, last), POS));
     }
 
-    private SqlNode toSql(RelDataType type) {
-      switch (dialect.getDatabaseProduct()) {
-      case MYSQL:
-        switch (type.getSqlTypeName()) {
-        case VARCHAR:
-          // MySQL doesn't have a VARCHAR type, only CHAR.
-          return new SqlDataTypeSpec(new SqlIdentifier("CHAR", POS),
-              type.getPrecision(), -1, null, null, POS);
-        case INTEGER:
-          return new SqlDataTypeSpec(new SqlIdentifier("_UNSIGNED", POS),
-              type.getPrecision(), -1, null, null, POS);
-        }
-        break;
-      }
-      if (type instanceof BasicSqlType) {
-        return new SqlDataTypeSpec(
-            new SqlIdentifier(type.getSqlTypeName().name(), POS),
-            type.getPrecision(),
-            type.getScale(),
-            type.getCharset() != null
-            && dialect.supportsCharSet()
-                ? type.getCharset().name()
-                : null,
-            null,
-            POS);
-      }
-      return SqlTypeUtil.convertTypeToSpec(type);
-    }
-
     private List<SqlNode> toSql(RexProgram program, List<RexNode> operandList) {
       final List<SqlNode> list = new ArrayList<>();
       for (RexNode rex : operandList) {
@@ -1195,7 +1071,7 @@ public abstract class SqlImplementor {
   public class Builder {
     private final RelNode rel;
     final List<Clause> clauses;
-    private final SqlSelect select;
+    final SqlSelect select;
     public final Context context;
     private final Map<String, RelDataType> aliases;
 
@@ -1244,14 +1120,16 @@ public abstract class SqlImplementor {
 
     public void addOrderItem(List<SqlNode> orderByList,
         RelFieldCollation field) {
-      if (field.nullDirection != RelFieldCollation.NullDirection.UNSPECIFIED
-          && dialect.getDatabaseProduct() == SqlDialect.DatabaseProduct.MYSQL) {
-        orderByList.add(
-            ISNULL_FUNCTION.createCall(POS,
-                context.field(field.getFieldIndex())));
-        field = new RelFieldCollation(field.getFieldIndex(),
-            field.getDirection(),
-            RelFieldCollation.NullDirection.UNSPECIFIED);
+      if (field.nullDirection != RelFieldCollation.NullDirection.UNSPECIFIED) {
+        boolean first = field.nullDirection == RelFieldCollation.NullDirection.FIRST;
+        SqlNode nullDirectionNode = dialect.emulateNullDirection(
+            context.field(field.getFieldIndex()), first);
+        if (nullDirectionNode != null) {
+          orderByList.add(nullDirectionNode);
+          field = new RelFieldCollation(field.getFieldIndex(),
+              field.getDirection(),
+              RelFieldCollation.NullDirection.UNSPECIFIED);
+        }
       }
       orderByList.add(context.toSql(field));
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/SqlAbstractDateTimeLiteral.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlAbstractDateTimeLiteral.java b/core/src/main/java/org/apache/calcite/sql/SqlAbstractDateTimeLiteral.java
index 59dfc69..05d77d1 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlAbstractDateTimeLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlAbstractDateTimeLiteral.java
@@ -33,7 +33,7 @@ import org.apache.calcite.util.TimestampString;
  * <li><code>TIMESTAMP '1969-07-21 03:15 GMT'</code></li>
  * </ul>
  */
-abstract class SqlAbstractDateTimeLiteral extends SqlLiteral {
+public abstract class SqlAbstractDateTimeLiteral extends SqlLiteral {
   //~ Instance fields --------------------------------------------------------
 
   protected final boolean hasTimeZone;

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/SqlDateLiteral.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlDateLiteral.java b/core/src/main/java/org/apache/calcite/sql/SqlDateLiteral.java
index 27cc8b4..ceb2156 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDateLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDateLiteral.java
@@ -65,14 +65,7 @@ public class SqlDateLiteral extends SqlAbstractDateTimeLiteral {
       SqlWriter writer,
       int leftPrec,
       int rightPrec) {
-    switch (writer.getDialect().getDatabaseProduct()) {
-    case MSSQL:
-      writer.literal("'" + this.toFormattedString() + "'");
-      break;
-    default:
-      writer.literal(this.toString());
-      break;
-    }
+    writer.getDialect().unparseDateTimeLiteral(writer, this, leftPrec, rightPrec);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlDialect.java b/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
index dafa8d9..f9cdf67 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
@@ -18,23 +18,29 @@ package org.apache.calcite.sql;
 
 import org.apache.calcite.avatica.util.DateTimeUtils;
 import org.apache.calcite.config.NullCollation;
-import org.apache.calcite.linq4j.function.Experimental;
 import org.apache.calcite.rel.RelFieldCollation;
-import org.apache.calcite.sql.dialect.HsqldbHandler;
-import org.apache.calcite.sql.dialect.MssqlHandler;
-import org.apache.calcite.sql.dialect.MysqlHandler;
-import org.apache.calcite.sql.dialect.OracleHandler;
-import org.apache.calcite.sql.dialect.PostgresqlHandler;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.dialect.AnsiSqlDialect;
+import org.apache.calcite.sql.dialect.CalciteSqlDialect;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.BasicSqlType;
+import org.apache.calcite.sql.type.SqlTypeUtil;
 
 import com.google.common.base.Preconditions;
+import com.google.common.base.Supplier;
+import com.google.common.base.Suppliers;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.sql.DatabaseMetaData;
-import java.sql.SQLException;
+import java.sql.ResultSet;
 import java.sql.Timestamp;
 import java.text.SimpleDateFormat;
 import java.util.List;
 import java.util.Locale;
 import java.util.regex.Pattern;
+import javax.annotation.Nonnull;
 
 /**
  * <code>SqlDialect</code> encapsulates the differences between dialects of SQL.
@@ -45,24 +51,21 @@ import java.util.regex.Pattern;
 public class SqlDialect {
   //~ Static fields/initializers ---------------------------------------------
 
-  private static final Handler DEFAULT_HANDLER = new BaseHandler();
+  protected static final Logger LOGGER =
+      LoggerFactory.getLogger(SqlDialect.class);
 
-  /**
-   * A dialect useful for generating generic SQL. If you need to do something
-   * database-specific like quoting identifiers, don't rely on this dialect to
-   * do what you want.
-   */
+  /** Empty context. */
+  public static final Context EMPTY_CONTEXT = emptyContext();
+
+  /** @deprecated Use {@link AnsiSqlDialect#DEFAULT} instead. */
+  @Deprecated // to be removed before 2.0
   public static final SqlDialect DUMMY =
-      DatabaseProduct.UNKNOWN.getDialect();
+      AnsiSqlDialect.DEFAULT;
 
-  /**
-   * A dialect useful for generating SQL which can be parsed by the
-   * Calcite parser, in particular quoting literals and identifiers. If you
-   * want a dialect that knows the full capabilities of the database, create
-   * one from a connection.
-   */
+  /** @deprecated Use {@link CalciteSqlDialect#DEFAULT} instead. */
+  @Deprecated // to be removed before 2.0
   public static final SqlDialect CALCITE =
-      DatabaseProduct.CALCITE.getDialect();
+      CalciteSqlDialect.DEFAULT;
 
   //~ Instance fields --------------------------------------------------------
 
@@ -71,7 +74,6 @@ public class SqlDialect {
   private final String identifierEscapedQuote;
   private final DatabaseProduct databaseProduct;
   private final NullCollation nullCollation;
-  private final Handler handler;
 
   //~ Constructors -----------------------------------------------------------
 
@@ -82,63 +84,22 @@ public class SqlDialect {
    * importantly, to its {@link java.sql.Connection} -- after this call has
    * returned.
    *
-   * @param databaseMetaData used to determine which dialect of SQL to
-   *                         generate
+   * @param databaseMetaData used to determine which dialect of SQL to generate
+   *
+   * @deprecated Replaced by {@link SqlDialectFactory}
    */
+  @Deprecated // to be removed before 2.0
   public static SqlDialect create(DatabaseMetaData databaseMetaData) {
-    String identifierQuoteString;
-    try {
-      identifierQuoteString = databaseMetaData.getIdentifierQuoteString();
-    } catch (SQLException e) {
-      throw FakeUtil.newInternal(e, "while quoting identifier");
-    }
-    String databaseProductName;
-    try {
-      databaseProductName = databaseMetaData.getDatabaseProductName();
-    } catch (SQLException e) {
-      throw FakeUtil.newInternal(e, "while detecting database product");
-    }
-    String databaseProductVersion;
-    try {
-      databaseProductVersion = databaseMetaData.getDatabaseProductVersion();
-    } catch (SQLException e) {
-      throw FakeUtil.newInternal(e, "while detecting database version");
-    }
-    final DatabaseProduct databaseProduct =
-        getProduct(databaseProductName, databaseProductVersion);
-    NullCollation nullCollation;
-    try {
-      if (databaseMetaData.nullsAreSortedAtEnd()) {
-        nullCollation = NullCollation.LAST;
-      } else if (databaseMetaData.nullsAreSortedAtStart()) {
-        nullCollation = NullCollation.FIRST;
-      } else if (databaseMetaData.nullsAreSortedLow()) {
-        nullCollation = NullCollation.LOW;
-      } else if (databaseMetaData.nullsAreSortedHigh()) {
-        nullCollation = NullCollation.HIGH;
-      } else {
-        throw new IllegalArgumentException("cannot deduce null collation");
-      }
-    } catch (SQLException e) {
-      throw new IllegalArgumentException("cannot deduce null collation", e);
-    }
-    Handler handler = chooseHandler(databaseProduct);
-    return new SqlDialect(databaseProduct, databaseProductName,
-        identifierQuoteString, nullCollation, handler);
+    return new SqlDialectFactoryImpl().create(databaseMetaData);
   }
 
   @Deprecated // to be removed before 2.0
   public SqlDialect(DatabaseProduct databaseProduct, String databaseProductName,
       String identifierQuoteString) {
-    this(databaseProduct, databaseProductName, identifierQuoteString,
-        NullCollation.HIGH, DEFAULT_HANDLER);
-  }
-
-  /** Creates a SqlDialect with the default handler. */
-  public SqlDialect(DatabaseProduct databaseProduct, String databaseProductName,
-      String identifierQuoteString, NullCollation nullCollation) {
-    this(databaseProduct, databaseProductName, identifierQuoteString,
-        nullCollation, DEFAULT_HANDLER);
+    this(EMPTY_CONTEXT
+        .withDatabaseProduct(databaseProduct)
+        .withDatabaseProductName(databaseProductName)
+        .withIdentifierQuoteString(identifierQuoteString));
   }
 
   /**
@@ -150,16 +111,29 @@ public class SqlDialect {
    *                              is not supported. If "[", close quote is
    *                              deemed to be "]".
    * @param nullCollation         Whether NULL values appear first or last
-   * @param handler               Handler for un-parsing
+   *
+   * @deprecated Use {@link #SqlDialect(Context)}
    */
-  @Experimental
+  @Deprecated // to be removed before 2.0
   public SqlDialect(DatabaseProduct databaseProduct, String databaseProductName,
-      String identifierQuoteString, NullCollation nullCollation,
-      Handler handler) {
-    this.nullCollation = Preconditions.checkNotNull(nullCollation);
-    Preconditions.checkNotNull(databaseProductName);
-    this.databaseProduct = Preconditions.checkNotNull(databaseProduct);
-    this.handler = Preconditions.checkNotNull(handler);
+      String identifierQuoteString, NullCollation nullCollation) {
+    this(EMPTY_CONTEXT
+        .withDatabaseProduct(databaseProduct)
+        .withDatabaseProductName(databaseProductName)
+        .withIdentifierQuoteString(identifierQuoteString)
+        .withNullCollation(nullCollation));
+  }
+
+  /**
+   * Creates a SqlDialect.
+   *
+   * @param context All the information necessary to create a dialect
+   */
+  public SqlDialect(Context context) {
+    this.nullCollation = Preconditions.checkNotNull(context.nullCollation());
+    this.databaseProduct =
+        Preconditions.checkNotNull(context.databaseProduct());
+    String identifierQuoteString = context.identifierQuoteString();
     if (identifierQuoteString != null) {
       identifierQuoteString = identifierQuoteString.trim();
       if (identifierQuoteString.equals("")) {
@@ -168,19 +142,22 @@ public class SqlDialect {
     }
     this.identifierQuoteString = identifierQuoteString;
     this.identifierEndQuoteString =
-        identifierQuoteString == null
-            ? null
-            : identifierQuoteString.equals("[")
-                ? "]"
-                : identifierQuoteString;
+        identifierQuoteString == null ? null
+            : identifierQuoteString.equals("[") ? "]"
+            : identifierQuoteString;
     this.identifierEscapedQuote =
-        identifierQuoteString == null
-            ? null
+        identifierQuoteString == null ? null
             : this.identifierEndQuoteString + this.identifierEndQuoteString;
   }
 
   //~ Methods ----------------------------------------------------------------
 
+  /** Creates an empty context. Use {@link #EMPTY_CONTEXT} if possible. */
+  protected static Context emptyContext() {
+    return new ContextImpl(DatabaseProduct.UNKNOWN, null, null, null,
+        NullCollation.HIGH);
+  }
+
   /**
    * Converts a product name and version (per the JDBC driver) into a product
    * enumeration.
@@ -189,6 +166,7 @@ public class SqlDialect {
    * @param productVersion Product version
    * @return database product
    */
+  @Deprecated // to be removed before 2.0
   public static DatabaseProduct getProduct(
       String productName,
       String productVersion) {
@@ -252,24 +230,6 @@ public class SqlDialect {
     }
   }
 
-  /** Chooses the best handler for a given database product. */
-  private static Handler chooseHandler(DatabaseProduct databaseProduct) {
-    switch (databaseProduct) {
-    case HSQLDB:
-      return HsqldbHandler.INSTANCE;
-    case MSSQL:
-      return MssqlHandler.INSTANCE;
-    case MYSQL:
-      return MysqlHandler.INSTANCE;
-    case ORACLE:
-      return OracleHandler.INSTANCE;
-    case POSTGRESQL:
-      return PostgresqlHandler.INSTANCE;
-    default:
-      return DEFAULT_HANDLER;
-    }
-  }
-
   /**
    * Encloses an identifier in quotation marks appropriate for the current SQL
    * dialect.
@@ -365,7 +325,12 @@ public class SqlDialect {
 
   public void unparseCall(SqlWriter writer, SqlCall call, int leftPrec,
       int rightPrec) {
-    handler.unparseCall(writer, call, leftPrec, rightPrec);
+    call.getOperator().unparse(writer, call, leftPrec, rightPrec);
+  }
+
+  public void unparseDateTimeLiteral(SqlWriter writer,
+      SqlAbstractDateTimeLiteral literal, int leftPrec, int rightPrec) {
+    writer.literal(literal.toString());
   }
 
   /**
@@ -434,18 +399,12 @@ public class SqlDialect {
   }
 
   protected boolean allowsAs() {
-    switch (databaseProduct) {
-    case ORACLE:
-    case HIVE:
-      return false;
-    default:
-      return true;
-    }
+    return true;
   }
 
   // -- behaviors --
   protected boolean requiresAliasForFromItems() {
-    return getDatabaseProduct() == DatabaseProduct.POSTGRESQL;
+    return false;
   }
 
   /** Returns whether a qualified table in the FROM clause has an implicit alias
@@ -471,12 +430,7 @@ public class SqlDialect {
    * <p>Returns true for all databases except DB2.
    */
   public boolean hasImplicitTableAlias() {
-    switch (databaseProduct) {
-    case DB2:
-      return false;
-    default:
-      return true;
-    }
+    return true;
   }
 
   /**
@@ -511,8 +465,15 @@ public class SqlDialect {
    * Returns the database this dialect belongs to,
    * {@link SqlDialect.DatabaseProduct#UNKNOWN} if not known, never null.
    *
+   * <p>Please be judicious in how you use this method. If you wish to determine
+   * whether a dialect has a particular capability or behavior, it is usually
+   * better to add a method to SqlDialect and override that method in particular
+   * sub-classes of SqlDialect.
+   *
    * @return Database product
+   * @deprecated To be removed without replacement
    */
+  @Deprecated // to be removed before 2.0
   public DatabaseProduct getDatabaseProduct() {
     return databaseProduct;
   }
@@ -522,18 +483,59 @@ public class SqlDialect {
    * data type, for instance {@code VARCHAR(30) CHARACTER SET `ISO-8859-1`}.
    */
   public boolean supportsCharSet() {
-    switch (databaseProduct) {
-    case DB2:
-    case H2:
-    case HSQLDB:
-    case MYSQL:
-    case ORACLE:
-    case PHOENIX:
-    case POSTGRESQL:
-      return false;
-    default:
+    return true;
+  }
+
+  public boolean supportsAggregateFunction(SqlKind kind) {
+    switch (kind) {
+    case COUNT:
+    case SUM:
+    case SUM0:
+    case MIN:
+    case MAX:
       return true;
     }
+    return false;
+  }
+
+  public CalendarPolicy getCalendarPolicy() {
+    return CalendarPolicy.NULL;
+  }
+
+  public SqlNode getCastSpec(RelDataType type) {
+    if (type instanceof BasicSqlType) {
+      return new SqlDataTypeSpec(
+          new SqlIdentifier(type.getSqlTypeName().name(), SqlParserPos.ZERO),
+              type.getPrecision(),
+              type.getScale(),
+              type.getCharset() != null
+                  && supportsCharSet()
+                  ? type.getCharset().name()
+                  : null,
+              null,
+              SqlParserPos.ZERO);
+    }
+    return SqlTypeUtil.convertTypeToSpec(type);
+  }
+
+  /** Rewrite SINGLE_VALUE into expression based on database variants
+   *  E.g. HSQLDB, MYSQL, ORACLE, etc
+   */
+  public SqlNode rewriteSingleValueExpr(SqlNode aggCall) {
+    LOGGER.debug("SINGLE_VALUE rewrite not supported for {}", databaseProduct);
+    return aggCall;
+  }
+
+  /**
+   * Returns the SqlNode for emulating the null direction for the given field
+   * or <code>null</code> if no emulation needs to be done.
+   *
+   * @param node The SqlNode representing the expression
+   * @param nullsFirst <code>true</code> if nulls should come first, <code>false</code> otherwise
+   * @return A SqlNode for null direction emulation or <code>null</code> if not required
+   */
+  public SqlNode emulateNullDirection(SqlNode node, boolean nullsFirst) {
+    return null;
   }
 
   /**
@@ -544,14 +546,7 @@ public class SqlDialect {
    * {@code LIMIT 20 OFFSET 10}.
    */
   public boolean supportsOffsetFetch() {
-    switch (databaseProduct) {
-    case MYSQL:
-    case HIVE:
-    case REDSHIFT:
-      return false;
-    default:
-      return true;
-    }
+    return true;
   }
 
   /**
@@ -559,14 +554,7 @@ public class SqlDialect {
    * {@code SELECT SUM(SUM(1)) }.
    */
   public boolean supportsNestedAggregations() {
-    switch (databaseProduct) {
-    case MYSQL:
-    case VERTICA:
-    case POSTGRESQL:
-      return false;
-    default:
-      return true;
-    }
+    return true;
   }
 
   /** Returns how NULL values are sorted if an ORDER BY item does not contain
@@ -641,6 +629,17 @@ public class SqlDialect {
     }
   }
 
+
+  /** Whether this JDBC driver needs you to pass a Calendar object to methods
+   * such as {@link ResultSet#getTimestamp(int, java.util.Calendar)}. */
+  public enum CalendarPolicy {
+    NONE,
+    NULL,
+    LOCAL,
+    DIRECT,
+    SHIFT;
+  }
+
   /**
    * Rough list of flavors of database.
    *
@@ -693,7 +692,22 @@ public class SqlDialect {
      */
     UNKNOWN("Unknown", "`", NullCollation.HIGH);
 
-    private SqlDialect dialect = null;
+    private final Supplier<SqlDialect> dialect =
+        Suppliers.memoize(new Supplier<SqlDialect>() {
+          public SqlDialect get() {
+            final SqlDialect dialect =
+                SqlDialectFactoryImpl.simple(DatabaseProduct.this);
+            if (dialect != null) {
+              return dialect;
+            }
+            return new SqlDialect(SqlDialect.EMPTY_CONTEXT
+                .withDatabaseProduct(DatabaseProduct.this)
+                .withDatabaseProductName(databaseProductName)
+                .withIdentifierQuoteString(quoteString)
+                .withNullCollation(nullCollation));
+          }
+        });
+
     private String databaseProductName;
     private String quoteString;
     private final NullCollation nullCollation;
@@ -718,34 +732,89 @@ public class SqlDialect {
      * all versions of this database
      */
     public SqlDialect getDialect() {
-      if (dialect == null) {
-        final Handler handler = chooseHandler(this);
-        dialect =
-            new SqlDialect(this, databaseProductName, quoteString,
-                nullCollation, handler);
-      }
-      return dialect;
+      return dialect.get();
     }
   }
 
-  /**
-   * A handler for converting {@link SqlNode} into SQL text of a particular
-   * dialect.
+  /** Information for creating a dialect.
    *
-   * <p>Instances are stateless and therefore immutable.
-   */
-  @Experimental
-  public interface Handler {
-    void unparseCall(SqlWriter writer, SqlCall call, int leftPrec,
-        int rightPrec);
-  }
-
-  /** Base class for dialect handlers. */
-  @Experimental
-  public static class BaseHandler implements Handler {
-    public void unparseCall(SqlWriter writer, SqlCall call,
-        int leftPrec, int rightPrec) {
-      call.getOperator().unparse(writer, call, leftPrec, rightPrec);
+   * <p>It is immutable; to "set" a property, call one of the "with" methods,
+   * which returns a new context with the desired property value. */
+  public interface Context {
+    @Nonnull DatabaseProduct databaseProduct();
+    Context withDatabaseProduct(@Nonnull DatabaseProduct databaseProduct);
+    String databaseProductName();
+    Context withDatabaseProductName(String databaseProductName);
+    String databaseVersion();
+    Context withDatabaseVersion(String databaseVersion);
+    String identifierQuoteString();
+    Context withIdentifierQuoteString(String identifierQuoteString);
+    @Nonnull NullCollation nullCollation();
+    Context withNullCollation(@Nonnull NullCollation nullCollation);
+  }
+
+  /** Implementation of Context. */
+  private static class ContextImpl implements Context {
+    private final DatabaseProduct databaseProduct;
+    private final String databaseProductName;
+    private final String databaseVersion;
+    private final String identifierQuoteString;
+    private final NullCollation nullCollation;
+
+    private ContextImpl(DatabaseProduct databaseProduct,
+        String databaseProductName, String databaseVersion,
+        String identifierQuoteString, NullCollation nullCollation) {
+      this.databaseProduct = Preconditions.checkNotNull(databaseProduct);
+      this.databaseProductName = databaseProductName;
+      this.databaseVersion = databaseVersion;
+      this.identifierQuoteString = identifierQuoteString;
+      this.nullCollation = Preconditions.checkNotNull(nullCollation);
+    }
+
+    @Nonnull public DatabaseProduct databaseProduct() {
+      return databaseProduct;
+    }
+
+    public Context withDatabaseProduct(
+        @Nonnull DatabaseProduct databaseProduct) {
+      return new ContextImpl(databaseProduct, databaseProductName,
+          databaseVersion, identifierQuoteString, nullCollation);
+    }
+
+    public String databaseProductName() {
+      return databaseProductName;
+    }
+
+    public Context withDatabaseProductName(String databaseProductName) {
+      return new ContextImpl(databaseProduct, databaseProductName,
+          databaseVersion, identifierQuoteString, nullCollation);
+    }
+
+    public String databaseVersion() {
+      return databaseVersion;
+    }
+
+    public Context withDatabaseVersion(String databaseVersion) {
+      return new ContextImpl(databaseProduct, databaseProductName,
+          databaseVersion, identifierQuoteString, nullCollation);
+    }
+
+    public String identifierQuoteString() {
+      return identifierQuoteString;
+    }
+
+    public Context withIdentifierQuoteString(String identifierQuoteString) {
+      return new ContextImpl(databaseProduct, databaseProductName,
+          databaseVersion, identifierQuoteString, nullCollation);
+    }
+
+    @Nonnull public NullCollation nullCollation() {
+      return nullCollation;
+    }
+
+    public Context withNullCollation(@Nonnull NullCollation nullCollation) {
+      return new ContextImpl(databaseProduct, databaseProductName,
+          databaseVersion, identifierQuoteString, nullCollation);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/SqlDialectFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlDialectFactory.java b/core/src/main/java/org/apache/calcite/sql/SqlDialectFactory.java
new file mode 100644
index 0000000..73de073
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDialectFactory.java
@@ -0,0 +1,42 @@
+/*
+ * 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;
+
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+
+/**
+ * <code>SqlDialectFactory</code> constructs a <code>SqlDialect</code> appropriate
+ * for a given database metadata object.
+ */
+public interface SqlDialectFactory {
+
+  /**
+   * Creates a <code>SqlDialect</code> from a DatabaseMetaData.
+   *
+   * <p>Does not maintain a reference to the DatabaseMetaData -- or, more
+   * importantly, to its {@link Connection} -- after this call has
+   * returned.
+   *
+   * @param databaseMetaData used to determine which dialect of SQL to
+   *                         generate
+   */
+  SqlDialect create(DatabaseMetaData databaseMetaData);
+
+}
+
+// End SqlDialectFactory.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/SqlDialectFactoryImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlDialectFactoryImpl.java b/core/src/main/java/org/apache/calcite/sql/SqlDialectFactoryImpl.java
new file mode 100644
index 0000000..0457735
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDialectFactoryImpl.java
@@ -0,0 +1,218 @@
+/*
+ * 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;
+
+import org.apache.calcite.config.NullCollation;
+import org.apache.calcite.sql.dialect.AccessSqlDialect;
+import org.apache.calcite.sql.dialect.AnsiSqlDialect;
+import org.apache.calcite.sql.dialect.CalciteSqlDialect;
+import org.apache.calcite.sql.dialect.Db2SqlDialect;
+import org.apache.calcite.sql.dialect.DerbySqlDialect;
+import org.apache.calcite.sql.dialect.FirebirdSqlDialect;
+import org.apache.calcite.sql.dialect.H2SqlDialect;
+import org.apache.calcite.sql.dialect.HiveSqlDialect;
+import org.apache.calcite.sql.dialect.HsqldbSqlDialect;
+import org.apache.calcite.sql.dialect.InfobrightSqlDialect;
+import org.apache.calcite.sql.dialect.InformixSqlDialect;
+import org.apache.calcite.sql.dialect.IngresSqlDialect;
+import org.apache.calcite.sql.dialect.InterbaseSqlDialect;
+import org.apache.calcite.sql.dialect.LucidDbSqlDialect;
+import org.apache.calcite.sql.dialect.MssqlSqlDialect;
+import org.apache.calcite.sql.dialect.MysqlSqlDialect;
+import org.apache.calcite.sql.dialect.NeoviewSqlDialect;
+import org.apache.calcite.sql.dialect.NetezzaSqlDialect;
+import org.apache.calcite.sql.dialect.OracleSqlDialect;
+import org.apache.calcite.sql.dialect.ParaccelSqlDialect;
+import org.apache.calcite.sql.dialect.PhoenixSqlDialect;
+import org.apache.calcite.sql.dialect.PostgresqlSqlDialect;
+import org.apache.calcite.sql.dialect.RedshiftSqlDialect;
+import org.apache.calcite.sql.dialect.SybaseSqlDialect;
+import org.apache.calcite.sql.dialect.TeradataSqlDialect;
+import org.apache.calcite.sql.dialect.VerticaSqlDialect;
+
+import java.sql.DatabaseMetaData;
+import java.sql.SQLException;
+import java.util.Locale;
+
+/**
+ * The default implementation of a <code>SqlDialectFactory</code>.
+ */
+public class SqlDialectFactoryImpl implements SqlDialectFactory {
+  public SqlDialect create(DatabaseMetaData databaseMetaData) {
+    String databaseProductName;
+    String databaseVersion;
+    try {
+      databaseProductName = databaseMetaData.getDatabaseProductName();
+      databaseVersion = databaseMetaData.getDatabaseProductVersion();
+    } catch (SQLException e) {
+      throw new RuntimeException("while detecting database product", e);
+    }
+    final String upperProductName =
+        databaseProductName.toUpperCase(Locale.ROOT).trim();
+    final String quoteString = getIdentifierQuoteString(databaseMetaData);
+    final NullCollation nullCollation = getNullCollation(databaseMetaData);
+    final SqlDialect.Context c = SqlDialect.EMPTY_CONTEXT
+        .withDatabaseProductName(databaseProductName)
+        .withDatabaseVersion(databaseVersion)
+        .withIdentifierQuoteString(quoteString)
+        .withNullCollation(nullCollation);
+    switch (upperProductName) {
+    case "ACCESS":
+      return new AccessSqlDialect(c);
+    case "APACHE DERBY":
+      return new DerbySqlDialect(c);
+    case "DBMS:CLOUDSCAPE":
+      return new DerbySqlDialect(c);
+    case "HIVE":
+      return new HiveSqlDialect(c);
+    case "INGRES":
+      return new IngresSqlDialect(c);
+    case "INTERBASE":
+      return new InterbaseSqlDialect(c);
+    case "LUCIDDB":
+      return new LucidDbSqlDialect(c);
+    case "ORACLE":
+      return new OracleSqlDialect(c);
+    case "PHOENIX":
+      return new PhoenixSqlDialect(c);
+    case "MYSQL (INFOBRIGHT)":
+      return new InfobrightSqlDialect(c);
+    case "MYSQL":
+      return new MysqlSqlDialect(c);
+    case "REDSHIFT":
+      return new RedshiftSqlDialect(c);
+    }
+    // Now the fuzzy matches.
+    if (databaseProductName.startsWith("DB2")) {
+      return new Db2SqlDialect(c);
+    } else if (upperProductName.contains("FIREBIRD")) {
+      return new FirebirdSqlDialect(c);
+    } else if (databaseProductName.startsWith("Informix")) {
+      return new InformixSqlDialect(c);
+    } else if (upperProductName.contains("NETEZZA")) {
+      return new NetezzaSqlDialect(c);
+    } else if (upperProductName.contains("PARACCEL")) {
+      return new ParaccelSqlDialect(c);
+    } else if (databaseProductName.startsWith("HP Neoview")) {
+      return new NeoviewSqlDialect(c);
+    } else if (upperProductName.contains("POSTGRE")) {
+      return new PostgresqlSqlDialect(c);
+    } else if (upperProductName.contains("SQL SERVER")) {
+      return new MssqlSqlDialect(c);
+    } else if (upperProductName.contains("SYBASE")) {
+      return new SybaseSqlDialect(c);
+    } else if (upperProductName.contains("TERADATA")) {
+      return new TeradataSqlDialect(c);
+    } else if (upperProductName.contains("HSQL")) {
+      return new HsqldbSqlDialect(c);
+    } else if (upperProductName.contains("H2")) {
+      return new H2SqlDialect(c);
+    } else if (upperProductName.contains("VERTICA")) {
+      return new VerticaSqlDialect(c);
+    } else {
+      return new AnsiSqlDialect(c);
+    }
+  }
+
+  private NullCollation getNullCollation(DatabaseMetaData databaseMetaData) {
+    try {
+      if (databaseMetaData.nullsAreSortedAtEnd()) {
+        return NullCollation.LAST;
+      } else if (databaseMetaData.nullsAreSortedAtStart()) {
+        return NullCollation.FIRST;
+      } else if (databaseMetaData.nullsAreSortedLow()) {
+        return NullCollation.LOW;
+      } else if (databaseMetaData.nullsAreSortedHigh()) {
+        return NullCollation.HIGH;
+      } else {
+        throw new IllegalArgumentException("cannot deduce null collation");
+      }
+    } catch (SQLException e) {
+      throw new IllegalArgumentException("cannot deduce null collation", e);
+    }
+  }
+
+  private String getIdentifierQuoteString(DatabaseMetaData databaseMetaData) {
+    try {
+      return databaseMetaData.getIdentifierQuoteString();
+    } catch (SQLException e) {
+      throw new IllegalArgumentException("cannot deduce identifier quote string", e);
+    }
+  }
+
+  /** Returns a basic dialect for a given product, or null if none is known. */
+  static SqlDialect simple(SqlDialect.DatabaseProduct databaseProduct) {
+    switch (databaseProduct) {
+    case ACCESS:
+      return AccessSqlDialect.DEFAULT;
+    case CALCITE:
+      return CalciteSqlDialect.DEFAULT;
+    case DB2:
+      return Db2SqlDialect.DEFAULT;
+    case DERBY:
+      return DerbySqlDialect.DEFAULT;
+    case FIREBIRD:
+      return FirebirdSqlDialect.DEFAULT;
+    case H2:
+      return H2SqlDialect.DEFAULT;
+    case HIVE:
+      return HiveSqlDialect.DEFAULT;
+    case HSQLDB:
+      return HsqldbSqlDialect.DEFAULT;
+    case INFOBRIGHT:
+      return InfobrightSqlDialect.DEFAULT;
+    case INFORMIX:
+      return InformixSqlDialect.DEFAULT;
+    case INGRES:
+      return IngresSqlDialect.DEFAULT;
+    case INTERBASE:
+      return InterbaseSqlDialect.DEFAULT;
+    case LUCIDDB:
+      return LucidDbSqlDialect.DEFAULT;
+    case MSSQL:
+      return MssqlSqlDialect.DEFAULT;
+    case MYSQL:
+      return MysqlSqlDialect.DEFAULT;
+    case NEOVIEW:
+      return NeoviewSqlDialect.DEFAULT;
+    case NETEZZA:
+      return NetezzaSqlDialect.DEFAULT;
+    case ORACLE:
+      return OracleSqlDialect.DEFAULT;
+    case PARACCEL:
+      return ParaccelSqlDialect.DEFAULT;
+    case PHOENIX:
+      return PhoenixSqlDialect.DEFAULT;
+    case POSTGRESQL:
+      return PostgresqlSqlDialect.DEFAULT;
+    case REDSHIFT:
+      return RedshiftSqlDialect.DEFAULT;
+    case SYBASE:
+      return SybaseSqlDialect.DEFAULT;
+    case TERADATA:
+      return TeradataSqlDialect.DEFAULT;
+    case VERTICA:
+      return VerticaSqlDialect.DEFAULT;
+    case SQLSTREAM:
+    case UNKNOWN:
+    default:
+      return null;
+    }
+  }
+}
+
+// End SqlDialectFactoryImpl.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/SqlNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlNode.java b/core/src/main/java/org/apache/calcite/sql/SqlNode.java
index 1d22f96..97080ab 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlNode.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlNode.java
@@ -16,6 +16,7 @@
  */
 package org.apache.calcite.sql;
 
+import org.apache.calcite.sql.dialect.AnsiSqlDialect;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.pretty.SqlPrettyWriter;
 import org.apache.calcite.sql.util.SqlString;
@@ -142,7 +143,7 @@ public abstract class SqlNode implements Cloneable {
    */
   public SqlString toSqlString(SqlDialect dialect, boolean forceParens) {
     if (dialect == null) {
-      dialect = SqlDialect.DUMMY;
+      dialect = AnsiSqlDialect.DEFAULT;
     }
     SqlPrettyWriter writer = new SqlPrettyWriter(dialect);
     writer.setAlwaysUseParentheses(forceParens);

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/SqlNodeList.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlNodeList.java b/core/src/main/java/org/apache/calcite/sql/SqlNodeList.java
index 0925864..b538013 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlNodeList.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlNodeList.java
@@ -184,6 +184,29 @@ public class SqlNodeList extends SqlNode implements Iterable<SqlNode> {
     return false;
   }
 
+  public static SqlNodeList of(SqlNode node1) {
+    SqlNodeList list = new SqlNodeList(SqlParserPos.ZERO);
+    list.add(node1);
+    return list;
+  }
+
+  public static SqlNodeList of(SqlNode node1, SqlNode node2) {
+    SqlNodeList list = new SqlNodeList(SqlParserPos.ZERO);
+    list.add(node1);
+    list.add(node2);
+    return list;
+  }
+
+  public static SqlNodeList of(SqlNode node1, SqlNode node2, SqlNode... nodes) {
+    SqlNodeList list = new SqlNodeList(SqlParserPos.ZERO);
+    list.add(node1);
+    list.add(node2);
+    for (SqlNode node : nodes) {
+      list.add(node);
+    }
+    return list;
+  }
+
   public void validateExpr(SqlValidator validator, SqlValidatorScope scope) {
     // While a SqlNodeList is not always a valid expression, this
     // implementation makes that assumption. It just validates the members

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/SqlSampleSpec.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlSampleSpec.java b/core/src/main/java/org/apache/calcite/sql/SqlSampleSpec.java
index 498b9ed..73e9d08 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlSampleSpec.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlSampleSpec.java
@@ -16,6 +16,8 @@
  */
 package org.apache.calcite.sql;
 
+import org.apache.calcite.sql.dialect.CalciteSqlDialect;
+
 /**
  * Specification of a SQL sample.
  *
@@ -94,7 +96,7 @@ public abstract class SqlSampleSpec {
 
     public String toString() {
       return "SUBSTITUTE("
-          + SqlDialect.CALCITE.quoteStringLiteral(name)
+          + CalciteSqlDialect.DEFAULT.quoteStringLiteral(name)
           + ")";
     }
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/SqlTimeLiteral.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlTimeLiteral.java b/core/src/main/java/org/apache/calcite/sql/SqlTimeLiteral.java
index be3a04d..dabd596 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlTimeLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlTimeLiteral.java
@@ -58,6 +58,13 @@ public class SqlTimeLiteral extends SqlAbstractDateTimeLiteral {
   public String toFormattedString() {
     return getTime().toString(precision);
   }
+
+  public void unparse(
+      SqlWriter writer,
+      int leftPrec,
+      int rightPrec) {
+    writer.getDialect().unparseDateTimeLiteral(writer, this, leftPrec, rightPrec);
+  }
 }
 
 // End SqlTimeLiteral.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/SqlTimestampLiteral.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlTimestampLiteral.java b/core/src/main/java/org/apache/calcite/sql/SqlTimestampLiteral.java
index 41d9fe4..7422f9b 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlTimestampLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlTimestampLiteral.java
@@ -63,14 +63,7 @@ public class SqlTimestampLiteral extends SqlAbstractDateTimeLiteral {
       SqlWriter writer,
       int leftPrec,
       int rightPrec) {
-    switch (writer.getDialect().getDatabaseProduct()) {
-    case MSSQL:
-      writer.literal("'" + this.toFormattedString() + "'");
-      break;
-    default:
-      writer.literal(this.toString());
-      break;
-    }
+    writer.getDialect().unparseDateTimeLiteral(writer, this, leftPrec, rightPrec);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/dialect/AccessSqlDialect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/AccessSqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/AccessSqlDialect.java
new file mode 100644
index 0000000..1c2861f
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/AccessSqlDialect.java
@@ -0,0 +1,36 @@
+/*
+ * 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.dialect;
+
+import org.apache.calcite.sql.SqlDialect;
+
+/**
+ * A <code>SqlDialect</code> implementation for the Access database.
+ */
+public class AccessSqlDialect extends SqlDialect {
+  public static final SqlDialect DEFAULT =
+      new AccessSqlDialect(EMPTY_CONTEXT
+          .withDatabaseProduct(DatabaseProduct.ACCESS)
+          .withIdentifierQuoteString("\""));
+
+  /** Creates an AccessSqlDialect. */
+  public AccessSqlDialect(Context context) {
+    super(context);
+  }
+}
+
+// End AccessSqlDialect.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/dialect/AnsiSqlDialect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/AnsiSqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/AnsiSqlDialect.java
new file mode 100644
index 0000000..14faa6f
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/AnsiSqlDialect.java
@@ -0,0 +1,41 @@
+/*
+ * 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.dialect;
+
+import org.apache.calcite.sql.SqlDialect;
+
+/**
+ * A <code>SqlDialect</code> implementation for an unknown ANSI compatible database.
+ */
+public class AnsiSqlDialect extends SqlDialect {
+  /**
+   * A dialect useful for generating generic SQL. If you need to do something
+   * database-specific like quoting identifiers, don't rely on this dialect to
+   * do what you want.
+   */
+  public static final SqlDialect DEFAULT =
+      new AnsiSqlDialect(emptyContext()
+          .withDatabaseProduct(DatabaseProduct.UNKNOWN)
+          .withIdentifierQuoteString("`"));
+
+  /** Creates an AnsiSqlDialect. */
+  public AnsiSqlDialect(Context context) {
+    super(context);
+  }
+}
+
+// End AnsiSqlDialect.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/dialect/CalciteSqlDialect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/CalciteSqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/CalciteSqlDialect.java
new file mode 100644
index 0000000..8b89df6
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/CalciteSqlDialect.java
@@ -0,0 +1,43 @@
+/*
+ * 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.dialect;
+
+import org.apache.calcite.sql.SqlDialect;
+
+/**
+ * A <code>SqlDialect</code> implementation that produces SQL that can be parsed
+ * by Apache Calcite.
+ */
+public class CalciteSqlDialect extends SqlDialect {
+  /**
+   * A dialect useful for generating SQL which can be parsed by the Apache
+   * Calcite parser, in particular quoting literals and identifiers. If you
+   * want a dialect that knows the full capabilities of the database, create
+   * one from a connection.
+   */
+  public static final SqlDialect DEFAULT =
+      new CalciteSqlDialect(emptyContext()
+          .withDatabaseProduct(DatabaseProduct.CALCITE)
+          .withIdentifierQuoteString("\""));
+
+  /** Creates a CalciteSqlDialect. */
+  public CalciteSqlDialect(Context context) {
+    super(context);
+  }
+}
+
+// End CalciteSqlDialect.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/dialect/Db2SqlDialect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/Db2SqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/Db2SqlDialect.java
new file mode 100644
index 0000000..b0e6fe4
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/Db2SqlDialect.java
@@ -0,0 +1,43 @@
+/*
+ * 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.dialect;
+
+import org.apache.calcite.sql.SqlDialect;
+
+/**
+ * A <code>SqlDialect</code> implementation for the IBM DB2 database.
+ */
+public class Db2SqlDialect extends SqlDialect {
+  public static final SqlDialect DEFAULT =
+      new Db2SqlDialect(EMPTY_CONTEXT
+          .withDatabaseProduct(DatabaseProduct.DB2));
+
+  /** Creates a Db2SqlDialect. */
+  public Db2SqlDialect(Context context) {
+    super(context);
+  }
+
+  @Override public boolean supportsCharSet() {
+    return false;
+  }
+
+  @Override public boolean hasImplicitTableAlias() {
+    return false;
+  }
+}
+
+// End Db2SqlDialect.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/dialect/DerbySqlDialect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/DerbySqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/DerbySqlDialect.java
new file mode 100644
index 0000000..184b7fb
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/DerbySqlDialect.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.dialect;
+
+import org.apache.calcite.sql.SqlDialect;
+
+/**
+ * A <code>SqlDialect</code> implementation for the Apache Derby database.
+ */
+public class DerbySqlDialect extends SqlDialect {
+  public static final SqlDialect DEFAULT =
+      new DerbySqlDialect(EMPTY_CONTEXT
+          .withDatabaseProduct(DatabaseProduct.DERBY));
+
+  /** Creates a DerbySqlDialect. */
+  public DerbySqlDialect(Context context) {
+    super(context);
+  }
+}
+
+// End DerbySqlDialect.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/dialect/FirebirdSqlDialect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/FirebirdSqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/FirebirdSqlDialect.java
new file mode 100644
index 0000000..40eaefd
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/FirebirdSqlDialect.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.dialect;
+
+import org.apache.calcite.sql.SqlDialect;
+
+/**
+ * A <code>SqlDialect</code> implementation for the Firebird database.
+ */
+public class FirebirdSqlDialect extends SqlDialect {
+  public static final SqlDialect DEFAULT =
+      new FirebirdSqlDialect(EMPTY_CONTEXT
+          .withDatabaseProduct(DatabaseProduct.FIREBIRD));
+
+  /** Creates a FirebirdSqlDialect. */
+  public FirebirdSqlDialect(Context context) {
+    super(context);
+  }
+}
+
+// End FirebirdSqlDialect.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/dialect/H2SqlDialect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/H2SqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/H2SqlDialect.java
new file mode 100644
index 0000000..005482a
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/H2SqlDialect.java
@@ -0,0 +1,40 @@
+/*
+ * 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.dialect;
+
+import org.apache.calcite.sql.SqlDialect;
+
+/**
+ * A <code>SqlDialect</code> implementation for the H2 database.
+ */
+public class H2SqlDialect extends SqlDialect {
+  public static final SqlDialect DEFAULT =
+      new H2SqlDialect(EMPTY_CONTEXT
+          .withDatabaseProduct(DatabaseProduct.H2)
+          .withIdentifierQuoteString("\""));
+
+  /** Creates an H2SqlDialect. */
+  public H2SqlDialect(Context context) {
+    super(context);
+  }
+
+  @Override public boolean supportsCharSet() {
+    return false;
+  }
+}
+
+// End H2SqlDialect.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/dialect/HiveSqlDialect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/HiveSqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/HiveSqlDialect.java
new file mode 100644
index 0000000..52f0a81
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/HiveSqlDialect.java
@@ -0,0 +1,43 @@
+/*
+ * 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.dialect;
+
+import org.apache.calcite.sql.SqlDialect;
+
+/**
+ * A <code>SqlDialect</code> implementation for the Apache Hive database.
+ */
+public class HiveSqlDialect extends SqlDialect {
+  public static final SqlDialect DEFAULT =
+      new HiveSqlDialect(
+          EMPTY_CONTEXT.withDatabaseProduct(DatabaseProduct.HIVE));
+
+  /** Creates a HiveSqlDialect. */
+  public HiveSqlDialect(Context context) {
+    super(context);
+  }
+
+  @Override protected boolean allowsAs() {
+    return false;
+  }
+
+  @Override public boolean supportsOffsetFetch() {
+    return false;
+  }
+}
+
+// End HiveSqlDialect.java


[06/15] calcite git commit: [CALCITE-1986] Add RelBuilder.match and methods for building patterns (Dian Fu)

Posted by jh...@apache.org.
[CALCITE-1986] Add RelBuilder.match and methods for building patterns (Dian Fu)

Add methods for building patterns, and documentation. (Julian Hyde)

Close apache/calcite#538


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

Branch: refs/heads/master
Commit: 3e97cff7253691bbd7df690721981de4c2d9f88b
Parents: 2773c48
Author: Dian Fu <fu...@alibaba-inc.com>
Authored: Thu Sep 14 12:47:41 2017 +0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Mon Oct 2 11:13:42 2017 -0700

----------------------------------------------------------------------
 .../apache/calcite/rel/core/RelFactories.java   |  16 +-
 .../calcite/rel/logical/LogicalMatch.java       |   3 +-
 .../java/org/apache/calcite/rex/RexBuilder.java |  13 ++
 .../calcite/sql2rel/SqlToRelConverter.java      |   2 +-
 .../org/apache/calcite/tools/RelBuilder.java    | 154 +++++++++++++++++++
 .../org/apache/calcite/test/RelBuilderTest.java |  90 +++++++++++
 site/_docs/algebra.md                           |  24 ++-
 7 files changed, 288 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/3e97cff7/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java b/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
index 477bbd4..b4ebbca 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
@@ -49,7 +49,7 @@ import com.google.common.collect.ImmutableList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.TreeSet;
+import java.util.SortedSet;
 
 /**
  * Contains factory interface and default implementation for creating various
@@ -396,11 +396,12 @@ public class RelFactories {
    */
   public interface MatchFactory {
     /** Creates a {@link Match}. */
-    RelNode createMatchRecognize(RelNode input, RexNode pattern,
+    RelNode createMatch(RelNode input, RexNode pattern,
         RelDataType rowType, boolean strictStart, boolean strictEnd,
         Map<String, RexNode> patternDefinitions, Map<String, RexNode> measures,
-        RexNode after, Map<String, TreeSet<String>> subsets, boolean allRows,
-        List<RexNode> partitionKeys, RelCollation orderKeys, RexNode interval);
+        RexNode after, Map<String, ? extends SortedSet<String>> subsets,
+        boolean allRows, List<RexNode> partitionKeys, RelCollation orderKeys,
+        RexNode interval);
   }
 
   /**
@@ -408,11 +409,12 @@ public class RelFactories {
    * that returns a {@link LogicalMatch}.
    */
   private static class MatchFactoryImpl implements MatchFactory {
-    public RelNode createMatchRecognize(RelNode input, RexNode pattern,
+    public RelNode createMatch(RelNode input, RexNode pattern,
         RelDataType rowType, boolean strictStart, boolean strictEnd,
         Map<String, RexNode> patternDefinitions, Map<String, RexNode> measures,
-        RexNode after, Map<String, TreeSet<String>> subsets, boolean allRows,
-        List<RexNode> partitionKeys, RelCollation orderKeys, RexNode interval) {
+        RexNode after, Map<String, ? extends SortedSet<String>> subsets,
+        boolean allRows, List<RexNode> partitionKeys, RelCollation orderKeys,
+        RexNode interval) {
       return LogicalMatch.create(input, rowType, pattern, strictStart,
           strictEnd, patternDefinitions, measures, after, subsets, allRows,
           partitionKeys, orderKeys, interval);

http://git-wip-us.apache.org/repos/asf/calcite/blob/3e97cff7/core/src/main/java/org/apache/calcite/rel/logical/LogicalMatch.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalMatch.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalMatch.java
index f0e3729..1a840f5 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalMatch.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalMatch.java
@@ -29,7 +29,6 @@ import org.apache.calcite.rex.RexNode;
 import java.util.List;
 import java.util.Map;
 import java.util.SortedSet;
-import java.util.TreeSet;
 
 /**
  * Sub-class of {@link Match}
@@ -74,7 +73,7 @@ public class LogicalMatch extends Match {
   public static LogicalMatch create(RelNode input, RelDataType rowType,
       RexNode pattern, boolean strictStart, boolean strictEnd,
       Map<String, RexNode> patternDefinitions, Map<String, RexNode> measures,
-      RexNode after, Map<String, TreeSet<String>> subsets, boolean allRows,
+      RexNode after, Map<String, ? extends SortedSet<String>> subsets, boolean allRows,
       List<RexNode> partitionKeys, RelCollation orderKeys, RexNode interval) {
     final RelOptCluster cluster = input.getCluster();
     final RelTraitSet traitSet = cluster.traitSetOf(Convention.NONE);

http://git-wip-us.apache.org/repos/asf/calcite/blob/3e97cff7/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexBuilder.java b/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
index 2144ab8..bd6579d 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
@@ -809,6 +809,19 @@ public class RexBuilder {
   }
 
   /**
+   * Creates a reference to a given field of the pattern.
+   *
+   * @param alpha the pattern name
+   * @param type Type of field
+   * @param i    Ordinal of field
+   * @return Reference to field of pattern
+   */
+  public RexPatternFieldRef makePatternFieldRef(String alpha, RelDataType type, int i) {
+    type = SqlTypeUtil.addCharsetAndCollation(type, typeFactory);
+    return new RexPatternFieldRef(alpha, i, type);
+  }
+
+  /**
    * Creates a literal representing a flag.
    *
    * @param flag Flag value

http://git-wip-us.apache.org/repos/asf/calcite/blob/3e97cff7/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
index 235dfb4..9967bd5 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -2239,7 +2239,7 @@ public class SqlToRelConverter {
     final RelFactories.MatchFactory factory =
         RelFactories.DEFAULT_MATCH_FACTORY;
     final RelNode rel =
-        factory.createMatchRecognize(input, patternNode,
+        factory.createMatch(input, patternNode,
             rowType, matchRecognize.getStrictStart().booleanValue(),
             matchRecognize.getStrictEnd().booleanValue(),
             definitionNodes.build(), measureNodes.build(), after,

http://git-wip-us.apache.org/repos/asf/calcite/blob/3e97cff7/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/tools/RelBuilder.java b/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
index 0a726c7..e55015a 100644
--- a/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
+++ b/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
@@ -75,6 +75,7 @@ import com.google.common.base.Function;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
@@ -89,6 +90,7 @@ import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Locale;
+import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
 import java.util.SortedSet;
@@ -132,6 +134,7 @@ public class RelBuilder {
   private final RelFactories.CorrelateFactory correlateFactory;
   private final RelFactories.ValuesFactory valuesFactory;
   private final RelFactories.TableScanFactory scanFactory;
+  private final RelFactories.MatchFactory matchFactory;
   private final Deque<Frame> stack = new ArrayDeque<>();
   private final boolean simplify;
   private final RexSimplify simplifier;
@@ -175,6 +178,9 @@ public class RelBuilder {
     this.scanFactory =
         Util.first(context.unwrap(RelFactories.TableScanFactory.class),
             RelFactories.DEFAULT_TABLE_SCAN_FACTORY);
+    this.matchFactory =
+        Util.first(context.unwrap(RelFactories.MatchFactory.class),
+            RelFactories.DEFAULT_MATCH_FACTORY);
     final RexExecutor executor =
         Util.first(context.unwrap(RexExecutor.class),
             Util.first(cluster.getPlanner().getExecutor(), RexUtil.EXECUTOR));
@@ -773,6 +779,89 @@ public class RelBuilder {
     return aggregateCall(SqlStdOperatorTable.MAX, false, null, alias, operand);
   }
 
+  // Methods for patterns
+
+  /**
+   * Creates a reference to a given field of the pattern.
+   *
+   * @param alpha the pattern name
+   * @param type Type of field
+   * @param i Ordinal of field
+   * @return Reference to field of pattern
+   */
+  public RexNode patternField(String alpha, RelDataType type, int i) {
+    return getRexBuilder().makePatternFieldRef(alpha, type, i);
+  }
+
+  /** Creates a call that concatenates patterns;
+   * for use in {@link #match}. */
+  public RexNode patternConcat(Iterable<? extends RexNode> nodes) {
+    final ImmutableList<RexNode> list = ImmutableList.copyOf(nodes);
+    if (list.size() > 2) {
+      // Convert into binary calls
+      return patternConcat(patternConcat(Util.skipLast(list)), Util.last(list));
+    }
+    final RelDataType t = getTypeFactory().createSqlType(SqlTypeName.NULL);
+    return getRexBuilder().makeCall(t, SqlStdOperatorTable.PATTERN_CONCAT,
+        list);
+  }
+
+  /** Creates a call that concatenates patterns;
+   * for use in {@link #match}. */
+  public RexNode patternConcat(RexNode... nodes) {
+    return patternConcat(ImmutableList.copyOf(nodes));
+  }
+
+  /** Creates a call that creates alternate patterns;
+   * for use in {@link #match}. */
+  public RexNode patternAlter(Iterable<? extends RexNode> nodes) {
+    final RelDataType t = getTypeFactory().createSqlType(SqlTypeName.NULL);
+    return getRexBuilder().makeCall(t, SqlStdOperatorTable.PATTERN_ALTER,
+        ImmutableList.copyOf(nodes));
+  }
+
+  /** Creates a call that creates alternate patterns;
+   * for use in {@link #match}. */
+  public RexNode patternAlter(RexNode... nodes) {
+    return patternAlter(ImmutableList.copyOf(nodes));
+  }
+
+  /** Creates a call that creates quantify patterns;
+   * for use in {@link #match}. */
+  public RexNode patternQuantify(Iterable<? extends RexNode> nodes) {
+    final RelDataType t = getTypeFactory().createSqlType(SqlTypeName.NULL);
+    return getRexBuilder().makeCall(t, SqlStdOperatorTable.PATTERN_QUANTIFIER,
+        ImmutableList.copyOf(nodes));
+  }
+
+  /** Creates a call that creates quantify patterns;
+   * for use in {@link #match}. */
+  public RexNode patternQuantify(RexNode... nodes) {
+    return patternQuantify(ImmutableList.copyOf(nodes));
+  }
+
+  /** Creates a call that creates permute patterns;
+   * for use in {@link #match}. */
+  public RexNode patternPermute(Iterable<? extends RexNode> nodes) {
+    final RelDataType t = getTypeFactory().createSqlType(SqlTypeName.NULL);
+    return getRexBuilder().makeCall(t, SqlStdOperatorTable.PATTERN_PERMUTE,
+        ImmutableList.copyOf(nodes));
+  }
+
+  /** Creates a call that creates permute patterns;
+   * for use in {@link #match}. */
+  public RexNode patternPermute(RexNode... nodes) {
+    return patternPermute(ImmutableList.copyOf(nodes));
+  }
+
+  /** Creates a call that creates an exclude pattern;
+   * for use in {@link #match}. */
+  public RexNode patternExclude(RexNode node) {
+    final RelDataType t = getTypeFactory().createSqlType(SqlTypeName.NULL);
+    return getRexBuilder().makeCall(t, SqlStdOperatorTable.PATTERN_EXCLUDE,
+        ImmutableList.of(node));
+  }
+
   // Methods that create relational expressions
 
   /** Creates a {@link org.apache.calcite.rel.core.TableScan} of the table
@@ -1711,6 +1800,71 @@ public class RelBuilder {
             }));
   }
 
+  /** Creates a {@link org.apache.calcite.rel.core.Match}. */
+  public RelBuilder match(RexNode pattern, boolean strictStart,
+      boolean strictEnd, Map<String, RexNode> patternDefinitions,
+      Iterable<? extends RexNode> measureList, RexNode after,
+      Map<String, ? extends SortedSet<String>> subsets, boolean allRows,
+      Iterable<? extends RexNode> partitionKeys,
+      Iterable<? extends RexNode> orderKeys, RexNode interval) {
+    final List<RelFieldCollation> fieldCollations = new ArrayList<>();
+    for (RexNode orderKey : orderKeys) {
+      final RelFieldCollation.Direction direction;
+      switch (orderKey.getKind()) {
+      case DESCENDING:
+        direction = RelFieldCollation.Direction.DESCENDING;
+        orderKey = ((RexCall) orderKey).getOperands().get(0);
+        break;
+      case NULLS_FIRST:
+      case NULLS_LAST:
+        throw new AssertionError();
+      default:
+        direction = RelFieldCollation.Direction.ASCENDING;
+        break;
+      }
+      final RelFieldCollation.NullDirection nullDirection =
+          direction.defaultNullDirection();
+      final RexInputRef ref = (RexInputRef) orderKey;
+      fieldCollations.add(
+          new RelFieldCollation(ref.getIndex(), direction, nullDirection));
+    }
+
+    final RelDataTypeFactory.Builder typeBuilder = cluster.getTypeFactory().builder();
+    for (RexNode partitionKey : partitionKeys) {
+      typeBuilder.add(partitionKey.toString(), partitionKey.getType());
+    }
+    if (allRows) {
+      for (RexNode orderKey : orderKeys) {
+        if (!typeBuilder.nameExists(orderKey.toString())) {
+          typeBuilder.add(orderKey.toString(), orderKey.getType());
+        }
+      }
+
+      final RelDataType inputRowType = peek().getRowType();
+      for (RelDataTypeField fs : inputRowType.getFieldList()) {
+        if (!typeBuilder.nameExists(fs.getName())) {
+          typeBuilder.add(fs);
+        }
+      }
+    }
+
+    final ImmutableMap.Builder<String, RexNode> measures = ImmutableMap.builder();
+    for (RexNode measure : measureList) {
+      List<RexNode> operands = ((RexCall) measure).getOperands();
+      String alias = operands.get(1).toString();
+      typeBuilder.add(alias, operands.get(0).getType());
+      measures.put(alias, operands.get(0));
+    }
+
+    final RelNode match = matchFactory.createMatch(peek(), pattern,
+        typeBuilder.build(), strictStart, strictEnd, patternDefinitions,
+        measures.build(), after, subsets, allRows,
+        ImmutableList.copyOf(partitionKeys), RelCollations.of(fieldCollations),
+        interval);
+    stack.push(new Frame(match));
+    return this;
+  }
+
   /** Clears the stack.
    *
    * <p>The builder's state is now the same as when it was created. */

http://git-wip-us.apache.org/repos/asf/calcite/blob/3e97cff7/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java b/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
index 0f66db3..93e3b30 100644
--- a/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
@@ -27,12 +27,14 @@ import org.apache.calcite.rel.core.TableFunctionScan;
 import org.apache.calcite.rel.core.TableModify;
 import org.apache.calcite.rel.core.Window;
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexCorrelVariable;
 import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.runtime.CalciteException;
 import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.SqlMatchRecognize;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParser;
 import org.apache.calcite.sql.type.SqlTypeName;
@@ -46,6 +48,7 @@ import org.apache.calcite.util.Util;
 import org.apache.calcite.util.mapping.Mappings;
 
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 
@@ -54,6 +57,7 @@ import org.junit.Test;
 import java.sql.PreparedStatement;
 import java.util.Arrays;
 import java.util.List;
+import java.util.TreeSet;
 
 import static org.hamcrest.CoreMatchers.containsString;
 import static org.hamcrest.CoreMatchers.is;
@@ -1861,6 +1865,92 @@ public class RelBuilderTest {
       assertThat(e.getMessage(), containsString("cannot derive type"));
     }
   }
+
+  @Test public void testMatchRecognize() {
+    // Equivalent SQL:
+    //   SELECT *
+    //   FROM emp
+    //   MATCH_RECOGNIZE (
+    //     PARTITION BY deptno
+    //     ORDER BY empno asc
+    //     MEASURES
+    //       STRT.mgr as start_nw,
+    //       LAST(DOWN.mgr) as bottom_nw,
+    //     PATTERN (STRT DOWN+ UP+) WITHIN INTERVAL '5' SECOND
+    //     DEFINE
+    //       DOWN as DOWN.mgr < PREV(DOWN.mgr),
+    //       UP as UP.mgr > PREV(UP.mgr)
+    //   )
+    final RelBuilder builder = RelBuilder.create(config().build()).scan("EMP");
+    final RelDataTypeFactory typeFactory = builder.getTypeFactory();
+    final RelDataType intType = typeFactory.createSqlType(SqlTypeName.INTEGER);
+
+    RexNode pattern = builder.patternConcat(
+        builder.literal("STRT"),
+        builder.patternQuantify(builder.literal("DOWN"), builder.literal(1),
+            builder.literal(-1), builder.literal(false)),
+        builder.patternQuantify(builder.literal("UP"), builder.literal(1),
+            builder.literal(-1), builder.literal(false)));
+
+    ImmutableMap.Builder<String, RexNode> pdBuilder = new ImmutableMap.Builder<>();
+    RexNode downDefinition = builder.call(SqlStdOperatorTable.LESS_THAN,
+        builder.call(SqlStdOperatorTable.PREV,
+            builder.patternField("DOWN", intType, 3),
+            builder.literal(0)),
+        builder.call(SqlStdOperatorTable.PREV,
+            builder.patternField("DOWN", intType, 3),
+            builder.literal(1)));
+    pdBuilder.put("DOWN", downDefinition);
+    RexNode upDefinition = builder.call(SqlStdOperatorTable.GREATER_THAN,
+        builder.call(SqlStdOperatorTable.PREV,
+            builder.patternField("UP", intType, 3),
+            builder.literal(0)),
+        builder.call(SqlStdOperatorTable.PREV,
+            builder.patternField("UP", intType, 3),
+            builder.literal(1)));
+    pdBuilder.put("UP", upDefinition);
+
+    ImmutableList.Builder<RexNode> measuresBuilder = new ImmutableList.Builder<>();
+    measuresBuilder.add(
+        builder.alias(builder.patternField("STRT", intType, 3),
+            "start_nw"));
+    measuresBuilder.add(
+        builder.alias(
+            builder.call(SqlStdOperatorTable.LAST,
+                builder.patternField("DOWN", intType, 3),
+                builder.literal(0)),
+            "bottom_nw"));
+
+    RexNode after = builder.getRexBuilder().makeFlag(
+        SqlMatchRecognize.AfterOption.SKIP_TO_NEXT_ROW);
+
+    ImmutableList.Builder<RexNode> partitionKeysBuilder = new ImmutableList.Builder<>();
+    partitionKeysBuilder.add(builder.field("DEPTNO"));
+
+    ImmutableList.Builder<RexNode> orderKeysBuilder = new ImmutableList.Builder<>();
+    orderKeysBuilder.add(builder.field("EMPNO"));
+
+    RexNode interval = builder.literal("INTERVAL '5' SECOND");
+
+    final ImmutableMap<String, TreeSet<String>> subsets = ImmutableMap.of();
+    final RelNode root = builder
+        .match(pattern, false, false, pdBuilder.build(),
+            measuresBuilder.build(), after, subsets, false,
+            partitionKeysBuilder.build(), orderKeysBuilder.build(), interval)
+        .build();
+    final String expected = "LogicalMatch(partition=[[$7]], order=[[0]], "
+        + "outputFields=[[$7, 'start_nw', 'bottom_nw']], allRows=[false], "
+        + "after=[FLAG(SKIP TO NEXT ROW)], pattern=[(('STRT', "
+        + "PATTERN_QUANTIFIER('DOWN', 1, -1, false)), "
+        + "PATTERN_QUANTIFIER('UP', 1, -1, false))], "
+        + "isStrictStarts=[false], isStrictEnds=[false], "
+        + "interval=['INTERVAL ''5'' SECOND'], subsets=[[]], "
+        + "patternDefinitions=[[<(PREV(DOWN.$3, 0), PREV(DOWN.$3, 1)), "
+        + ">(PREV(UP.$3, 0), PREV(UP.$3, 1))]], "
+        + "inputFields=[[EMPNO, ENAME, JOB, MGR, HIREDATE, SAL, COMM, DEPTNO]])\n"
+        + "  LogicalTableScan(table=[[scott, EMP]])\n";
+    assertThat(str(root), is(expected));
+  }
 }
 
 // End RelBuilderTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/3e97cff7/site/_docs/algebra.md
----------------------------------------------------------------------
diff --git a/site/_docs/algebra.md b/site/_docs/algebra.md
index 490ca4f..527b2e1 100644
--- a/site/_docs/algebra.md
+++ b/site/_docs/algebra.md
@@ -274,12 +274,14 @@ return the `RelBuilder`.
 | `union(all [, n])` | Creates a [Union]({{ site.apiRoot }}/org/apache/calcite/rel/core/Union.html) of the `n` (default two) most recent relational expressions.
 | `intersect(all [, n])` | Creates an [Intersect]({{ site.apiRoot }}/org/apache/calcite/rel/core/Intersect.html) of the `n` (default two) most recent relational expressions.
 | `minus(all)` | Creates a [Minus]({{ site.apiRoot }}/org/apache/calcite/rel/core/Minus.html) of the two most recent relational expressions.
+| `match(pattern, strictStart,` `strictEnd, patterns, measures,` `after, subsets, allRows,` `partitionKeys, orderKeys,` `interval)` | Creates a [Match]({{ site.apiRoot }}/org/apache/calcite/rel/core/Match.html).
 
 Argument types:
 
-* `expr`  [RexNode]({{ site.apiRoot }}/org/apache/calcite/rex/RexNode.html)
+* `expr`, `interval` [RexNode]({{ site.apiRoot }}/org/apache/calcite/rex/RexNode.html)
 * `expr...` Array of [RexNode]({{ site.apiRoot }}/org/apache/calcite/rex/RexNode.html)
-* `exprList` Iterable of [RexNode]({{ site.apiRoot }}/org/apache/calcite/rex/RexNode.html)
+* `exprList`, `measureList`, `partitionKeys`, `orderKeys` Iterable of
+  [RexNode]({{ site.apiRoot }}/org/apache/calcite/rex/RexNode.html)
 * `fieldOrdinal` Ordinal of a field within its row (starting from 0)
 * `fieldName` Name of a field, unique within its row
 * `fieldName...` Array of String
@@ -291,12 +293,14 @@ Argument types:
 * `value...` Array of Object
 * `value` Object
 * `tupleList` Iterable of List of [RexLiteral]({{ site.apiRoot }}/org/apache/calcite/rex/RexLiteral.html)
-* `all` boolean
-* `distinct` boolean
+* `all`, `distinct`, `strictStart`, `strictEnd`, `allRows` boolean
 * `alias` String
 * `varHolder` [Holder]({{ site.apiRoot }}/org/apache/calcite/util/Holder.html) of [RexCorrelVariable]({{ site.apiRoot }}/org/apache/calcite/rex/RexCorrelVariable.html)
+* `patterns` Map whose key is String, value is [RexNode]({{ site.apiRoot }}/org/apache/calcite/rex/RexNode.html)
+* `subsets` Map whose key is String, value is a sorted set of String
 
 The builder methods perform various optimizations, including:
+
 * `project` returns its input if asked to project all columns in order
 * `filter` flattens the condition (so an `AND` and `OR` may have more than 2 children),
   simplifies (converting say `x = 1 AND TRUE` to `x = 1`)
@@ -355,6 +359,18 @@ added to the stack.
 | `nullsFirst(expr)` | Changes sort order to nulls first (only valid as an argument to `sort` or `sortLimit`)
 | `nullsLast(expr)` | Changes sort order to nulls last (only valid as an argument to `sort` or `sortLimit`)
 
+#### Pattern methods
+
+The following methods return patterns for use in `match`.
+
+| Method              | Description
+|:------------------- |:-----------
+| `patternConcat(pattern...)` | Concatenates patterns
+| `patternAlter(pattern...)` | Alternates patterns
+| `patternQuantify(pattern, min, max)` | Quantifies a pattern
+| `patternPermute(pattern...)` | Permutes a pattern
+| `patternExclude(pattern)` | Excludes a pattern
+
 ### Group key methods
 
 The following methods return a


[04/15] calcite git commit: Cosmetic

Posted by jh...@apache.org.
Cosmetic


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

Branch: refs/heads/master
Commit: 7a9857504af654d518004a07f05798ebdfd253c1
Parents: 4d0e83e
Author: Julian Hyde <jh...@apache.org>
Authored: Mon Sep 11 13:32:42 2017 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Mon Oct 2 11:13:41 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/calcite/rel/logical/LogicalTableScan.java  | 3 ++-
 .../main/java/org/apache/calcite/sql/SqlJdbcFunctionCall.java  | 5 +++--
 .../src/main/java/org/apache/calcite/util/ImmutableBitSet.java | 6 +++---
 3 files changed, 8 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/7a985750/core/src/main/java/org/apache/calcite/rel/logical/LogicalTableScan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalTableScan.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalTableScan.java
index 788e5a2..abad97d 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalTableScan.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalTableScan.java
@@ -90,7 +90,8 @@ public final class LogicalTableScan extends TableScan {
   }
 
   /** Creates a LogicalTableScan.
-   *  @param cluster Cluster
+   *
+   * @param cluster Cluster
    * @param relOptTable Table
    */
   public static LogicalTableScan create(RelOptCluster cluster,

http://git-wip-us.apache.org/repos/asf/calcite/blob/7a985750/core/src/main/java/org/apache/calcite/sql/SqlJdbcFunctionCall.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlJdbcFunctionCall.java b/core/src/main/java/org/apache/calcite/sql/SqlJdbcFunctionCall.java
index 20a01da..a057697 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlJdbcFunctionCall.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlJdbcFunctionCall.java
@@ -595,8 +595,9 @@ public class SqlJdbcFunctionCall extends SqlFunction {
      * <p>The reordering is specified by an int array where the value of
      * element at position <code>i</code> indicates to which element in a
      * new SqlNode[] array the operand goes.
-     *  @param operator Operator
-     * @param order    Order
+     *
+     * @param operator Operator
+     * @param order Order
      */
     PermutingMakeCall(SqlOperator operator, int[] order) {
       super(operator);

http://git-wip-us.apache.org/repos/asf/calcite/blob/7a985750/core/src/main/java/org/apache/calcite/util/ImmutableBitSet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/ImmutableBitSet.java b/core/src/main/java/org/apache/calcite/util/ImmutableBitSet.java
index c03d5c4..568fe3e 100644
--- a/core/src/main/java/org/apache/calcite/util/ImmutableBitSet.java
+++ b/core/src/main/java/org/apache/calcite/util/ImmutableBitSet.java
@@ -698,7 +698,7 @@ public class ImmutableBitSet
   /** Returns a bit set with all the bits in this set that are not in
    * another.
    *
-   *  @see BitSet#andNot(java.util.BitSet) */
+   * @see BitSet#andNot(java.util.BitSet) */
   public ImmutableBitSet except(ImmutableBitSet that) {
     final Builder builder = rebuild();
     builder.removeAll(that);
@@ -706,9 +706,9 @@ public class ImmutableBitSet
   }
 
   /** Returns a bit set with all the bits set in both this set and in
-   *  another.
+   * another.
    *
-   *  @see BitSet#and */
+   * @see BitSet#and */
   public ImmutableBitSet intersect(ImmutableBitSet that) {
     final Builder builder = rebuild();
     builder.intersect(that);


[10/15] calcite git commit: [CALCITE-1960] RelMdPredicates.getPredicates is slow if there are many equivalent columns (Rheet Wong)

Posted by jh...@apache.org.
[CALCITE-1960] RelMdPredicates.getPredicates is slow if there are many equivalent columns (Rheet Wong)

The algorithm to generate candidate mappings was generating very many
duplicates. They were filtered out eventually, but caused an
exponential running time.

Close apache/calcite#530


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

Branch: refs/heads/master
Commit: f1a002ea8386fc89abcebc185b1f0e7a7bd30246
Parents: dc2b86a
Author: LeoWangLZ <wa...@163.com>
Authored: Fri Aug 25 15:42:32 2017 +0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Mon Oct 2 11:13:44 2017 -0700

----------------------------------------------------------------------
 .../calcite/rel/metadata/RelMdPredicates.java   |  4 ++-
 .../apache/calcite/test/RelMetadataTest.java    | 27 ++++++++++++++++++++
 2 files changed, 30 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/f1a002ea/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java
index feeabd4..1b7dbc3 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java
@@ -827,7 +827,9 @@ public class RelMdPredicates
           if (level == 0) {
             nextMapping = null;
           } else {
-            iterationIdx[level] = 0;
+            int tmp = columnSets[level].nextSetBit(0);
+            nextMapping.set(columns[level], tmp);
+            iterationIdx[level] = tmp + 1;
             computeNextMapping(level - 1);
           }
         } else {

http://git-wip-us.apache.org/repos/asf/calcite/blob/f1a002ea/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java b/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
index 115da8e..f2bed82 100644
--- a/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
@@ -1438,6 +1438,33 @@ public class RelMetadataTest extends SqlToRelTestBase {
     assertThat(pulledUpPredicates, sortsAs("[=($0, 1)]"));
   }
 
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1960">[CALCITE-1960]
+   * RelMdPredicates.getPredicates is slow if there are many equivalent
+   * columns</a>. Since this is a performance problem, the test result does not
+   * change, but takes over 15 minutes before the fix and 6 seconds after. */
+  @Test(timeout = 20_000) public void testPullUpPredicatesForExprsItr() {
+    final String sql = "select a.EMPNO, a.ENAME\n"
+        + "from (select * from sales.emp ) a\n"
+        + "join (select * from sales.emp  ) b\n"
+        + "on a.empno = b.deptno\n"
+        + "  and a.comm = b.comm\n"
+        + "  and a.mgr=b.mgr\n"
+        + "  and (a.empno < 10 or a.comm < 3 or a.deptno < 10\n"
+        + "    or a.job ='abc' or a.ename='abc' or a.sal='30' or a.mgr >3\n"
+        + "    or a.slacker is not null  or a.HIREDATE is not null\n"
+        + "    or b.empno < 9 or b.comm < 3 or b.deptno < 10 or b.job ='abc'\n"
+        + "    or b.ename='abc' or b.sal='30' or b.mgr >3 or b.slacker )\n"
+        + "join emp c\n"
+        + "on b.mgr =a.mgr and a.empno =b.deptno and a.comm=b.comm\n"
+        + "  and a.deptno=b.deptno and a.job=b.job and a.ename=b.ename\n"
+        + "  and a.mgr=b.deptno and a.slacker=b.slacker";
+    final RelNode rel = convertSql(sql);
+    final RelMetadataQuery mq = RelMetadataQuery.instance();
+    RelOptPredicateList inputSet = mq.getPulledUpPredicates(rel.getInput(0));
+    assertThat(inputSet.pulledUpPredicates.size(), is(131089));
+  }
+
   @Test public void testPullUpPredicatesOnConstant() {
     final String sql = "select deptno, mgr, x, 'y' as y, z from (\n"
         + "  select deptno, mgr, cast(null as integer) as x, cast('1' as int) as z\n"


[11/15] calcite git commit: Make Travis CI builds work (Christian Beikov)

Posted by jh...@apache.org.
Make Travis CI builds work (Christian Beikov)

Close apache/calcite#542


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

Branch: refs/heads/master
Commit: dc2b86afcf46bfefbadfc4d56b2ad5482e757244
Parents: 796a28f
Author: Christian Beikov <ch...@gmail.com>
Authored: Fri Sep 22 10:29:39 2017 +0200
Committer: Julian Hyde <jh...@apache.org>
Committed: Mon Oct 2 11:13:44 2017 -0700

----------------------------------------------------------------------
 .travis.yml | 11 +++++++----
 pom.xml     |  4 +++-
 2 files changed, 10 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/dc2b86af/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index 7232f1f..f8cd9b2 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -17,9 +17,6 @@
 # limitations under the License.
 #
 language: java
-jdk:
-  - oraclejdk8
-  - oraclejdk7
 branches:
   only:
     - master
@@ -30,11 +27,17 @@ branches:
 install:
   - mvn install -DskipTests=true -Dmaven.javadoc.skip=true -B -V
 script:
-  - mvn -Dsurefire.useFile=false test
+  - unset _JAVA_OPTIONS
+  - mvn -Dsurefire.useFile=false -Dsurefire.parallel= test
 git:
   depth: 10000
 sudo: false
 cache:
   directories:
     - $HOME/.m2
+matrix:
+  fast_finish: true
+  include:
+    - jdk: openjdk7
+    - jdk: oraclejdk8
 # End .travis.yml

http://git-wip-us.apache.org/repos/asf/calcite/blob/dc2b86af/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index f459a1c..1e25c0c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -46,6 +46,8 @@ limitations under the License.
   </mailingLists>
 
   <properties>
+    <surefire.parallel>both</surefire.parallel>
+
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
     <top.dir>${project.basedir}</top.dir>
     <version.major>1</version.major>
@@ -817,7 +819,7 @@ limitations under the License.
           <configuration>
             <threadCount>1</threadCount>
             <perCoreThreadCount>true</perCoreThreadCount>
-            <parallel>both</parallel>
+            <parallel>${surefire.parallel}</parallel>
             <forkMode>once</forkMode>
             <systemProperties>
               <systemProperty>


[05/15] calcite git commit: [CALCITE-1989] Check dependencies for vulnerabilities each release

Posted by jh...@apache.org.
[CALCITE-1989] Check dependencies for vulnerabilities each release

Run maven with -Ppedantic to generate a vulnerability report.

Upgrade Apache Spark.


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

Branch: refs/heads/master
Commit: d173640c202238c8cb6bdb87d20ab4f3f9fcc88b
Parents: 3e97cff
Author: Julian Hyde <jh...@apache.org>
Authored: Mon Sep 18 18:00:16 2017 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Mon Oct 2 11:13:42 2017 -0700

----------------------------------------------------------------------
 pom.xml             | 34 +++++++++++++++++++++++++++++++++-
 site/_docs/howto.md |  3 +++
 2 files changed, 36 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/d173640c/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 6b76026..f459a1c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -51,6 +51,9 @@ limitations under the License.
     <version.major>1</version.major>
     <version.minor>14</version.minor>
 
+    <!-- Don't fail the build for vulnerabilities below this threshold. -->
+    <failBuildOnCVSS>8</failBuildOnCVSS>
+
     <!-- This list is in alphabetical order. -->
     <airlift-tpch.version>0.1</airlift-tpch.version>
     <avatica.version>1.10.0</avatica.version>
@@ -111,6 +114,7 @@ limitations under the License.
     <natty.version>0.13</natty.version>
     <opencsv.version>2.3</opencsv.version>
     <oracle-jdbc6-driver.version>11.2.0.2.0</oracle-jdbc6-driver.version>
+    <owasp-dependency-check.version>2.1.1</owasp-dependency-check.version>
     <pig.version>0.16.0</pig.version>
     <aggdesigner.version>6.0</aggdesigner.version>
     <postgresql.version>9.3-1102-jdbc3</postgresql.version>
@@ -119,7 +123,7 @@ limitations under the License.
     <scott-data-hsqldb.version>0.1</scott-data-hsqldb.version>
     <servlet.version>3.0.1</servlet.version>
     <slf4j.version>1.7.13</slf4j.version>
-    <spark.version>1.6.1</spark.version>
+    <spark.version>1.6.3</spark.version>
     <sqlline.version>1.3.0</sqlline.version>
     <xalan.version>2.7.1</xalan.version>
     <xerces.version>2.9.1</xerces.version>
@@ -840,6 +844,11 @@ limitations under the License.
           <version>${javacc-maven-plugin.version}</version>
         </plugin>
         <plugin>
+          <groupId>org.owasp</groupId>
+          <artifactId>dependency-check-maven</artifactId>
+          <version>${owasp-dependency-check.version}</version>
+        </plugin>
+        <plugin>
           <groupId>pl.project13.maven</groupId>
           <artifactId>git-commit-id-plugin</artifactId>
           <version>${git-commit-id-plugin.version}</version>
@@ -1045,5 +1054,28 @@ limitations under the License.
         </plugins>
       </build>
     </profile>
+    <profile>
+      <!-- Extra checks that are disabled in the regular build, enabled for
+      releases and on demand. -->
+      <id>pedantic</id>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.owasp</groupId>
+            <artifactId>dependency-check-maven</artifactId>
+            <configuration>
+              <failBuildOnCVSS>${failBuildOnCVSS}</failBuildOnCVSS>
+            </configuration>
+            <executions>
+              <execution>
+                <goals>
+                  <goal>check</goal>
+                </goals>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
   </profiles>
 </project>

http://git-wip-us.apache.org/repos/asf/calcite/blob/d173640c/site/_docs/howto.md
----------------------------------------------------------------------
diff --git a/site/_docs/howto.md b/site/_docs/howto.md
index 1a6b27e..26744dd 100644
--- a/site/_docs/howto.md
+++ b/site/_docs/howto.md
@@ -436,6 +436,9 @@ Before you start:
 * Make sure build and tests succeed, including with `-P it,it-oracle`.
 * Make sure that `mvn javadoc:javadoc javadoc:test-javadoc` succeeds
   (i.e. gives no errors; warnings are OK)
+* Generate a report of vulnerabilities that occur among dependencies,
+  using `-Ppedantic`; if you like, run again with `-DfailBuildOnCVSS=8` to see
+  whether serious vulnerabilities exist.
 * Make sure that `mvn apache-rat:check` succeeds. (It will be run as part of
   the release, but it's better to trouble-shoot early.)
 * Decide the supported configurations of JDK, operating system and


[02/15] calcite git commit: Add test case for 'select * as x' parser bug

Posted by jh...@apache.org.
Add test case for 'select * as x' parser bug


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

Branch: refs/heads/master
Commit: 43fa8e94249d796d0e42b366feb4a8c886994aa4
Parents: 7a98575
Author: Julian Hyde <jh...@apache.org>
Authored: Mon Sep 11 15:58:17 2017 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Mon Oct 2 11:13:41 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/calcite/sql/parser/SqlParserTest.java     | 7 +++++++
 1 file changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/43fa8e94/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
index e91a76c..117862e 100644
--- a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
@@ -699,6 +699,13 @@ public class SqlParserTest {
         "Lexical error at line 1, column 10\\.  Encountered: \"#\" \\(35\\), after : \"\"");
   }
 
+  // TODO: should fail in parser
+  @Test public void testStarAsFails() {
+    sql("select * as x from emp")
+        .ok("SELECT * AS `X`\n"
+            + "FROM `EMP`");
+  }
+
   @Test public void testDerivedColumnList() {
     check("select * from emp as e (empno, gender) where true",
         "SELECT *\n"


[13/15] calcite git commit: [CALCITE-1913] Replace usages of DatabaseProduct with dialect methods, and introduce a configurable SqlDialectFactory (Christian Beikov)

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/test/java/org/apache/calcite/util/UtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/util/UtilTest.java b/core/src/test/java/org/apache/calcite/util/UtilTest.java
index 3711654..d7053c9 100644
--- a/core/src/test/java/org/apache/calcite/util/UtilTest.java
+++ b/core/src/test/java/org/apache/calcite/util/UtilTest.java
@@ -31,7 +31,7 @@ import org.apache.calcite.runtime.Resources;
 import org.apache.calcite.runtime.SqlFunctions;
 import org.apache.calcite.runtime.Utilities;
 import org.apache.calcite.sql.SqlCollation;
-import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.dialect.CalciteSqlDialect;
 import org.apache.calcite.sql.util.SqlBuilder;
 import org.apache.calcite.sql.util.SqlString;
 import org.apache.calcite.test.DiffTestCase;
@@ -653,7 +653,7 @@ public class UtilTest {
    * Tests SQL builders.
    */
   @Test public void testSqlBuilder() {
-    final SqlBuilder buf = new SqlBuilder(SqlDialect.CALCITE);
+    final SqlBuilder buf = new SqlBuilder(CalciteSqlDialect.DEFAULT);
     assertEquals(0, buf.length());
     buf.append("select ");
     assertEquals("select ", buf.getSql());
@@ -666,7 +666,7 @@ public class UtilTest {
     assertEquals("select \"x\", \"y\".\"a b\"", buf.getSql());
 
     final SqlString sqlString = buf.toSqlString();
-    assertEquals(SqlDialect.CALCITE, sqlString.getDialect());
+    assertEquals(CalciteSqlDialect.DEFAULT, sqlString.getDialect());
     assertEquals(buf.getSql(), sqlString.getSql());
 
     assertTrue(buf.getSql().length() > 0);


[14/15] calcite git commit: [CALCITE-1913] Replace usages of DatabaseProduct with dialect methods, and introduce a configurable SqlDialectFactory (Christian Beikov)

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/dialect/HsqldbHandler.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/HsqldbHandler.java b/core/src/main/java/org/apache/calcite/sql/dialect/HsqldbHandler.java
deleted file mode 100644
index 62e54bf..0000000
--- a/core/src/main/java/org/apache/calcite/sql/dialect/HsqldbHandler.java
+++ /dev/null
@@ -1,82 +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.dialect;
-
-import org.apache.calcite.avatica.util.TimeUnitRange;
-import org.apache.calcite.sql.SqlCall;
-import org.apache.calcite.sql.SqlDialect;
-import org.apache.calcite.sql.SqlLiteral;
-import org.apache.calcite.sql.SqlWriter;
-import org.apache.calcite.sql.fun.SqlFloorFunction;
-
-/**
- * Defines how a SQL parse tree should be unparsed to SQL
- * for execution against an HSQLDB database.
- *
- * <p>It reverts to the unparse method of the operator
- * if this database's implementation is standard.
- */
-public class HsqldbHandler extends SqlDialect.BaseHandler {
-  public static final HsqldbHandler INSTANCE = new HsqldbHandler();
-
-  @Override public void unparseCall(SqlWriter writer, SqlCall call,
-      int leftPrec, int rightPrec) {
-    switch (call.getKind()) {
-    case FLOOR:
-      if (call.operandCount() != 2) {
-        super.unparseCall(writer, call, leftPrec, rightPrec);
-        return;
-      }
-
-      final SqlLiteral timeUnitNode = call.operand(1);
-      final TimeUnitRange timeUnit = timeUnitNode.getValueAs(TimeUnitRange.class);
-
-      final String translatedLit = convertTimeUnit(timeUnit);
-      SqlCall call2 = SqlFloorFunction.replaceTimeUnitOperand(call, translatedLit,
-          timeUnitNode.getParserPosition());
-      SqlFloorFunction.unparseDatetimeFunction(writer, call2, "TRUNC", true);
-      break;
-
-    default:
-      super.unparseCall(writer, call, leftPrec, rightPrec);
-    }
-  }
-
-  private static String convertTimeUnit(TimeUnitRange unit) {
-    switch (unit) {
-    case YEAR:
-      return "YYYY";
-    case MONTH:
-      return "MM";
-    case DAY:
-      return "DD";
-    case WEEK:
-      return "WW";
-    case HOUR:
-      return "HH24";
-    case MINUTE:
-      return "MI";
-    case SECOND:
-      return "SS";
-    default:
-      throw new AssertionError("could not convert time unit to HSQLDB equivalent: "
-          + unit);
-    }
-  }
-}
-
-// End HsqldbHandler.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/dialect/HsqldbSqlDialect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/HsqldbSqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/HsqldbSqlDialect.java
new file mode 100644
index 0000000..d2ac74a
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/HsqldbSqlDialect.java
@@ -0,0 +1,126 @@
+/*
+ * 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.dialect;
+
+import org.apache.calcite.avatica.util.TimeUnitRange;
+import org.apache.calcite.sql.SqlBasicCall;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.fun.SqlCase;
+import org.apache.calcite.sql.fun.SqlFloorFunction;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParserPos;
+
+/**
+ * A <code>SqlDialect</code> implementation for the Hsqldb database.
+ */
+public class HsqldbSqlDialect extends SqlDialect {
+  public static final SqlDialect DEFAULT =
+      new HsqldbSqlDialect(EMPTY_CONTEXT
+          .withDatabaseProduct(DatabaseProduct.HSQLDB));
+
+  /** Creates an HsqldbSqlDialect. */
+  public HsqldbSqlDialect(Context context) {
+    super(context);
+  }
+
+  @Override public boolean supportsCharSet() {
+    return false;
+  }
+
+  @Override public void unparseCall(SqlWriter writer, SqlCall call,
+      int leftPrec, int rightPrec) {
+    switch (call.getKind()) {
+    case FLOOR:
+      if (call.operandCount() != 2) {
+        super.unparseCall(writer, call, leftPrec, rightPrec);
+        return;
+      }
+
+      final SqlLiteral timeUnitNode = call.operand(1);
+      final TimeUnitRange timeUnit = timeUnitNode.getValueAs(TimeUnitRange.class);
+
+      final String translatedLit = convertTimeUnit(timeUnit);
+      SqlCall call2 = SqlFloorFunction.replaceTimeUnitOperand(call, translatedLit,
+          timeUnitNode.getParserPosition());
+      SqlFloorFunction.unparseDatetimeFunction(writer, call2, "TRUNC", true);
+      break;
+
+    default:
+      super.unparseCall(writer, call, leftPrec, rightPrec);
+    }
+  }
+
+  @Override public SqlNode rewriteSingleValueExpr(SqlNode aggCall) {
+    final SqlNode operand = ((SqlBasicCall) aggCall).operand(0);
+    final SqlLiteral nullLiteral = SqlLiteral.createNull(SqlParserPos.ZERO);
+    final SqlNode unionOperand = SqlStdOperatorTable.VALUES.createCall(SqlParserPos.ZERO,
+        SqlLiteral.createApproxNumeric("0", SqlParserPos.ZERO));
+    // For hsqldb, generate
+    //   CASE COUNT(*)
+    //   WHEN 0 THEN NULL
+    //   WHEN 1 THEN MIN(<result>)
+    //   ELSE (VALUES 1 UNION ALL VALUES 1)
+    //   END
+    final SqlNode caseExpr =
+        new SqlCase(SqlParserPos.ZERO,
+            SqlStdOperatorTable.COUNT.createCall(SqlParserPos.ZERO, operand),
+            SqlNodeList.of(
+                SqlLiteral.createExactNumeric("0", SqlParserPos.ZERO),
+                SqlLiteral.createExactNumeric("1", SqlParserPos.ZERO)
+            ),
+            SqlNodeList.of(
+                nullLiteral,
+                SqlStdOperatorTable.MIN.createCall(SqlParserPos.ZERO, operand)
+            ),
+            SqlStdOperatorTable.SCALAR_QUERY.createCall(SqlParserPos.ZERO,
+                SqlStdOperatorTable.UNION_ALL
+                    .createCall(SqlParserPos.ZERO, unionOperand, unionOperand)));
+
+    LOGGER.debug("SINGLE_VALUE rewritten into [{}]", caseExpr);
+
+    return caseExpr;
+  }
+
+  private static String convertTimeUnit(TimeUnitRange unit) {
+    switch (unit) {
+    case YEAR:
+      return "YYYY";
+    case MONTH:
+      return "MM";
+    case DAY:
+      return "DD";
+    case WEEK:
+      return "WW";
+    case HOUR:
+      return "HH24";
+    case MINUTE:
+      return "MI";
+    case SECOND:
+      return "SS";
+    default:
+      throw new AssertionError("could not convert time unit to HSQLDB equivalent: "
+          + unit);
+    }
+  }
+}
+
+// End HsqldbSqlDialect.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/dialect/InfobrightSqlDialect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/InfobrightSqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/InfobrightSqlDialect.java
new file mode 100644
index 0000000..78ba060
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/InfobrightSqlDialect.java
@@ -0,0 +1,36 @@
+/*
+ * 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.dialect;
+
+import org.apache.calcite.sql.SqlDialect;
+
+/**
+ * A <code>SqlDialect</code> implementation for the Infobright database.
+ */
+public class InfobrightSqlDialect extends SqlDialect {
+  public static final SqlDialect DEFAULT =
+      new InfobrightSqlDialect(EMPTY_CONTEXT
+          .withDatabaseProduct(DatabaseProduct.INFOBRIGHT)
+          .withIdentifierQuoteString("`"));
+
+  /** Creates an InfobrightSqlDialect. */
+  public InfobrightSqlDialect(Context context) {
+    super(context);
+  }
+}
+
+// End InfobrightSqlDialect.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/dialect/InformixSqlDialect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/InformixSqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/InformixSqlDialect.java
new file mode 100644
index 0000000..cf5c280
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/InformixSqlDialect.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.dialect;
+
+import org.apache.calcite.sql.SqlDialect;
+
+/**
+ * A <code>SqlDialect</code> implementation for the Informix database.
+ */
+public class InformixSqlDialect extends SqlDialect {
+  public static final SqlDialect DEFAULT =
+      new InformixSqlDialect(EMPTY_CONTEXT
+          .withDatabaseProduct(DatabaseProduct.INFORMIX));
+
+  /** Creates an InformixSqlDialect. */
+  public InformixSqlDialect(Context context) {
+    super(context);
+  }
+}
+
+// End InformixSqlDialect.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/dialect/IngresSqlDialect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/IngresSqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/IngresSqlDialect.java
new file mode 100644
index 0000000..b9cff58
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/IngresSqlDialect.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.dialect;
+
+import org.apache.calcite.sql.SqlDialect;
+
+/**
+ * A <code>SqlDialect</code> implementation for the Ingres database.
+ */
+public class IngresSqlDialect extends SqlDialect {
+  public static final SqlDialect DEFAULT =
+      new IngresSqlDialect(EMPTY_CONTEXT
+          .withDatabaseProduct(DatabaseProduct.INGRES));
+
+  /** Creates an IngresSqlDialect. */
+  public IngresSqlDialect(Context context) {
+    super(context);
+  }
+}
+
+// End IngresSqlDialect.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/dialect/InterbaseSqlDialect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/InterbaseSqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/InterbaseSqlDialect.java
new file mode 100644
index 0000000..1712fc9
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/InterbaseSqlDialect.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.dialect;
+
+import org.apache.calcite.sql.SqlDialect;
+
+/**
+ * A <code>SqlDialect</code> implementation for the Interbase database.
+ */
+public class InterbaseSqlDialect extends SqlDialect {
+  public static final SqlDialect DEFAULT =
+      new InterbaseSqlDialect(EMPTY_CONTEXT
+          .withDatabaseProduct(DatabaseProduct.INTERBASE));
+
+  /** Creates an InterbaseSqlDialect. */
+  public InterbaseSqlDialect(Context context) {
+    super(context);
+  }
+}
+
+// End InterbaseSqlDialect.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/dialect/LucidDbSqlDialect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/LucidDbSqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/LucidDbSqlDialect.java
new file mode 100644
index 0000000..7d2f72a
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/LucidDbSqlDialect.java
@@ -0,0 +1,36 @@
+/*
+ * 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.dialect;
+
+import org.apache.calcite.sql.SqlDialect;
+
+/**
+ * A <code>SqlDialect</code> implementation for the LucidDB database.
+ */
+public class LucidDbSqlDialect extends SqlDialect {
+  public static final SqlDialect DEFAULT =
+      new LucidDbSqlDialect(EMPTY_CONTEXT
+          .withDatabaseProduct(DatabaseProduct.LUCIDDB)
+          .withIdentifierQuoteString("\""));
+
+  /** Creates a LucidDbSqlDialect. */
+  public LucidDbSqlDialect(Context context) {
+    super(context);
+  }
+}
+
+// End LucidDbSqlDialect.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/dialect/MssqlHandler.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/MssqlHandler.java b/core/src/main/java/org/apache/calcite/sql/dialect/MssqlHandler.java
deleted file mode 100644
index 31aa287..0000000
--- a/core/src/main/java/org/apache/calcite/sql/dialect/MssqlHandler.java
+++ /dev/null
@@ -1,128 +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.dialect;
-
-import org.apache.calcite.avatica.util.TimeUnitRange;
-import org.apache.calcite.sql.SqlCall;
-import org.apache.calcite.sql.SqlDialect;
-import org.apache.calcite.sql.SqlFunction;
-import org.apache.calcite.sql.SqlFunctionCategory;
-import org.apache.calcite.sql.SqlKind;
-import org.apache.calcite.sql.SqlLiteral;
-import org.apache.calcite.sql.SqlUtil;
-import org.apache.calcite.sql.SqlWriter;
-import org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.calcite.sql.type.ReturnTypes;
-
-/**
- * Defines how a SQL parse tree should be unparsed to SQL
- * for execution against a Microsoft SQL Server database.
- *
- * <p>It reverts to the unparse method of the operator
- * if this database's implementation is standard.
- */
-public class MssqlHandler extends SqlDialect.BaseHandler {
-  public static final MssqlHandler INSTANCE = new MssqlHandler();
-  public static final SqlFunction MSSQL_SUBSTRING =
-      new SqlFunction("SUBSTRING", SqlKind.OTHER_FUNCTION,
-          ReturnTypes.ARG0_NULLABLE_VARYING, null, null,
-          SqlFunctionCategory.STRING);
-
-  @Override public void unparseCall(SqlWriter writer, SqlCall call,
-      int leftPrec, int rightPrec) {
-    if (call.getOperator() == SqlStdOperatorTable.SUBSTRING) {
-      if (call.operandCount() != 3) {
-        throw new IllegalArgumentException("MSSQL SUBSTRING requires FROM and FOR arguments");
-      }
-      SqlUtil.unparseFunctionSyntax(MSSQL_SUBSTRING, writer, call);
-
-    } else {
-      switch (call.getKind()) {
-      case FLOOR:
-        if (call.operandCount() != 2) {
-          super.unparseCall(writer, call, leftPrec, rightPrec);
-          return;
-        }
-        unparseFloor(writer, call);
-        break;
-
-      default:
-        super.unparseCall(writer, call, leftPrec, rightPrec);
-      }
-    }
-  }
-
-  /**
-   * Unparses datetime floor for Microsoft SQL Server.
-   * There is no TRUNC function, so simulate this using calls to CONVERT.
-   *
-   * @param writer Writer
-   * @param call Call
-   */
-  private void unparseFloor(SqlWriter writer, SqlCall call) {
-    SqlLiteral node = call.operand(1);
-    TimeUnitRange unit = (TimeUnitRange) node.getValue();
-
-    switch (unit) {
-    case YEAR:
-      unparseFloorWithUnit(writer, call, 4, "-01-01");
-      break;
-    case MONTH:
-      unparseFloorWithUnit(writer, call, 7, "-01");
-      break;
-    case WEEK:
-      writer.print("CONVERT(DATETIME, CONVERT(VARCHAR(10), "
-          + "DATEADD(day, - (6 + DATEPART(weekday, ");
-      call.operand(0).unparse(writer, 0, 0);
-      writer.print(")) % 7, ");
-      call.operand(0).unparse(writer, 0, 0);
-      writer.print("), 126))");
-      break;
-    case DAY:
-      unparseFloorWithUnit(writer, call, 10, "");
-      break;
-    case HOUR:
-      unparseFloorWithUnit(writer, call, 13, ":00:00");
-      break;
-    case MINUTE:
-      unparseFloorWithUnit(writer, call, 16, ":00");
-      break;
-    case SECOND:
-      unparseFloorWithUnit(writer, call, 19, ":00");
-      break;
-    default:
-      throw new IllegalArgumentException("MSSQL does not support FLOOR for time unit: "
-          + unit);
-    }
-  }
-
-  private void unparseFloorWithUnit(SqlWriter writer, SqlCall call, int charLen,
-      String offset) {
-    writer.print("CONVERT");
-    SqlWriter.Frame frame = writer.startList("(", ")");
-    writer.print("DATETIME, CONVERT(VARCHAR(" + charLen + "), ");
-    call.operand(0).unparse(writer, 0, 0);
-    writer.print(", 126)");
-
-    if (offset.length() > 0) {
-      writer.print("+'" + offset + "'");
-    }
-    writer.endList(frame);
-  }
-}
-
-// End MssqlHandler.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/dialect/MssqlSqlDialect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/MssqlSqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/MssqlSqlDialect.java
new file mode 100644
index 0000000..3ee35e1
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/MssqlSqlDialect.java
@@ -0,0 +1,139 @@
+/*
+ * 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.dialect;
+
+import org.apache.calcite.avatica.util.TimeUnitRange;
+import org.apache.calcite.sql.SqlAbstractDateTimeLiteral;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.ReturnTypes;
+
+/**
+ * A <code>SqlDialect</code> implementation for the Microsoft SQL Server
+ * database.
+ */
+public class MssqlSqlDialect extends SqlDialect {
+  public static final SqlDialect DEFAULT =
+      new MssqlSqlDialect(EMPTY_CONTEXT
+          .withDatabaseProduct(DatabaseProduct.MSSQL)
+          .withIdentifierQuoteString("["));
+
+  private static final SqlFunction MSSQL_SUBSTRING =
+      new SqlFunction("SUBSTRING", SqlKind.OTHER_FUNCTION,
+          ReturnTypes.ARG0_NULLABLE_VARYING, null, null,
+          SqlFunctionCategory.STRING);
+
+  /** Creates a MssqlSqlDialect. */
+  public MssqlSqlDialect(Context context) {
+    super(context);
+  }
+
+  @Override public void unparseDateTimeLiteral(SqlWriter writer,
+      SqlAbstractDateTimeLiteral literal, int leftPrec, int rightPrec) {
+    writer.literal("'" + literal.toFormattedString() + "'");
+  }
+
+  @Override public void unparseCall(SqlWriter writer, SqlCall call,
+      int leftPrec, int rightPrec) {
+    if (call.getOperator() == SqlStdOperatorTable.SUBSTRING) {
+      if (call.operandCount() != 3) {
+        throw new IllegalArgumentException("MSSQL SUBSTRING requires FROM and FOR arguments");
+      }
+      SqlUtil.unparseFunctionSyntax(MSSQL_SUBSTRING, writer, call);
+    } else {
+      switch (call.getKind()) {
+      case FLOOR:
+        if (call.operandCount() != 2) {
+          super.unparseCall(writer, call, leftPrec, rightPrec);
+          return;
+        }
+        unparseFloor(writer, call);
+        break;
+
+      default:
+        super.unparseCall(writer, call, leftPrec, rightPrec);
+      }
+    }
+  }
+
+  /**
+   * Unparses datetime floor for Microsoft SQL Server.
+   * There is no TRUNC function, so simulate this using calls to CONVERT.
+   *
+   * @param writer Writer
+   * @param call Call
+   */
+  private void unparseFloor(SqlWriter writer, SqlCall call) {
+    SqlLiteral node = call.operand(1);
+    TimeUnitRange unit = (TimeUnitRange) node.getValue();
+
+    switch (unit) {
+    case YEAR:
+      unparseFloorWithUnit(writer, call, 4, "-01-01");
+      break;
+    case MONTH:
+      unparseFloorWithUnit(writer, call, 7, "-01");
+      break;
+    case WEEK:
+      writer.print("CONVERT(DATETIME, CONVERT(VARCHAR(10), "
+          + "DATEADD(day, - (6 + DATEPART(weekday, ");
+      call.operand(0).unparse(writer, 0, 0);
+      writer.print(")) % 7, ");
+      call.operand(0).unparse(writer, 0, 0);
+      writer.print("), 126))");
+      break;
+    case DAY:
+      unparseFloorWithUnit(writer, call, 10, "");
+      break;
+    case HOUR:
+      unparseFloorWithUnit(writer, call, 13, ":00:00");
+      break;
+    case MINUTE:
+      unparseFloorWithUnit(writer, call, 16, ":00");
+      break;
+    case SECOND:
+      unparseFloorWithUnit(writer, call, 19, ":00");
+      break;
+    default:
+      throw new IllegalArgumentException("MSSQL does not support FLOOR for time unit: "
+          + unit);
+    }
+  }
+
+  private void unparseFloorWithUnit(SqlWriter writer, SqlCall call, int charLen,
+      String offset) {
+    writer.print("CONVERT");
+    SqlWriter.Frame frame = writer.startList("(", ")");
+    writer.print("DATETIME, CONVERT(VARCHAR(" + charLen + "), ");
+    call.operand(0).unparse(writer, 0, 0);
+    writer.print(", 126)");
+
+    if (offset.length() > 0) {
+      writer.print("+'" + offset + "'");
+    }
+    writer.endList(frame);
+  }
+}
+
+// End MssqlSqlDialect.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/dialect/MysqlHandler.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/MysqlHandler.java b/core/src/main/java/org/apache/calcite/sql/dialect/MysqlHandler.java
deleted file mode 100644
index adf7061..0000000
--- a/core/src/main/java/org/apache/calcite/sql/dialect/MysqlHandler.java
+++ /dev/null
@@ -1,108 +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.dialect;
-
-import org.apache.calcite.avatica.util.TimeUnitRange;
-import org.apache.calcite.sql.SqlCall;
-import org.apache.calcite.sql.SqlDialect;
-import org.apache.calcite.sql.SqlLiteral;
-import org.apache.calcite.sql.SqlWriter;
-
-/**
- * Defines how a SQL parse tree should be unparsed to SQL
- * for execution against a MySQL database.
- *
- * <p>It reverts to the unparse method of the operator
- * if this database's implementation is standard.
- */
-public class MysqlHandler extends SqlDialect.BaseHandler {
-  public static final MysqlHandler INSTANCE = new MysqlHandler();
-
-  @Override public void unparseCall(SqlWriter writer, SqlCall call,
-      int leftPrec, int rightPrec) {
-    switch (call.getKind()) {
-    case FLOOR:
-      if (call.operandCount() != 2) {
-        super.unparseCall(writer, call, leftPrec, rightPrec);
-        return;
-      }
-
-      unparseFloor(writer, call);
-      break;
-
-    default:
-      super.unparseCall(writer, call, leftPrec, rightPrec);
-    }
-  }
-
-  /**
-   * Unparses datetime floor for MySQL. There is no TRUNC function, so simulate
-   * this using calls to DATE_FORMAT.
-   *
-   * @param writer Writer
-   * @param call Call
-   */
-  private void unparseFloor(SqlWriter writer, SqlCall call) {
-    SqlLiteral node = call.operand(1);
-    TimeUnitRange unit = (TimeUnitRange) node.getValue();
-
-    if (unit == TimeUnitRange.WEEK) {
-      writer.print("STR_TO_DATE");
-      SqlWriter.Frame frame = writer.startList("(", ")");
-
-      writer.print("DATE_FORMAT(");
-      call.operand(0).unparse(writer, 0, 0);
-      writer.print(", '%x%v-1'), '%x%v-%w'");
-      writer.endList(frame);
-      return;
-    }
-
-    String format;
-    switch (unit) {
-    case YEAR:
-      format = "%Y-01-01";
-      break;
-    case MONTH:
-      format = "%Y-%m-01";
-      break;
-    case DAY:
-      format = "%Y-%m-%d";
-      break;
-    case HOUR:
-      format = "%Y-%m-%d %k:00:00";
-      break;
-    case MINUTE:
-      format = "%Y-%m-%d %k:%i:00";
-      break;
-    case SECOND:
-      format = "%Y-%m-%d %k:%i:%s";
-      break;
-    default:
-      throw new AssertionError("MYSQL does not support FLOOR for time unit: "
-          + unit);
-    }
-
-    writer.print("DATE_FORMAT");
-    SqlWriter.Frame frame = writer.startList("(", ")");
-    call.operand(0).unparse(writer, 0, 0);
-    writer.sep(",", true);
-    writer.print("'" + format + "'");
-    writer.endList(frame);
-  }
-}
-
-// End MysqlHandler.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/dialect/MysqlSqlDialect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/MysqlSqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/MysqlSqlDialect.java
new file mode 100644
index 0000000..e9ae479
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/MysqlSqlDialect.java
@@ -0,0 +1,215 @@
+/*
+ * 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.dialect;
+
+import org.apache.calcite.avatica.util.TimeUnitRange;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlBasicCall;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlSelect;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.fun.SqlCase;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.InferTypes;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+
+/**
+ * A <code>SqlDialect</code> implementation for the MySQL database.
+ */
+public class MysqlSqlDialect extends SqlDialect {
+  public static final SqlDialect DEFAULT =
+      new MysqlSqlDialect(EMPTY_CONTEXT
+          .withDatabaseProduct(DatabaseProduct.MYSQL)
+          .withIdentifierQuoteString("`"));
+
+  /** MySQL specific function. */
+  public static final SqlFunction ISNULL_FUNCTION =
+      new SqlFunction("ISNULL", SqlKind.OTHER_FUNCTION,
+          ReturnTypes.BOOLEAN, InferTypes.FIRST_KNOWN,
+          OperandTypes.ANY, SqlFunctionCategory.SYSTEM);
+
+  /** Creates a MysqlSqlDialect. */
+  public MysqlSqlDialect(Context context) {
+    super(context);
+  }
+
+  @Override public boolean supportsCharSet() {
+    return false;
+  }
+
+  @Override public boolean supportsOffsetFetch() {
+    return false;
+  }
+
+  @Override public SqlNode emulateNullDirection(SqlNode node, boolean nullsFirst) {
+    node = ISNULL_FUNCTION.createCall(SqlParserPos.ZERO, node);
+    if (nullsFirst) {
+      node = SqlStdOperatorTable.DESC.createCall(SqlParserPos.ZERO, node);
+    }
+    return node;
+  }
+
+  @Override public boolean supportsAggregateFunction(SqlKind kind) {
+    switch (kind) {
+    case COUNT:
+    case SUM:
+    case SUM0:
+    case MIN:
+    case MAX:
+    case SINGLE_VALUE:
+      return true;
+    }
+    return false;
+  }
+
+  @Override public boolean supportsNestedAggregations() {
+    return false;
+  }
+
+  @Override public CalendarPolicy getCalendarPolicy() {
+    return CalendarPolicy.SHIFT;
+  }
+
+  @Override public SqlNode getCastSpec(RelDataType type) {
+    switch (type.getSqlTypeName()) {
+    case VARCHAR:
+      // MySQL doesn't have a VARCHAR type, only CHAR.
+      return new SqlDataTypeSpec(new SqlIdentifier("CHAR", SqlParserPos.ZERO),
+          type.getPrecision(), -1, null, null, SqlParserPos.ZERO);
+    case INTEGER:
+      return new SqlDataTypeSpec(new SqlIdentifier("_UNSIGNED", SqlParserPos.ZERO),
+          type.getPrecision(), -1, null, null, SqlParserPos.ZERO);
+    }
+    return super.getCastSpec(type);
+  }
+
+  @Override public SqlNode rewriteSingleValueExpr(SqlNode aggCall) {
+    final SqlNode operand = ((SqlBasicCall) aggCall).operand(0);
+    final SqlLiteral nullLiteral = SqlLiteral.createNull(SqlParserPos.ZERO);
+    final SqlNode unionOperand = new SqlSelect(SqlParserPos.ZERO, SqlNodeList.EMPTY,
+        SqlNodeList.of(nullLiteral), null, null, null, null, SqlNodeList.EMPTY, null, null, null);
+    // For MySQL, generate
+    //   CASE COUNT(*)
+    //   WHEN 0 THEN NULL
+    //   WHEN 1 THEN <result>
+    //   ELSE (SELECT NULL UNION ALL SELECT NULL)
+    //   END
+    final SqlNode caseExpr =
+        new SqlCase(SqlParserPos.ZERO,
+            SqlStdOperatorTable.COUNT.createCall(SqlParserPos.ZERO, operand),
+            SqlNodeList.of(
+                SqlLiteral.createExactNumeric("0", SqlParserPos.ZERO),
+                SqlLiteral.createExactNumeric("1", SqlParserPos.ZERO)
+            ),
+            SqlNodeList.of(
+                nullLiteral,
+                operand
+            ),
+            SqlStdOperatorTable.SCALAR_QUERY.createCall(SqlParserPos.ZERO,
+                SqlStdOperatorTable.UNION_ALL
+                    .createCall(SqlParserPos.ZERO, unionOperand, unionOperand)));
+
+    LOGGER.debug("SINGLE_VALUE rewritten into [{}]", caseExpr);
+
+    return caseExpr;
+  }
+
+  @Override public void unparseCall(SqlWriter writer, SqlCall call,
+      int leftPrec, int rightPrec) {
+    switch (call.getKind()) {
+    case FLOOR:
+      if (call.operandCount() != 2) {
+        super.unparseCall(writer, call, leftPrec, rightPrec);
+        return;
+      }
+
+      unparseFloor(writer, call);
+      break;
+
+    default:
+      super.unparseCall(writer, call, leftPrec, rightPrec);
+    }
+  }
+
+  /**
+   * Unparses datetime floor for MySQL. There is no TRUNC function, so simulate
+   * this using calls to DATE_FORMAT.
+   *
+   * @param writer Writer
+   * @param call Call
+   */
+  private void unparseFloor(SqlWriter writer, SqlCall call) {
+    SqlLiteral node = call.operand(1);
+    TimeUnitRange unit = (TimeUnitRange) node.getValue();
+
+    if (unit == TimeUnitRange.WEEK) {
+      writer.print("STR_TO_DATE");
+      SqlWriter.Frame frame = writer.startList("(", ")");
+
+      writer.print("DATE_FORMAT(");
+      call.operand(0).unparse(writer, 0, 0);
+      writer.print(", '%x%v-1'), '%x%v-%w'");
+      writer.endList(frame);
+      return;
+    }
+
+    String format;
+    switch (unit) {
+    case YEAR:
+      format = "%Y-01-01";
+      break;
+    case MONTH:
+      format = "%Y-%m-01";
+      break;
+    case DAY:
+      format = "%Y-%m-%d";
+      break;
+    case HOUR:
+      format = "%Y-%m-%d %k:00:00";
+      break;
+    case MINUTE:
+      format = "%Y-%m-%d %k:%i:00";
+      break;
+    case SECOND:
+      format = "%Y-%m-%d %k:%i:%s";
+      break;
+    default:
+      throw new AssertionError("MYSQL does not support FLOOR for time unit: "
+          + unit);
+    }
+
+    writer.print("DATE_FORMAT");
+    SqlWriter.Frame frame = writer.startList("(", ")");
+    call.operand(0).unparse(writer, 0, 0);
+    writer.sep(",", true);
+    writer.print("'" + format + "'");
+    writer.endList(frame);
+  }
+}
+
+// End MysqlSqlDialect.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/dialect/NeoviewSqlDialect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/NeoviewSqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/NeoviewSqlDialect.java
new file mode 100644
index 0000000..7b0b67a
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/NeoviewSqlDialect.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.dialect;
+
+import org.apache.calcite.sql.SqlDialect;
+
+/**
+ * A <code>SqlDialect</code> implementation for the Neoview database.
+ */
+public class NeoviewSqlDialect extends SqlDialect {
+  public static final SqlDialect DEFAULT =
+      new NeoviewSqlDialect(EMPTY_CONTEXT
+          .withDatabaseProduct(DatabaseProduct.NEOVIEW));
+
+  /** Creates a NeoviewSqlDialect. */
+  public NeoviewSqlDialect(Context context) {
+    super(context);
+  }
+}
+
+// End NeoviewSqlDialect.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/dialect/NetezzaSqlDialect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/NetezzaSqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/NetezzaSqlDialect.java
new file mode 100644
index 0000000..60be00c
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/NetezzaSqlDialect.java
@@ -0,0 +1,36 @@
+/*
+ * 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.dialect;
+
+import org.apache.calcite.sql.SqlDialect;
+
+/**
+ * A <code>SqlDialect</code> implementation for the Netezza database.
+ */
+public class NetezzaSqlDialect extends SqlDialect {
+  public static final SqlDialect DEFAULT =
+      new NetezzaSqlDialect(EMPTY_CONTEXT
+          .withDatabaseProduct(DatabaseProduct.NETEZZA)
+          .withIdentifierQuoteString("\""));
+
+  /** Creates a NetezzaSqlDialect. */
+  public NetezzaSqlDialect(Context context) {
+    super(context);
+  }
+}
+
+// End NetezzaSqlDialect.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/dialect/OracleHandler.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/OracleHandler.java b/core/src/main/java/org/apache/calcite/sql/dialect/OracleHandler.java
deleted file mode 100644
index a67eb01..0000000
--- a/core/src/main/java/org/apache/calcite/sql/dialect/OracleHandler.java
+++ /dev/null
@@ -1,67 +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.dialect;
-
-import org.apache.calcite.avatica.util.TimeUnitRange;
-import org.apache.calcite.sql.SqlCall;
-import org.apache.calcite.sql.SqlDialect;
-import org.apache.calcite.sql.SqlLiteral;
-import org.apache.calcite.sql.SqlUtil;
-import org.apache.calcite.sql.SqlWriter;
-import org.apache.calcite.sql.fun.OracleSqlOperatorTable;
-import org.apache.calcite.sql.fun.SqlFloorFunction;
-import org.apache.calcite.sql.fun.SqlStdOperatorTable;
-
-/**
- * Defines how a SQL parse tree should be unparsed to SQL
- * for execution against an Oracle database.
- *
- * <p>It reverts to the unparse method of the operator
- * if this database's implementation is standard.
- */
-public class OracleHandler extends SqlDialect.BaseHandler {
-  public static final OracleHandler INSTANCE = new OracleHandler();
-
-  @Override public void unparseCall(SqlWriter writer, SqlCall call,
-      int leftPrec, int rightPrec) {
-    if (call.getOperator() == SqlStdOperatorTable.SUBSTRING) {
-      SqlUtil.unparseFunctionSyntax(OracleSqlOperatorTable.SUBSTR, writer, call);
-
-    } else {
-      switch (call.getKind()) {
-      case FLOOR:
-        if (call.operandCount() != 2) {
-          super.unparseCall(writer, call, leftPrec, rightPrec);
-          return;
-        }
-
-        final SqlLiteral timeUnitNode = call.operand(1);
-        final TimeUnitRange timeUnit = timeUnitNode.getValueAs(TimeUnitRange.class);
-
-        SqlCall call2 = SqlFloorFunction.replaceTimeUnitOperand(call, timeUnit.name(),
-            timeUnitNode.getParserPosition());
-        SqlFloorFunction.unparseDatetimeFunction(writer, call2, "TRUNC", true);
-        break;
-
-      default:
-        super.unparseCall(writer, call, leftPrec, rightPrec);
-      }
-    }
-  }
-}
-
-// End OracleHandler.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/dialect/OracleSqlDialect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/OracleSqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/OracleSqlDialect.java
new file mode 100644
index 0000000..cdac9d7
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/OracleSqlDialect.java
@@ -0,0 +1,78 @@
+/*
+ * 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.dialect;
+
+import org.apache.calcite.avatica.util.TimeUnitRange;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.fun.OracleSqlOperatorTable;
+import org.apache.calcite.sql.fun.SqlFloorFunction;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+
+/**
+ * A <code>SqlDialect</code> implementation for the Oracle database.
+ */
+public class OracleSqlDialect extends SqlDialect {
+  public static final SqlDialect DEFAULT =
+      new OracleSqlDialect(EMPTY_CONTEXT
+          .withDatabaseProduct(DatabaseProduct.ORACLE)
+          .withIdentifierQuoteString("\""));
+
+  /** Creates an OracleSqlDialect. */
+  public OracleSqlDialect(Context context) {
+    super(context);
+  }
+
+  @Override public boolean supportsCharSet() {
+    return false;
+  }
+
+  @Override protected boolean allowsAs() {
+    return false;
+  }
+
+  @Override public void unparseCall(SqlWriter writer, SqlCall call,
+      int leftPrec, int rightPrec) {
+    if (call.getOperator() == SqlStdOperatorTable.SUBSTRING) {
+      SqlUtil.unparseFunctionSyntax(OracleSqlOperatorTable.SUBSTR, writer, call);
+    } else {
+      switch (call.getKind()) {
+      case FLOOR:
+        if (call.operandCount() != 2) {
+          super.unparseCall(writer, call, leftPrec, rightPrec);
+          return;
+        }
+
+        final SqlLiteral timeUnitNode = call.operand(1);
+        final TimeUnitRange timeUnit = timeUnitNode.getValueAs(TimeUnitRange.class);
+
+        SqlCall call2 = SqlFloorFunction.replaceTimeUnitOperand(call, timeUnit.name(),
+            timeUnitNode.getParserPosition());
+        SqlFloorFunction.unparseDatetimeFunction(writer, call2, "TRUNC", true);
+        break;
+
+      default:
+        super.unparseCall(writer, call, leftPrec, rightPrec);
+      }
+    }
+  }
+}
+
+// End OracleSqlDialect.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/dialect/ParaccelSqlDialect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/ParaccelSqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/ParaccelSqlDialect.java
new file mode 100644
index 0000000..07b2d24
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/ParaccelSqlDialect.java
@@ -0,0 +1,36 @@
+/*
+ * 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.dialect;
+
+import org.apache.calcite.sql.SqlDialect;
+
+/**
+ * A <code>SqlDialect</code> implementation for the Paraccel database.
+ */
+public class ParaccelSqlDialect extends SqlDialect {
+  public static final SqlDialect DEFAULT =
+      new ParaccelSqlDialect(EMPTY_CONTEXT
+          .withDatabaseProduct(DatabaseProduct.PARACCEL)
+          .withIdentifierQuoteString("\""));
+
+  /** Creates a ParaccelSqlDialect. */
+  public ParaccelSqlDialect(Context context) {
+    super(context);
+  }
+}
+
+// End ParaccelSqlDialect.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/dialect/PhoenixSqlDialect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/PhoenixSqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/PhoenixSqlDialect.java
new file mode 100644
index 0000000..cb409fe
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/PhoenixSqlDialect.java
@@ -0,0 +1,40 @@
+/*
+ * 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.dialect;
+
+import org.apache.calcite.sql.SqlDialect;
+
+/**
+ * A <code>SqlDialect</code> implementation for the Apache Phoenix database.
+ */
+public class PhoenixSqlDialect extends SqlDialect {
+  public static final SqlDialect DEFAULT =
+      new PhoenixSqlDialect(EMPTY_CONTEXT
+          .withDatabaseProduct(DatabaseProduct.PHOENIX)
+          .withIdentifierQuoteString("\""));
+
+  /** Creates a PhoenixSqlDialect. */
+  public PhoenixSqlDialect(Context context) {
+    super(context);
+  }
+
+  @Override public boolean supportsCharSet() {
+    return false;
+  }
+}
+
+// End PhoenixSqlDialect.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/dialect/PostgresqlHandler.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/PostgresqlHandler.java b/core/src/main/java/org/apache/calcite/sql/dialect/PostgresqlHandler.java
deleted file mode 100644
index 8b35346..0000000
--- a/core/src/main/java/org/apache/calcite/sql/dialect/PostgresqlHandler.java
+++ /dev/null
@@ -1,59 +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.dialect;
-
-import org.apache.calcite.avatica.util.TimeUnitRange;
-import org.apache.calcite.sql.SqlCall;
-import org.apache.calcite.sql.SqlDialect;
-import org.apache.calcite.sql.SqlLiteral;
-import org.apache.calcite.sql.SqlWriter;
-import org.apache.calcite.sql.fun.SqlFloorFunction;
-
-/**
- * Defines how a SQL parse tree should be unparsed to SQL
- * for execution against a Postgresql database.
- *
- * <p>It reverts to the unparse method of the operator
- * if this database's implementation is standard.
- */
-public class PostgresqlHandler extends SqlDialect.BaseHandler {
-  public static final PostgresqlHandler INSTANCE = new PostgresqlHandler();
-
-  @Override public void unparseCall(SqlWriter writer, SqlCall call,
-      int leftPrec, int rightPrec) {
-    switch (call.getKind()) {
-    case FLOOR:
-      if (call.operandCount() != 2) {
-        super.unparseCall(writer, call, leftPrec, rightPrec);
-        return;
-      }
-
-      final SqlLiteral timeUnitNode = call.operand(1);
-      final TimeUnitRange timeUnit = timeUnitNode.getValueAs(TimeUnitRange.class);
-
-      SqlCall call2 = SqlFloorFunction.replaceTimeUnitOperand(call, timeUnit.name(),
-          timeUnitNode.getParserPosition());
-      SqlFloorFunction.unparseDatetimeFunction(writer, call2, "DATE_TRUNC", false);
-      break;
-
-    default:
-      super.unparseCall(writer, call, leftPrec, rightPrec);
-    }
-  }
-}
-
-// End PostgresqlHandler.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/dialect/PostgresqlSqlDialect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/PostgresqlSqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/PostgresqlSqlDialect.java
new file mode 100644
index 0000000..6a6014e
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/PostgresqlSqlDialect.java
@@ -0,0 +1,75 @@
+/*
+ * 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.dialect;
+
+import org.apache.calcite.avatica.util.TimeUnitRange;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.fun.SqlFloorFunction;
+
+/**
+ * A <code>SqlDialect</code> implementation for the PostgreSQL database.
+ */
+public class PostgresqlSqlDialect extends SqlDialect {
+  public static final SqlDialect DEFAULT =
+      new PostgresqlSqlDialect(EMPTY_CONTEXT
+          .withDatabaseProduct(DatabaseProduct.POSTGRESQL)
+          .withIdentifierQuoteString("\""));
+
+  /** Creates a PostgresqlSqlDialect. */
+  public PostgresqlSqlDialect(Context context) {
+    super(context);
+  }
+
+  @Override public boolean supportsCharSet() {
+    return false;
+  }
+
+  @Override protected boolean requiresAliasForFromItems() {
+    return true;
+  }
+
+  @Override public boolean supportsNestedAggregations() {
+    return false;
+  }
+
+  @Override public void unparseCall(SqlWriter writer, SqlCall call,
+      int leftPrec, int rightPrec) {
+    switch (call.getKind()) {
+    case FLOOR:
+      if (call.operandCount() != 2) {
+        super.unparseCall(writer, call, leftPrec, rightPrec);
+        return;
+      }
+
+      final SqlLiteral timeUnitNode = call.operand(1);
+      final TimeUnitRange timeUnit = timeUnitNode.getValueAs(TimeUnitRange.class);
+
+      SqlCall call2 = SqlFloorFunction.replaceTimeUnitOperand(call, timeUnit.name(),
+          timeUnitNode.getParserPosition());
+      SqlFloorFunction.unparseDatetimeFunction(writer, call2, "DATE_TRUNC", false);
+      break;
+
+    default:
+      super.unparseCall(writer, call, leftPrec, rightPrec);
+    }
+  }
+}
+
+// End PostgresqlSqlDialect.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/dialect/RedshiftSqlDialect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/RedshiftSqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/RedshiftSqlDialect.java
new file mode 100644
index 0000000..33ba101
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/RedshiftSqlDialect.java
@@ -0,0 +1,40 @@
+/*
+ * 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.dialect;
+
+import org.apache.calcite.sql.SqlDialect;
+
+/**
+ * A <code>SqlDialect</code> implementation for the Redshift database.
+ */
+public class RedshiftSqlDialect extends SqlDialect {
+  public static final SqlDialect DEFAULT =
+      new RedshiftSqlDialect(EMPTY_CONTEXT
+          .withDatabaseProduct(DatabaseProduct.REDSHIFT)
+          .withIdentifierQuoteString("\""));
+
+  /** Creates a RedshiftSqlDialect. */
+  public RedshiftSqlDialect(Context context) {
+    super(context);
+  }
+
+  @Override public boolean supportsOffsetFetch() {
+    return false;
+  }
+}
+
+// End RedshiftSqlDialect.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/dialect/SybaseSqlDialect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/SybaseSqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/SybaseSqlDialect.java
new file mode 100644
index 0000000..0b50e4b
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/SybaseSqlDialect.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.dialect;
+
+import org.apache.calcite.sql.SqlDialect;
+
+/**
+ * A <code>SqlDialect</code> implementation for the Sybase database.
+ */
+public class SybaseSqlDialect extends SqlDialect {
+  public static final SqlDialect DEFAULT =
+      new SybaseSqlDialect(EMPTY_CONTEXT
+          .withDatabaseProduct(DatabaseProduct.SYBASE));
+
+  /** Creates a SybaseSqlDialect. */
+  public SybaseSqlDialect(Context context) {
+    super(context);
+  }
+}
+
+// End SybaseSqlDialect.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/dialect/TeradataSqlDialect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/TeradataSqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/TeradataSqlDialect.java
new file mode 100644
index 0000000..b2a52fc
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/TeradataSqlDialect.java
@@ -0,0 +1,36 @@
+/*
+ * 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.dialect;
+
+import org.apache.calcite.sql.SqlDialect;
+
+/**
+ * A <code>SqlDialect</code> implementation for the Teradata database.
+ */
+public class TeradataSqlDialect extends SqlDialect {
+  public static final SqlDialect DEFAULT =
+      new TeradataSqlDialect(EMPTY_CONTEXT
+          .withDatabaseProduct(DatabaseProduct.TERADATA)
+          .withIdentifierQuoteString("\""));
+
+  /** Creates a TeradataSqlDialect. */
+  public TeradataSqlDialect(Context context) {
+    super(context);
+  }
+}
+
+// End TeradataSqlDialect.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/dialect/VerticaSqlDialect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/VerticaSqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/VerticaSqlDialect.java
new file mode 100644
index 0000000..5781feb
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/VerticaSqlDialect.java
@@ -0,0 +1,40 @@
+/*
+ * 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.dialect;
+
+import org.apache.calcite.sql.SqlDialect;
+
+/**
+ * A <code>SqlDialect</code> implementation for the Vertica database.
+ */
+public class VerticaSqlDialect extends SqlDialect {
+  public static final SqlDialect DEFAULT =
+      new VerticaSqlDialect(EMPTY_CONTEXT
+          .withDatabaseProduct(DatabaseProduct.VERTICA)
+          .withIdentifierQuoteString("\""));
+
+  /** Creates a VerticaSqlDialect. */
+  public VerticaSqlDialect(Context context) {
+    super(context);
+  }
+
+  @Override public boolean supportsNestedAggregations() {
+    return false;
+  }
+}
+
+// End VerticaSqlDialect.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/pretty/SqlPrettyWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/pretty/SqlPrettyWriter.java b/core/src/main/java/org/apache/calcite/sql/pretty/SqlPrettyWriter.java
index dde8d28..0afd951 100644
--- a/core/src/main/java/org/apache/calcite/sql/pretty/SqlPrettyWriter.java
+++ b/core/src/main/java/org/apache/calcite/sql/pretty/SqlPrettyWriter.java
@@ -20,6 +20,7 @@ import org.apache.calcite.avatica.util.Spaces;
 import org.apache.calcite.sql.SqlDialect;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.dialect.AnsiSqlDialect;
 import org.apache.calcite.sql.util.SqlBuilder;
 import org.apache.calcite.sql.util.SqlString;
 import org.apache.calcite.util.Unsafe;
@@ -131,7 +132,7 @@ public class SqlPrettyWriter implements SqlWriter {
    * Bean holding the default property values.
    */
   private static final Bean DEFAULT_BEAN =
-      new SqlPrettyWriter(SqlDialect.DUMMY).getBean();
+      new SqlPrettyWriter(AnsiSqlDialect.DEFAULT).getBean();
   protected static final String NL = System.getProperty("line.separator");
 
   //~ Instance fields --------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/main/java/org/apache/calcite/sql/type/IntervalSqlType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/IntervalSqlType.java b/core/src/main/java/org/apache/calcite/sql/type/IntervalSqlType.java
index d4b2284..4789c06 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/IntervalSqlType.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/IntervalSqlType.java
@@ -22,6 +22,7 @@ import org.apache.calcite.rel.type.RelDataTypeFactoryImpl;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.sql.SqlDialect;
 import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.dialect.AnsiSqlDialect;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.pretty.SqlPrettyWriter;
 import org.apache.calcite.sql.util.SqlString;
@@ -56,7 +57,7 @@ public class IntervalSqlType extends AbstractSqlType {
 
   protected void generateTypeString(StringBuilder sb, boolean withDetail) {
     sb.append("INTERVAL ");
-    final SqlDialect dialect = SqlDialect.DUMMY;
+    final SqlDialect dialect = AnsiSqlDialect.DEFAULT;
     final SqlPrettyWriter writer = new SqlPrettyWriter(dialect);
     writer.setAlwaysUseParentheses(false);
     writer.setSelectListItemsOnSeparateLines(false);

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java b/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
index fb5d118..6716d3d 100644
--- a/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
@@ -30,6 +30,7 @@ import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.sql.SqlDialect;
 import org.apache.calcite.sql.SqlDialect.DatabaseProduct;
 import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.dialect.CalciteSqlDialect;
 import org.apache.calcite.sql.parser.SqlParser;
 import org.apache.calcite.sql2rel.SqlToRelConverter;
 import org.apache.calcite.test.CalciteAssert;
@@ -74,7 +75,7 @@ public class RelToSqlConverterTest {
   /** Initiates a test case with a given SQL query. */
   private Sql sql(String sql) {
     return new Sql(CalciteAssert.SchemaSpec.JDBC_FOODMART, sql,
-        SqlDialect.CALCITE, DEFAULT_REL_CONFIG,
+        CalciteSqlDialect.DEFAULT, DEFAULT_REL_CONFIG,
         ImmutableList.<Function<RelNode, RelNode>>of());
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
index 117862e..1042e20 100644
--- a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
@@ -18,10 +18,10 @@ package org.apache.calcite.sql.parser;
 
 import org.apache.calcite.avatica.util.Casing;
 import org.apache.calcite.avatica.util.Quoting;
-import org.apache.calcite.sql.SqlDialect;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlSetOption;
+import org.apache.calcite.sql.dialect.CalciteSqlDialect;
 import org.apache.calcite.sql.parser.impl.SqlParserImpl;
 import org.apache.calcite.sql.pretty.SqlPrettyWriter;
 import org.apache.calcite.sql.validate.SqlConformance;
@@ -7259,11 +7259,11 @@ public class SqlParserTest {
     SqlNode node = getSqlParser("alter system set schema = true").parseStmt();
     SqlSetOption opt = (SqlSetOption) node;
     assertThat(opt.getScope(), equalTo("SYSTEM"));
-    SqlPrettyWriter writer = new SqlPrettyWriter(SqlDialect.CALCITE);
+    SqlPrettyWriter writer = new SqlPrettyWriter(CalciteSqlDialect.DEFAULT);
     assertThat(writer.format(opt.getName()), equalTo("\"SCHEMA\""));
-    writer = new SqlPrettyWriter(SqlDialect.CALCITE);
+    writer = new SqlPrettyWriter(CalciteSqlDialect.DEFAULT);
     assertThat(writer.format(opt.getValue()), equalTo("TRUE"));
-    writer = new SqlPrettyWriter(SqlDialect.CALCITE);
+    writer = new SqlPrettyWriter(CalciteSqlDialect.DEFAULT);
     assertThat(writer.format(opt),
         equalTo("ALTER SYSTEM SET \"SCHEMA\" = TRUE"));
 
@@ -7291,10 +7291,10 @@ public class SqlParserTest {
     node = getSqlParser("reset schema").parseStmt();
     opt = (SqlSetOption) node;
     assertThat(opt.getScope(), equalTo(null));
-    writer = new SqlPrettyWriter(SqlDialect.CALCITE);
+    writer = new SqlPrettyWriter(CalciteSqlDialect.DEFAULT);
     assertThat(writer.format(opt.getName()), equalTo("\"SCHEMA\""));
     assertThat(opt.getValue(), equalTo(null));
-    writer = new SqlPrettyWriter(SqlDialect.CALCITE);
+    writer = new SqlPrettyWriter(CalciteSqlDialect.DEFAULT);
     assertThat(writer.format(opt),
         equalTo("RESET \"SCHEMA\""));
 
@@ -8185,7 +8185,7 @@ public class SqlParserTest {
       // Unparse again in Calcite dialect (which we can parse), and
       // minimal parentheses.
       final String sql1 =
-          sqlNode.toSqlString(SqlDialect.CALCITE, false).getSql();
+          sqlNode.toSqlString(CalciteSqlDialect.DEFAULT, false).getSql();
 
       // Parse and unparse again.
       SqlNode sqlNode2;
@@ -8197,7 +8197,7 @@ public class SqlParserTest {
         quoting = q;
       }
       final String sql2 =
-          sqlNode2.toSqlString(SqlDialect.CALCITE, false).getSql();
+          sqlNode2.toSqlString(CalciteSqlDialect.DEFAULT, false).getSql();
 
       // Should be the same as we started with.
       assertEquals(sql1, sql2);
@@ -8219,7 +8219,7 @@ public class SqlParserTest {
       // Unparse again in Calcite dialect (which we can parse), and
       // minimal parentheses.
       final String sql1 =
-          sqlNode.toSqlString(SqlDialect.CALCITE, false).getSql();
+          sqlNode.toSqlString(CalciteSqlDialect.DEFAULT, false).getSql();
 
       // Parse and unparse again.
       SqlNode sqlNode2;
@@ -8231,7 +8231,7 @@ public class SqlParserTest {
         quoting = q;
       }
       final String sql2 =
-          sqlNode2.toSqlString(SqlDialect.CALCITE, false).getSql();
+          sqlNode2.toSqlString(CalciteSqlDialect.DEFAULT, false).getSql();
 
       // Should be the same as we started with.
       assertEquals(sql1, sql2);

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/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 15ddb13..7c0be3f 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
@@ -26,7 +26,6 @@ import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlCallBinding;
 import org.apache.calcite.sql.SqlDataTypeSpec;
-import org.apache.calcite.sql.SqlDialect;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlJdbcFunctionCall;
 import org.apache.calcite.sql.SqlLiteral;
@@ -34,6 +33,8 @@ import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlNodeList;
 import org.apache.calcite.sql.SqlOperandCountRange;
 import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.dialect.AnsiSqlDialect;
+import org.apache.calcite.sql.dialect.CalciteSqlDialect;
 import org.apache.calcite.sql.fun.OracleSqlOperatorTable;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParserPos;
@@ -6733,7 +6734,7 @@ public abstract class SqlOperatorBaseTest {
         SqlLiteral literal =
             type.getSqlTypeName().createLiteral(o, SqlParserPos.ZERO);
         SqlString literalString =
-            literal.toSqlString(SqlDialect.DUMMY);
+            literal.toSqlString(AnsiSqlDialect.DEFAULT);
         final String expr =
             "CAST(" + literalString
                 + " AS " + type + ")";
@@ -6783,7 +6784,7 @@ public abstract class SqlOperatorBaseTest {
         SqlLiteral literal =
             type.getSqlTypeName().createLiteral(o, SqlParserPos.ZERO);
         SqlString literalString =
-            literal.toSqlString(SqlDialect.DUMMY);
+            literal.toSqlString(AnsiSqlDialect.DEFAULT);
 
         if ((type.getSqlTypeName() == SqlTypeName.BIGINT)
             || ((type.getSqlTypeName() == SqlTypeName.DECIMAL)
@@ -6900,7 +6901,7 @@ public abstract class SqlOperatorBaseTest {
             continue;
           }
           final SqlPrettyWriter writer =
-              new SqlPrettyWriter(SqlDialect.CALCITE);
+              new SqlPrettyWriter(CalciteSqlDialect.DEFAULT);
           op.unparse(writer, call, 0, 0);
           final String s = writer.toSqlString().toString();
           if (s.startsWith("OVERLAY(")

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/test/java/org/apache/calcite/sql/test/SqlPrettyWriterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlPrettyWriterTest.java b/core/src/test/java/org/apache/calcite/sql/test/SqlPrettyWriterTest.java
index 83d5a7e..ce7a806 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlPrettyWriterTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlPrettyWriterTest.java
@@ -17,9 +17,9 @@
 package org.apache.calcite.sql.test;
 
 import org.apache.calcite.sql.SqlCall;
-import org.apache.calcite.sql.SqlDialect;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.dialect.AnsiSqlDialect;
 import org.apache.calcite.sql.parser.SqlParseException;
 import org.apache.calcite.sql.parser.SqlParser;
 import org.apache.calcite.sql.pretty.SqlPrettyWriter;
@@ -78,7 +78,7 @@ public class SqlPrettyWriterTest {
       String expected) {
     final SqlNode node = parseQuery(sql);
     final SqlPrettyWriter prettyWriter =
-        new SqlPrettyWriter(SqlDialect.DUMMY);
+        new SqlPrettyWriter(AnsiSqlDialect.DEFAULT);
     prettyWriter.setAlwaysUseParentheses(false);
     if (newlines) {
       prettyWriter.setCaseClausesOnNewLines(true);
@@ -101,7 +101,7 @@ public class SqlPrettyWriterTest {
     final SqlCall rowCall = valuesCall.operand(0);
     final SqlNode node = rowCall.operand(0);
     final SqlPrettyWriter prettyWriter =
-        new SqlPrettyWriter(SqlDialect.DUMMY);
+        new SqlPrettyWriter(AnsiSqlDialect.DEFAULT);
     prettyWriter.setAlwaysUseParentheses(false);
     if (newlines) {
       prettyWriter.setCaseClausesOnNewLines(true);
@@ -153,34 +153,34 @@ public class SqlPrettyWriterTest {
 
   @Test public void testDefault() throws Exception {
     final SqlPrettyWriter prettyWriter =
-        new SqlPrettyWriter(SqlDialect.DUMMY);
+        new SqlPrettyWriter(AnsiSqlDialect.DEFAULT);
     checkSimple(prettyWriter, "${desc}", "${formatted}");
   }
 
   @Test public void testIndent8() throws Exception {
     final SqlPrettyWriter prettyWriter =
-        new SqlPrettyWriter(SqlDialect.DUMMY);
+        new SqlPrettyWriter(AnsiSqlDialect.DEFAULT);
     prettyWriter.setIndentation(8);
     checkSimple(prettyWriter, "${desc}", "${formatted}");
   }
 
   @Test public void testClausesNotOnNewLine() throws Exception {
     final SqlPrettyWriter prettyWriter =
-        new SqlPrettyWriter(SqlDialect.DUMMY);
+        new SqlPrettyWriter(AnsiSqlDialect.DEFAULT);
     prettyWriter.setClauseStartsLine(false);
     checkSimple(prettyWriter, "${desc}", "${formatted}");
   }
 
   @Test public void testSelectListItemsOnSeparateLines() throws Exception {
     final SqlPrettyWriter prettyWriter =
-        new SqlPrettyWriter(SqlDialect.DUMMY);
+        new SqlPrettyWriter(AnsiSqlDialect.DEFAULT);
     prettyWriter.setSelectListItemsOnSeparateLines(true);
     checkSimple(prettyWriter, "${desc}", "${formatted}");
   }
 
   @Test public void testSelectListExtraIndentFlag() throws Exception {
     final SqlPrettyWriter prettyWriter =
-        new SqlPrettyWriter(SqlDialect.DUMMY);
+        new SqlPrettyWriter(AnsiSqlDialect.DEFAULT);
     prettyWriter.setSelectListItemsOnSeparateLines(true);
     prettyWriter.setSelectListExtraIndentFlag(false);
     checkSimple(prettyWriter, "${desc}", "${formatted}");
@@ -188,21 +188,21 @@ public class SqlPrettyWriterTest {
 
   @Test public void testKeywordsLowerCase() throws Exception {
     final SqlPrettyWriter prettyWriter =
-        new SqlPrettyWriter(SqlDialect.DUMMY);
+        new SqlPrettyWriter(AnsiSqlDialect.DEFAULT);
     prettyWriter.setKeywordsLowerCase(true);
     checkSimple(prettyWriter, "${desc}", "${formatted}");
   }
 
   @Test public void testParenthesizeAllExprs() throws Exception {
     final SqlPrettyWriter prettyWriter =
-        new SqlPrettyWriter(SqlDialect.DUMMY);
+        new SqlPrettyWriter(AnsiSqlDialect.DEFAULT);
     prettyWriter.setAlwaysUseParentheses(true);
     checkSimple(prettyWriter, "${desc}", "${formatted}");
   }
 
   @Test public void testOnlyQuoteIdentifiersWhichNeedIt() throws Exception {
     final SqlPrettyWriter prettyWriter =
-        new SqlPrettyWriter(SqlDialect.DUMMY);
+        new SqlPrettyWriter(AnsiSqlDialect.DEFAULT);
     prettyWriter.setQuoteAllIdentifiers(false);
     checkSimple(prettyWriter, "${desc}", "${formatted}");
   }
@@ -211,7 +211,7 @@ public class SqlPrettyWriterTest {
     // Note that ( is at the indent, SELECT is on the same line, and ) is
     // below it.
     final SqlPrettyWriter prettyWriter =
-        new SqlPrettyWriter(SqlDialect.DUMMY);
+        new SqlPrettyWriter(AnsiSqlDialect.DEFAULT);
     prettyWriter.setSubQueryStyle(SqlWriter.SubQueryStyle.BLACK);
     checkSimple(prettyWriter, "${desc}", "${formatted}");
   }
@@ -330,7 +330,7 @@ public class SqlPrettyWriterTest {
 
   private void checkPrettySeparateLines(String sql) {
     final SqlPrettyWriter prettyWriter =
-        new SqlPrettyWriter(SqlDialect.DUMMY);
+        new SqlPrettyWriter(AnsiSqlDialect.DEFAULT);
     prettyWriter.setSelectListItemsOnSeparateLines(true);
     prettyWriter.setSelectListExtraIndentFlag(false);
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/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 f4b2fbd..c96c4e8 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
@@ -24,13 +24,13 @@ 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.SqlDialect;
 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;
@@ -475,7 +475,7 @@ public class SqlTesterImpl implements SqlTester, AutoCloseable {
       String expectedRewrite) {
     SqlNode rewrittenNode = parseAndValidate(validator, query);
     String actualRewrite =
-        rewrittenNode.toSqlString(SqlDialect.DUMMY, false).getSql();
+        rewrittenNode.toSqlString(AnsiSqlDialect.DEFAULT, false).getSql();
     TestUtil.assertEqualsVerbose(expectedRewrite, Util.toLinux(actualRewrite));
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/test/java/org/apache/calcite/test/SqlLimitsTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlLimitsTest.java b/core/src/test/java/org/apache/calcite/test/SqlLimitsTest.java
index 86c55f0..1bb26d4 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlLimitsTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlLimitsTest.java
@@ -21,8 +21,8 @@ import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.calcite.sql.SqlDialect;
 import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.dialect.AnsiSqlDialect;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.BasicSqlType;
 import org.apache.calcite.sql.type.SqlTypeName;
@@ -204,7 +204,7 @@ public class SqlLimitsTest {
     SqlLiteral literal =
         type.getSqlTypeName().createLiteral(o, SqlParserPos.ZERO);
     pw.print("; as SQL: ");
-    pw.print(literal.toSqlString(SqlDialect.DUMMY));
+    pw.print(literal.toSqlString(AnsiSqlDialect.DEFAULT));
     pw.println();
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/test/java/org/apache/calcite/tools/FrameworksTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/tools/FrameworksTest.java b/core/src/test/java/org/apache/calcite/tools/FrameworksTest.java
index 6170563..6de282e 100644
--- a/core/src/test/java/org/apache/calcite/tools/FrameworksTest.java
+++ b/core/src/test/java/org/apache/calcite/tools/FrameworksTest.java
@@ -39,10 +39,10 @@ import org.apache.calcite.schema.Schemas;
 import org.apache.calcite.schema.Table;
 import org.apache.calcite.schema.impl.AbstractTable;
 import org.apache.calcite.server.CalciteServerStatement;
-import org.apache.calcite.sql.SqlDialect;
 import org.apache.calcite.sql.SqlExplainFormat;
 import org.apache.calcite.sql.SqlExplainLevel;
 import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.dialect.AnsiSqlDialect;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.test.CalciteAssert;
@@ -186,7 +186,7 @@ public class FrameworksTest {
     SqlNode val = planner.validate(parse);
 
     String valStr =
-        val.toSqlString(SqlDialect.DUMMY, false).getSql();
+        val.toSqlString(AnsiSqlDialect.DEFAULT, false).getSql();
 
     String expandedStr =
         "SELECT `emps`.`empid`, `emps`.`deptno`, `emps`.`name`, `emps`.`salary`, `emps`.`commission`\n"

http://git-wip-us.apache.org/repos/asf/calcite/blob/914b5cfb/core/src/test/java/org/apache/calcite/util/Smalls.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/util/Smalls.java b/core/src/test/java/org/apache/calcite/util/Smalls.java
index 6ea55bf..1b8824a 100644
--- a/core/src/test/java/org/apache/calcite/util/Smalls.java
+++ b/core/src/test/java/org/apache/calcite/util/Smalls.java
@@ -47,8 +47,8 @@ import org.apache.calcite.schema.TranslatableTable;
 import org.apache.calcite.schema.impl.AbstractTable;
 import org.apache.calcite.schema.impl.ViewTable;
 import org.apache.calcite.sql.SqlCall;
-import org.apache.calcite.sql.SqlDialect;
 import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.dialect.CalciteSqlDialect;
 import org.apache.calcite.sql.type.SqlTypeName;
 
 import com.google.common.collect.ImmutableList;
@@ -357,7 +357,7 @@ public class Smalls {
             return typeFactory.builder().add("c", SqlTypeName.VARCHAR, 100)
                     .build();
           }
-        }, "values (" + SqlDialect.CALCITE.quoteStringLiteral(s) + ")",
+        }, "values (" + CalciteSqlDialect.DEFAULT.quoteStringLiteral(s) + ")",
         ImmutableList.<String>of(), Arrays.asList("view"));
   }
 
@@ -371,8 +371,8 @@ public class Smalls {
                 .build();
           }
         },
-        "values " + SqlDialect.CALCITE.quoteStringLiteral(o.toString())
-            + ", " + SqlDialect.CALCITE.quoteStringLiteral(p.toString()),
+        "values " + CalciteSqlDialect.DEFAULT.quoteStringLiteral(o.toString())
+            + ", " + CalciteSqlDialect.DEFAULT.quoteStringLiteral(p.toString()),
         ImmutableList.<String>of(), Arrays.asList("view"));
   }
 


[12/15] calcite git commit: Fix URL in FileSchemaFactory javadoc (Marc Prud'hommeaux)

Posted by jh...@apache.org.
Fix URL in FileSchemaFactory javadoc (Marc Prud'hommeaux)

Close apache/calcite#537


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

Branch: refs/heads/master
Commit: 1a71a66bac99f588e33896c385174eed51f1de16
Parents: f1a002e
Author: Marc Prud'hommeaux <mw...@cornell.edu>
Authored: Wed Sep 13 10:27:15 2017 -0400
Committer: Julian Hyde <jh...@apache.org>
Committed: Mon Oct 2 11:13:44 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/calcite/adapter/file/FileSchemaFactory.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/1a71a66b/file/src/main/java/org/apache/calcite/adapter/file/FileSchemaFactory.java
----------------------------------------------------------------------
diff --git a/file/src/main/java/org/apache/calcite/adapter/file/FileSchemaFactory.java b/file/src/main/java/org/apache/calcite/adapter/file/FileSchemaFactory.java
index c92a729..4fae382 100644
--- a/file/src/main/java/org/apache/calcite/adapter/file/FileSchemaFactory.java
+++ b/file/src/main/java/org/apache/calcite/adapter/file/FileSchemaFactory.java
@@ -29,7 +29,7 @@ import java.util.Map;
  * Factory that creates a {@link FileSchema}.
  *
  * <p>Allows a custom schema to be included in a model.json file.
- * See <a href="http://calcite.apache.org/docs/file.html">File adapter</a>.
+ * See <a href="http://calcite.apache.org/docs/file_adapter.html">File adapter</a>.
  */
 @SuppressWarnings("UnusedDeclaration")
 public class FileSchemaFactory implements SchemaFactory {


[03/15] calcite git commit: [CALCITE-1946] JDBC adapter should generate sub-SELECT if dialect does not support nested aggregate functions (Pawel Ruchaj)

Posted by jh...@apache.org.
[CALCITE-1946] JDBC adapter should generate sub-SELECT if dialect does not support nested aggregate functions (Pawel Ruchaj)

Close apache/calcite#520


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

Branch: refs/heads/master
Commit: 4d0e83e7e51b28acff7305e1342c5107f1357b1c
Parents: 41687f3
Author: Pawel <pa...@profimedia.co.uk>
Authored: Thu Aug 17 15:38:48 2017 +0100
Committer: Julian Hyde <jh...@apache.org>
Committed: Mon Oct 2 11:13:41 2017 -0700

----------------------------------------------------------------------
 .../calcite/rel/rel2sql/SqlImplementor.java     | 32 +++++++++++++++
 .../java/org/apache/calcite/sql/SqlDialect.java | 15 +++++++
 .../rel/rel2sql/RelToSqlConverterTest.java      | 41 ++++++++++++++++++++
 3 files changed, 88 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/4d0e83e7/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java b/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java
index 57155b7..3a4b649 100644
--- a/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java
+++ b/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java
@@ -23,6 +23,7 @@ import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.logical.LogicalAggregate;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexCall;
@@ -41,6 +42,7 @@ import org.apache.calcite.rex.RexSubQuery;
 import org.apache.calcite.rex.RexWindow;
 import org.apache.calcite.rex.RexWindowBound;
 import org.apache.calcite.sql.JoinType;
+import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlBasicCall;
 import org.apache.calcite.sql.SqlBinaryOperator;
 import org.apache.calcite.sql.SqlCall;
@@ -92,6 +94,7 @@ import java.util.AbstractList;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
@@ -1023,6 +1026,12 @@ public abstract class SqlImplementor {
           needNew = true;
         }
       }
+      if (rel instanceof LogicalAggregate
+          && !dialect.supportsNestedAggregations()
+          && hasNestedAggregations((LogicalAggregate) rel)) {
+        needNew = true;
+      }
+
       SqlSelect select;
       Expressions.FluentList<Clause> clauseList = Expressions.list();
       if (needNew) {
@@ -1066,6 +1075,29 @@ public abstract class SqlImplementor {
           needNew ? null : aliases);
     }
 
+    private boolean hasNestedAggregations(LogicalAggregate rel) {
+      List<AggregateCall> aggCallList = rel.getAggCallList();
+      HashSet<Integer> aggregatesArgs = new HashSet<>();
+      for (AggregateCall aggregateCall: aggCallList) {
+        aggregatesArgs.addAll(aggregateCall.getArgList());
+      }
+      for (Integer aggregatesArg : aggregatesArgs) {
+        SqlNode selectNode = ((SqlSelect) node).getSelectList().get(aggregatesArg);
+        if (!(selectNode instanceof SqlBasicCall)) {
+          continue;
+        }
+        for (SqlNode operand : ((SqlBasicCall) selectNode).getOperands()) {
+          if (operand instanceof SqlCall) {
+            final SqlOperator operator = ((SqlCall) operand).getOperator();
+            if (operator instanceof SqlAggFunction) {
+              return true;
+            }
+          }
+        }
+      }
+      return false;
+    }
+
     // make private?
     public Clause maxClause() {
       Clause maxClause = null;

http://git-wip-us.apache.org/repos/asf/calcite/blob/4d0e83e7/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlDialect.java b/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
index 6bc25ca..dafa8d9 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
@@ -554,6 +554,21 @@ public class SqlDialect {
     }
   }
 
+  /**
+   * Returns whether the dialect supports nested aggregations, for instance
+   * {@code SELECT SUM(SUM(1)) }.
+   */
+  public boolean supportsNestedAggregations() {
+    switch (databaseProduct) {
+    case MYSQL:
+    case VERTICA:
+    case POSTGRESQL:
+      return false;
+    default:
+      return true;
+    }
+  }
+
   /** Returns how NULL values are sorted if an ORDER BY item does not contain
    * NULLS ASCENDING or NULLS DESCENDING. */
   public NullCollation getNullCollation() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/4d0e83e7/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java b/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
index b72edfc..fb5d118 100644
--- a/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
@@ -190,6 +190,47 @@ public class RelToSqlConverterTest {
     sql(query).ok(expected);
   }
 
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1946">[CALCITE-1946]
+   * JDBC adapter should generate sub-SELECT if dialect does not support nested
+   * aggregate functions</a>. */
+  @Test public void testNestedAggregates() {
+    // PostgreSQL, MySQL, Vertica do not support nested aggregate functions, so
+    // for these, the JDBC adapter generates a SELECT in the FROM clause.
+    // Oracle can do it in a single SELECT.
+    final String query = "select\n"
+        + "    SUM(\"net_weight1\") as \"net_weight_converted\"\n"
+        + "  from ("
+        + "    select\n"
+        + "       SUM(\"net_weight\") as \"net_weight1\"\n"
+        + "    from \"foodmart\".\"product\"\n"
+        + "    group by \"product_id\")";
+    final String expectedOracle = "SELECT SUM(SUM(\"net_weight\")) \"net_weight_converted\"\n"
+        + "FROM \"foodmart\".\"product\"\n"
+        + "GROUP BY \"product_id\"";
+    final String expectedMySQL = "SELECT SUM(`net_weight1`) AS `net_weight_converted`\n"
+        + "FROM (SELECT SUM(`net_weight`) AS `net_weight1`\n"
+        + "FROM `foodmart`.`product`\n"
+        + "GROUP BY `product_id`) AS `t1`";
+    final String expectedVertica = "SELECT SUM(\"net_weight1\") AS \"net_weight_converted\"\n"
+        + "FROM (SELECT SUM(\"net_weight\") AS \"net_weight1\"\n"
+        + "FROM \"foodmart\".\"product\"\n"
+        + "GROUP BY \"product_id\") AS \"t1\"";
+    final String expectedPostgresql = "SELECT SUM(\"net_weight1\") AS \"net_weight_converted\"\n"
+        + "FROM (SELECT SUM(\"net_weight\") AS \"net_weight1\"\n"
+        + "FROM \"foodmart\".\"product\"\n"
+        + "GROUP BY \"product_id\") AS \"t1\"";
+    sql(query)
+        .dialect(DatabaseProduct.ORACLE.getDialect())
+        .ok(expectedOracle)
+        .dialect(DatabaseProduct.MYSQL.getDialect())
+        .ok(expectedMySQL)
+        .dialect(DatabaseProduct.VERTICA.getDialect())
+        .ok(expectedVertica)
+        .dialect(DatabaseProduct.POSTGRESQL.getDialect())
+        .ok(expectedPostgresql);
+  }
+
   @Test public void testSelectQueryWithGroupByAndProjectList1() {
     String query =
         "select count(*)  from \"product\" group by \"product_class_id\", \"product_id\"";


[07/15] calcite git commit: [CALCITE-1980] RelBuilder.aggregate should rename underlying fields if groupKey contains alias

Posted by jh...@apache.org.
[CALCITE-1980] RelBuilder.aggregate should rename underlying fields if groupKey contains alias

Test case by Pavel Gubin, in the following PR; did not use the rest of
the PR.

Close apache/calcite#535


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

Branch: refs/heads/master
Commit: 2773c4846a67360de8301680e375779ce3b1304b
Parents: 43fa8e9
Author: Julian Hyde <jh...@apache.org>
Authored: Wed Sep 13 12:00:35 2017 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Mon Oct 2 11:13:42 2017 -0700

----------------------------------------------------------------------
 .../org/apache/calcite/tools/RelBuilder.java    | 92 ++++++++++++--------
 .../org/apache/calcite/test/RelBuilderTest.java | 44 ++++++++++
 2 files changed, 102 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/2773c484/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/tools/RelBuilder.java b/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
index fe822f0..0a726c7 100644
--- a/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
+++ b/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
@@ -1049,10 +1049,12 @@ public class RelBuilder {
   /** Creates an {@link org.apache.calcite.rel.core.Aggregate} with a list of
    * calls. */
   public RelBuilder aggregate(GroupKey groupKey, Iterable<AggCall> aggCalls) {
-    final List<RexNode> extraNodes = new ArrayList<>(fields());
+    final Registrar registrar = new Registrar();
+    registrar.extraNodes.addAll(fields());
+    registrar.names.addAll(peek().getRowType().getFieldNames());
     final GroupKeyImpl groupKey_ = (GroupKeyImpl) groupKey;
     final ImmutableBitSet groupSet =
-        ImmutableBitSet.of(registerExpressions(extraNodes, groupKey_.nodes));
+        ImmutableBitSet.of(registrar.registerExpressions(groupKey_.nodes));
   label:
     if (Iterables.isEmpty(aggCalls) && !groupKey_.indicator) {
       final RelMetadataQuery mq = peek().getCluster().getMetadataQuery();
@@ -1064,10 +1066,12 @@ public class RelBuilder {
           break label;
         }
       }
-      final Boolean unique = mq.areColumnsUnique(peek(), groupSet);
-      if (unique != null && unique) {
-        // Rel is already unique. Nothing to do.
-        return this;
+      if (registrar.extraNodes.size() == fields().size()) {
+        final Boolean unique = mq.areColumnsUnique(peek(), groupSet);
+        if (unique != null && unique) {
+          // Rel is already unique. Nothing to do.
+          return this;
+        }
       }
       final Double maxRowCount = mq.getMaxRowCount(peek());
       if (maxRowCount != null && maxRowCount <= 1D) {
@@ -1077,12 +1081,12 @@ public class RelBuilder {
     }
     final ImmutableList<ImmutableBitSet> groupSets;
     if (groupKey_.nodeLists != null) {
-      final int sizeBefore = extraNodes.size();
+      final int sizeBefore = registrar.extraNodes.size();
       final SortedSet<ImmutableBitSet> groupSetSet =
           new TreeSet<>(ImmutableBitSet.ORDERING);
       for (ImmutableList<RexNode> nodeList : groupKey_.nodeLists) {
         final ImmutableBitSet groupSet2 =
-            ImmutableBitSet.of(registerExpressions(extraNodes, nodeList));
+            ImmutableBitSet.of(registrar.registerExpressions(nodeList));
         if (!groupSet.contains(groupSet2)) {
           throw new IllegalArgumentException("group set element " + nodeList
               + " must be a subset of group key");
@@ -1090,10 +1094,11 @@ public class RelBuilder {
         groupSetSet.add(groupSet2);
       }
       groupSets = ImmutableList.copyOf(groupSetSet);
-      if (extraNodes.size() > sizeBefore) {
+      if (registrar.extraNodes.size() > sizeBefore) {
         throw new IllegalArgumentException(
             "group sets contained expressions not in group key: "
-                + extraNodes.subList(sizeBefore, extraNodes.size()));
+                + registrar.extraNodes.subList(sizeBefore,
+                registrar.extraNodes.size()));
       }
     } else {
       groupSets = ImmutableList.of(groupSet);
@@ -1101,13 +1106,14 @@ public class RelBuilder {
     for (AggCall aggCall : aggCalls) {
       if (aggCall instanceof AggCallImpl) {
         final AggCallImpl aggCall1 = (AggCallImpl) aggCall;
-        registerExpressions(extraNodes, aggCall1.operands);
+        registrar.registerExpressions(aggCall1.operands);
         if (aggCall1.filter != null) {
-          registerExpression(extraNodes, aggCall1.filter);
+          registrar.registerExpression(aggCall1.filter);
         }
       }
     }
-    project(extraNodes);
+    project(registrar.extraNodes);
+    rename(registrar.names);
     final Frame frame = stack.pop();
     final RelNode r = frame.rel;
     final List<AggregateCall> aggregateCalls = new ArrayList<>();
@@ -1115,9 +1121,10 @@ public class RelBuilder {
       final AggregateCall aggregateCall;
       if (aggCall instanceof AggCallImpl) {
         final AggCallImpl aggCall1 = (AggCallImpl) aggCall;
-        final List<Integer> args = registerExpressions(extraNodes, aggCall1.operands);
+        final List<Integer> args =
+            registrar.registerExpressions(aggCall1.operands);
         final int filterArg = aggCall1.filter == null ? -1
-            : registerExpression(extraNodes, aggCall1.filter);
+            : registrar.registerExpression(aggCall1.filter);
         if (aggCall1.distinct && !aggCall1.aggFunction.isQuantifierAllowed()) {
           throw new IllegalArgumentException("DISTINCT not allowed");
         }
@@ -1147,7 +1154,7 @@ public class RelBuilder {
     int i = 0;
     // first, group fields
     for (Integer groupField : groupSet.asList()) {
-      RexNode node = extraNodes.get(groupField);
+      RexNode node = registrar.extraNodes.get(groupField);
       final SqlKind kind = node.getKind();
       switch (kind) {
       case INPUT_REF:
@@ -1184,24 +1191,6 @@ public class RelBuilder {
     return this;
   }
 
-  private static int registerExpression(List<RexNode> exprList, RexNode node) {
-    int i = exprList.indexOf(node);
-    if (i < 0) {
-      i = exprList.size();
-      exprList.add(node);
-    }
-    return i;
-  }
-
-  private static List<Integer> registerExpressions(List<RexNode> extraNodes,
-      Iterable<? extends RexNode> nodes) {
-    final List<Integer> builder = new ArrayList<>();
-    for (RexNode node : nodes) {
-      builder.add(registerExpression(extraNodes, node));
-    }
-    return builder;
-  }
-
   private RelBuilder setOp(boolean all, SqlKind kind, int n) {
     List<RelNode> inputs = new LinkedList<>();
     for (int i = 0; i < n; i++) {
@@ -1800,6 +1789,41 @@ public class RelBuilder {
     }
   }
 
+  /** Collects the extra expressions needed for {@link #aggregate}.
+   *
+   * <p>The extra expressions come from the group key and as arguments to
+   * aggregate calls, and later there will be a {@link #project} or a
+   * {@link #rename(List)} if necessary. */
+  private static class Registrar {
+    final List<RexNode> extraNodes = new ArrayList<>();
+    final List<String> names = new ArrayList<>();
+
+    int registerExpression(RexNode node) {
+      switch (node.getKind()) {
+      case AS:
+        final List<RexNode> operands = ((RexCall) node).operands;
+        int i = registerExpression(operands.get(0));
+        names.set(i, RexLiteral.stringValue(operands.get(1)));
+        return i;
+      }
+      int i = extraNodes.indexOf(node);
+      if (i < 0) {
+        i = extraNodes.size();
+        extraNodes.add(node);
+        names.add(null);
+      }
+      return i;
+    }
+
+    List<Integer> registerExpressions(Iterable<? extends RexNode> nodes) {
+      final List<Integer> builder = new ArrayList<>();
+      for (RexNode node : nodes) {
+        builder.add(registerExpression(node));
+      }
+      return builder;
+    }
+  }
+
   /** Builder stack frame.
    *
    * <p>Describes a previously created relational expression and

http://git-wip-us.apache.org/repos/asf/calcite/blob/2773c484/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java b/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
index 8a337cc..0f66db3 100644
--- a/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
@@ -702,6 +702,50 @@ public class RelBuilderTest {
     assertThat(str(root), is(expected));
   }
 
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1980">[CALCITE-1980]
+   * RelBuilder gives NPE if groupKey contains alias</a>.
+   *
+   * <p>Now, the alias does not cause a new expression to be added to the input,
+   * but causes the referenced fields to be renamed. */
+  @Test public void testAggregateProjectWithAliases() {
+    final RelBuilder builder = RelBuilder.create(config().build());
+    RelNode root =
+        builder.scan("EMP")
+            .project(builder.field("DEPTNO"))
+            .aggregate(
+                builder.groupKey(
+                    builder.alias(builder.field("DEPTNO"), "departmentNo")))
+            .build();
+    final String expected = ""
+        + "LogicalAggregate(group=[{0}])\n"
+        + "  LogicalProject(departmentNo=[$0])\n"
+        + "    LogicalProject(DEPTNO=[$7])\n"
+        + "      LogicalTableScan(table=[[scott, EMP]])\n";
+    assertThat(str(root), is(expected));
+  }
+
+  @Test public void testAggregateProjectWithExpression() {
+    final RelBuilder builder = RelBuilder.create(config().build());
+    RelNode root =
+        builder.scan("EMP")
+            .project(builder.field("DEPTNO"))
+            .aggregate(
+                builder.groupKey(
+                    builder.alias(
+                        builder.call(SqlStdOperatorTable.PLUS,
+                            builder.field("DEPTNO"), builder.literal(3)),
+                        "d3")))
+            .build();
+    final String expected = ""
+        + "LogicalAggregate(group=[{1}])\n"
+        + "  LogicalProject(DEPTNO=[$0], d3=[$1])\n"
+        + "    LogicalProject(DEPTNO=[$0], $f1=[+($0, 3)])\n"
+        + "      LogicalProject(DEPTNO=[$7])\n"
+        + "        LogicalTableScan(table=[[scott, EMP]])\n";
+    assertThat(str(root), is(expected));
+  }
+
   @Test public void testAggregateGroupingKeyOutOfRangeFails() {
     final RelBuilder builder = RelBuilder.create(config().build());
     try {