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/07/27 17:28:26 UTC

[1/3] calcite git commit: [CALCITE-1899] When reading JSON model, give error if mandatory JSON attributes are missing

Repository: calcite
Updated Branches:
  refs/heads/master 0cbd2a182 -> bdaa33f9c


[CALCITE-1899] When reading JSON model, give error if mandatory JSON attributes are missing


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

Branch: refs/heads/master
Commit: b516216b6a51fb17b9cebb47e5b84c31207bab8f
Parents: ee0afb6
Author: Julian Hyde <jh...@apache.org>
Authored: Sat Jul 22 20:34:25 2017 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Mon Jul 24 21:39:43 2017 -0700

----------------------------------------------------------------------
 .../org/apache/calcite/model/JsonColumn.java    |  4 ++
 .../org/apache/calcite/model/ModelHandler.java  | 44 ++++++++++++++++++--
 .../java/org/apache/calcite/test/ModelTest.java | 31 ++++++++++++++
 3 files changed, 75 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/b516216b/core/src/main/java/org/apache/calcite/model/JsonColumn.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/model/JsonColumn.java b/core/src/main/java/org/apache/calcite/model/JsonColumn.java
index c57fd29..b337ef3 100644
--- a/core/src/main/java/org/apache/calcite/model/JsonColumn.java
+++ b/core/src/main/java/org/apache/calcite/model/JsonColumn.java
@@ -29,6 +29,10 @@ public class JsonColumn {
    * <p>Required, and must be unique within the table.
    */
   public String name;
+
+  public void accept(ModelHandler handler) {
+    handler.visit(this);
+  }
 }
 
 // End JsonColumn.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/b516216b/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 adfdd07..ee6373e 100644
--- a/core/src/main/java/org/apache/calcite/model/ModelHandler.java
+++ b/core/src/main/java/org/apache/calcite/model/ModelHandler.java
@@ -48,6 +48,7 @@ import com.google.common.collect.ImmutableMap;
 
 import java.io.File;
 import java.io.IOException;
+import java.lang.reflect.Field;
 import java.sql.SQLException;
 import java.util.ArrayDeque;
 import java.util.Collections;
@@ -135,18 +136,36 @@ public class ModelHandler {
         + "'initAdd', 'merge' and 'result' methods.");
   }
 
