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:15 UTC

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

[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