-  public void visit(JsonRoot root) {
+  private void checkRequiredAttributes(Object json, String... attributeNames) {
+    for (String attributeName : attributeNames) {
+      try {
+        final Class<?> c = json.getClass();
+        final Field f = c.getField(attributeName);
+        final Object o = f.get(json);
+        if (o == null) {
+          throw new RuntimeException("Field '" + attributeName
+              + "' is required in " + c.getSimpleName());
+        }
+      } catch (NoSuchFieldException | IllegalAccessException e) {
+        throw new RuntimeException("while accessing field " + attributeName,
+            e);
+      }
+    }
+  }
+
+  public void visit(JsonRoot jsonRoot) {
+    checkRequiredAttributes(jsonRoot, "version");
     final Pair<String, SchemaPlus> pair =
         Pair.of(null, connection.getRootSchema());
     schemaStack.push(pair);
-    for (JsonSchema schema : root.schemas) {
+    for (JsonSchema schema : jsonRoot.schemas) {
       schema.accept(this);
     }
     final Pair<String, SchemaPlus> p = schemaStack.pop();
     assert p == pair;
-    if (root.defaultSchema != null) {
+    if (jsonRoot.defaultSchema != null) {
       try {
-        connection.setSchema(root.defaultSchema);
+        connection.setSchema(jsonRoot.defaultSchema);
       } catch (SQLException e) {
         throw new RuntimeException(e);
       }
@@ -154,6 +173,7 @@ public class ModelHandler {
   }
 
   public void visit(JsonMapSchema jsonSchema) {
+    checkRequiredAttributes(jsonSchema, "name");
     final SchemaPlus parentSchema = currentMutableSchema("schema");
     final SchemaPlus schema =
         parentSchema.add(jsonSchema.name, new AbstractSchema());
@@ -208,6 +228,7 @@ public class ModelHandler {
   public void visit(JsonCustomSchema jsonSchema) {
     try {
       final SchemaPlus parentSchema = currentMutableSchema("sub-schema");
+      checkRequiredAttributes(jsonSchema, "name", "factory");
       final SchemaFactory schemaFactory =
           AvaticaUtils.instantiatePlugin(SchemaFactory.class,
               jsonSchema.factory);
@@ -259,6 +280,7 @@ public class ModelHandler {
   }
 
   public void visit(JsonJdbcSchema jsonSchema) {
+    checkRequiredAttributes(jsonSchema, "name");
     final SchemaPlus parentSchema = currentMutableSchema("jdbc schema");
     final DataSource dataSource =
         JdbcSchema.dataSource(jsonSchema.jdbcUrl,
@@ -274,6 +296,7 @@ public class ModelHandler {
 
   public void visit(JsonMaterialization jsonMaterialization) {
     try {
+      checkRequiredAttributes(jsonMaterialization, "sql");
       final SchemaPlus schema = currentSchema();
       if (!schema.isMutable()) {
         throw new RuntimeException(
@@ -307,6 +330,7 @@ public class ModelHandler {
 
   public void visit(JsonLattice jsonLattice) {
     try {
+      checkRequiredAttributes(jsonLattice, "name", "sql");
       final SchemaPlus schema = currentSchema();
       if (!schema.isMutable()) {
         throw new RuntimeException("Cannot define lattice; parent schema '"
@@ -347,6 +371,7 @@ public class ModelHandler {
 
   public void visit(JsonCustomTable jsonTable) {
     try {
+      checkRequiredAttributes(jsonTable, "name", "factory");
       final SchemaPlus schema = currentMutableSchema("table");
       final TableFactory tableFactory =
           AvaticaUtils.instantiatePlugin(TableFactory.class,
@@ -354,14 +379,22 @@ public class ModelHandler {
       final Table table =
           tableFactory.create(schema, jsonTable.name,
               operandMap(null, jsonTable.operand), null);
+      for (JsonColumn column : jsonTable.columns) {
+        column.accept(this);
+      }
       schema.add(jsonTable.name, table);
     } catch (Exception e) {
       throw new RuntimeException("Error instantiating " + jsonTable, e);
     }
   }
 
+  public void visit(JsonColumn jsonColumn) {
+    checkRequiredAttributes(jsonColumn, "name");
+  }
+
   public void visit(JsonView jsonView) {
     try {
+      checkRequiredAttributes(jsonView, "name");
       final SchemaPlus schema = currentMutableSchema("view");
       final List<String> path = Util.first(jsonView.path, currentSchemaPath());
       final List<String> viewPath = ImmutableList.<String>builder().addAll(path)
@@ -396,6 +429,8 @@ public class ModelHandler {
   }
 
   public void visit(JsonFunction jsonFunction) {
+    // "name" is not required - a class can have several functions
+    checkRequiredAttributes(jsonFunction, "className");
     try {
       final SchemaPlus schema = currentMutableSchema("function");
       final List<String> path =
@@ -411,6 +446,7 @@ public class ModelHandler {
   }
 
   public void visit(JsonMeasure jsonMeasure) {
+    checkRequiredAttributes(jsonMeasure, "agg");
     assert latticeBuilder != null;
     final Lattice.Measure measure =
         latticeBuilder.resolveMeasure(jsonMeasure.agg, jsonMeasure.args);

http://git-wip-us.apache.org/repos/asf/calcite/blob/b516216b/core/src/test/java/org/apache/calcite/test/ModelTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/ModelTest.java b/core/src/test/java/org/apache/calcite/test/ModelTest.java
index 989ddf4..35e78fb 100644
--- a/core/src/test/java/org/apache/calcite/test/ModelTest.java
+++ b/core/src/test/java/org/apache/calcite/test/ModelTest.java
@@ -198,6 +198,37 @@ public class ModelTest {
             + "is not a SemiMutableSchema");
   }
 
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1899">[CALCITE-1899]
+   * When reading model, give error if mandatory JSON attributes are
+   * missing</a>.
+   *
+   * <p>Schema without name should give useful error, not
+   * NullPointerException. */
+  @Test public void testSchemaWithoutName() throws Exception {
+    final String model = "{\n"
+        + "  version: '1.0',\n"
+        + "  defaultSchema: 'adhoc',\n"
+        + "  schemas: [ {\n"
+        + "  } ]\n"
+        + "}";
+    CalciteAssert.model(model)
+        .connectThrows("Field 'name' is required in JsonMapSchema");
+  }
+
+  @Test public void testCustomSchemaWithoutFactory() throws Exception {
+    final String model = "{\n"
+        + "  version: '1.0',\n"
+        + "  defaultSchema: 'adhoc',\n"
+        + "  schemas: [ {\n"
+        + "    type: 'custom',\n"
+        + "    name: 'my_custom_schema'\n"
+        + "  } ]\n"
+        + "}";
+    CalciteAssert.model(model)
+        .connectThrows("Field 'factory' is required in JsonCustomSchema");
+  }
+
   /** Tests a model containing a lattice and some views. */
   @Test public void testReadLattice() throws IOException {
     final ObjectMapper mapper = mapper();


[2/3] calcite git commit: [CALCITE-1898] LIKE must match '.' (period) literally

Posted by jh...@apache.org.
[CALCITE-1898] LIKE must match '.' (period) literally


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

Branch: refs/heads/master
Commit: ee0afb6e38ce09f82cc1883aae757306b55ce8fe
Parents: 0cbd2a1
Author: Julian Hyde <jh...@apache.org>
Authored: Sat Jul 22 17:25:47 2017 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Mon Jul 24 21:39:43 2017 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/calcite/runtime/Like.java |  2 +-
 .../calcite/sql/test/SqlOperatorBaseTest.java      | 17 +++++++++++++++++
 2 files changed, 18 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/ee0afb6e/core/src/main/java/org/apache/calcite/runtime/Like.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/Like.java b/core/src/main/java/org/apache/calcite/runtime/Like.java
index 74b2c82..bdb6785 100644
--- a/core/src/main/java/org/apache/calcite/runtime/Like.java
+++ b/core/src/main/java/org/apache/calcite/runtime/Like.java
@@ -21,7 +21,7 @@ package org.apache.calcite.runtime;
  * to regular expressions.
  */
 public class Like {
-  private static final String JAVA_REGEX_SPECIALS = "[]()|^-+*?{}$\\";
+  private static final String JAVA_REGEX_SPECIALS = "[]()|^-+*?{}$\\.";
   private static final String SQL_SIMILAR_SPECIALS = "[]()|^-+*_%?{}";
   private static final String [] REG_CHAR_CLASSES = {
     "[:ALPHA:]", "\\p{Alpha}",

http://git-wip-us.apache.org/repos/asf/calcite/blob/ee0afb6e/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 b1d7cde..7925bf6 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
@@ -3417,6 +3417,15 @@ public abstract class SqlOperatorBaseTest {
     tester.checkBoolean("'ab\ncd\nef' like '%cde%'", Boolean.FALSE);
   }
 
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1898">[CALCITE-1898]
+   * LIKE must match '.' (period) literally</a>. */
+  @Test public void testLikeDot() {
+    tester.checkBoolean("'abc' like 'a.c'", Boolean.FALSE);
+    tester.checkBoolean("'abcde' like '%c.e'", Boolean.FALSE);
+    tester.checkBoolean("'abc.e' like '%c.e'", Boolean.TRUE);
+  }
+
   @Test public void testNotSimilarToOperator() {
     tester.setFor(SqlStdOperatorTable.NOT_SIMILAR_TO, VM_EXPAND);
     tester.checkBoolean("'ab' not similar to 'a_'", false);
@@ -3594,6 +3603,14 @@ public abstract class SqlOperatorBaseTest {
     tester.checkBoolean("'y' similar to 'x?+y'", Boolean.TRUE);
     tester.checkBoolean("'y' similar to 'x*+y'", Boolean.TRUE);
 
+    // dot is a wildcard for SIMILAR TO but not LIKE
+    tester.checkBoolean("'abc' similar to 'a.c'", Boolean.TRUE);
+    tester.checkBoolean("'a.c' similar to 'a.c'", Boolean.TRUE);
+    tester.checkBoolean("'abcd' similar to 'a.*d'", Boolean.TRUE);
+    tester.checkBoolean("'abc' like 'a.c'", Boolean.FALSE);
+    tester.checkBoolean("'a.c' like 'a.c'", Boolean.TRUE);
+    tester.checkBoolean("'abcd' like 'a.*d'", Boolean.FALSE);
+
     // The following two tests throws exception(They probably should).
     // "Dangling meta character '*' near index 2"
 


[3/3] calcite git commit: [CALCITE-1841] Create handlers for JDBC dialect-specific generated SQL (Chris Baynes)

Posted by jh...@apache.org.
[CALCITE-1841] Create handlers for JDBC dialect-specific generated SQL (Chris Baynes)

Add Oracle SUBSTR function.

Close apache/calcite#501


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

Branch: refs/heads/master
Commit: bdaa33f9cac508c3802ad0fcced34f1bb37a63c0
Parents: b516216
Author: Chris Baynes <bi...@gmail.com>
Authored: Fri Jul 14 16:30:10 2017 +0200
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Jul 26 16:33:04 2017 -0700

----------------------------------------------------------------------
 .../calcite/rel/rel2sql/SqlImplementor.java     |  13 --
 .../java/org/apache/calcite/sql/SqlCall.java    |   5 +-
 .../java/org/apache/calcite/sql/SqlDialect.java |  96 ++++++--
 .../calcite/sql/dialect/HsqldbHandler.java      |  82 +++++++
 .../calcite/sql/dialect/MssqlHandler.java       | 111 +++++++++
 .../calcite/sql/dialect/MysqlHandler.java       | 108 +++++++++
 .../calcite/sql/dialect/OracleHandler.java      |  67 ++++++
 .../calcite/sql/dialect/PostgresqlHandler.java  |  59 +++++
 .../calcite/sql/dialect/package-info.java       |  26 +++
 .../calcite/sql/fun/OracleSqlOperatorTable.java |   9 +
 .../calcite/sql/fun/SqlFloorFunction.java       | 226 ++-----------------
 .../rel/rel2sql/RelToSqlConverterTest.java      |  44 ++++
 12 files changed, 603 insertions(+), 243 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/bdaa33f9/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 3851a27..5ff48a2 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
@@ -105,13 +105,6 @@ public abstract class SqlImplementor {
 
   public static final SqlParserPos POS = SqlParserPos.ZERO;
 
-  /** Oracle's {@code SUBSTR} function.
-   * Oracle does not support {@link SqlStdOperatorTable#SUBSTRING}. */
-  public static final SqlFunction ORACLE_SUBSTR =
-      new SqlFunction("SUBSTR", SqlKind.OTHER_FUNCTION,
-          ReturnTypes.ARG0_NULLABLE_VARYING, null, null,
-          SqlFunctionCategory.STRING);
-
   /** MySQL specific function. */
   public static final SqlFunction ISNULL_FUNCTION =
       new SqlFunction("ISNULL", SqlKind.OTHER_FUNCTION,
@@ -631,12 +624,6 @@ public abstract class SqlImplementor {
           // SqlCall requires exactly 2. So, convert to a left-deep binary tree.
           return createLeftCall(op, nodeList);
         }
-        if (op == SqlStdOperatorTable.SUBSTRING) {
-          switch (dialect.getDatabaseProduct()) {
-          case ORACLE:
-            return ORACLE_SUBSTR.createCall(new SqlNodeList(nodeList, POS));
-          }
-        }
         return op.createCall(new SqlNodeList(nodeList, POS));
       }
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/bdaa33f9/core/src/main/java/org/apache/calcite/sql/SqlCall.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlCall.java b/core/src/main/java/org/apache/calcite/sql/SqlCall.java
index 18a2160..9e50417 100755
--- a/core/src/main/java/org/apache/calcite/sql/SqlCall.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlCall.java
@@ -92,14 +92,15 @@ public abstract class SqlCall extends SqlNode {
       int leftPrec,
       int rightPrec) {
     final SqlOperator operator = getOperator();
+    final SqlDialect dialect = writer.getDialect();
     if (leftPrec > operator.getLeftPrec()
         || (operator.getRightPrec() <= rightPrec && (rightPrec != 0))
         || writer.isAlwaysUseParentheses() && isA(SqlKind.EXPRESSION)) {
       final SqlWriter.Frame frame = writer.startList("(", ")");
-      operator.unparse(writer, this, 0, 0);
+      dialect.unparseCall(writer, this, 0, 0);
       writer.endList(frame);
     } else {
-      operator.unparse(writer, this, leftPrec, rightPrec);
+      dialect.unparseCall(writer, this, leftPrec, rightPrec);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/bdaa33f9/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 27691fb..9eab2b7 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
@@ -19,6 +19,11 @@ package org.apache.calcite.sql;
 import org.apache.calcite.avatica.util.DateTimeUtils;
 import org.apache.calcite.config.NullCollation;
 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 com.google.common.base.Preconditions;
 
@@ -39,6 +44,8 @@ import java.util.regex.Pattern;
 public class SqlDialect {
   //~ Static fields/initializers ---------------------------------------------
 
+  private static final Handler DEFAULT_HANDLER = new BaseHandler();
+
   /**
    * 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
@@ -63,6 +70,7 @@ public class SqlDialect {
   private final String identifierEscapedQuote;
   private final DatabaseProduct databaseProduct;
   private final NullCollation nullCollation;
+  private final Handler handler;
 
   //~ Constructors -----------------------------------------------------------
 
@@ -113,26 +121,23 @@ public class SqlDialect {
     } catch (SQLException e) {
       throw new IllegalArgumentException("cannot deduce null collation", e);
     }
+    Handler handler = chooseHandler(databaseProduct);
     return new SqlDialect(databaseProduct, databaseProductName,
-        identifierQuoteString, nullCollation);
+        identifierQuoteString, nullCollation, handler);
   }
 
-  /**
-   * Creates a SqlDialect.
-   *
-   * @param databaseProduct       Database product; may be UNKNOWN, never null
-   * @param databaseProductName   Database product name from JDBC driver
-   * @param identifierQuoteString String to quote identifiers. Null if quoting
-   *                              is not supported. If "[", close quote is
-   *                              deemed to be "]".
-   */
   @Deprecated // to be removed before 2.0
-  public SqlDialect(
-      DatabaseProduct databaseProduct,
-      String databaseProductName,
+  public SqlDialect(DatabaseProduct databaseProduct, String databaseProductName,
       String identifierQuoteString) {
     this(databaseProduct, databaseProductName, identifierQuoteString,
-        NullCollation.HIGH);
+        NullCollation.HIGH, DEFAULT_HANDLER);
+  }
+
+  @Deprecated // to be removed before 2.0
+  public SqlDialect(DatabaseProduct databaseProduct, String databaseProductName,
+      String identifierQuoteString, NullCollation nullCollation) {
+    this(databaseProduct, databaseProductName, identifierQuoteString,
+        nullCollation, DEFAULT_HANDLER);
   }
 
   /**
@@ -144,14 +149,15 @@ 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
    */
-  public SqlDialect(
-      DatabaseProduct databaseProduct,
-      String databaseProductName,
-      String identifierQuoteString, NullCollation nullCollation) {
-    Preconditions.checkNotNull(this.nullCollation = nullCollation);
+  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);
     if (identifierQuoteString != null) {
       identifierQuoteString = identifierQuoteString.trim();
       if (identifierQuoteString.equals("")) {
@@ -244,7 +250,23 @@ public class SqlDialect {
     }
   }
 
-  // -- detect various databases --
+  /** 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
@@ -339,6 +361,11 @@ public class SqlDialect {
     }
   }
 
+  public void unparseCall(SqlWriter writer, SqlCall call, int leftPrec,
+      int rightPrec) {
+    handler.unparseCall(writer, call, leftPrec, rightPrec);
+  }
+
   /**
    * Returns whether the string contains any characters outside the
    * comfortable 7-bit ASCII range (32 through 127).
@@ -656,9 +683,10 @@ public class SqlDialect {
 
     DatabaseProduct(String databaseProductName, String quoteString,
         NullCollation nullCollation) {
-      this.databaseProductName = databaseProductName;
+      this.databaseProductName =
+          Preconditions.checkNotNull(databaseProductName);
       this.quoteString = quoteString;
-      this.nullCollation = nullCollation;
+      this.nullCollation = Preconditions.checkNotNull(nullCollation);
     }
 
     /**
@@ -669,18 +697,38 @@ public class SqlDialect {
      * use a dialect created from an actual connection's metadata
      * (see {@link SqlDialect#create(java.sql.DatabaseMetaData)}).
      *
-     * @return Dialect representing lowest-common-demoninator behavior for
+     * @return Dialect representing lowest-common-denominator behavior for
      * all versions of this database
      */
     public SqlDialect getDialect() {
       if (dialect == null) {
+        final Handler handler = chooseHandler(this);
         dialect =
             new SqlDialect(this, databaseProductName, quoteString,
-                nullCollation);
+                nullCollation, handler);
       }
       return dialect;
     }
   }
+
+  /**
+   * A handler for converting {@link SqlNode} into SQL text of a particular
+   * dialect.
+   *
+   * <p>Instances are stateless and therefore immutable.
+   */
+  public interface Handler {
+    void unparseCall(SqlWriter writer, SqlCall call, int leftPrec,
+        int rightPrec);
+  }
+
+  /** Base class for dialect handlers. */
+  public static class BaseHandler implements Handler {
+    public void unparseCall(SqlWriter writer, SqlCall call,
+        int leftPrec, int rightPrec) {
+      call.getOperator().unparse(writer, call, leftPrec, rightPrec);
+    }
+  }
 }
 
 // End SqlDialect.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/bdaa33f9/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
new file mode 100644
index 0000000..62e54bf
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/HsqldbHandler.java
@@ -0,0 +1,82 @@
+/*
+ * 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/bdaa33f9/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
new file mode 100644
index 0000000..96f4a89
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/MssqlHandler.java
@@ -0,0 +1,111 @@
+/*
+ * 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 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();
+
+  @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 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 AssertionError("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/bdaa33f9/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
new file mode 100644
index 0000000..adf7061
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/MysqlHandler.java
@@ -0,0 +1,108 @@
+/*
+ * 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/bdaa33f9/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
new file mode 100644
index 0000000..a67eb01
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/OracleHandler.java
@@ -0,0 +1,67 @@
+/*
+ * 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/bdaa33f9/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
new file mode 100644
index 0000000..8b35346
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/PostgresqlHandler.java
@@ -0,0 +1,59 @@
+/*
+ * 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/bdaa33f9/core/src/main/java/org/apache/calcite/sql/dialect/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/package-info.java b/core/src/main/java/org/apache/calcite/sql/dialect/package-info.java
new file mode 100644
index 0000000..348e27f
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/package-info.java
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+/**
+ * SQL unparsers for JDBC dialects.
+ */
+@PackageMarker
+package org.apache.calcite.sql.dialect;
+
+import org.apache.calcite.avatica.util.PackageMarker;
+
+// End package-info.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/bdaa33f9/core/src/main/java/org/apache/calcite/sql/fun/OracleSqlOperatorTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/OracleSqlOperatorTable.java b/core/src/main/java/org/apache/calcite/sql/fun/OracleSqlOperatorTable.java
index e45a43c..a4a3175 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/OracleSqlOperatorTable.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/OracleSqlOperatorTable.java
@@ -88,6 +88,15 @@ public class OracleSqlOperatorTable extends ReflectiveSqlOperatorTable {
               SqlTypeTransforms.TO_VARYING), null,
           OperandTypes.STRING, SqlFunctionCategory.STRING);
 
+  /** Oracle's "SUBSTR(string, position [, substringLength ])" function.
+   *
+   * <p>It has similar semantics to standard SQL's
+   * {@link SqlStdOperatorTable#SUBSTRING} function but different syntax. */
+  public static final SqlFunction SUBSTR =
+      new SqlFunction("SUBSTR", SqlKind.OTHER_FUNCTION,
+          ReturnTypes.ARG0_NULLABLE_VARYING, null, null,
+          SqlFunctionCategory.STRING);
+
   /** The "GREATEST(value, value)" function. */
   public static final SqlFunction GREATEST =
       new SqlFunction("GREATEST", SqlKind.GREATEST,

http://git-wip-us.apache.org/repos/asf/calcite/blob/bdaa33f9/core/src/main/java/org/apache/calcite/sql/fun/SqlFloorFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlFloorFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlFloorFunction.java
index 5d751f7..d23876b 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlFloorFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlFloorFunction.java
@@ -16,12 +16,14 @@
  */
 package org.apache.calcite.sql.fun;
 
-import org.apache.calcite.avatica.util.TimeUnitRange;
 import org.apache.calcite.sql.SqlCall;
+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.SqlNode;
 import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.SqlUtil;
 import org.apache.calcite.sql.SqlWriter;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.OperandTypes;
@@ -58,62 +60,17 @@ public class SqlFloorFunction extends SqlMonotonicUnaryFunction {
 
   @Override public void unparse(SqlWriter writer, SqlCall call, int leftPrec,
       int rightPrec) {
+    final SqlWriter.Frame frame = writer.startFunCall(getName());
     if (call.operandCount() == 2) {
-      unparseDatetime(writer, call);
+      call.operand(0).unparse(writer, 0, 100);
+      writer.sep("TO");
+      call.operand(1).unparse(writer, 100, 0);
     } else {
-      unparseNumeric(writer, call);
+      call.operand(0).unparse(writer, 0, 0);
     }
-  }
-
-  private void unparseNumeric(SqlWriter writer, SqlCall call) {
-    final SqlWriter.Frame frame = writer.startFunCall(getName());
-    call.operand(0).unparse(writer, 0, 0);
     writer.endFunCall(frame);
   }
 
-  private void unparseDatetime(SqlWriter writer, SqlCall call) {
-    // FLOOR (not CEIL) is the only function that works in most dialects
-    if (kind != SqlKind.FLOOR) {
-      unparseDatetimeDefault(writer, call);
-      return;
-    }
-
-    switch (writer.getDialect().getDatabaseProduct()) {
-    case UNKNOWN:
-    case CALCITE:
-      unparseDatetimeDefault(writer, call);
-      return;
-    }
-
-    final SqlLiteral timeUnitNode = call.operand(1);
-    final TimeUnitRange timeUnit = timeUnitNode.getValueAs(TimeUnitRange.class);
-
-    final SqlCall call2;
-    switch (writer.getDialect().getDatabaseProduct()) {
-    case ORACLE:
-      call2 = replaceTimeUnitOperand(call, timeUnit.name(), timeUnitNode.getParserPosition());
-      unparseDatetimeFunction(writer, call2, "TRUNC", true);
-      break;
-    case HSQLDB:
-      final String translatedLit = convertToHsqlDb(timeUnit);
-      call2 = replaceTimeUnitOperand(call, translatedLit, timeUnitNode.getParserPosition());
-      unparseDatetimeFunction(writer, call2, "TRUNC", true);
-      break;
-    case POSTGRESQL:
-      call2 = replaceTimeUnitOperand(call, timeUnit.name(), timeUnitNode.getParserPosition());
-      unparseDatetimeFunction(writer, call2, "DATE_TRUNC", false);
-      break;
-    case MSSQL:
-      unparseDatetimeMssql(writer, call);
-      break;
-    case MYSQL:
-      unparseDatetimeMysql(writer, call);
-      break;
-    default:
-      unparseDatetimeDefault(writer, call);
-    }
-  }
-
   /**
    * Copies a {@link SqlCall}, replacing the time unit operand with the given
    * literal.
@@ -123,27 +80,13 @@ public class SqlFloorFunction extends SqlMonotonicUnaryFunction {
    * @param pos Parser position
    * @return Modified call
    */
-  private SqlCall replaceTimeUnitOperand(SqlCall call, String literal, SqlParserPos pos) {
+  public static SqlCall replaceTimeUnitOperand(SqlCall call, String literal, SqlParserPos pos) {
     SqlLiteral literalNode = SqlLiteral.createCharString(literal, null, pos);
     return call.getOperator().createCall(call.getFunctionQuantifier(), pos,
         call.getOperandList().get(0), literalNode);
   }
 
   /**
-   * Default datetime unparse method if the specific dialect was not matched.
-   *
-   * @param writer SqlWriter
-   * @param call SqlCall
-   */
-  private void unparseDatetimeDefault(SqlWriter writer, SqlCall call) {
-    final SqlWriter.Frame frame = writer.startFunCall(getName());
-    call.operand(0).unparse(writer, 0, 100);
-    writer.sep("TO");
-    call.operand(1).unparse(writer, 100, 0);
-    writer.endFunCall(frame);
-  }
-
-  /**
    * Most dialects that natively support datetime floor will use this.
    * In those cases the call will look like TRUNC(datetime, 'year').
    *
@@ -153,149 +96,24 @@ public class SqlFloorFunction extends SqlMonotonicUnaryFunction {
    * @param datetimeFirst Specify the order of the datetime &amp; timeUnit
    * arguments
    */
-  private void unparseDatetimeFunction(SqlWriter writer, SqlCall call,
+  public static void unparseDatetimeFunction(SqlWriter writer, SqlCall call,
       String funName, Boolean datetimeFirst) {
-    final SqlWriter.Frame frame = writer.startFunCall(funName);
-    int firstOpIndex = datetimeFirst ? 0 : 1;
-    int secondOpIndex = datetimeFirst ? 1 : 0;
-    call.operand(firstOpIndex).unparse(writer, 0, 0);
-    writer.sep(",", true);
-    call.operand(secondOpIndex).unparse(writer, 0, 0);
-    writer.endFunCall(frame);
-  }
-
-  /**
-   * Unparse datetime floor for MS SQL. There is no TRUNC function, so simulate this
-   * using calls to CONVERT.
-   *
-   * @param writer SqlWriter
-   * @param call SqlCall
-   */
-  private void unparseDatetimeMssql(SqlWriter writer, SqlCall call) {
-    SqlLiteral node = call.operand(1);
-    TimeUnitRange unit = (TimeUnitRange) node.getValue();
-
-    switch (unit) {
-    case YEAR:
-      unparseMssql(writer, call, 4, "-01-01");
-      break;
-    case MONTH:
-      unparseMssql(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:
-      unparseMssql(writer, call, 10, "");
-      break;
-    case HOUR:
-      unparseMssql(writer, call, 13, ":00:00");
-      break;
-    case MINUTE:
-      unparseMssql(writer, call, 16, ":00");
-      break;
-    case SECOND:
-      unparseMssql(writer, call, 19, ":00");
-      break;
-    default:
-      throw new AssertionError("MSSQL does not support FLOOR for time unit: "
-          + unit);
-    }
-  }
-
-  private void unparseMssql(SqlWriter writer, SqlCall call, Integer charLen, String offset) {
-    writer.print("CONVERT");
-    SqlWriter.Frame frame = writer.startList("(", ")");
-    writer.print("DATETIME, CONVERT(VARCHAR(" + charLen.toString() + "), ");
-    call.operand(0).unparse(writer, 0, 0);
-    writer.print(", 126)");
-
-    if (offset.length() > 0) {
-      writer.print("+'" + offset + "'");
-    }
-    writer.endList(frame);
-  }
+    SqlFunction func = new SqlFunction(funName, SqlKind.OTHER_FUNCTION,
+        ReturnTypes.ARG0_NULLABLE_VARYING, null, null,
+        SqlFunctionCategory.STRING);
 
-  private static String convertToHsqlDb(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 an HsqlDb equivalent: "
-        + unit);
-    }
-  }
-
-  /**
-   * Unparse datetime floor for MySQL. There is no TRUNC function, so simulate this
-   * using calls to DATE_FORMAT.
-   *
-   * @param writer SqlWriter
-   * @param call SqlCall
-   */
-  private void unparseDatetimeMysql(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;
-    }
+    SqlCall call1;
+    if (datetimeFirst) {
+      call1 = call;
+    } else {
+      // switch order of operands
+      SqlNode op1 = call.operand(0);
+      SqlNode op2 = call.operand(1);
 
-    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);
+      call1 = call.getOperator().createCall(call.getParserPosition(), op2, op1);
     }
 
-    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);
+    SqlUtil.unparseFunctionSyntax(func, writer, call1);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/bdaa33f9/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 d1a1b87..274f559 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
@@ -785,6 +785,50 @@ public class RelToSqlConverterTest {
         .ok(expectedMysql);
   }
 
+  @Test public void testSubstring() {
+    final String query = "select substring(\"brand_name\" from 2) "
+        + "from \"product\"\n";
+    final String expectedOracle = "SELECT SUBSTR(\"brand_name\", 2)\n"
+        + "FROM \"foodmart\".\"product\"";
+    final String expectedPostgresql = "SELECT SUBSTRING(\"brand_name\" FROM 2)\n"
+        + "FROM \"foodmart\".\"product\"";
+    final String expectedMysql = "SELECT SUBSTRING(`brand_name` FROM 2)\n"
+        + "FROM `foodmart`.`product`";
+    final String expectedMssql = "SELECT SUBSTRING([brand_name] FROM 2)\n"
+        + "FROM [foodmart].[product]";
+    sql(query)
+        .dialect(DatabaseProduct.ORACLE.getDialect())
+        .ok(expectedOracle)
+        .dialect(DatabaseProduct.POSTGRESQL.getDialect())
+        .ok(expectedPostgresql)
+        .dialect(DatabaseProduct.MYSQL.getDialect())
+        .ok(expectedMysql)
+        .dialect(DatabaseProduct.MSSQL.getDialect())
+        .ok(expectedMssql);
+  }
+
+  @Test public void testSubstringWithFor() {
+    final String query = "select substring(\"brand_name\" from 2 for 3) "
+        + "from \"product\"\n";
+    final String expectedOracle = "SELECT SUBSTR(\"brand_name\", 2, 3)\n"
+        + "FROM \"foodmart\".\"product\"";
+    final String expectedPostgresql = "SELECT SUBSTRING(\"brand_name\" FROM 2 FOR 3)\n"
+        + "FROM \"foodmart\".\"product\"";
+    final String expectedMysql = "SELECT SUBSTRING(`brand_name` FROM 2 FOR 3)\n"
+        + "FROM `foodmart`.`product`";
+    final String expectedMssql = "SELECT SUBSTRING([brand_name] FROM 2 FOR 3)\n"
+        + "FROM [foodmart].[product]";
+    sql(query)
+        .dialect(DatabaseProduct.ORACLE.getDialect())
+        .ok(expectedOracle)
+        .dialect(DatabaseProduct.POSTGRESQL.getDialect())
+        .ok(expectedPostgresql)
+        .dialect(DatabaseProduct.MYSQL.getDialect())
+        .ok(expectedMysql)
+        .dialect(DatabaseProduct.MSSQL.getDialect())
+        .ok(expectedMssql);
+  }
+
   @Test public void testMatchRecognizePatternExpression() {
     String sql = "select *\n"
         + "  from \"product\" match_recognize\n"