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/05/31 17:54:30 UTC

[1/5] calcite git commit: [CALCITE-1807] Upgrade to Avatica 1.10.0

Repository: calcite
Updated Branches:
  refs/heads/master c8462e1ac -> f22a9965a


[CALCITE-1807] Upgrade to Avatica 1.10.0

Includes fix to [CALCITE-1664] CAST('<string>' as TIMESTAMP) adds part of sub-second fraction to the value

Replace uses of DateTimeUtils.GMT_Zone with UTC_ZONE.


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

Branch: refs/heads/master
Commit: 915c218b0fcbff26652071d267903d97875c7b9e
Parents: c8462e1
Author: Julian Hyde <jh...@apache.org>
Authored: Tue May 30 11:18:38 2017 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed May 31 08:17:27 2017 -0700

----------------------------------------------------------------------
 core/src/main/codegen/templates/Parser.jj                   | 6 +++---
 core/src/main/java/org/apache/calcite/rex/RexLiteral.java   | 4 ++--
 core/src/main/java/org/apache/calcite/sql/SqlDialect.java   | 4 ++--
 core/src/main/java/org/apache/calcite/util/Util.java        | 2 +-
 .../java/org/apache/calcite/sql/parser/SqlParserTest.java   | 9 ++++++---
 .../test/java/org/apache/calcite/test/CalciteAssert.java    | 2 +-
 core/src/test/java/org/apache/calcite/test/JdbcTest.java    | 2 +-
 .../test/java/org/apache/calcite/test/SqlLimitsTest.java    | 2 +-
 core/src/test/resources/sql/misc.iq                         | 4 +---
 .../apache/calcite/adapter/druid/DruidConnectionImpl.java   | 2 +-
 .../org/apache/calcite/adapter/file/FileRowConverter.java   | 2 +-
 pom.xml                                                     | 2 +-
 12 files changed, 21 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/915c218b/core/src/main/codegen/templates/Parser.jj
----------------------------------------------------------------------
diff --git a/core/src/main/codegen/templates/Parser.jj b/core/src/main/codegen/templates/Parser.jj
index c84ebe6..b776e92 100644
--- a/core/src/main/codegen/templates/Parser.jj
+++ b/core/src/main/codegen/templates/Parser.jj
@@ -418,7 +418,7 @@ JAVACODE void checkNonQueryExpression(ExprContext exprContext)
 JAVACODE SqlDateLiteral parseDateLiteral(String s, SqlParserPos pos) {
     String dateStr = SqlParserUtil.parseString(s);
     Calendar cal = DateTimeUtils.parseDateFormat(
-        dateStr, DateTimeUtils.DATE_FORMAT_STRING, DateTimeUtils.GMT_ZONE);
+        dateStr, DateTimeUtils.DATE_FORMAT_STRING, DateTimeUtils.UTC_ZONE);
     if (null == cal) {
         throw SqlUtil.newContextException(pos,
             RESOURCE.illegalLiteral("DATE", s,
@@ -431,7 +431,7 @@ JAVACODE SqlTimeLiteral parseTimeLiteral(String s, SqlParserPos pos) {
     String dateStr = SqlParserUtil.parseString(s);
     DateTimeUtils.PrecisionTime pt =
     DateTimeUtils.parsePrecisionDateTimeLiteral(
-        dateStr, DateTimeUtils.TIME_FORMAT_STRING, DateTimeUtils.GMT_ZONE);
+        dateStr, DateTimeUtils.TIME_FORMAT_STRING, DateTimeUtils.UTC_ZONE);
     if (null == pt) {
         throw SqlUtil.newContextException(pos,
             RESOURCE.illegalLiteral("TIME", s,
@@ -444,7 +444,7 @@ JAVACODE SqlTimestampLiteral parseTimestampLiteral(String s, SqlParserPos pos) {
     String dateStr = SqlParserUtil.parseString(s);
     DateTimeUtils.PrecisionTime pt =
     DateTimeUtils.parsePrecisionDateTimeLiteral(
-        dateStr, DateTimeUtils.TIMESTAMP_FORMAT_STRING, DateTimeUtils.GMT_ZONE);
+        dateStr, DateTimeUtils.TIMESTAMP_FORMAT_STRING, DateTimeUtils.UTC_ZONE);
     if (null == pt) {
         throw SqlUtil.newContextException(pos,
             RESOURCE.illegalLiteral("TIMESTAMP", s,

http://git-wip-us.apache.org/repos/asf/calcite/blob/915c218b/core/src/main/java/org/apache/calcite/rex/RexLiteral.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexLiteral.java b/core/src/main/java/org/apache/calcite/rex/RexLiteral.java
index 5bfd08e..f9be2bb 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexLiteral.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexLiteral.java
@@ -216,7 +216,7 @@ public class RexLiteral extends RexNode {
     case TIME:
     case TIMESTAMP:
       return value instanceof Calendar
-          && ((Calendar) value).getTimeZone().equals(DateTimeUtils.GMT_ZONE);
+          && ((Calendar) value).getTimeZone().equals(DateTimeUtils.UTC_ZONE);
     case INTERVAL_YEAR:
     case INTERVAL_YEAR_MONTH:
     case INTERVAL_MONTH:
@@ -520,7 +520,7 @@ public class RexLiteral extends RexNode {
     case TIME:
     case TIMESTAMP:
       String format = getCalendarFormat(typeName);
-      TimeZone tz = DateTimeUtils.GMT_ZONE;
+      TimeZone tz = DateTimeUtils.UTC_ZONE;
       Calendar cal = null;
       if (typeName == SqlTypeName.DATE) {
         cal =

http://git-wip-us.apache.org/repos/asf/calcite/blob/915c218b/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 3f9e6b0..27691fb 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
@@ -463,7 +463,7 @@ public class SqlDialect {
    * <blockquote><code>quoteTimestampLiteral(new Timestamp(0));</code>
    * </blockquote>
    *
-   * returns {@code TIMESTAMP '1970-01-01 00:00:00'}, regardless of the JVM's
+   * <p>returns {@code TIMESTAMP '1970-01-01 00:00:00'}, regardless of the JVM's
    * time zone.
    *
    * @param timestamp Timestamp
@@ -474,7 +474,7 @@ public class SqlDialect {
         new SimpleDateFormat(
             "'TIMESTAMP' ''yyyy-MM-DD HH:mm:SS''",
             Locale.ROOT);
-    format.setTimeZone(DateTimeUtils.GMT_ZONE);
+    format.setTimeZone(DateTimeUtils.UTC_ZONE);
     return format.format(timestamp);
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/915c218b/core/src/main/java/org/apache/calcite/util/Util.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Util.java b/core/src/main/java/org/apache/calcite/util/Util.java
index 24d9ab5..5e26fd7 100644
--- a/core/src/main/java/org/apache/calcite/util/Util.java
+++ b/core/src/main/java/org/apache/calcite/util/Util.java
@@ -2417,7 +2417,7 @@ public class Util {
   /** Creates a {@link Calendar} in the GMT time zone and root locale.
    * Does not use the time zone or locale. */
   public static Calendar calendar() {
-    return Calendar.getInstance(DateTimeUtils.GMT_ZONE, Locale.ROOT);
+    return Calendar.getInstance(DateTimeUtils.UTC_ZONE, Locale.ROOT);
   }
 
   //~ Inner Classes ----------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/calcite/blob/915c218b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
index bb74daa..f4baa86 100644
--- a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
@@ -2233,10 +2233,10 @@ public class SqlParserTest {
         "TIMESTAMP '2004-06-01 15:55:55.123'");
     checkExp(
         "TIMESTAMP '2004-06-01 15:55:55.1236'",
-        "TIMESTAMP '2004-06-01 15:55:55.124'");
+        "TIMESTAMP '2004-06-01 15:55:55.123'");
     checkExp(
         "TIMESTAMP '2004-06-01 15:55:55.9999'",
-        "TIMESTAMP '2004-06-01 15:55:56.000'");
+        "TIMESTAMP '2004-06-01 15:55:55.999'");
     checkExpSame("NULL");
   }
 
@@ -3684,10 +3684,14 @@ public class SqlParserTest {
 
     // Date literals
     checkExp("DATE '2004-12-01'", "DATE '2004-12-01'");
+
+    // Time literals
     checkExp("TIME '12:01:01'", "TIME '12:01:01'");
     checkExp("TIME '12:01:01.'", "TIME '12:01:01'");
     checkExp("TIME '12:01:01.000'", "TIME '12:01:01.000'");
     checkExp("TIME '12:01:01.001'", "TIME '12:01:01.001'");
+
+    // Timestamp literals
     checkExp(
         "TIMESTAMP '2004-12-01 12:01:01'",
         "TIMESTAMP '2004-12-01 12:01:01'");
@@ -3697,7 +3701,6 @@ public class SqlParserTest {
     checkExp(
         "TIMESTAMP '2004-12-01 12:01:01.'",
         "TIMESTAMP '2004-12-01 12:01:01'");
-    checkExpSame("TIMESTAMP '2004-12-01 12:01:01.1'");
 
     // Failures.
     checkFails("^DATE '12/21/99'^", "(?s).*Illegal DATE literal.*");

http://git-wip-us.apache.org/repos/asf/calcite/blob/915c218b/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/CalciteAssert.java b/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
index 370ba99..9829997 100644
--- a/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
+++ b/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
@@ -131,7 +131,7 @@ public class CalciteAssert {
   private static final DateFormat UTC_TIME_FORMAT;
   private static final DateFormat UTC_TIMESTAMP_FORMAT;
   static {
-    final TimeZone utc = DateTimeUtils.GMT_ZONE;
+    final TimeZone utc = DateTimeUtils.UTC_ZONE;
     UTC_DATE_FORMAT = new SimpleDateFormat("yyyy-MM-dd", Locale.ROOT);
     UTC_DATE_FORMAT.setTimeZone(utc);
     UTC_TIME_FORMAT = new SimpleDateFormat("HH:mm:ss", Locale.ROOT);

http://git-wip-us.apache.org/repos/asf/calcite/blob/915c218b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcTest.java b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
index d97035d..825babe 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
@@ -5017,7 +5017,7 @@ public class JdbcTest {
                     fail("expected error, got " + resultSet);
                   } catch (SQLException e) {
                     assertThat(e.getMessage(),
-                        equalTo(
+                        containsString(
                             "exception while executing query: unbound parameter"));
                   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/915c218b/core/src/test/java/org/apache/calcite/test/SqlLimitsTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlLimitsTest.java b/core/src/test/java/org/apache/calcite/test/SqlLimitsTest.java
index 298830b..86c55f0 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlLimitsTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlLimitsTest.java
@@ -195,7 +195,7 @@ public class SqlLimitsTest {
     } else if (o instanceof Calendar) {
       Calendar calendar = (Calendar) o;
       DateFormat dateFormat = getDateFormat(type.getSqlTypeName());
-      dateFormat.setTimeZone(DateTimeUtils.GMT_ZONE);
+      dateFormat.setTimeZone(DateTimeUtils.UTC_ZONE);
       s = dateFormat.format(calendar.getTime());
     } else {
       s = o.toString();

http://git-wip-us.apache.org/repos/asf/calcite/blob/915c218b/core/src/test/resources/sql/misc.iq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/misc.iq b/core/src/test/resources/sql/misc.iq
index 0ba3335..cbf12fe 100644
--- a/core/src/test/resources/sql/misc.iq
+++ b/core/src/test/resources/sql/misc.iq
@@ -1915,8 +1915,6 @@ EnumerableCalc(expr#0=[{inputs}], expr#1=[2016-02-26 19:06:00.123], expr#2=[2016
 !plan
 
 # [CALCITE-1664] CAST('<string>' as TIMESTAMP) adds part of sub-second fraction to the value
-# The following output is incorrect (i.e. demonstrates the bug), but will become
-# correct when we upgrade to Avatica-1.10 (where the bug is fixed).
 select
   TIMESTAMP '2016-02-26 19:06:00.12345678',
   CAST('2016-02-26 19:06:00.12345678' as TIMESTAMP),
@@ -1926,7 +1924,7 @@ select
 +---------------------+---------------------+--------+
 | EXPR$0              | EXPR$1              | EXPR$2 |
 +---------------------+---------------------+--------+
-| 2016-02-26 19:06:00 | 2016-02-26 22:31:46 | 123456 |
+| 2016-02-26 19:06:00 | 2016-02-26 19:06:00 |      0 |
 +---------------------+---------------------+--------+
 (1 row)
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/915c218b/druid/src/main/java/org/apache/calcite/adapter/druid/DruidConnectionImpl.java
----------------------------------------------------------------------
diff --git a/druid/src/main/java/org/apache/calcite/adapter/druid/DruidConnectionImpl.java b/druid/src/main/java/org/apache/calcite/adapter/druid/DruidConnectionImpl.java
index a1975eb..2e278e8 100644
--- a/druid/src/main/java/org/apache/calcite/adapter/druid/DruidConnectionImpl.java
+++ b/druid/src/main/java/org/apache/calcite/adapter/druid/DruidConnectionImpl.java
@@ -72,7 +72,7 @@ class DruidConnectionImpl implements DruidConnection {
   private static final SimpleDateFormat UTC_TIMESTAMP_FORMAT;
 
   static {
-    final TimeZone utc = DateTimeUtils.GMT_ZONE;
+    final TimeZone utc = DateTimeUtils.UTC_ZONE;
     UTC_TIMESTAMP_FORMAT =
         new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'", Locale.ROOT);
     UTC_TIMESTAMP_FORMAT.setTimeZone(utc);

http://git-wip-us.apache.org/repos/asf/calcite/blob/915c218b/file/src/main/java/org/apache/calcite/adapter/file/FileRowConverter.java
----------------------------------------------------------------------
diff --git a/file/src/main/java/org/apache/calcite/adapter/file/FileRowConverter.java b/file/src/main/java/org/apache/calcite/adapter/file/FileRowConverter.java
index 827bf59..3a635e4 100644
--- a/file/src/main/java/org/apache/calcite/adapter/file/FileRowConverter.java
+++ b/file/src/main/java/org/apache/calcite/adapter/file/FileRowConverter.java
@@ -324,7 +324,7 @@ class FileRowConverter {
     }
 
     private java.util.Date parseDate(String string) {
-      Parser parser = new Parser(DateTimeUtils.GMT_ZONE);
+      Parser parser = new Parser(DateTimeUtils.UTC_ZONE);
       List<DateGroup> groups = parser.parse(string);
       DateGroup group = groups.get(0);
       return group.getDates().get(0);

http://git-wip-us.apache.org/repos/asf/calcite/blob/915c218b/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index ff04980..1a9d420 100644
--- a/pom.xml
+++ b/pom.xml
@@ -53,7 +53,7 @@ limitations under the License.
 
     <!-- This list is in alphabetical order. -->
     <airlift-tpch.version>0.1</airlift-tpch.version>
-    <avatica.version>1.9.0</avatica.version>
+    <avatica.version>1.10.0</avatica.version>
     <build-helper-maven-plugin.version>1.9</build-helper-maven-plugin.version>
     <cassandra-driver-core.version>3.1.4</cassandra-driver-core.version>
     <checksum-maven-plugin.version>1.2</checksum-maven-plugin.version>


[2/5] calcite git commit: [CALCITE-1690] Calcite timestamp literals cannot express precision above millisecond, TIMESTAMP(3)

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/main/java/org/apache/calcite/util/DateString.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/DateString.java b/core/src/main/java/org/apache/calcite/util/DateString.java
new file mode 100644
index 0000000..cea9df3
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/util/DateString.java
@@ -0,0 +1,98 @@
+/*
+ * 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.util;
+
+import org.apache.calcite.avatica.util.DateTimeUtils;
+
+import com.google.common.base.Preconditions;
+
+import java.util.Calendar;
+import java.util.regex.Pattern;
+
+/**
+ * Date literal.
+ *
+ * <p>Immutable, internally represented as a string (in ISO format).
+ */
+public class DateString implements Comparable<DateString> {
+  private static final Pattern PATTERN =
+      Pattern.compile("[0-9][0-9][0-9][0-9]-[0-9][0-9]-[0-9][0-9]");
+
+  final String v;
+
+  /** Creates a DateString. */
+  public DateString(String v) {
+    this.v = v;
+    Preconditions.checkArgument(PATTERN.matcher(v).matches(), v);
+  }
+
+  /** Creates a DateString for year, month, day values. */
+  public DateString(int year, int month, int day) {
+    this(TimestampString.ymd(new StringBuilder(), year, month, day).toString());
+  }
+
+  @Override public String toString() {
+    return v;
+  }
+
+  @Override public boolean equals(Object o) {
+    // The value is in canonical form.
+    return o == this
+        || o instanceof DateString
+        && ((DateString) o).v.equals(v);
+  }
+
+  @Override public int hashCode() {
+    return v.hashCode();
+  }
+
+  @Override public int compareTo(DateString o) {
+    return v.compareTo(o.v);
+  }
+
+  /** Creates a DateString from a Calendar. */
+  public static DateString fromCalendarFields(Calendar calendar) {
+    return new DateString(calendar.get(Calendar.YEAR),
+        calendar.get(Calendar.MONTH) + 1,
+        calendar.get(Calendar.DAY_OF_MONTH));
+  }
+
+  /** Returns the number of days since the epoch. */
+  public int getDaysSinceEpoch() {
+    int year = Integer.valueOf(v.substring(0, 4));
+    int month = Integer.valueOf(v.substring(5, 7));
+    int day = Integer.valueOf(v.substring(8, 10));
+    return DateTimeUtils.ymdToUnixDate(year, month, day);
+  }
+
+  /** Creates a DateString that is a given number of days since the epoch. */
+  public static DateString fromDaysSinceEpoch(int days) {
+    return new DateString(DateTimeUtils.unixDateToString(days));
+  }
+
+  /** Returns the number of milliseconds since the epoch. Always a multiple of
+   * 86,400,000 (the number of milliseconds in a day). */
+  public long getMillisSinceEpoch() {
+    return getDaysSinceEpoch() * DateTimeUtils.MILLIS_PER_DAY;
+  }
+
+  public Calendar toCalendar() {
+    return Util.calendar(getMillisSinceEpoch());
+  }
+}
+
+// End DateString.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/main/java/org/apache/calcite/util/TimeString.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/TimeString.java b/core/src/main/java/org/apache/calcite/util/TimeString.java
new file mode 100644
index 0000000..75aa96b
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/util/TimeString.java
@@ -0,0 +1,192 @@
+/*
+ * 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.util;
+
+import org.apache.calcite.avatica.util.DateTimeUtils;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+
+import java.util.Calendar;
+import java.util.regex.Pattern;
+
+/**
+ * Time literal.
+ *
+ * <p>Immutable, internally represented as a string (in ISO format),
+ * and can support unlimited precision (milliseconds, nanoseconds).
+ */
+public class TimeString implements Comparable<TimeString> {
+  private static final Pattern PATTERN =
+      Pattern.compile("[0-9][0-9]:[0-9][0-9]:[0-9][0-9](\\.[0-9]*[1-9])?");
+
+  final String v;
+
+  /** Creates a TimeString. */
+  public TimeString(String v) {
+    this.v = v;
+    Preconditions.checkArgument(PATTERN.matcher(v).matches(), v);
+  }
+
+  /** Creates a TimeString for hour, minute, second and millisecond values. */
+  public TimeString(int h, int m, int s) {
+    this(TimestampString.hms(new StringBuilder(), h, m, s).toString());
+  }
+
+  /** Sets the fraction field of a {@code TimeString} to a given number
+   * of milliseconds. Nukes the value set via {@link #withNanos}.
+   *
+   * <p>For example,
+   * {@code new TimeString(1970, 1, 1, 2, 3, 4).withMillis(56)}
+   * yields {@code TIME '1970-01-01 02:03:04.056'}. */
+  public TimeString withMillis(int millis) {
+    Preconditions.checkArgument(millis >= 0 && millis < 1000);
+    return withFraction(TimestampString.pad(3, millis));
+  }
+
+  /** Sets the fraction field of a {@code TimeString} to a given number
+   * of nanoseconds. Nukes the value set via {@link #withMillis(int)}.
+   *
+   * <p>For example,
+   * {@code new TimeString(1970, 1, 1, 2, 3, 4).withNanos(56789)}
+   * yields {@code TIME '1970-01-01 02:03:04.000056789'}. */
+  public TimeString withNanos(int nanos) {
+    Preconditions.checkArgument(nanos >= 0 && nanos < 1000000000);
+    return withFraction(TimestampString.pad(9, nanos));
+  }
+
+  /** Sets the fraction field of a {@code TimeString}.
+   * The precision is determined by the number of leading zeros.
+   * Trailing zeros are stripped.
+   *
+   * <p>For example,
+   * {@code new TimeString(1970, 1, 1, 2, 3, 4).withFraction("00506000")}
+   * yields {@code TIME '1970-01-01 02:03:04.00506'}. */
+  public TimeString withFraction(String fraction) {
+    String v = this.v;
+    int i = v.indexOf('.');
+    if (i >= 0) {
+      v = v.substring(0, i);
+    }
+    while (fraction.endsWith("0")) {
+      fraction = fraction.substring(0, fraction.length() - 1);
+    }
+    if (fraction.length() > 0) {
+      v = v + "." + fraction;
+    }
+    return new TimeString(v);
+  }
+
+  @Override public String toString() {
+    return v;
+  }
+
+  @Override public boolean equals(Object o) {
+    // The value is in canonical form (no trailing zeros).
+    return o == this
+        || o instanceof TimeString
+        && ((TimeString) o).v.equals(v);
+  }
+
+  @Override public int hashCode() {
+    return v.hashCode();
+  }
+
+  @Override public int compareTo(TimeString o) {
+    return v.compareTo(o.v);
+  }
+
+  /** Creates a TimeString from a Calendar. */
+  public static TimeString fromCalendarFields(Calendar calendar) {
+    return new TimeString(
+        calendar.get(Calendar.HOUR_OF_DAY),
+        calendar.get(Calendar.MINUTE),
+        calendar.get(Calendar.SECOND))
+        .withMillis(calendar.get(Calendar.MILLISECOND));
+  }
+
+  public static TimeString fromMillisOfDay(int i) {
+    return new TimeString(DateTimeUtils.unixTimeToString(i))
+        .withMillis((int) DateTimeUtils.floorMod(i, 1000));
+  }
+
+  public TimeString round(int precision) {
+    Preconditions.checkArgument(precision >= 0);
+    int targetLength = 9 + precision;
+    if (v.length() <= targetLength) {
+      return this;
+    }
+    String v = this.v.substring(0, targetLength);
+    while (v.length() >= 9 && (v.endsWith("0") || v.endsWith("."))) {
+      v = v.substring(0, v.length() - 1);
+    }
+    return new TimeString(v);
+  }
+
+  public int getMillisOfDay() {
+    int h = Integer.valueOf(v.substring(0, 2));
+    int m = Integer.valueOf(v.substring(3, 5));
+    int s = Integer.valueOf(v.substring(6, 8));
+    int ms = getMillisInSecond();
+    return (int) (h * DateTimeUtils.MILLIS_PER_HOUR
+        + m * DateTimeUtils.MILLIS_PER_MINUTE
+        + s * DateTimeUtils.MILLIS_PER_SECOND
+        + ms);
+  }
+
+  private int getMillisInSecond() {
+    switch (v.length()) {
+    case 8: // "12:34:56"
+      return 0;
+    case 10: // "12:34:56.7"
+      return Integer.valueOf(v.substring(9)) * 100;
+    case 11: // "12:34:56.78"
+      return Integer.valueOf(v.substring(9)) * 10;
+    case 12: // "12:34:56.789"
+    default: // "12:34:56.7890000012345"
+      return Integer.valueOf(v.substring(9, 12));
+    }
+  }
+
+  public Calendar toCalendar() {
+    return Util.calendar(getMillisOfDay());
+  }
+
+  /** Converts this TimestampString to a string, truncated or padded with
+   * zeroes to a given precision. */
+  public String toString(int precision) {
+    Preconditions.checkArgument(precision >= 0);
+    final int p = precision();
+    if (precision < p) {
+      return round(precision).toString(precision);
+    }
+    if (precision > p) {
+      String s = v;
+      if (p == 0) {
+        s += ".";
+      }
+      return s + Strings.repeat("0", precision - p);
+    }
+    return v;
+  }
+
+  private int precision() {
+    return v.length() < 9 ? 0 : (v.length() - 9);
+  }
+}
+
+// End TimeString.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/main/java/org/apache/calcite/util/TimestampString.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/TimestampString.java b/core/src/main/java/org/apache/calcite/util/TimestampString.java
new file mode 100644
index 0000000..4e392f0
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/util/TimestampString.java
@@ -0,0 +1,252 @@
+/*
+ * 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.util;
+
+import org.apache.calcite.avatica.util.DateTimeUtils;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+
+import java.util.Calendar;
+import java.util.regex.Pattern;
+
+/**
+ * Timestamp literal.
+ *
+ * <p>Immutable, internally represented as a string (in ISO format),
+ * and can support unlimited precision (milliseconds, nanoseconds).
+ */
+public class TimestampString implements Comparable<TimestampString> {
+  private static final Pattern PATTERN =
+      Pattern.compile("[0-9][0-9][0-9][0-9]-[0-9][0-9]-[0-9][0-9]"
+          + " "
+          + "[0-9][0-9]:[0-9][0-9]:[0-9][0-9](\\.[0-9]*[1-9])?");
+
+  final String v;
+
+  /** Creates a TimeString. */
+  public TimestampString(String v) {
+    this.v = v;
+    Preconditions.checkArgument(PATTERN.matcher(v).matches(), v);
+  }
+
+  /** Creates a TimestampString for year, month, day, hour, minute, second,
+   *  millisecond values. */
+  public TimestampString(int year, int month, int day, int h, int m, int s) {
+    this(ymdhms(new StringBuilder(), year, month, day, h, m, s).toString());
+  }
+
+  /** Sets the fraction field of a {@code TimestampString} to a given number
+   * of milliseconds. Nukes the value set via {@link #withNanos}.
+   *
+   * <p>For example,
+   * {@code new TimestampString(1970, 1, 1, 2, 3, 4).withMillis(56)}
+   * yields {@code TIMESTAMP '1970-01-01 02:03:04.056'}. */
+  public TimestampString withMillis(int millis) {
+    Preconditions.checkArgument(millis >= 0 && millis < 1000);
+    return withFraction(pad(3, millis));
+  }
+
+  /** Sets the fraction field of a {@code TimestampString} to a given number
+   * of nanoseconds. Nukes the value set via {@link #withMillis(int)}.
+   *
+   * <p>For example,
+   * {@code new TimestampString(1970, 1, 1, 2, 3, 4).withNanos(56789)}
+   * yields {@code TIMESTAMP '1970-01-01 02:03:04.000056789'}. */
+  public TimestampString withNanos(int nanos) {
+    Preconditions.checkArgument(nanos >= 0 && nanos < 1000000000);
+    return withFraction(pad(9, nanos));
+  }
+
+  /** Sets the fraction field of a {@code TimestampString}.
+   * The precision is determined by the number of leading zeros.
+   * Trailing zeros are stripped.
+   *
+   * <p>For example,
+   * {@code new TimestampString(1970, 1, 1, 2, 3, 4).withFraction("00506000")}
+   * yields {@code TIMESTAMP '1970-01-01 02:03:04.00506'}. */
+  public TimestampString withFraction(String fraction) {
+    String v = this.v;
+    int i = v.indexOf('.');
+    if (i >= 0) {
+      v = v.substring(0, i);
+    }
+    while (fraction.endsWith("0")) {
+      fraction = fraction.substring(0, fraction.length() - 1);
+    }
+    if (fraction.length() > 0) {
+      v = v + "." + fraction;
+    }
+    return new TimestampString(v);
+  }
+
+  @Override public String toString() {
+    return v;
+  }
+
+  @Override public boolean equals(Object o) {
+    // The value is in canonical form (no trailing zeros).
+    return o == this
+        || o instanceof TimestampString
+        && ((TimestampString) o).v.equals(v);
+  }
+
+  @Override public int hashCode() {
+    return v.hashCode();
+  }
+
+  @Override public int compareTo(TimestampString o) {
+    return v.compareTo(o.v);
+  }
+
+  static StringBuilder hms(StringBuilder b, int h, int m, int s) {
+    int2(b, h);
+    b.append(':');
+    int2(b, m);
+    b.append(':');
+    int2(b, s);
+    return b;
+  }
+
+  static StringBuilder ymdhms(StringBuilder b, int year, int month, int day,
+      int h, int m, int s) {
+    ymd(b, year, month, day);
+    b.append(' ');
+    hms(b, h, m, s);
+    return b;
+  }
+
+  static StringBuilder ymd(StringBuilder b, int year, int month, int day) {
+    int4(b, year);
+    b.append('-');
+    int2(b, month);
+    b.append('-');
+    int2(b, day);
+    return b;
+  }
+
+  private static void int4(StringBuilder buf, int i) {
+    buf.append((char) ('0' + (i / 1000) % 10));
+    buf.append((char) ('0' + (i / 100) % 10));
+    buf.append((char) ('0' + (i / 10) % 10));
+    buf.append((char) ('0' + i % 10));
+  }
+
+  private static void int2(StringBuilder buf, int i) {
+    buf.append((char) ('0' + (i / 10) % 10));
+    buf.append((char) ('0' + i % 10));
+  }
+
+  /** Creates a TimestampString from a Calendar. */
+  public static TimestampString fromCalendarFields(Calendar calendar) {
+    return new TimestampString(
+        calendar.get(Calendar.YEAR),
+        calendar.get(Calendar.MONTH) + 1,
+        calendar.get(Calendar.DAY_OF_MONTH),
+        calendar.get(Calendar.HOUR_OF_DAY),
+        calendar.get(Calendar.MINUTE),
+        calendar.get(Calendar.SECOND))
+        .withMillis(calendar.get(Calendar.MILLISECOND));
+  }
+
+  public TimestampString round(int precision) {
+    Preconditions.checkArgument(precision >= 0);
+    int targetLength = 20 + precision;
+    if (v.length() <= targetLength) {
+      return this;
+    }
+    String v = this.v.substring(0, targetLength);
+    while (v.length() >= 20 && (v.endsWith("0") || v.endsWith("."))) {
+      v = v.substring(0, v.length() - 1);
+    }
+    return new TimestampString(v);
+  }
+
+  /** Returns the number of milliseconds since the epoch. */
+  public long getMillisSinceEpoch() {
+    final int year = Integer.valueOf(v.substring(0, 4));
+    final int month = Integer.valueOf(v.substring(5, 7));
+    final int day = Integer.valueOf(v.substring(8, 10));
+    final int h = Integer.valueOf(v.substring(11, 13));
+    final int m = Integer.valueOf(v.substring(14, 16));
+    final int s = Integer.valueOf(v.substring(17, 19));
+    final int ms = getMillisInSecond();
+    final int d = DateTimeUtils.ymdToUnixDate(year, month, day);
+    return d * DateTimeUtils.MILLIS_PER_DAY
+        + h * DateTimeUtils.MILLIS_PER_HOUR
+        + m * DateTimeUtils.MILLIS_PER_MINUTE
+        + s * DateTimeUtils.MILLIS_PER_SECOND
+        + ms;
+  }
+
+  private int getMillisInSecond() {
+    switch (v.length()) {
+    case 19: // "1999-12-31 12:34:56"
+      return 0;
+    case 21: // "1999-12-31 12:34:56.7"
+      return Integer.valueOf(v.substring(20)) * 100;
+    case 22: // "1999-12-31 12:34:56.78"
+      return Integer.valueOf(v.substring(20)) * 10;
+    case 23: // "1999-12-31 12:34:56.789"
+    default:  // "1999-12-31 12:34:56.789123456"
+      return Integer.valueOf(v.substring(20, 23));
+    }
+  }
+
+  /** Creates a TimestampString that is a given number of milliseconds since
+   * the epoch. */
+  public static TimestampString fromMillisSinceEpoch(long millis) {
+    return new TimestampString(DateTimeUtils.unixTimestampToString(millis))
+        .withMillis((int) DateTimeUtils.floorMod(millis, 1000));
+  }
+
+  static String pad(int length, long v) {
+    StringBuilder s = new StringBuilder(Long.toString(v));
+    while (s.length() < length) {
+      s.insert(0, "0");
+    }
+    return s.toString();
+  }
+
+  public Calendar toCalendar() {
+    return Util.calendar(getMillisSinceEpoch());
+  }
+
+  /** Converts this TimestampString to a string, truncated or padded with
+   * zeroes to a given precision. */
+  public String toString(int precision) {
+    Preconditions.checkArgument(precision >= 0);
+    final int p = precision();
+    if (precision < p) {
+      return round(precision).toString(precision);
+    }
+    if (precision > p) {
+      String s = v;
+      if (p == 0) {
+        s += ".";
+      }
+      return s + Strings.repeat("0", precision - p);
+    }
+    return v;
+  }
+
+  private int precision() {
+    return v.length() < 20 ? 0 : (v.length() - 20);
+  }
+}
+
+// End TimestampString.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/main/java/org/apache/calcite/util/Util.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Util.java b/core/src/main/java/org/apache/calcite/util/Util.java
index 5e26fd7..e0746e2 100644
--- a/core/src/main/java/org/apache/calcite/util/Util.java
+++ b/core/src/main/java/org/apache/calcite/util/Util.java
@@ -665,7 +665,7 @@ public class Util {
    * underscore followed by the hex code of the character; and underscores are
    * doubled.</p>
    *
-   * Examples:
+   * <p>Examples:
    *
    * <ul>
    * <li><code>toJavaId("foo")</code> returns <code>"foo"</code>
@@ -740,7 +740,7 @@ public class Util {
   /**
    * Converts a list of a string, with commas between elements.
    *
-   * For example,
+   * <p>For example,
    * <code>commaList(Arrays.asList({"a", "b"}))</code>
    * returns "a, b".
    *
@@ -944,7 +944,7 @@ public class Util {
    * <pre><code>int x = Util.deprecated(0, false);</code></pre>
    * </blockquote>
    *
-   * but the usual usage is to pass in a descriptive string.
+   * <p>but the usual usage is to pass in a descriptive string.
    *
    * <h3>Examples</h3>
    *
@@ -1265,7 +1265,7 @@ public class Util {
    * <blockquote>"std offset dst [offset],start[/time],end[/time]"
    * </blockquote>
    *
-   * where:
+   * <p>where:
    *
    * <ul>
    * <li>'std' specifies the abbrev of the time zone.
@@ -1736,7 +1736,7 @@ public class Util {
    * &nbsp;&nbsp;&nbsp;&nbsp;print(i);<br>
    * }</code></blockquote>
    *
-   * will print 1, 2, 4.
+   * <p>will print 1, 2, 4.
    *
    * @param iterable      Iterable
    * @param includeFilter Class whose instances to include
@@ -2414,12 +2414,20 @@ public class Util {
     return reader(new FileInputStream(file));
   }
 
-  /** Creates a {@link Calendar} in the GMT time zone and root locale.
+  /** Creates a {@link Calendar} in the UTC time zone and root locale.
    * Does not use the time zone or locale. */
   public static Calendar calendar() {
     return Calendar.getInstance(DateTimeUtils.UTC_ZONE, Locale.ROOT);
   }
 
+  /** Creates a {@link Calendar} in the UTC time zone and root locale
+   * with a given time. */
+  public static Calendar calendar(long millis) {
+    Calendar calendar = calendar();
+    calendar.setTimeInMillis(millis);
+    return calendar;
+  }
+
   //~ Inner Classes ----------------------------------------------------------
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/main/java/org/apache/calcite/util/ZonelessDate.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/ZonelessDate.java b/core/src/main/java/org/apache/calcite/util/ZonelessDate.java
deleted file mode 100644
index 0d5fc16..0000000
--- a/core/src/main/java/org/apache/calcite/util/ZonelessDate.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.calcite.util;
-
-import org.apache.calcite.avatica.util.DateTimeUtils;
-
-import java.sql.Date;
-import java.text.DateFormat;
-import java.util.Calendar;
-import java.util.TimeZone;
-
-/**
- * ZonelessDate is a date value without a time zone.
- */
-public class ZonelessDate extends ZonelessDatetime {
-  //~ Static fields/initializers ---------------------------------------------
-
-  /**
-   * SerialVersionUID created with JDK 1.5 serialver tool.
-   */
-  private static final long serialVersionUID = -6385775986251759394L;
-
-  //~ Instance fields --------------------------------------------------------
-
-  protected transient Date tempDate;
-
-  //~ Constructors -----------------------------------------------------------
-
-  /**
-   * Constructs a ZonelessDate.
-   */
-  public ZonelessDate() {
-  }
-
-  //~ Methods ----------------------------------------------------------------
-
-  // override ZonelessDatetime
-  public void setZonelessTime(long value) {
-    super.setZonelessTime(value);
-    clearTime();
-  }
-
-  // override ZonelessDatetime
-  public void setZonedTime(long value, TimeZone zone) {
-    super.setZonedTime(value, zone);
-    clearTime();
-  }
-
-  // implement ZonelessDatetime
-  public Object toJdbcObject() {
-    return new Date(getJdbcDate(DateTimeUtils.DEFAULT_ZONE));
-  }
-
-  /**
-   * Converts this ZonelessDate to a java.sql.Date and formats it via the
-   * {@link java.sql.Date#toString() toString()} method of that class.
-   *
-   * @return the formatted date string
-   */
-  public String toString() {
-    Date jdbcDate = getTempDate(getJdbcDate(DateTimeUtils.DEFAULT_ZONE));
-    return jdbcDate.toString();
-  }
-
-  /**
-   * Formats this ZonelessDate via a SimpleDateFormat.
-   *
-   * @param format Format string, as required by
-   *     {@link java.text.SimpleDateFormat}
-   * @return the formatted date string
-   */
-  public String toString(String format) {
-    DateFormat formatter = getFormatter(format);
-    Date jdbcDate = getTempDate(getTime());
-    return formatter.format(jdbcDate);
-  }
-
-  /**
-   * Parses a string as a ZonelessDate.
-   *
-   * @param s a string representing a date in ISO format, i.e. according to
-   *          the SimpleDateFormat string "yyyy-MM-dd"
-   * @return the parsed date, or null if parsing failed
-   */
-  public static ZonelessDate parse(String s) {
-    return parse(s, DateTimeUtils.DATE_FORMAT_STRING);
-  }
-
-  /**
-   * Parses a string as a ZonelessDate with a given format string.
-   *
-   * @param s      a string representing a date in ISO format, i.e. according to
-   *               the SimpleDateFormat string "yyyy-MM-dd"
-   * @param format Format string as per {@link java.text.SimpleDateFormat}
-   * @return the parsed date, or null if parsing failed
-   */
-  public static ZonelessDate parse(String s, String format) {
-    Calendar cal =
-        DateTimeUtils.parseDateFormat(s, format, DateTimeUtils.GMT_ZONE);
-    if (cal == null) {
-      return null;
-    }
-    ZonelessDate zd = new ZonelessDate();
-    zd.setZonelessTime(cal.getTimeInMillis());
-    return zd;
-  }
-
-  /**
-   * Gets a temporary Date object. The same object is returned every time.
-   */
-  protected Date getTempDate(long value) {
-    if (tempDate == null) {
-      tempDate = new Date(value);
-    } else {
-      tempDate.setTime(value);
-    }
-    return tempDate;
-  }
-}
-
-// End ZonelessDate.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/main/java/org/apache/calcite/util/ZonelessDatetime.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/ZonelessDatetime.java b/core/src/main/java/org/apache/calcite/util/ZonelessDatetime.java
deleted file mode 100644
index ab311e7..0000000
--- a/core/src/main/java/org/apache/calcite/util/ZonelessDatetime.java
+++ /dev/null
@@ -1,199 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.calcite.util;
-
-import org.apache.calcite.avatica.util.DateTimeUtils;
-
-import java.io.Serializable;
-import java.text.DateFormat;
-import java.util.Calendar;
-import java.util.Locale;
-import java.util.TimeZone;
-
-/**
- * ZonelessDatetime is an abstract class for dates, times, or timestamps that
- * contain a zoneless time value.
- */
-public abstract class ZonelessDatetime implements BasicDatetime, Serializable {
-  //~ Static fields/initializers ---------------------------------------------
-
-  /**
-   * SerialVersionUID created with JDK 1.5 serialver tool.
-   */
-  private static final long serialVersionUID = -1274713852537224763L;
-
-  //~ Instance fields --------------------------------------------------------
-
-  /**
-   * Treat this as a protected field. It is only made public to simplify Java
-   * code generation.
-   */
-  public long internalTime;
-
-  // The following fields are workspace and are not serialized.
-
-  protected transient Calendar tempCal;
-  protected transient DateFormat tempFormatter;
-  protected transient String lastFormat;
-
-  //~ Methods ----------------------------------------------------------------
-
-  // implement BasicDatetime
-  public long getTime() {
-    return internalTime;
-  }
-
-  // implement BasicDatetime
-  public void setZonelessTime(long value) {
-    this.internalTime = value;
-  }
-
-  // implement BasicDatetime
-  public void setZonedTime(long value, TimeZone zone) {
-    this.internalTime = value + zone.getOffset(value);
-  }
-
-  /**
-   * Gets the time portion of this zoneless datetime.
-   */
-  public long getTimeValue() {
-    // Value must be non-negative, even for negative timestamps, and
-    // unfortunately the '%' operator returns a negative value if its LHS
-    // is negative.
-    long timePart = internalTime % DateTimeUtils.MILLIS_PER_DAY;
-    if (timePart < 0) {
-      timePart += DateTimeUtils.MILLIS_PER_DAY;
-    }
-    return timePart;
-  }
-
-  /**
-   * Gets the date portion of this zoneless datetime.
-   */
-  public long getDateValue() {
-    return internalTime - getTimeValue();
-  }
-
-  /**
-   * Clears the date component of this datetime
-   */
-  public void clearDate() {
-    internalTime = getTimeValue();
-  }
-
-  /**
-   * Clears the time component of this datetime
-   */
-  public void clearTime() {
-    internalTime = getDateValue();
-  }
-
-  /**
-   * Gets the value of this datetime as a milliseconds value for
-   * {@link java.sql.Time}.
-   *
-   * @param zone time zone in which to generate a time value for
-   */
-  public long getJdbcTime(TimeZone zone) {
-    long timeValue = getTimeValue();
-    return timeValue - zone.getOffset(timeValue);
-  }
-
-  /**
-   * Gets the value of this datetime as a milliseconds value for
-   * {@link java.sql.Date}.
-   *
-   * @param zone time zone in which to generate a time value for
-   */
-  public long getJdbcDate(TimeZone zone) {
-    Calendar cal = getCalendar(DateTimeUtils.GMT_ZONE);
-    cal.setTimeInMillis(getDateValue());
-
-    int year = cal.get(Calendar.YEAR);
-    int doy = cal.get(Calendar.DAY_OF_YEAR);
-
-    cal.clear();
-    cal.setTimeZone(zone);
-    cal.set(Calendar.YEAR, year);
-    cal.set(Calendar.DAY_OF_YEAR, doy);
-    return cal.getTimeInMillis();
-  }
-
-  /**
-   * Gets the value of this datetime as a milliseconds value for
-   * {@link java.sql.Timestamp}.
-   *
-   * @param zone time zone in which to generate a time value for
-   */
-  public long getJdbcTimestamp(TimeZone zone) {
-    Calendar cal = getCalendar(DateTimeUtils.GMT_ZONE);
-    cal.setTimeInMillis(internalTime);
-
-    int year = cal.get(Calendar.YEAR);
-    int doy = cal.get(Calendar.DAY_OF_YEAR);
-    int hour = cal.get(Calendar.HOUR_OF_DAY);
-    int minute = cal.get(Calendar.MINUTE);
-    int second = cal.get(Calendar.SECOND);
-    int millis = cal.get(Calendar.MILLISECOND);
-
-    cal.clear();
-    cal.setTimeZone(zone);
-    cal.set(Calendar.YEAR, year);
-    cal.set(Calendar.DAY_OF_YEAR, doy);
-    cal.set(Calendar.HOUR_OF_DAY, hour);
-    cal.set(Calendar.MINUTE, minute);
-    cal.set(Calendar.SECOND, second);
-    cal.set(Calendar.MILLISECOND, millis);
-    return cal.getTimeInMillis();
-  }
-
-  /**
-   * Returns this datetime as a Jdbc object
-   */
-  public abstract Object toJdbcObject();
-
-  /**
-   * Gets a temporary Calendar set to the specified time zone. The same
-   * Calendar is returned on subsequent calls.
-   */
-  protected Calendar getCalendar(TimeZone zone) {
-    if (tempCal == null) {
-      tempCal = Calendar.getInstance(zone, Locale.ROOT);
-    } else {
-      tempCal.setTimeZone(zone);
-    }
-    return tempCal;
-  }
-
-  /**
-   * Gets a temporary formatter for a zoneless date time. The same formatter
-   * is returned on subsequent calls.
-   *
-   * @param format a {@link java.text.SimpleDateFormat} format string
-   */
-  protected DateFormat getFormatter(String format) {
-    if ((tempFormatter != null) && lastFormat.equals(format)) {
-      return tempFormatter;
-    }
-    tempFormatter = DateTimeUtils.newDateFormat(format);
-    tempFormatter.setTimeZone(DateTimeUtils.GMT_ZONE);
-    lastFormat = format;
-    return tempFormatter;
-  }
-}
-
-// End ZonelessDatetime.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/main/java/org/apache/calcite/util/ZonelessTime.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/ZonelessTime.java b/core/src/main/java/org/apache/calcite/util/ZonelessTime.java
deleted file mode 100644
index 28a788c..0000000
--- a/core/src/main/java/org/apache/calcite/util/ZonelessTime.java
+++ /dev/null
@@ -1,175 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.calcite.util;
-
-import org.apache.calcite.avatica.util.DateTimeUtils;
-
-import java.sql.Time;
-import java.text.DateFormat;
-import java.text.SimpleDateFormat;
-import java.util.Calendar;
-import java.util.TimeZone;
-
-/**
- * ZonelessTime is a time value without a time zone.
- */
-public class ZonelessTime extends ZonelessDatetime {
-  //~ Static fields/initializers ---------------------------------------------
-
-  /**
-   * SerialVersionUID created with JDK 1.5 serialver tool.
-   */
-  private static final long serialVersionUID = 906156904798141861L;
-
-  //~ Instance fields --------------------------------------------------------
-
-  protected final int precision;
-  protected transient Time tempTime;
-
-  //~ Constructors -----------------------------------------------------------
-
-  /**
-   * Constructs a ZonelessTime
-   */
-  public ZonelessTime() {
-    precision = 0;
-  }
-
-  /**
-   * Constructs a ZonelessTime with precision.
-   *
-   * <p>The precision is the number of digits to the right of the decimal
-   * point in the seconds value. For example, a <code>TIME(6)</code> has a
-   * precision to microseconds.
-   *
-   * @param precision Number of digits of precision
-   */
-  public ZonelessTime(int precision) {
-    this.precision = precision;
-  }
-
-  //~ Methods ----------------------------------------------------------------
-
-  // override ZonelessDatetime
-  public void setZonelessTime(long value) {
-    super.setZonelessTime(value);
-    clearDate();
-  }
-
-  // override ZonelessDatetime
-  public void setZonedTime(long value, TimeZone zone) {
-    super.setZonedTime(value, zone);
-    clearDate();
-  }
-
-  // implement ZonelessDatetime
-  public Object toJdbcObject() {
-    return new Time(getJdbcTime(DateTimeUtils.DEFAULT_ZONE));
-  }
-
-  /**
-   * Override ZonelessDatetime.
-   *
-   * <p>NOTE: the returned timestamp is based on the current date of the
-   * specified time zone, rather than the context variable for current_date,
-   * as specified by the SQL standard.
-   */
-  public long getJdbcTimestamp(TimeZone zone) {
-    Calendar cal = getCalendar(DateTimeUtils.GMT_ZONE);
-    cal.setTimeInMillis(getTime());
-    int hour = cal.get(Calendar.HOUR_OF_DAY);
-    int minute = cal.get(Calendar.MINUTE);
-    int second = cal.get(Calendar.SECOND);
-    int millis = cal.get(Calendar.MILLISECOND);
-
-    cal.setTimeZone(zone);
-    cal.setTimeInMillis(System.currentTimeMillis());
-    cal.set(Calendar.HOUR_OF_DAY, hour);
-    cal.set(Calendar.MINUTE, minute);
-    cal.set(Calendar.SECOND, second);
-    cal.set(Calendar.MILLISECOND, millis);
-    return cal.getTimeInMillis();
-  }
-
-  /**
-   * Converts this ZonelessTime to a java.sql.Time and formats it via the
-   * {@link java.sql.Time#toString() toString()} method of that class.
-   *
-   * @return the formatted time string
-   */
-  public String toString() {
-    Time jdbcTime = getTempTime(getJdbcTime(DateTimeUtils.DEFAULT_ZONE));
-    return jdbcTime.toString();
-  }
-
-  /**
-   * Formats this ZonelessTime via a SimpleDateFormat
-   *
-   * @param format format string, as required by SimpleDateFormat
-   * @return the formatted time string
-   */
-  public String toString(String format) {
-    DateFormat formatter = getFormatter(format);
-    Time jdbcTime = getTempTime(getTime());
-    return formatter.format(jdbcTime);
-  }
-
-  /**
-   * Parses a string as a ZonelessTime.
-   *
-   * @param s a string representing a time in ISO format, i.e. according to
-   *          the {@link SimpleDateFormat} string "HH:mm:ss"
-   * @return the parsed time, or null if parsing failed
-   */
-  public static ZonelessTime parse(String s) {
-    return parse(s, DateTimeUtils.TIME_FORMAT_STRING);
-  }
-
-  /**
-   * Parses a string as a ZonelessTime using a given format string.
-   *
-   * @param s      a string representing a time the given format
-   * @param format format string as per {@link SimpleDateFormat}
-   * @return the parsed time, or null if parsing failed
-   */
-  public static ZonelessTime parse(String s, String format) {
-    DateTimeUtils.PrecisionTime pt =
-        DateTimeUtils.parsePrecisionDateTimeLiteral(s,
-            format,
-            DateTimeUtils.GMT_ZONE);
-    if (pt == null) {
-      return null;
-    }
-    ZonelessTime zt = new ZonelessTime(pt.getPrecision());
-    zt.setZonelessTime(pt.getCalendar().getTime().getTime());
-    return zt;
-  }
-
-  /**
-   * Gets a temporary Time object. The same object is returned every time.
-   */
-  protected Time getTempTime(long value) {
-    if (tempTime == null) {
-      tempTime = new Time(value);
-    } else {
-      tempTime.setTime(value);
-    }
-    return tempTime;
-  }
-}
-
-// End ZonelessTime.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/main/java/org/apache/calcite/util/ZonelessTimestamp.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/ZonelessTimestamp.java b/core/src/main/java/org/apache/calcite/util/ZonelessTimestamp.java
deleted file mode 100644
index e58f687..0000000
--- a/core/src/main/java/org/apache/calcite/util/ZonelessTimestamp.java
+++ /dev/null
@@ -1,160 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.calcite.util;
-
-import org.apache.calcite.avatica.util.DateTimeUtils;
-
-import java.sql.Timestamp;
-import java.text.DateFormat;
-
-/**
- * ZonelessTimestamp is a timestamp value without a time zone.
- */
-public class ZonelessTimestamp extends ZonelessDatetime {
-  //~ Static fields/initializers ---------------------------------------------
-
-  /**
-   * SerialVersionUID created with JDK 1.5 serialver tool.
-   */
-  private static final long serialVersionUID = -6829714640541648394L;
-
-  //~ Instance fields --------------------------------------------------------
-
-  protected final int precision;
-
-  protected transient Timestamp tempTimestamp;
-
-  //~ Constructors -----------------------------------------------------------
-
-  /**
-   * Constructs a ZonelessTimestamp.
-   */
-  public ZonelessTimestamp() {
-    this.precision = 0;
-  }
-
-  /**
-   * Constructs a ZonelessTimestamp with precision.
-   *
-   * <p>The precision is the number of digits to the right of the decimal
-   * point in the seconds value. For example, a <code>TIMESTAMP(3)</code> has
-   * a precision to milliseconds.
-   *
-   * @param precision Number of digits of precision
-   */
-  public ZonelessTimestamp(int precision) {
-    this.precision = precision;
-  }
-
-  //~ Methods ----------------------------------------------------------------
-
-  // implement ZonelessDatetime
-  public Object toJdbcObject() {
-    return new Timestamp(getJdbcTimestamp(DateTimeUtils.DEFAULT_ZONE));
-  }
-
-  /**
-   * Converts this ZonelessTimestamp to a java.sql.Timestamp and formats it
-   * via the {@link java.sql.Timestamp#toString() toString()} method of that
-   * class.
-   *
-   * <p>Note: Jdbc formatting always includes a decimal point and at least one
-   * digit of milliseconds precision. Trailing zeros, except for the first one
-   * after the decimal point, do not appear in the output.
-   *
-   * @return the formatted time string
-   */
-  public String toString() {
-    Timestamp ts =
-        getTempTimestamp(getJdbcTimestamp(DateTimeUtils.DEFAULT_ZONE));
-
-    // Remove trailing '.0' so that format is consistent with SQL spec for
-    // CAST(TIMESTAMP(0) TO VARCHAR). E.g. "1969-12-31 16:00:00.0"
-    // becomes "1969-12-31 16:00:00"
-    String sts = ts.toString();
-    if (sts.length() > 19 && ts.getNanos() == 0) {
-      sts = sts.substring(0, 19);
-    }
-    return sts;
-  }
-
-  /**
-   * Formats this ZonelessTimestamp via a SimpleDateFormat. This method does
-   * not display milliseconds precision.
-   *
-   * @param format format string, as required by SimpleDateFormat
-   * @return the formatted timestamp string
-   */
-  public String toString(String format) {
-    DateFormat formatter = getFormatter(format);
-    Timestamp ts = getTempTimestamp(getTime());
-    return formatter.format(ts);
-  }
-
-  /**
-   * Parses a string as a ZonelessTimestamp.
-   *
-   * <p>This method's parsing is strict and may parse fractional seconds (as
-   * opposed to just milliseconds.)
-   *
-   * @param s a string representing a time in ISO format, i.e. according to
-   *          the SimpleDateFormat string "yyyy-MM-dd HH:mm:ss"
-   * @return the parsed time, or null if parsing failed
-   */
-  public static ZonelessTimestamp parse(String s) {
-    return parse(s, DateTimeUtils.TIMESTAMP_FORMAT_STRING);
-  }
-
-  /**
-   * Parses a string as a ZonelessTimestamp using a given format string.
-   *
-   * <p>This method's parsing is strict and may parse fractional seconds (as
-   * opposed to just milliseconds.)
-   *
-   * @param s      a string representing a time in ISO format, i.e. according to
-   *               the SimpleDateFormat string "yyyy-MM-dd HH:mm:ss"
-   * @param format Format string as per {@link java.text.SimpleDateFormat}
-   * @return the parsed timestamp, or null if parsing failed
-   */
-  public static ZonelessTimestamp parse(String s, String format) {
-    DateTimeUtils.PrecisionTime pt =
-        DateTimeUtils.parsePrecisionDateTimeLiteral(s,
-            format,
-            DateTimeUtils.GMT_ZONE);
-    if (pt == null) {
-      return null;
-    }
-    ZonelessTimestamp zt = new ZonelessTimestamp(pt.getPrecision());
-    zt.setZonelessTime(pt.getCalendar().getTime().getTime());
-    return zt;
-  }
-
-  /**
-   * Gets a temporary Timestamp object. The same object is returned every
-   * time.
-   */
-  protected Timestamp getTempTimestamp(long value) {
-    if (tempTimestamp == null) {
-      tempTimestamp = new Timestamp(value);
-    } else {
-      tempTimestamp.setTime(value);
-    }
-    return tempTimestamp;
-  }
-}
-
-// End ZonelessTimestamp.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/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 8f5015c..4e4db08 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
@@ -659,6 +659,26 @@ public class RelToSqlConverterTest {
     sql(query).ok(expected);
   }
 
+  @Test public void testLiteral() {
+    checkLiteral("DATE '1978-05-02'");
+    checkLiteral("TIME '12:34:56'");
+    checkLiteral("TIME '12:34:56.78'");
+    checkLiteral("TIMESTAMP '1978-05-02 12:34:56.78'");
+    checkLiteral("'I can''t explain'");
+    checkLiteral("''");
+    checkLiteral("TRUE");
+    checkLiteral("123");
+    checkLiteral("123.45");
+    checkLiteral("-123.45");
+  }
+
+  private void checkLiteral(String s) {
+    sql("VALUES " + s)
+        .dialect(DatabaseProduct.HSQLDB.getDialect())
+        .ok("SELECT *\n"
+            + "FROM (VALUES  (" + s + "))");
+  }
+
   /** Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-1798">[CALCITE-1798]
    * Generate dialect-specific SQL for FLOOR operator</a>. */

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/test/java/org/apache/calcite/rex/RexBuilderTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/rex/RexBuilderTest.java b/core/src/test/java/org/apache/calcite/rex/RexBuilderTest.java
index bbf95f0..0fd99f8 100644
--- a/core/src/test/java/org/apache/calcite/rex/RexBuilderTest.java
+++ b/core/src/test/java/org/apache/calcite/rex/RexBuilderTest.java
@@ -16,18 +16,28 @@
  */
 package org.apache.calcite.rex;
 
+import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.sql.type.SqlTypeFactoryImpl;
 import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.DateString;
+import org.apache.calcite.util.TimeString;
+import org.apache.calcite.util.TimestampString;
+import org.apache.calcite.util.Util;
 
 import org.junit.Test;
 
+import java.util.Calendar;
+
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.core.Is.is;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertThat;
 
 /**
- * Test for {@link RexBuilder}
+ * Test for {@link RexBuilder}.
  */
 public class RexBuilderTest {
 
@@ -80,6 +90,203 @@ public class RexBuilderTest {
     assertEquals(ensuredNode.getType(), typeFactory.createSqlType(SqlTypeName.INTEGER));
   }
 
+  private static final long MOON = -14159025000L;
+
+  private static final int MOON_DAY = -164;
+
+  private static final int MOON_TIME = 10575000;
+
+  /** Tests {@link RexBuilder#makeTimestampLiteral(TimestampString, int)}. */
+  @Test public void testTimestampLiteral() {
+    final RelDataTypeFactory typeFactory =
+        new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
+    final RelDataType timestampType =
+        typeFactory.createSqlType(SqlTypeName.TIMESTAMP);
+    final RelDataType timestampType3 =
+        typeFactory.createSqlType(SqlTypeName.TIMESTAMP, 3);
+    final RelDataType timestampType9 =
+        typeFactory.createSqlType(SqlTypeName.TIMESTAMP, 9);
+    final RelDataType timestampType18 =
+        typeFactory.createSqlType(SqlTypeName.TIMESTAMP, 18);
+    final RexBuilder builder = new RexBuilder(typeFactory);
+
+    // Old way: provide a Calendar
+    final Calendar calendar = Util.calendar();
+    calendar.set(1969, Calendar.JULY, 21, 2, 56, 15); // one small step
+    calendar.set(Calendar.MILLISECOND, 0);
+    checkTimestamp(builder.makeLiteral(calendar, timestampType, false));
+
+    // Old way #2: Provide a Long
+    checkTimestamp(builder.makeLiteral(MOON, timestampType, false));
+
+    // The new way
+    final TimestampString ts = new TimestampString(1969, 7, 21, 2, 56, 15);
+    checkTimestamp(builder.makeLiteral(ts, timestampType, false));
+
+    // Now with milliseconds
+    final TimestampString ts2 = ts.withMillis(56);
+    assertThat(ts2.toString(), is("1969-07-21 02:56:15.056"));
+    final RexNode literal2 = builder.makeLiteral(ts2, timestampType3, false);
+    assertThat(((RexLiteral) literal2).getValueAs(TimestampString.class)
+            .toString(), is("1969-07-21 02:56:15.056"));
+
+    // Now with nanoseconds
+    final TimestampString ts3 = ts.withNanos(56);
+    final RexNode literal3 = builder.makeLiteral(ts3, timestampType9, false);
+    assertThat(((RexLiteral) literal3).getValueAs(TimestampString.class)
+            .toString(), is("1969-07-21 02:56:15"));
+    final TimestampString ts3b = ts.withNanos(2345678);
+    final RexNode literal3b = builder.makeLiteral(ts3b, timestampType9, false);
+    assertThat(((RexLiteral) literal3b).getValueAs(TimestampString.class)
+            .toString(), is("1969-07-21 02:56:15.002"));
+
+    // Now with a very long fraction
+    final TimestampString ts4 = ts.withFraction("102030405060708090102");
+    final RexNode literal4 = builder.makeLiteral(ts4, timestampType18, false);
+    assertThat(((RexLiteral) literal4).getValueAs(TimestampString.class)
+            .toString(), is("1969-07-21 02:56:15.102"));
+
+    // toString
+    assertThat(ts2.round(1).toString(), is("1969-07-21 02:56:15"));
+    assertThat(ts2.round(2).toString(), is("1969-07-21 02:56:15.05"));
+    assertThat(ts2.round(3).toString(), is("1969-07-21 02:56:15.056"));
+    assertThat(ts2.round(4).toString(), is("1969-07-21 02:56:15.056"));
+
+    assertThat(ts2.toString(6), is("1969-07-21 02:56:15.056000"));
+    assertThat(ts2.toString(1), is("1969-07-21 02:56:15.0"));
+    assertThat(ts2.toString(0), is("1969-07-21 02:56:15"));
+
+    assertThat(ts2.round(0).toString(), is("1969-07-21 02:56:15"));
+    assertThat(ts2.round(0).toString(0), is("1969-07-21 02:56:15"));
+    assertThat(ts2.round(0).toString(1), is("1969-07-21 02:56:15.0"));
+    assertThat(ts2.round(0).toString(2), is("1969-07-21 02:56:15.00"));
+
+    assertThat(TimestampString.fromMillisSinceEpoch(1456513560123L).toString(),
+        is("2016-02-26 19:06:00.123"));
+  }
+
+  private void checkTimestamp(RexNode node) {
+    assertThat(node.toString(), is("1969-07-21 02:56:15"));
+    RexLiteral literal = (RexLiteral) node;
+    assertThat(literal.getValue() instanceof Calendar, is(true));
+    assertThat(literal.getValue2() instanceof Long, is(true));
+    assertThat(literal.getValue3() instanceof Long, is(true));
+    assertThat((Long) literal.getValue2(), is(MOON));
+    assertThat(literal.getValueAs(Calendar.class), notNullValue());
+    assertThat(literal.getValueAs(TimestampString.class), notNullValue());
+  }
+
+  /** Tests {@link RexBuilder#makeTimeLiteral(TimeString, int)}. */
+  @Test public void testTimeLiteral() {
+    final RelDataTypeFactory typeFactory =
+        new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
+    RelDataType timeType = typeFactory.createSqlType(SqlTypeName.TIME);
+    final RelDataType timeType3 =
+        typeFactory.createSqlType(SqlTypeName.TIME, 3);
+    final RelDataType timeType9 =
+        typeFactory.createSqlType(SqlTypeName.TIME, 9);
+    final RelDataType timeType18 =
+        typeFactory.createSqlType(SqlTypeName.TIME, 18);
+    final RexBuilder builder = new RexBuilder(typeFactory);
+
+    // Old way: provide a Calendar
+    final Calendar calendar = Util.calendar();
+    calendar.set(1969, Calendar.JULY, 21, 2, 56, 15); // one small step
+    calendar.set(Calendar.MILLISECOND, 0);
+    checkTime(builder.makeLiteral(calendar, timeType, false));
+
+    // Old way #2: Provide a Long
+    checkTime(builder.makeLiteral(MOON_TIME, timeType, false));
+
+    // The new way
+    final TimeString t = new TimeString(2, 56, 15);
+    assertThat(t.getMillisOfDay(), is(10575000));
+    checkTime(builder.makeLiteral(t, timeType, false));
+
+    // Now with milliseconds
+    final TimeString t2 = t.withMillis(56);
+    assertThat(t2.getMillisOfDay(), is(10575056));
+    assertThat(t2.toString(), is("02:56:15.056"));
+    final RexNode literal2 = builder.makeLiteral(t2, timeType3, false);
+    assertThat(((RexLiteral) literal2).getValueAs(TimeString.class)
+        .toString(), is("02:56:15.056"));
+
+    // Now with nanoseconds
+    final TimeString t3 = t.withNanos(2345678);
+    assertThat(t3.getMillisOfDay(), is(10575002));
+    final RexNode literal3 = builder.makeLiteral(t3, timeType9, false);
+    assertThat(((RexLiteral) literal3).getValueAs(TimeString.class)
+        .toString(), is("02:56:15.002"));
+
+    // Now with a very long fraction
+    final TimeString t4 = t.withFraction("102030405060708090102");
+    assertThat(t4.getMillisOfDay(), is(10575102));
+    final RexNode literal4 = builder.makeLiteral(t4, timeType18, false);
+    assertThat(((RexLiteral) literal4).getValueAs(TimeString.class)
+        .toString(), is("02:56:15.102"));
+
+    // toString
+    assertThat(t2.round(1).toString(), is("02:56:15"));
+    assertThat(t2.round(2).toString(), is("02:56:15.05"));
+    assertThat(t2.round(3).toString(), is("02:56:15.056"));
+    assertThat(t2.round(4).toString(), is("02:56:15.056"));
+
+    assertThat(t2.toString(6), is("02:56:15.056000"));
+    assertThat(t2.toString(1), is("02:56:15.0"));
+    assertThat(t2.toString(0), is("02:56:15"));
+
+    assertThat(t2.round(0).toString(), is("02:56:15"));
+    assertThat(t2.round(0).toString(0), is("02:56:15"));
+    assertThat(t2.round(0).toString(1), is("02:56:15.0"));
+    assertThat(t2.round(0).toString(2), is("02:56:15.00"));
+
+    assertThat(TimeString.fromMillisOfDay(53560123).toString(),
+        is("14:52:40.123"));
+  }
+
+  private void checkTime(RexNode node) {
+    assertThat(node.toString(), is("02:56:15"));
+    RexLiteral literal = (RexLiteral) node;
+    assertThat(literal.getValue() instanceof Calendar, is(true));
+    assertThat(literal.getValue2() instanceof Integer, is(true));
+    assertThat(literal.getValue3() instanceof Integer, is(true));
+    assertThat((Integer) literal.getValue2(), is(MOON_TIME));
+    assertThat(literal.getValueAs(Calendar.class), notNullValue());
+    assertThat(literal.getValueAs(TimeString.class), notNullValue());
+  }
+
+  /** Tests {@link RexBuilder#makeDateLiteral(DateString)}. */
+  @Test public void testDateLiteral() {
+    final RelDataTypeFactory typeFactory =
+        new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
+    RelDataType dateType = typeFactory.createSqlType(SqlTypeName.DATE);
+    final RexBuilder builder = new RexBuilder(typeFactory);
+
+    // Old way: provide a Calendar
+    final Calendar calendar = Util.calendar();
+    calendar.set(1969, Calendar.JULY, 21); // one small step
+    calendar.set(Calendar.MILLISECOND, 0);
+    checkDate(builder.makeLiteral(calendar, dateType, false));
+
+    // Old way #2: Provide in Integer
+    checkDate(builder.makeLiteral(MOON_DAY, dateType, false));
+
+    // The new way
+    final DateString d = new DateString(1969, 7, 21);
+    checkDate(builder.makeLiteral(d, dateType, false));
+  }
+
+  private void checkDate(RexNode node) {
+    assertThat(node.toString(), is("1969-07-21"));
+    RexLiteral literal = (RexLiteral) node;
+    assertThat(literal.getValue() instanceof Calendar, is(true));
+    assertThat(literal.getValue2() instanceof Integer, is(true));
+    assertThat(literal.getValue3() instanceof Integer, is(true));
+    assertThat((Integer) literal.getValue2(), is(MOON_DAY));
+    assertThat(literal.getValueAs(Calendar.class), notNullValue());
+    assertThat(literal.getValueAs(DateString.class), notNullValue());
+  }
+
 }
 
 // End RexBuilderTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/test/java/org/apache/calcite/rex/RexExecutorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/rex/RexExecutorTest.java b/core/src/test/java/org/apache/calcite/rex/RexExecutorTest.java
index 46190df..d421033 100644
--- a/core/src/test/java/org/apache/calcite/rex/RexExecutorTest.java
+++ b/core/src/test/java/org/apache/calcite/rex/RexExecutorTest.java
@@ -36,6 +36,7 @@ import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.util.DateString;
 import org.apache.calcite.util.NlsString;
 import org.apache.calcite.util.Util;
 
@@ -47,7 +48,6 @@ import org.junit.Test;
 
 import java.math.BigDecimal;
 import java.util.ArrayList;
-import java.util.Calendar;
 import java.util.List;
 import java.util.Random;
 
@@ -159,20 +159,22 @@ public class RexExecutorTest {
     checkConstant(true,
         new Function<RexBuilder, RexNode>() {
           public RexNode apply(RexBuilder rexBuilder) {
-            Calendar calendar = Util.calendar();
+            final DateString d =
+                DateString.fromCalendarFields(Util.calendar());
             return rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL,
-                rexBuilder.makeDateLiteral(calendar),
-                rexBuilder.makeDateLiteral(calendar));
+                rexBuilder.makeDateLiteral(d),
+                rexBuilder.makeDateLiteral(d));
           }
         });
     // date 'today' < date 'today' -> false
     checkConstant(false,
         new Function<RexBuilder, RexNode>() {
           public RexNode apply(RexBuilder rexBuilder) {
-            Calendar calendar = Util.calendar();
+            final DateString d =
+                DateString.fromCalendarFields(Util.calendar());
             return rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN,
-                rexBuilder.makeDateLiteral(calendar),
-                rexBuilder.makeDateLiteral(calendar));
+                rexBuilder.makeDateLiteral(d),
+                rexBuilder.makeDateLiteral(d));
           }
         });
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
index f4baa86..95d70d3 100644
--- a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
@@ -2230,13 +2230,13 @@ public class SqlParserTest {
     checkExpSame("TIMESTAMP '2004-06-01 15:55:55.900'");
     checkExp(
         "TIMESTAMP '2004-06-01 15:55:55.1234'",
-        "TIMESTAMP '2004-06-01 15:55:55.123'");
+        "TIMESTAMP '2004-06-01 15:55:55.1234'");
     checkExp(
         "TIMESTAMP '2004-06-01 15:55:55.1236'",
-        "TIMESTAMP '2004-06-01 15:55:55.123'");
+        "TIMESTAMP '2004-06-01 15:55:55.1236'");
     checkExp(
         "TIMESTAMP '2004-06-01 15:55:55.9999'",
-        "TIMESTAMP '2004-06-01 15:55:55.999'");
+        "TIMESTAMP '2004-06-01 15:55:55.9999'");
     checkExpSame("NULL");
   }
 
@@ -3690,6 +3690,7 @@ public class SqlParserTest {
     checkExp("TIME '12:01:01.'", "TIME '12:01:01'");
     checkExp("TIME '12:01:01.000'", "TIME '12:01:01.000'");
     checkExp("TIME '12:01:01.001'", "TIME '12:01:01.001'");
+    checkExp("TIME '12:01:01.01023456789'", "TIME '12:01:01.01023456789'");
 
     // Timestamp literals
     checkExp(
@@ -3701,6 +3702,10 @@ public class SqlParserTest {
     checkExp(
         "TIMESTAMP '2004-12-01 12:01:01.'",
         "TIMESTAMP '2004-12-01 12:01:01'");
+    checkExp(
+        "TIMESTAMP  '2004-12-01 12:01:01.010234567890'",
+        "TIMESTAMP '2004-12-01 12:01:01.010234567890'");
+    checkExpSame("TIMESTAMP '2004-12-01 12:01:01.01023456789'");
 
     // Failures.
     checkFails("^DATE '12/21/99'^", "(?s).*Illegal DATE literal.*");

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/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 63443cb..d3e19dc 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
@@ -51,6 +51,7 @@ import org.apache.calcite.test.SqlLimitsTest;
 import org.apache.calcite.util.Bug;
 import org.apache.calcite.util.Holder;
 import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.TimestampString;
 import org.apache.calcite.util.Util;
 
 import com.google.common.base.Function;
@@ -6883,9 +6884,8 @@ public abstract class SqlOperatorBaseTest {
       case VARCHAR:
         return SqlLiteral.createCharString(value.toString(), SqlParserPos.ZERO);
       case TIMESTAMP:
-        Calendar calendar = Util.calendar();
-        calendar.setTimeInMillis((Long) value);
-        return SqlLiteral.createTimestamp(calendar, type.getPrecision(),
+        TimestampString ts = TimestampString.fromMillisSinceEpoch((Long) value);
+        return SqlLiteral.createTimestamp(ts, type.getPrecision(),
             SqlParserPos.ZERO);
       default:
         throw new AssertionError(type);

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/test/java/org/apache/calcite/test/RexImplicationCheckerTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RexImplicationCheckerTest.java b/core/src/test/java/org/apache/calcite/test/RexImplicationCheckerTest.java
index ee2511d..f089170 100644
--- a/core/src/test/java/org/apache/calcite/test/RexImplicationCheckerTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RexImplicationCheckerTest.java
@@ -36,8 +36,11 @@ import org.apache.calcite.server.CalciteServerStatement;
 import org.apache.calcite.sql.SqlCollation;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.util.DateString;
 import org.apache.calcite.util.Holder;
 import org.apache.calcite.util.NlsString;
+import org.apache.calcite.util.TimeString;
+import org.apache.calcite.util.TimestampString;
 import org.apache.calcite.util.Util;
 
 import org.junit.Ignore;
@@ -50,7 +53,6 @@ import java.math.BigDecimal;
 import java.sql.Date;
 import java.sql.Time;
 import java.sql.Timestamp;
-import java.util.Calendar;
 
 /**
  * Unit tests for {@link RexImplicationChecker}.
@@ -179,9 +181,9 @@ public class RexImplicationCheckerTest {
   @Ignore("work in progress")
   @Test public void testSimpleDate() {
     final Fixture f = new Fixture();
-    final Calendar instance = Util.calendar();
-    final RexNode node1 = f.ge(f.dt, f.rexBuilder.makeDateLiteral(instance));
-    final RexNode node2 = f.eq(f.dt, f.rexBuilder.makeDateLiteral(instance));
+    final DateString d = DateString.fromCalendarFields(Util.calendar());
+    final RexNode node1 = f.ge(f.dt, f.rexBuilder.makeDateLiteral(d));
+    final RexNode node2 = f.eq(f.dt, f.rexBuilder.makeDateLiteral(d));
 
     f.checkImplies(node2, node1);
     f.checkNotImplies(node1, node2);
@@ -190,9 +192,10 @@ public class RexImplicationCheckerTest {
   @Ignore("work in progress")
   @Test public void testSimpleTimeStamp() {
     final Fixture f = new Fixture();
-    final Calendar calendar = Util.calendar();
-    final RexNode node1 = f.le(f.ts, f.timestampLiteral(calendar));
-    final RexNode node2 = f.le(f.ts, f.timestampLiteral(calendar));
+    final TimestampString ts =
+        TimestampString.fromCalendarFields(Util.calendar());
+    final RexNode node1 = f.le(f.ts, f.timestampLiteral(ts));
+    final RexNode node2 = f.le(f.ts, f.timestampLiteral(ts));
 
     f.checkImplies(node1, node2);
     f.checkNotImplies(node2, node1);
@@ -201,9 +204,9 @@ public class RexImplicationCheckerTest {
   @Ignore("work in progress")
   @Test public void testSimpleTime() {
     final Fixture f = new Fixture();
-    final Calendar calendar = Util.calendar();
-    final RexNode node1 = f.le(f.ts, f.timeLiteral(calendar));
-    final RexNode node2 = f.le(f.ts, f.timeLiteral(calendar));
+    final TimeString t = TimeString.fromCalendarFields(Util.calendar());
+    final RexNode node1 = f.le(f.ts, f.timeLiteral(t));
+    final RexNode node2 = f.le(f.ts, f.timeLiteral(t));
 
     f.checkImplies(node1, node2);
     f.checkNotImplies(node2, node1);
@@ -491,14 +494,13 @@ public class RexImplicationCheckerTest {
           new NlsString(z, null, SqlCollation.COERCIBLE));
     }
 
-    public RexNode timestampLiteral(Calendar calendar) {
-      return rexBuilder.makeTimestampLiteral(
-          calendar, timeStampDataType.getPrecision());
+    public RexNode timestampLiteral(TimestampString ts) {
+      return rexBuilder.makeTimestampLiteral(ts,
+          timeStampDataType.getPrecision());
     }
 
-    public RexNode timeLiteral(Calendar calendar) {
-      return rexBuilder.makeTimestampLiteral(
-          calendar, timeDataType.getPrecision());
+    public RexNode timeLiteral(TimeString t) {
+      return rexBuilder.makeTimeLiteral(t, timeDataType.getPrecision());
     }
 
     public RexNode cast(RelDataType type, RexNode exp) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RexProgramTest.java b/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
index 408cb73..f889347 100644
--- a/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
@@ -38,9 +38,12 @@ import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.type.SqlTypeAssignmentRules;
 import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.DateString;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.NlsString;
 import org.apache.calcite.util.TestUtil;
+import org.apache.calcite.util.TimeString;
+import org.apache.calcite.util.TimestampString;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
@@ -1397,12 +1400,11 @@ public class RexProgramTest {
     literals.add((RexLiteral)
         rexBuilder.makeLiteral(new ByteString(new byte[] {1, 2, -34, 0, -128}),
             typeFactory.createSqlType(SqlTypeName.BINARY, 5), false));
-    literals.add(
-        rexBuilder.makeDateLiteral(cal(1974, Calendar.AUGUST, 9, 0, 0, 0)));
-    literals.add(rexBuilder.makeTimeLiteral(cal(0, 0, 0, 1, 23, 45), 0));
+    literals.add(rexBuilder.makeDateLiteral(new DateString(1974, 8, 9)));
+    literals.add(rexBuilder.makeTimeLiteral(new TimeString(1, 23, 45), 0));
     literals.add(
         rexBuilder.makeTimestampLiteral(
-            cal(1974, Calendar.AUGUST, 9, 1, 23, 45), 0));
+            new TimestampString(1974, 8, 9, 1, 23, 45), 0));
 
     final Multimap<SqlTypeName, RexLiteral> map = LinkedHashMultimap.create();
     for (RexLiteral literal : literals) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/test/resources/sql/misc.iq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/misc.iq b/core/src/test/resources/sql/misc.iq
index cbf12fe..9ecd55d 100644
--- a/core/src/test/resources/sql/misc.iq
+++ b/core/src/test/resources/sql/misc.iq
@@ -1910,7 +1910,7 @@ select TIMESTAMP '2016-02-26 19:06:00.123456789',
 (1 row)
 
 !ok
-EnumerableCalc(expr#0=[{inputs}], expr#1=[2016-02-26 19:06:00.123], expr#2=[2016-02-26 19:06:00], expr#3=[2016-02-26 19:06:00.1], expr#4=[2016-02-26 19:06:00.12], expr#5=[2016-02-26 19:06:00.123], expr#6=[2016-02-26 19:06:00.123], EXPR$0=[$t1], EXPR$1=[$t2], EXPR$2=[$t2], EXPR$3=[$t3], EXPR$4=[$t4], EXPR$5=[$t1], EXPR$6=[$t5], EXPR$7=[$t6])
+EnumerableCalc(expr#0=[{inputs}], expr#1=[2016-02-26 19:06:00.123], expr#2=[2016-02-26 19:06:00], expr#3=[2016-02-26 19:06:00.1], expr#4=[2016-02-26 19:06:00.12], EXPR$0=[$t1], EXPR$1=[$t2], EXPR$2=[$t2], EXPR$3=[$t3], EXPR$4=[$t4], EXPR$5=[$t1], EXPR$6=[$t1], EXPR$7=[$t1])
   EnumerableValues(tuples=[[{ 0 }]])
 !plan
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/druid/src/main/java/org/apache/calcite/adapter/druid/DruidDateTimeUtils.java
----------------------------------------------------------------------
diff --git a/druid/src/main/java/org/apache/calcite/adapter/druid/DruidDateTimeUtils.java b/druid/src/main/java/org/apache/calcite/adapter/druid/DruidDateTimeUtils.java
index 0e2b3d3..d6065e2 100644
--- a/druid/src/main/java/org/apache/calcite/adapter/druid/DruidDateTimeUtils.java
+++ b/druid/src/main/java/org/apache/calcite/adapter/druid/DruidDateTimeUtils.java
@@ -24,6 +24,8 @@ import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.DateString;
+import org.apache.calcite.util.TimestampString;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.trace.CalciteTrace;
 
@@ -37,9 +39,7 @@ import com.google.common.collect.TreeRangeSet;
 import org.slf4j.Logger;
 
 import java.util.ArrayList;
-import java.util.Calendar;
 import java.util.List;
-import java.util.regex.Pattern;
 
 /**
  * Utilities for generating intervals from RexNode.
@@ -49,10 +49,6 @@ public class DruidDateTimeUtils {
 
   protected static final Logger LOGGER = CalciteTrace.getPlannerTracer();
 
-  private static final Pattern TIMESTAMP_PATTERN =
-      Pattern.compile("[0-9][0-9][0-9][0-9]-[0-9][0-9]-[0-9][0-9]"
-          + " [0-9][0-9]:[0-9][0-9]:[0-9][0-9]");
-
   private DruidDateTimeUtils() {
   }
 
@@ -63,7 +59,7 @@ public class DruidDateTimeUtils {
    */
   public static List<LocalInterval> createInterval(RelDataType type,
       RexNode e) {
-    final List<Range<Calendar>> ranges = extractRanges(e, false);
+    final List<Range<TimestampString>> ranges = extractRanges(e, false);
     if (ranges == null) {
       // We did not succeed, bail out
       return null;
@@ -78,18 +74,18 @@ public class DruidDateTimeUtils {
     return toInterval(ImmutableList.<Range>copyOf(condensedRanges.asRanges()));
   }
 
-  protected static List<LocalInterval> toInterval(List<Range<Calendar>> ranges) {
+  protected static List<LocalInterval> toInterval(List<Range<TimestampString>> ranges) {
     List<LocalInterval> intervals = Lists.transform(ranges,
-        new Function<Range<Calendar>, LocalInterval>() {
-          public LocalInterval apply(Range<Calendar> range) {
+        new Function<Range<TimestampString>, LocalInterval>() {
+          public LocalInterval apply(Range<TimestampString> range) {
             if (!range.hasLowerBound() && !range.hasUpperBound()) {
               return DruidTable.DEFAULT_INTERVAL;
             }
             long start = range.hasLowerBound()
-                ? range.lowerEndpoint().getTime().getTime()
+                ? range.lowerEndpoint().getMillisSinceEpoch()
                 : DruidTable.DEFAULT_INTERVAL.getStartMillis();
             long end = range.hasUpperBound()
-                ? range.upperEndpoint().getTime().getTime()
+                ? range.upperEndpoint().getMillisSinceEpoch()
                 : DruidTable.DEFAULT_INTERVAL.getEndMillis();
             if (range.hasLowerBound()
                 && range.lowerBoundType() == BoundType.OPEN) {
@@ -108,7 +104,7 @@ public class DruidDateTimeUtils {
     return intervals;
   }
 
-  protected static List<Range<Calendar>> extractRanges(RexNode node,
+  protected static List<Range<TimestampString>> extractRanges(RexNode node,
       boolean withNot) {
     switch (node.getKind()) {
     case EQUALS:
@@ -125,9 +121,9 @@ public class DruidDateTimeUtils {
 
     case OR: {
       RexCall call = (RexCall) node;
-      List<Range<Calendar>> intervals = Lists.newArrayList();
+      List<Range<TimestampString>> intervals = Lists.newArrayList();
       for (RexNode child : call.getOperands()) {
-        List<Range<Calendar>> extracted = extractRanges(child, withNot);
+        List<Range<TimestampString>> extracted = extractRanges(child, withNot);
         if (extracted != null) {
           intervals.addAll(extracted);
         }
@@ -137,9 +133,9 @@ public class DruidDateTimeUtils {
 
     case AND: {
       RexCall call = (RexCall) node;
-      List<Range<Calendar>> ranges = new ArrayList<>();
+      List<Range<TimestampString>> ranges = new ArrayList<>();
       for (RexNode child : call.getOperands()) {
-        List<Range<Calendar>> extractedRanges = extractRanges(child, false);
+        List<Range<TimestampString>> extractedRanges = extractRanges(child, false);
         if (extractedRanges == null || extractedRanges.isEmpty()) {
           // We could not extract, we bail out
           return null;
@@ -148,7 +144,7 @@ public class DruidDateTimeUtils {
           ranges.addAll(extractedRanges);
           continue;
         }
-        List<Range<Calendar>> overlapped = new ArrayList<>();
+        List<Range<TimestampString>> overlapped = new ArrayList<>();
         for (Range current : ranges) {
           for (Range interval : extractedRanges) {
             if (current.isConnected(interval)) {
@@ -166,7 +162,7 @@ public class DruidDateTimeUtils {
     }
   }
 
-  protected static List<Range<Calendar>> leafToRanges(RexCall call,
+  protected static List<Range<TimestampString>> leafToRanges(RexCall call,
       boolean withNot) {
     switch (call.getKind()) {
     case EQUALS:
@@ -175,7 +171,7 @@ public class DruidDateTimeUtils {
     case GREATER_THAN:
     case GREATER_THAN_OR_EQUAL:
     {
-      final Calendar value;
+      final TimestampString value;
       if (call.getOperands().get(0) instanceof RexInputRef
           && literalValue(call.getOperands().get(1)) != null) {
         value = literalValue(call.getOperands().get(1));
@@ -203,8 +199,8 @@ public class DruidDateTimeUtils {
     }
     case BETWEEN:
     {
-      final Calendar value1;
-      final Calendar value2;
+      final TimestampString value1;
+      final TimestampString value2;
       if (literalValue(call.getOperands().get(2)) != null
           && literalValue(call.getOperands().get(3)) != null) {
         value1 = literalValue(call.getOperands().get(2));
@@ -223,9 +219,9 @@ public class DruidDateTimeUtils {
     }
     case IN:
     {
-      ImmutableList.Builder<Range<Calendar>> ranges = ImmutableList.builder();
+      ImmutableList.Builder<Range<TimestampString>> ranges = ImmutableList.builder();
       for (RexNode operand : Util.skip(call.operands)) {
-        final Calendar element = literalValue(operand);
+        final TimestampString element = literalValue(operand);
         if (element == null) {
           return null;
         }
@@ -243,13 +239,16 @@ public class DruidDateTimeUtils {
     }
   }
 
-  private static Calendar literalValue(RexNode node) {
+  private static TimestampString literalValue(RexNode node) {
     switch (node.getKind()) {
     case LITERAL:
-      assert node instanceof RexLiteral;
-      Object value = ((RexLiteral) node).getValue();
-      if (value instanceof  Calendar) {
-        return (Calendar) value;
+      switch (((RexLiteral) node).getTypeName()) {
+      case TIMESTAMP:
+        return ((RexLiteral) node).getValueAs(TimestampString.class);
+      case DATE:
+        // For uniformity, treat dates as timestamps
+        final DateString d = ((RexLiteral) node).getValueAs(DateString.class);
+        return TimestampString.fromMillisSinceEpoch(d.getMillisSinceEpoch());
       }
       break;
     case CAST:

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/druid/src/test/java/org/apache/calcite/test/DruidDateRangeRulesTest.java
----------------------------------------------------------------------
diff --git a/druid/src/test/java/org/apache/calcite/test/DruidDateRangeRulesTest.java b/druid/src/test/java/org/apache/calcite/test/DruidDateRangeRulesTest.java
index b2cf321..74ce10c 100644
--- a/druid/src/test/java/org/apache/calcite/test/DruidDateRangeRulesTest.java
+++ b/druid/src/test/java/org/apache/calcite/test/DruidDateRangeRulesTest.java
@@ -23,6 +23,7 @@ import org.apache.calcite.rel.rules.DateRangeRules;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.test.RexImplicationCheckerTest.Fixture;
+import org.apache.calcite.util.TimestampString;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
@@ -123,12 +124,13 @@ public class DruidDateRangeRulesTest {
    * Push CAST of literals to Druid</a>. */
   @Test public void testFilterWithCast() {
     final Fixture2 f = new Fixture2();
-    Calendar from = Util.calendar();
-    from.clear();
-    from.set(2010, Calendar.JANUARY, 1);
-    Calendar to = Util.calendar();
-    to.clear();
-    to.set(2011, Calendar.JANUARY, 1);
+    final Calendar c = Util.calendar();
+    c.clear();
+    c.set(2010, Calendar.JANUARY, 1);
+    final TimestampString from = TimestampString.fromCalendarFields(c);
+    c.clear();
+    c.set(2011, Calendar.JANUARY, 1);
+    final TimestampString to = TimestampString.fromCalendarFields(c);
 
     // dt >= 2010-01-01 AND dt < 2011-01-01
     checkDateRangeNoSimplify(f,


[4/5] calcite git commit: Upgrade sqlline

Posted by jh...@apache.org.
Upgrade sqlline


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

Branch: refs/heads/master
Commit: f22a9965a4e0f97b4f60c29ad52c0328a6a1e221
Parents: 946b24f
Author: Julian Hyde <jh...@apache.org>
Authored: Tue May 30 20:14:10 2017 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed May 31 08:17:28 2017 -0700

----------------------------------------------------------------------
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/f22a9965/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1a9d420..d2e9a3a 100644
--- a/pom.xml
+++ b/pom.xml
@@ -118,7 +118,7 @@ limitations under the License.
     <servlet.version>3.0.1</servlet.version>
     <slf4j.version>1.7.13</slf4j.version>
     <spark.version>1.6.1</spark.version>
-    <sqlline.version>1.2.0</sqlline.version>
+    <sqlline.version>1.3.0</sqlline.version>
     <xalan.version>2.7.1</xalan.version>
     <xerces.version>2.9.1</xerces.version>
   </properties>


[3/5] calcite git commit: [CALCITE-1690] Calcite timestamp literals cannot express precision above millisecond, TIMESTAMP(3)

Posted by jh...@apache.org.
[CALCITE-1690] Calcite timestamp literals cannot express precision above millisecond, TIMESTAMP(3)


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

Branch: refs/heads/master
Commit: 205af8134857ba312415b9a5b4a48020fe0ce888
Parents: 915c218
Author: Julian Hyde <jh...@apache.org>
Authored: Mon May 1 21:53:12 2017 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed May 31 08:17:28 2017 -0700

----------------------------------------------------------------------
 core/src/main/codegen/templates/Parser.jj       |  73 +----
 .../adapter/enumerable/RexToLixTranslator.java  |  40 +--
 .../apache/calcite/interpreter/Interpreter.java |   2 +-
 .../apache/calcite/interpreter/SortNode.java    |   5 +-
 .../java/org/apache/calcite/plan/Strong.java    |   2 +-
 .../calcite/rel/metadata/RelMdCollation.java    |   8 +-
 .../rel/metadata/RelMdColumnUniqueness.java     |   4 +-
 .../apache/calcite/rel/metadata/RelMdSize.java  |   6 +-
 .../apache/calcite/rel/metadata/RelMdUtil.java  |  19 +-
 .../calcite/rel/rel2sql/SqlImplementor.java     |  25 +-
 .../calcite/rel/rules/DateRangeRules.java       |   7 +-
 .../java/org/apache/calcite/rex/RexBuilder.java | 124 +++++----
 .../java/org/apache/calcite/rex/RexLiteral.java | 275 +++++++++++++++----
 .../org/apache/calcite/rex/RexSimplify.java     |  10 +-
 .../calcite/rex/RexToSqlNodeConverterImpl.java  |  18 +-
 .../calcite/sql/SqlAbstractDateTimeLiteral.java |  81 +-----
 .../org/apache/calcite/sql/SqlDateLiteral.java  |  23 +-
 .../java/org/apache/calcite/sql/SqlLiteral.java |  96 ++++++-
 .../org/apache/calcite/sql/SqlTimeLiteral.java  |  48 +---
 .../apache/calcite/sql/SqlTimestampLiteral.java |  49 +---
 .../calcite/sql/parser/SqlParserUtil.java       |  85 ++++++
 .../calcite/sql/type/SqlTypeFactoryImpl.java    |   8 +
 .../apache/calcite/sql/type/SqlTypeName.java    |  15 +-
 .../calcite/sql/validate/SqlValidatorImpl.java  |   2 +-
 .../sql2rel/SqlNodeToRexConverterImpl.java      |  35 +--
 .../org/apache/calcite/util/BasicDatetime.java  |  63 -----
 .../org/apache/calcite/util/DateString.java     |  98 +++++++
 .../org/apache/calcite/util/TimeString.java     | 192 +++++++++++++
 .../apache/calcite/util/TimestampString.java    | 252 +++++++++++++++++
 .../main/java/org/apache/calcite/util/Util.java |  20 +-
 .../org/apache/calcite/util/ZonelessDate.java   | 135 ---------
 .../apache/calcite/util/ZonelessDatetime.java   | 199 --------------
 .../org/apache/calcite/util/ZonelessTime.java   | 175 ------------
 .../apache/calcite/util/ZonelessTimestamp.java  | 160 -----------
 .../rel/rel2sql/RelToSqlConverterTest.java      |  20 ++
 .../org/apache/calcite/rex/RexBuilderTest.java  | 209 +++++++++++++-
 .../org/apache/calcite/rex/RexExecutorTest.java |  16 +-
 .../calcite/sql/parser/SqlParserTest.java       |  11 +-
 .../calcite/sql/test/SqlOperatorBaseTest.java   |   6 +-
 .../calcite/test/RexImplicationCheckerTest.java |  34 +--
 .../org/apache/calcite/test/RexProgramTest.java |  10 +-
 core/src/test/resources/sql/misc.iq             |   2 +-
 .../adapter/druid/DruidDateTimeUtils.java       |  57 ++--
 .../calcite/test/DruidDateRangeRulesTest.java   |  14 +-
 44 files changed, 1505 insertions(+), 1228 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/main/codegen/templates/Parser.jj
----------------------------------------------------------------------
diff --git a/core/src/main/codegen/templates/Parser.jj b/core/src/main/codegen/templates/Parser.jj
index b776e92..3543da6 100644
--- a/core/src/main/codegen/templates/Parser.jj
+++ b/core/src/main/codegen/templates/Parser.jj
@@ -411,63 +411,6 @@ JAVACODE void checkNonQueryExpression(ExprContext exprContext)
     }
 }
 
-// The date/time parse utilities have to live here, instead of in the
-// SqlParserUtil class because ParseException is ambiguous, and
-// CommonParser has to live in multiple packages.
-
-JAVACODE SqlDateLiteral parseDateLiteral(String s, SqlParserPos pos) {
-    String dateStr = SqlParserUtil.parseString(s);
-    Calendar cal = DateTimeUtils.parseDateFormat(
-        dateStr, DateTimeUtils.DATE_FORMAT_STRING, DateTimeUtils.UTC_ZONE);
-    if (null == cal) {
-        throw SqlUtil.newContextException(pos,
-            RESOURCE.illegalLiteral("DATE", s,
-                RESOURCE.badFormat(DateTimeUtils.DATE_FORMAT_STRING).str()));
-    }
-    return SqlLiteral.createDate(cal, pos);
-}
-
-JAVACODE SqlTimeLiteral parseTimeLiteral(String s, SqlParserPos pos) {
-    String dateStr = SqlParserUtil.parseString(s);
-    DateTimeUtils.PrecisionTime pt =
-    DateTimeUtils.parsePrecisionDateTimeLiteral(
-        dateStr, DateTimeUtils.TIME_FORMAT_STRING, DateTimeUtils.UTC_ZONE);
-    if (null == pt) {
-        throw SqlUtil.newContextException(pos,
-            RESOURCE.illegalLiteral("TIME", s,
-                RESOURCE.badFormat(DateTimeUtils.TIME_FORMAT_STRING).str()));
-    }
-    return SqlLiteral.createTime(pt.getCalendar(), pt.getPrecision(), pos);
-}
-
-JAVACODE SqlTimestampLiteral parseTimestampLiteral(String s, SqlParserPos pos) {
-    String dateStr = SqlParserUtil.parseString(s);
-    DateTimeUtils.PrecisionTime pt =
-    DateTimeUtils.parsePrecisionDateTimeLiteral(
-        dateStr, DateTimeUtils.TIMESTAMP_FORMAT_STRING, DateTimeUtils.UTC_ZONE);
-    if (null == pt) {
-        throw SqlUtil.newContextException(pos,
-            RESOURCE.illegalLiteral("TIMESTAMP", s,
-                RESOURCE.badFormat(DateTimeUtils.TIMESTAMP_FORMAT_STRING).str()));
-    }
-    return SqlLiteral.createTimestamp(pt.getCalendar(), pt.getPrecision(), pos);
-}
-
-JAVACODE SqlIntervalLiteral parseIntervalLiteral(
-    SqlParserPos pos,
-    int sign,
-    String s,
-    SqlIntervalQualifier intervalQualifier) throws ParseException
-{
-    String intervalStr = SqlParserUtil.parseString(s);
-    if ("".equals(intervalStr)) {
-        throw new ParseException(
-            RESOURCE.illegalIntervalLiteral(s + " "
-                + intervalQualifier.toString(), pos.toString()).str());
-    }
-    return SqlLiteral.createInterval(sign, intervalStr, intervalQualifier, pos);
-}
-
 /**
  * Converts a ParseException (local to this particular instantiation
  * of the parser) into a SqlParseException (common to all parsers).
@@ -3693,33 +3636,33 @@ SqlLiteral DateTimeLiteral() :
         p = token.image;
     }
     <RBRACE> {
-        return parseDateLiteral(p, getPos());
+        return SqlParserUtil.parseDateLiteral(p, getPos());
     }
 |
     <LBRACE_T> <QUOTED_STRING> {
         p = token.image;
     }
     <RBRACE> {
-        return parseTimeLiteral(p, getPos());
+        return SqlParserUtil.parseTimeLiteral(p, getPos());
     }
 |
     <LBRACE_TS> { s = span(); } <QUOTED_STRING> {
         p = token.image;
     }
     <RBRACE> {
-        return parseTimestampLiteral(p, s.end(this));
+        return SqlParserUtil.parseTimestampLiteral(p, s.end(this));
     }
 |
     <DATE> { s = span(); } <QUOTED_STRING> {
-        return parseDateLiteral(token.image, s.end(this));
+        return SqlParserUtil.parseDateLiteral(token.image, s.end(this));
     }
 |
     <TIME> { s = span(); } <QUOTED_STRING> {
-        return parseTimeLiteral(token.image, s.end(this));
+        return SqlParserUtil.parseTimeLiteral(token.image, s.end(this));
     }
 |
     <TIMESTAMP> { s = span(); } <QUOTED_STRING> {
-        return parseTimestampLiteral(token.image, s.end(this));
+        return SqlParserUtil.parseTimestampLiteral(token.image, s.end(this));
     }
 }
 
@@ -3862,8 +3805,8 @@ SqlLiteral IntervalLiteral() :
     ]
     <QUOTED_STRING> { p = token.image; }
     intervalQualifier = IntervalQualifier() {
-        return parseIntervalLiteral(s.end(intervalQualifier), sign, p,
-            intervalQualifier);
+        return SqlParserUtil.parseIntervalLiteral(s.end(intervalQualifier),
+            sign, p, intervalQualifier);
     }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
index 7d5ef75..10e969f 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
@@ -48,7 +48,6 @@ import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.util.BuiltInMethod;
 import org.apache.calcite.util.ControlFlowException;
-import org.apache.calcite.util.NlsString;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
@@ -59,7 +58,6 @@ import java.lang.reflect.Modifier;
 import java.lang.reflect.Type;
 import java.math.BigDecimal;
 import java.util.ArrayList;
-import java.util.Calendar;
 import java.util.Collections;
 import java.util.LinkedHashMap;
 import java.util.List;
@@ -588,8 +586,7 @@ public class RexToLixTranslator {
       RelDataType type,
       JavaTypeFactory typeFactory,
       RexImpTable.NullAs nullAs) {
-    final Comparable value = literal.getValue();
-    if (value == null) {
+    if (literal.isNull()) {
       switch (nullAs) {
       case TRUE:
       case IS_NULL:
@@ -615,26 +612,22 @@ public class RexToLixTranslator {
     final Object value2;
     switch (literal.getType().getSqlTypeName()) {
     case DECIMAL:
+      final BigDecimal bd = literal.getValueAs(BigDecimal.class);
       if (javaClass == float.class) {
-        return Expressions.constant(value, javaClass);
+        return Expressions.constant(bd, javaClass);
       }
       assert javaClass == BigDecimal.class;
       return Expressions.new_(BigDecimal.class,
-          Expressions.constant(value.toString()));
+          Expressions.constant(bd.toString()));
     case DATE:
-      value2 = (int)
-          (((Calendar) value).getTimeInMillis() / DateTimeUtils.MILLIS_PER_DAY);
-      javaClass = int.class;
-      break;
     case TIME:
-      value2 = (int)
-          (((Calendar) value).getTimeInMillis() % DateTimeUtils.MILLIS_PER_DAY);
+    case INTERVAL_YEAR:
+    case INTERVAL_YEAR_MONTH:
+    case INTERVAL_MONTH:
+      value2 = literal.getValueAs(Integer.class);
       javaClass = int.class;
       break;
     case TIMESTAMP:
-      value2 = ((Calendar) value).getTimeInMillis();
-      javaClass = long.class;
-      break;
     case INTERVAL_DAY:
     case INTERVAL_DAY_HOUR:
     case INTERVAL_DAY_MINUTE:
@@ -645,32 +638,27 @@ public class RexToLixTranslator {
     case INTERVAL_MINUTE:
     case INTERVAL_MINUTE_SECOND:
     case INTERVAL_SECOND:
-      value2 = ((BigDecimal) value).longValue();
+      value2 = literal.getValueAs(Long.class);
       javaClass = long.class;
       break;
-    case INTERVAL_YEAR:
-    case INTERVAL_YEAR_MONTH:
-    case INTERVAL_MONTH:
-      value2 = ((BigDecimal) value).intValue();
-      javaClass = int.class;
-      break;
     case CHAR:
     case VARCHAR:
-      value2 = ((NlsString) value).getValue();
+      value2 = literal.getValueAs(String.class);
       break;
     case BINARY:
     case VARBINARY:
       return Expressions.new_(
           ByteString.class,
           Expressions.constant(
-              ((ByteString) value).getBytes(),
+              literal.getValueAs(byte[].class),
               byte[].class));
     case SYMBOL:
-      value2 = value;
-      javaClass = value.getClass();
+      value2 = literal.getValueAs(Enum.class);
+      javaClass = value2.getClass();
       break;
     default:
       final Primitive primitive = Primitive.ofBoxOr(javaClass);
+      final Comparable value = literal.getValueAs(Comparable.class);
       if (primitive != null && value instanceof Number) {
         value2 = primitive.number((Number) value);
       } else {

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/main/java/org/apache/calcite/interpreter/Interpreter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/interpreter/Interpreter.java b/core/src/main/java/org/apache/calcite/interpreter/Interpreter.java
index 51aa082..16b70de 100644
--- a/core/src/main/java/org/apache/calcite/interpreter/Interpreter.java
+++ b/core/src/main/java/org/apache/calcite/interpreter/Interpreter.java
@@ -238,7 +238,7 @@ public class Interpreter extends AbstractEnumerable<Object[]>
         public Object execute(Context context) {
           switch (node.getKind()) {
           case LITERAL:
-            return ((RexLiteral) node).getValue();
+            return ((RexLiteral) node).getValueAs(Comparable.class);
           case INPUT_REF:
             return context.values[((RexInputRef) node).getIndex()];
           default:

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/main/java/org/apache/calcite/interpreter/SortNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/interpreter/SortNode.java b/core/src/main/java/org/apache/calcite/interpreter/SortNode.java
index e2c0eeb..5eaaaf6 100644
--- a/core/src/main/java/org/apache/calcite/interpreter/SortNode.java
+++ b/core/src/main/java/org/apache/calcite/interpreter/SortNode.java
@@ -25,7 +25,6 @@ import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Ordering;
 
-import java.math.BigDecimal;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
@@ -43,11 +42,11 @@ public class SortNode extends AbstractSingleNode<Sort> {
     final int offset =
         rel.offset == null
             ? 0
-            : ((BigDecimal) ((RexLiteral) rel.offset).getValue()).intValue();
+            : ((RexLiteral) rel.offset).getValueAs(Integer.class);
     final int fetch =
         rel.fetch == null
             ? -1
-            : ((BigDecimal) ((RexLiteral) rel.fetch).getValue()).intValue();
+            : ((RexLiteral) rel.fetch).getValueAs(Integer.class);
     // In pure limit mode. No sort required.
     Row row;
   loop:

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/main/java/org/apache/calcite/plan/Strong.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/Strong.java b/core/src/main/java/org/apache/calcite/plan/Strong.java
index 99a1b4e..e631d90 100644
--- a/core/src/main/java/org/apache/calcite/plan/Strong.java
+++ b/core/src/main/java/org/apache/calcite/plan/Strong.java
@@ -118,7 +118,7 @@ public class Strong {
 
     switch (node.getKind()) {
     case LITERAL:
-      return ((RexLiteral) node).getValue() == null;
+      return ((RexLiteral) node).isNull();
     // We can only guarantee AND to return NULL if both inputs are NULL  (similar for OR)
     // AND(NULL, FALSE) = FALSE
     case AND:

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/main/java/org/apache/calcite/rel/metadata/RelMdCollation.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdCollation.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdCollation.java
index 2867657..32596ed 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdCollation.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdCollation.java
@@ -335,16 +335,16 @@ public class RelMdCollation
     case ASCENDING:
       return new Ordering<List<RexLiteral>>() {
         public int compare(List<RexLiteral> o1, List<RexLiteral> o2) {
-          final Comparable c1 = o1.get(x).getValue();
-          final Comparable c2 = o2.get(x).getValue();
+          final Comparable c1 = o1.get(x).getValueAs(Comparable.class);
+          final Comparable c2 = o2.get(x).getValueAs(Comparable.class);
           return RelFieldCollation.compare(c1, c2, nullComparison);
         }
       };
     default:
       return new Ordering<List<RexLiteral>>() {
         public int compare(List<RexLiteral> o1, List<RexLiteral> o2) {
-          final Comparable c1 = o1.get(x).getValue();
-          final Comparable c2 = o2.get(x).getValue();
+          final Comparable c1 = o1.get(x).getValueAs(Comparable.class);
+          final Comparable c2 = o2.get(x).getValueAs(Comparable.class);
           return RelFieldCollation.compare(c2, c1, -nullComparison);
         }
       };

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnUniqueness.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnUniqueness.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnUniqueness.java
index 9b5d38c..e7032c6 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnUniqueness.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnUniqueness.java
@@ -333,7 +333,9 @@ public class RelMdColumnUniqueness
     final List<Comparable> values = new ArrayList<>();
     for (ImmutableList<RexLiteral> tuple : rel.tuples) {
       for (RexLiteral literal : tuple) {
-        values.add(NullSentinel.mask(literal.getValue()));
+        values.add(literal.isNull()
+            ? NullSentinel.INSTANCE
+            : literal.getValueAs(Comparable.class));
       }
       if (!set.add(ImmutableList.copyOf(values))) {
         return false;

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/main/java/org/apache/calcite/rel/metadata/RelMdSize.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdSize.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdSize.java
index 744d241..bd12347 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdSize.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdSize.java
@@ -145,7 +145,8 @@ public class RelMdSize implements MetadataHandler<BuiltInMetadata.Size> {
       } else {
         d = 0;
         for (ImmutableList<RexLiteral> literals : rel.getTuples()) {
-          d += typeValueSize(field.getType(), literals.get(i).getValue());
+          d += typeValueSize(field.getType(),
+              literals.get(i).getValueAs(Comparable.class));
         }
         d /= rel.getTuples().size();
       }
@@ -372,7 +373,8 @@ public class RelMdSize implements MetadataHandler<BuiltInMetadata.Size> {
     case INPUT_REF:
       return inputColumnSizes.get(((RexInputRef) node).getIndex());
     case LITERAL:
-      return typeValueSize(node.getType(), ((RexLiteral) node).getValue());
+      return typeValueSize(node.getType(),
+          ((RexLiteral) node).getValueAs(Comparable.class));
     default:
       if (node instanceof RexCall) {
         RexCall call = (RexCall) node;

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUtil.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUtil.java
index 7b63ac9..6a896ad 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUtil.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUtil.java
@@ -86,24 +86,23 @@ public class RelMdUtil {
     RexBuilder rexBuilder = rel.getCluster().getRexBuilder();
     double selectivity =
         computeSemiJoinSelectivity(mq, rel.getLeft(), rel.getRight(), rel);
-    RexNode selec =
-        rexBuilder.makeApproxLiteral(new BigDecimal(selectivity));
-    return rexBuilder.makeCall(ARTIFICIAL_SELECTIVITY_FUNC, selec);
+    return rexBuilder.makeCall(ARTIFICIAL_SELECTIVITY_FUNC,
+        rexBuilder.makeApproxLiteral(new BigDecimal(selectivity)));
   }
 
   /**
-   * Returns the selectivity value stored in the rexnode
+   * Returns the selectivity value stored in a call.
    *
-   * @param artificialSelecFuncNode rexnode containing the selectivity value
+   * @param artificialSelectivityFuncNode Call containing the selectivity value
    * @return selectivity value
    */
-  public static double getSelectivityValue(RexNode artificialSelecFuncNode) {
-    assert artificialSelecFuncNode instanceof RexCall;
-    RexCall call = (RexCall) artificialSelecFuncNode;
+  public static double getSelectivityValue(
+      RexNode artificialSelectivityFuncNode) {
+    assert artificialSelectivityFuncNode instanceof RexCall;
+    RexCall call = (RexCall) artificialSelectivityFuncNode;
     assert call.getOperator() == ARTIFICIAL_SELECTIVITY_FUNC;
     RexNode operand = call.getOperands().get(0);
-    BigDecimal bd = (BigDecimal) ((RexLiteral) operand).getValue();
-    return bd.doubleValue();
+    return ((RexLiteral) operand).getValueAs(Double.class);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/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 826cebc..1e477d2 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
@@ -67,6 +67,9 @@ 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;
+import org.apache.calcite.util.TimestampString;
 import org.apache.calcite.util.Util;
 
 import com.google.common.base.Preconditions;
@@ -78,9 +81,9 @@ 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;
-import java.util.Calendar;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -481,7 +484,8 @@ public abstract class SqlImplementor {
         || node instanceof SqlMatchRecognize
         || node instanceof SqlCall
             && (((SqlCall) node).getOperator() instanceof SqlSetOperator
-                || ((SqlCall) node).getOperator() == SqlStdOperatorTable.AS)
+                || ((SqlCall) node).getOperator() == SqlStdOperatorTable.AS
+                || ((SqlCall) node).getOperator() == SqlStdOperatorTable.VALUES)
         : node;
     return new SqlSelect(POS, SqlNodeList.EMPTY, null, node, null, null, null,
         SqlNodeList.EMPTY, null, null, null);
@@ -542,20 +546,23 @@ public abstract class SqlImplementor {
           return SqlLiteral.createCharString((String) literal.getValue2(), POS);
         case NUMERIC:
         case EXACT_NUMERIC:
-          return SqlLiteral.createExactNumeric(literal.getValue().toString(),
-              POS);
+          return SqlLiteral.createExactNumeric(
+              literal.getValueAs(BigDecimal.class).toString(), POS);
         case APPROXIMATE_NUMERIC:
           return SqlLiteral.createApproxNumeric(
-              literal.getValue().toString(), POS);
+              literal.getValueAs(BigDecimal.class).toString(), POS);
         case BOOLEAN:
-          return SqlLiteral.createBoolean((Boolean) literal.getValue(), POS);
+          return SqlLiteral.createBoolean(literal.getValueAs(Boolean.class),
+              POS);
         case DATE:
-          return SqlLiteral.createDate((Calendar) literal.getValue(), POS);
+          return SqlLiteral.createDate(literal.getValueAs(DateString.class),
+              POS);
         case TIME:
-          return SqlLiteral.createTime((Calendar) literal.getValue(),
+          return SqlLiteral.createTime(literal.getValueAs(TimeString.class),
               literal.getType().getPrecision(), POS);
         case TIMESTAMP:
-          return SqlLiteral.createTimestamp((Calendar) literal.getValue(),
+          return SqlLiteral.createTimestamp(
+              literal.getValueAs(TimestampString.class),
               literal.getType().getPrecision(), POS);
         case ANY:
         case NULL:

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/main/java/org/apache/calcite/rel/rules/DateRangeRules.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/DateRangeRules.java b/core/src/main/java/org/apache/calcite/rel/rules/DateRangeRules.java
index 1bfb813..2a185a1 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/DateRangeRules.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/DateRangeRules.java
@@ -35,6 +35,7 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.tools.RelBuilderFactory;
 import org.apache.calcite.util.Bug;
+import org.apache.calcite.util.DateString;
 import org.apache.calcite.util.Util;
 
 import com.google.common.base.Predicate;
@@ -355,7 +356,8 @@ public abstract class DateRangeRules {
             : SqlStdOperatorTable.GREATER_THAN;
         nodes.add(
             rexBuilder.makeCall(op, operand,
-                rexBuilder.makeDateLiteral(r.lowerEndpoint())));
+                rexBuilder.makeDateLiteral(
+                    DateString.fromCalendarFields(r.lowerEndpoint()))));
       }
       if (r.hasUpperBound()) {
         final SqlBinaryOperator op = r.upperBoundType() == BoundType.CLOSED
@@ -363,7 +365,8 @@ public abstract class DateRangeRules {
             : SqlStdOperatorTable.LESS_THAN;
         nodes.add(
             rexBuilder.makeCall(op, operand,
-                rexBuilder.makeDateLiteral(r.upperEndpoint())));
+                rexBuilder.makeDateLiteral(
+                    DateString.fromCalendarFields(r.upperEndpoint()))));
       }
       return RexUtil.composeConjunction(rexBuilder, nodes, false);
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexBuilder.java b/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
index 1906227..adf7fee 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
@@ -28,7 +28,6 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.runtime.FlatLists;
-import org.apache.calcite.runtime.SqlFunctions;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlIntervalQualifier;
 import org.apache.calcite.sql.SqlKind;
@@ -43,8 +42,11 @@ import org.apache.calcite.sql.type.MultisetSqlType;
 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.util.DateString;
 import org.apache.calcite.util.NlsString;
 import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.TimeString;
+import org.apache.calcite.util.TimestampString;
 import org.apache.calcite.util.Util;
 
 import com.google.common.base.Function;
@@ -477,21 +479,6 @@ public class RexBuilder {
   }
 
   /**
-   * Rounds the time part of a TIME or TIMESTAMP value to the given precision.
-   *
-   * @param timestamp The value to be rounded, will change in place
-   * @param precision the desired precision
-   */
-  private void roundTime(Calendar timestamp, long precision) {
-    if (precision == RelDataType.PRECISION_NOT_SPECIFIED) {
-      precision = 0;
-    }
-    final long pow = DateTimeUtils.powerX(10, 3 - precision);
-    final long timeMs = SqlFunctions.round(timestamp.getTimeInMillis(), pow);
-    timestamp.setTimeInMillis(timeMs);
-  }
-
-  /**
    * Creates a call to the CAST operator, expanding if possible, and optionally
    * also preserving nullability.
    *
@@ -512,7 +499,7 @@ public class RexBuilder {
     final SqlTypeName sqlType = type.getSqlTypeName();
     if (exp instanceof RexLiteral) {
       RexLiteral literal = (RexLiteral) exp;
-      Comparable value = literal.getValue();
+      Comparable value = literal.getValueAs(Comparable.class);
       SqlTypeName typeName = literal.getTypeName();
       if (canRemoveCastFromLiteral(type, value, typeName)) {
         switch (typeName) {
@@ -836,7 +823,7 @@ public class RexBuilder {
   /**
    * Internal method to create a call to a literal. Code outside this package
    * should call one of the type-specific methods such as
-   * {@link #makeDateLiteral(Calendar)}, {@link #makeLiteral(boolean)},
+   * {@link #makeDateLiteral(DateString)}, {@link #makeLiteral(boolean)},
    * {@link #makeLiteral(String)}.
    *
    * @param o        Value of literal, must be appropriate for the type
@@ -850,6 +837,7 @@ public class RexBuilder {
       SqlTypeName typeName) {
     // All literals except NULL have NOT NULL types.
     type = typeFactory.createTypeWithNullability(type, o == null);
+    int p;
     switch (typeName) {
     case CHAR:
       // Character literals must have a charset and collation. Populate
@@ -868,8 +856,21 @@ public class RexBuilder {
       }
       break;
     case TIME:
+      assert o instanceof TimeString;
+      p = type.getPrecision();
+      if (p == RelDataType.PRECISION_NOT_SPECIFIED) {
+        p = 0;
+      }
+      o = ((TimeString) o).round(p);
+      break;
     case TIMESTAMP:
-      roundTime((Calendar) o, type.getPrecision());
+      assert o instanceof TimestampString;
+      p = type.getPrecision();
+      if (p == RelDataType.PRECISION_NOT_SPECIFIED) {
+        p = 0;
+      }
+      o = ((TimestampString) o).round(p);
+      break;
     }
     return new RexLiteral(o, type, typeName);
   }
@@ -1054,37 +1055,48 @@ public class RexBuilder {
     return makeLiteral(str, type, SqlTypeName.CHAR);
   }
 
+  /** @deprecated Use {@link #makeDateLiteral(DateString)}. */
+  @Deprecated // to be removed before 2.0
+  public RexLiteral makeDateLiteral(Calendar calendar) {
+    return makeDateLiteral(DateString.fromCalendarFields(calendar));
+  }
+
   /**
    * Creates a Date literal.
    */
-  public RexLiteral makeDateLiteral(Calendar date) {
-    assert date != null;
-    return makeLiteral(
-        date, typeFactory.createSqlType(SqlTypeName.DATE), SqlTypeName.DATE);
+  public RexLiteral makeDateLiteral(DateString date) {
+    return makeLiteral(Preconditions.checkNotNull(date),
+        typeFactory.createSqlType(SqlTypeName.DATE), SqlTypeName.DATE);
+  }
+
+  /** @deprecated Use {@link #makeTimeLiteral(TimeString, int)}. */
+  @Deprecated // to be removed before 2.0
+  public RexLiteral makeTimeLiteral(Calendar calendar, int precision) {
+    return makeTimeLiteral(TimeString.fromCalendarFields(calendar), precision);
   }
 
   /**
    * Creates a Time literal.
    */
-  public RexLiteral makeTimeLiteral(
-      Calendar time,
-      int precision) {
-    assert time != null;
-    return makeLiteral(
-        time,
+  public RexLiteral makeTimeLiteral(TimeString time, int precision) {
+    return makeLiteral(Preconditions.checkNotNull(time),
         typeFactory.createSqlType(SqlTypeName.TIME, precision),
         SqlTypeName.TIME);
   }
 
+  /** @deprecated Use {@link #makeTimestampLiteral(TimestampString, int)}. */
+  @Deprecated // to be removed before 2.0
+  public RexLiteral makeTimestampLiteral(Calendar calendar, int precision) {
+    return makeTimestampLiteral(TimestampString.fromCalendarFields(calendar),
+        precision);
+  }
+
   /**
    * Creates a Timestamp literal.
    */
-  public RexLiteral makeTimestampLiteral(
-      Calendar timestamp,
+  public RexLiteral makeTimestampLiteral(TimestampString timestamp,
       int precision) {
-    assert timestamp != null;
-    return makeLiteral(
-        timestamp,
+    return makeLiteral(Preconditions.checkNotNull(timestamp),
         typeFactory.createSqlType(SqlTypeName.TIMESTAMP, precision),
         SqlTypeName.TIMESTAMP);
   }
@@ -1275,11 +1287,11 @@ public class RexBuilder {
     case BOOLEAN:
       return (Boolean) value ? booleanTrue : booleanFalse;
     case TIME:
-      return makeTimeLiteral((Calendar) value, type.getPrecision());
+      return makeTimeLiteral((TimeString) value, type.getPrecision());
     case DATE:
-      return makeDateLiteral((Calendar) value);
+      return makeDateLiteral((DateString) value);
     case TIMESTAMP:
-      return makeTimestampLiteral((Calendar) value, type.getPrecision());
+      return makeTimestampLiteral((TimestampString) value, type.getPrecision());
     case INTERVAL_YEAR:
     case INTERVAL_YEAR_MONTH:
     case INTERVAL_MONTH:
@@ -1357,7 +1369,6 @@ public class RexBuilder {
     if (o == null) {
       return null;
     }
-    final Calendar calendar;
     switch (type.getSqlTypeName()) {
     case TINYINT:
     case SMALLINT:
@@ -1402,27 +1413,38 @@ public class RexBuilder {
       return new NlsString((String) o, type.getCharset().name(),
           type.getCollation());
     case TIME:
-      if (o instanceof Calendar) {
+      if (o instanceof TimeString) {
         return o;
+      } else if (o instanceof Calendar) {
+        if (!((Calendar) o).getTimeZone().equals(DateTimeUtils.UTC_ZONE)) {
+          throw new AssertionError();
+        }
+        return TimeString.fromCalendarFields((Calendar) o);
+      } else {
+        return TimeString.fromMillisOfDay((Integer) o);
       }
-      calendar = Util.calendar();
-      calendar.setTimeInMillis((Integer) o);
-      return calendar;
     case DATE:
-      if (o instanceof Calendar) {
+      if (o instanceof DateString) {
         return o;
+      } else if (o instanceof Calendar) {
+        if (!((Calendar) o).getTimeZone().equals(DateTimeUtils.UTC_ZONE)) {
+          throw new AssertionError();
+        }
+        return DateString.fromCalendarFields((Calendar) o);
+      } else {
+        return DateString.fromDaysSinceEpoch((Integer) o);
       }
-      calendar = Util.calendar();
-      calendar.setTimeInMillis(0);
-      calendar.add(Calendar.DAY_OF_YEAR, (Integer) o);
-      return calendar;
     case TIMESTAMP:
-      if (o instanceof Calendar) {
+      if (o instanceof TimestampString) {
         return o;
+      } else if (o instanceof Calendar) {
+        if (!((Calendar) o).getTimeZone().equals(DateTimeUtils.UTC_ZONE)) {
+          throw new AssertionError();
+        }
+        return TimestampString.fromCalendarFields((Calendar) o);
+      } else {
+        return TimestampString.fromMillisSinceEpoch((Long) o);
       }
-      calendar = Util.calendar();
-      calendar.setTimeInMillis((Long) o);
-      return calendar;
     default:
       return o;
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/main/java/org/apache/calcite/rex/RexLiteral.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexLiteral.java b/core/src/main/java/org/apache/calcite/rex/RexLiteral.java
index f9be2bb..97e3055 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexLiteral.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexLiteral.java
@@ -26,14 +26,13 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParserUtil;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.ConversionUtil;
+import org.apache.calcite.util.DateString;
 import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.NlsString;
 import org.apache.calcite.util.SaffronProperties;
+import org.apache.calcite.util.TimeString;
+import org.apache.calcite.util.TimestampString;
 import org.apache.calcite.util.Util;
-import org.apache.calcite.util.ZonelessDate;
-import org.apache.calcite.util.ZonelessDatetime;
-import org.apache.calcite.util.ZonelessTime;
-import org.apache.calcite.util.ZonelessTimestamp;
 
 import com.google.common.base.Preconditions;
 
@@ -42,9 +41,11 @@ import java.io.StringWriter;
 import java.math.BigDecimal;
 import java.nio.ByteBuffer;
 import java.nio.charset.Charset;
+import java.text.SimpleDateFormat;
 import java.util.AbstractList;
 import java.util.Calendar;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Objects;
 import java.util.TimeZone;
@@ -68,7 +69,7 @@ import java.util.TimeZone;
  * <caption>Allowable types for RexLiteral instances</caption>
  * <tr>
  * <th>TypeName</th>
- * <th>Meaing</th>
+ * <th>Meaning</th>
  * <th>Value type</th>
  * </tr>
  * <tr>
@@ -96,32 +97,63 @@ import java.util.TimeZone;
  * <tr>
  * <td>{@link SqlTypeName#DATE}</td>
  * <td>Date, for example <code>DATE '1969-04'29'</code></td>
- * <td>{@link Calendar}</td>
+ * <td>{@link Calendar};
+ *     also {@link Calendar} (UTC time zone)
+ *     and {@link Integer} (days since POSIX epoch)</td>
  * </tr>
  * <tr>
  * <td>{@link SqlTypeName#TIME}</td>
  * <td>Time, for example <code>TIME '18:37:42.567'</code></td>
- * <td>{@link Calendar}</td>
+ * <td>{@link Calendar};
+ *     also {@link Calendar} (UTC time zone)
+ *     and {@link Integer} (milliseconds since midnight)</td>
  * </tr>
  * <tr>
  * <td>{@link SqlTypeName#TIMESTAMP}</td>
  * <td>Timestamp, for example <code>TIMESTAMP '1969-04-29
  * 18:37:42.567'</code></td>
- * <td>{@link Calendar}</td>
+ * <td>{@link TimestampString};
+ *     also {@link Calendar} (UTC time zone)
+ *     and {@link Long} (milliseconds since POSIX epoch)</td>
+ * </tr>
+ * <tr>
+ * <td>{@link SqlTypeName#INTERVAL_DAY},
+ *     {@link SqlTypeName#INTERVAL_DAY_HOUR},
+ *     {@link SqlTypeName#INTERVAL_DAY_MINUTE},
+ *     {@link SqlTypeName#INTERVAL_DAY_SECOND},
+ *     {@link SqlTypeName#INTERVAL_HOUR},
+ *     {@link SqlTypeName#INTERVAL_HOUR_MINUTE},
+ *     {@link SqlTypeName#INTERVAL_HOUR_SECOND},
+ *     {@link SqlTypeName#INTERVAL_MINUTE},
+ *     {@link SqlTypeName#INTERVAL_MINUTE_SECOND},
+ *     {@link SqlTypeName#INTERVAL_SECOND}</td>
+ * <td>Interval, for example <code>INTERVAL '4:3:2' HOUR TO SECOND</code></td>
+ * <td>{@link BigDecimal};
+ *     also {@link Long} (milliseconds)</td>
+ * </tr>
+ * <tr>
+ * <td>{@link SqlTypeName#INTERVAL_YEAR},
+ *     {@link SqlTypeName#INTERVAL_YEAR_MONTH},
+ *     {@link SqlTypeName#INTERVAL_MONTH}</td>
+ * <td>Interval, for example <code>INTERVAL '2-3' YEAR TO MONTH</code></td>
+ * <td>{@link BigDecimal};
+ *     also {@link Integer} (months)</td>
  * </tr>
  * <tr>
  * <td>{@link SqlTypeName#CHAR}</td>
  * <td>Character constant, for example <code>'Hello, world!'</code>, <code>
  * ''</code>, <code>_N'Bonjour'</code>, <code>_ISO-8859-1'It''s superman!'
  * COLLATE SHIFT_JIS$ja_JP$2</code>. These are always CHAR, never VARCHAR.</td>
- * <td>{@link NlsString}</td>
+ * <td>{@link NlsString};
+ *     also {@link String}</td>
  * </tr>
  * <tr>
  * <td>{@link SqlTypeName#BINARY}</td>
  * <td>Binary constant, for example <code>X'7F34'</code>. (The number of hexits
  * must be even; see above.) These constants are always BINARY, never
  * VARBINARY.</td>
- * <td>{@link ByteBuffer}</td>
+ * <td>{@link ByteBuffer};
+ *     also {@code byte[]}</td>
  * </tr>
  * <tr>
  * <td>{@link SqlTypeName#SYMBOL}</td>
@@ -213,10 +245,11 @@ public class RexLiteral extends RexNode {
     case BIGINT:
       return value instanceof BigDecimal;
     case DATE:
+      return value instanceof DateString;
     case TIME:
+      return value instanceof TimeString;
     case TIMESTAMP:
-      return value instanceof Calendar
-          && ((Calendar) value).getTimeZone().equals(DateTimeUtils.UTC_ZONE);
+      return value instanceof TimestampString;
     case INTERVAL_YEAR:
     case INTERVAL_YEAR_MONTH:
     case INTERVAL_MONTH:
@@ -391,13 +424,16 @@ public class RexLiteral extends RexNode {
       pw.print(")");
       break;
     case DATE:
-      printDatetime(pw, new ZonelessDate(), value);
+      assert value instanceof DateString;
+      pw.print(value);
       break;
     case TIME:
-      printDatetime(pw, new ZonelessTime(), value);
+      assert value instanceof TimeString;
+      pw.print(value);
       break;
     case TIMESTAMP:
-      printDatetime(pw, new ZonelessTimestamp(), value);
+      assert value instanceof TimestampString;
+      pw.print(value);
       break;
     case INTERVAL_YEAR:
     case INTERVAL_YEAR_MONTH:
@@ -439,16 +475,6 @@ public class RexLiteral extends RexNode {
     }
   }
 
-  private static void printDatetime(
-      PrintWriter pw,
-      ZonelessDatetime datetime,
-      Comparable value) {
-    assert value instanceof Calendar;
-    datetime.setZonelessTime(
-        ((Calendar) value).getTimeInMillis());
-    pw.print(datetime);
-  }
-
   /**
    * Converts a Jdbc string into a RexLiteral. This method accepts a string,
    * as returned by the Jdbc method ResultSet.getString(), and restores the
@@ -521,25 +547,44 @@ public class RexLiteral extends RexNode {
     case TIMESTAMP:
       String format = getCalendarFormat(typeName);
       TimeZone tz = DateTimeUtils.UTC_ZONE;
-      Calendar cal = null;
-      if (typeName == SqlTypeName.DATE) {
-        cal =
-            DateTimeUtils.parseDateFormat(literal, format, tz);
-      } else {
+      final Comparable v;
+      switch (typeName) {
+      case DATE:
+        final Calendar cal = DateTimeUtils.parseDateFormat(literal,
+            new SimpleDateFormat(format, Locale.ROOT),
+            tz);
+        if (cal == null) {
+          throw new AssertionError("fromJdbcString: invalid date/time value '"
+              + literal + "'");
+        }
+        v = DateString.fromCalendarFields(cal);
+        break;
+      default:
         // Allow fractional seconds for times and timestamps
-        DateTimeUtils.PrecisionTime ts =
-            DateTimeUtils.parsePrecisionDateTimeLiteral(literal, format, tz);
-        if (ts != null) {
-          cal = ts.getCalendar();
+        assert format != null;
+        final DateTimeUtils.PrecisionTime ts =
+            DateTimeUtils.parsePrecisionDateTimeLiteral(literal,
+                new SimpleDateFormat(format, Locale.ROOT), tz, -1);
+        if (ts == null) {
+          throw new AssertionError("fromJdbcString: invalid date/time value '"
+              + literal + "'");
+        }
+        switch (typeName) {
+        case TIMESTAMP:
+          v = TimestampString.fromCalendarFields(ts.getCalendar())
+              .withFraction(ts.getFraction());
+          break;
+        case TIME:
+          v = TimeString.fromCalendarFields(ts.getCalendar())
+              .withFraction(ts.getFraction());
+          break;
+        default:
+          throw new AssertionError();
         }
       }
-      if (cal == null) {
-        throw new AssertionError("fromJdbcString: invalid date/time value '"
-            + literal + "'");
-      }
-      return new RexLiteral(cal, type, typeName);
-    case SYMBOL:
+      return new RexLiteral(v, type, typeName);
 
+    case SYMBOL:
       // Symbols are for internal use
     default:
       throw new AssertionError("fromJdbcString: unsupported type");
@@ -572,11 +617,31 @@ public class RexLiteral extends RexNode {
   }
 
   /**
+   * Returns whether this literal's value is null.
+   */
+  public boolean isNull() {
+    return value == null;
+  }
+
+  /**
    * Returns the value of this literal.
+   *
+   * <p>For backwards compatibility, returns DATE. TIME and TIMESTAMP as a
+   * {@link Calendar} value in UTC time zone.
    */
   public Comparable getValue() {
     assert valueMatchesType(value, typeName, true) : value;
-    return value;
+    if (value == null) {
+      return null;
+    }
+    switch (typeName) {
+    case TIME:
+    case DATE:
+    case TIMESTAMP:
+      return getValueAs(Calendar.class);
+    default:
+      return value;
+    }
   }
 
   /**
@@ -589,17 +654,13 @@ public class RexLiteral extends RexNode {
     }
     switch (typeName) {
     case CHAR:
-      return ((NlsString) value).getValue();
+      return getValueAs(String.class);
     case DECIMAL:
-      return ((BigDecimal) value).unscaledValue().longValue();
+    case TIMESTAMP:
+      return getValueAs(Long.class);
     case DATE:
-      return (int) (((Calendar) value).getTimeInMillis()
-          / DateTimeUtils.MILLIS_PER_DAY);
     case TIME:
-      return (int) (((Calendar) value).getTimeInMillis()
-          % DateTimeUtils.MILLIS_PER_DAY);
-    case TIMESTAMP:
-      return ((Calendar) value).getTimeInMillis();
+      return getValueAs(Integer.class);
     default:
       return value;
     }
@@ -619,6 +680,122 @@ public class RexLiteral extends RexNode {
     }
   }
 
+  /** Returns the value of this literal as an instance of the specified class.
+   *
+   * <p>The following SQL types allow more than one form:
+   *
+   * <ul>
+   * <li>CHAR as {@link NlsString} or {@link String}
+   * <li>TIME as {@link TimeString},
+   *   {@link Integer} (milliseconds since midnight),
+   *   {@link Calendar} (in UTC)
+   * <li>DATE as {@link DateString},
+   *   {@link Integer} (days since 1970-01-01),
+   *   {@link Calendar}
+   * <li>TIMESTAMP as {@link TimestampString},
+   *   {@link Long} (milliseconds since 1970-01-01 00:00:00),
+   *   {@link Calendar}
+   * <li>DECIMAL as {@link BigDecimal} or {@link Long}
+   * </ul>
+   *
+   * <p>Called with {@code clazz} = {@link Comparable}, returns the value in
+   * its native form.
+   *
+   * @param clazz Desired return type
+   * @param <T> Return type
+   * @return Value of this literal in the desired type
+   */
+  public <T> T getValueAs(Class<T> clazz) {
+    if (value == null || clazz.isInstance(value)) {
+      return clazz.cast(value);
+    }
+    switch (typeName) {
+    case BINARY:
+      if (clazz == byte[].class) {
+        return clazz.cast(((ByteString) value).getBytes());
+      }
+      break;
+    case CHAR:
+      if (clazz == String.class) {
+        return clazz.cast(((NlsString) value).getValue());
+      }
+      break;
+    case DECIMAL:
+      if (clazz == Long.class) {
+        return clazz.cast(((BigDecimal) value).unscaledValue().longValue());
+      }
+      // fall through
+    case BIGINT:
+    case INTEGER:
+    case SMALLINT:
+    case TINYINT:
+    case DOUBLE:
+    case REAL:
+    case FLOAT:
+      if (clazz == Long.class) {
+        return clazz.cast(((BigDecimal) value).longValue());
+      } else if (clazz == Integer.class) {
+        return clazz.cast(((BigDecimal) value).intValue());
+      } else if (clazz == Short.class) {
+        return clazz.cast(((BigDecimal) value).shortValue());
+      } else if (clazz == Byte.class) {
+        return clazz.cast(((BigDecimal) value).byteValue());
+      } else if (clazz == Double.class) {
+        return clazz.cast(((BigDecimal) value).doubleValue());
+      } else if (clazz == Float.class) {
+        return clazz.cast(((BigDecimal) value).floatValue());
+      }
+      break;
+    case DATE:
+      if (clazz == Integer.class) {
+        return clazz.cast(((DateString) value).getDaysSinceEpoch());
+      } else if (clazz == Calendar.class) {
+        return clazz.cast(((DateString) value).toCalendar());
+      }
+      break;
+    case TIME:
+      if (clazz == Integer.class) {
+        return clazz.cast(((TimeString) value).getMillisOfDay());
+      } else if (clazz == Calendar.class) {
+        // Note: Nanos are ignored
+        return clazz.cast(((TimeString) value).toCalendar());
+      }
+      break;
+    case TIMESTAMP:
+      if (clazz == Long.class) {
+        // Milliseconds since 1970-01-01 00:00:00
+        return clazz.cast(((TimestampString) value).getMillisSinceEpoch());
+      } else if (clazz == Calendar.class) {
+        // Note: Nanos are ignored
+        return clazz.cast(((TimestampString) value).toCalendar());
+      }
+      break;
+    case INTERVAL_YEAR:
+    case INTERVAL_YEAR_MONTH:
+    case INTERVAL_MONTH:
+      if (clazz == Integer.class) {
+        return clazz.cast(((BigDecimal) value).intValue());
+      }
+      break;
+    case INTERVAL_DAY:
+    case INTERVAL_DAY_HOUR:
+    case INTERVAL_DAY_MINUTE:
+    case INTERVAL_DAY_SECOND:
+    case INTERVAL_HOUR:
+    case INTERVAL_HOUR_MINUTE:
+    case INTERVAL_HOUR_SECOND:
+    case INTERVAL_MINUTE:
+    case INTERVAL_MINUTE_SECOND:
+    case INTERVAL_SECOND:
+      if (clazz == Long.class) {
+        return clazz.cast(((BigDecimal) value).longValue());
+      }
+      break;
+    }
+    throw new AssertionError("cannot convert " + typeName
+        + " literal to " + clazz);
+  }
+
   public static boolean booleanValue(RexNode node) {
     return (Boolean) ((RexLiteral) node).value;
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexSimplify.java b/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
index 9ac69f7..34f01a0 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
@@ -177,8 +177,8 @@ public class RexSimplify {
     if (o0.isA(SqlKind.LITERAL)
         && o1.isA(SqlKind.LITERAL)
         && o0.getType().equals(o1.getType())) {
-      final Comparable v0 = ((RexLiteral) o0).getValue();
-      final Comparable v1 = ((RexLiteral) o1).getValue();
+      final Comparable v0 = ((RexLiteral) o0).getValueAs(Comparable.class);
+      final Comparable v1 = ((RexLiteral) o1).getValueAs(Comparable.class);
       if (v0 == null || v1 == null) {
         return unknownAsFalse
             ? rexBuilder.makeLiteral(false)
@@ -358,7 +358,7 @@ public class RexSimplify {
     case CUSTOM:
       switch (a.getKind()) {
       case LITERAL:
-        return rexBuilder.makeLiteral(((RexLiteral) a).getValue() != null);
+        return rexBuilder.makeLiteral(!((RexLiteral) a).isNull());
       default:
         throw new AssertionError("every CUSTOM policy needs a handler, "
             + a.getKind());
@@ -793,7 +793,7 @@ public class RexSimplify {
     switch (operand.getKind()) {
     case LITERAL:
       final RexLiteral literal = (RexLiteral) operand;
-      final Comparable value = literal.getValue();
+      final Comparable value = literal.getValueAs(Comparable.class);
       final SqlTypeName typeName = literal.getTypeName();
 
       // First, try to remove the cast without changing the value.
@@ -834,7 +834,7 @@ public class RexSimplify {
   private static RexNode processRange(RexBuilder rexBuilder,
       List<RexNode> terms, Map<String, Pair<Range, List<RexNode>>> rangeTerms,
       RexNode term, RexNode ref, RexLiteral constant, SqlKind comparison) {
-    final Comparable v0 = constant.getValue();
+    final Comparable v0 = constant.getValueAs(Comparable.class);
     Pair<Range, List<RexNode>> p = rangeTerms.get(ref.toString());
     if (p == null) {
       Range r;

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/main/java/org/apache/calcite/rex/RexToSqlNodeConverterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexToSqlNodeConverterImpl.java b/core/src/main/java/org/apache/calcite/rex/RexToSqlNodeConverterImpl.java
index 16b4203..f9a8932 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexToSqlNodeConverterImpl.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexToSqlNodeConverterImpl.java
@@ -20,9 +20,10 @@ import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.util.DateString;
 import org.apache.calcite.util.NlsString;
-
-import java.util.Calendar;
+import org.apache.calcite.util.TimeString;
+import org.apache.calcite.util.TimestampString;
 
 /**
  * Standard implementation of {@link RexToSqlNodeConverter}.
@@ -83,7 +84,7 @@ public class RexToSqlNodeConverterImpl implements RexToSqlNodeConverter {
     if (SqlTypeFamily.TIMESTAMP.getTypeNames().contains(
         literal.getTypeName())) {
       return SqlLiteral.createTimestamp(
-          (Calendar) literal.getValue(),
+          literal.getValueAs(TimestampString.class),
           0,
           SqlParserPos.ZERO);
     }
@@ -92,7 +93,16 @@ public class RexToSqlNodeConverterImpl implements RexToSqlNodeConverter {
     if (SqlTypeFamily.DATE.getTypeNames().contains(
         literal.getTypeName())) {
       return SqlLiteral.createDate(
-          (Calendar) literal.getValue(),
+          literal.getValueAs(DateString.class),
+          SqlParserPos.ZERO);
+    }
+
+    // Time
+    if (SqlTypeFamily.TIME.getTypeNames().contains(
+        literal.getTypeName())) {
+      return SqlLiteral.createTime(
+          literal.getValueAs(TimeString.class),
+          0,
           SqlParserPos.ZERO);
     }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/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 324b799..59dfc69 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlAbstractDateTimeLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlAbstractDateTimeLiteral.java
@@ -20,12 +20,7 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.util.ZonelessDate;
-import org.apache.calcite.util.ZonelessTime;
-import org.apache.calcite.util.ZonelessTimestamp;
-
-import java.util.Calendar;
-import java.util.TimeZone;
+import org.apache.calcite.util.TimestampString;
 
 /**
  * A SQL literal representing a DATE, TIME or TIMESTAMP value.
@@ -42,57 +37,29 @@ abstract class SqlAbstractDateTimeLiteral extends SqlLiteral {
   //~ Instance fields --------------------------------------------------------
 
   protected final boolean hasTimeZone;
-  protected final String formatString;
   protected final int precision;
 
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Constructs a datetime literal based on a Calendar. If the literal is to
-   * represent a Timestamp, the Calendar is expected to follow java.sql
-   * semantics. If the Calendar is to represent a Time or Date, the Calendar
-   * is expected to follow {@link org.apache.calcite.util.ZonelessTime}
-   * and {@link org.apache.calcite.util.ZonelessDate}
-   * semantics.
+   * Constructs a datetime literal.
    */
-  protected SqlAbstractDateTimeLiteral(
-      Calendar d,
-      boolean tz,
-      SqlTypeName typeName,
-      int precision,
-      String formatString,
-      SqlParserPos pos) {
+  protected SqlAbstractDateTimeLiteral(Object d, boolean tz,
+      SqlTypeName typeName, int precision, SqlParserPos pos) {
     super(d, typeName, pos);
     this.hasTimeZone = tz;
     this.precision = precision;
-    this.formatString = formatString;
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  public int getPrec() {
-    return precision;
-  }
-
-  public String toValue() {
-    return Long.toString(getCal().getTimeInMillis());
-  }
-
-  public Calendar getCal() {
-    return (Calendar) value;
+  /** Converts this literal to a {@link TimestampString}. */
+  protected TimestampString getTimestamp() {
+    return (TimestampString) value;
   }
 
-  /**
-   * Returns time zone component of this literal. Technically, a SQL date
-   * doesn't come with a tz, but time and ts inherit this, and the calendar
-   * object has one, so it seems harmless.
-   *
-   * @return time zone
-   */
-  public TimeZone getTimeZone() {
-    assert hasTimeZone : "Attempt to get time zone on Literal date: "
-        + getCal() + ", which has no time zone";
-    return getCal().getTimeZone();
+  public int getPrec() {
+    return precision;
   }
 
   /**
@@ -117,36 +84,6 @@ abstract class SqlAbstractDateTimeLiteral extends SqlLiteral {
       int rightPrec) {
     writer.literal(this.toString());
   }
-
-  /**
-   * Converts this literal to a
-   * {@link org.apache.calcite.util.ZonelessDate} object.
-   */
-  protected ZonelessDate getDate() {
-    ZonelessDate zd = new ZonelessDate();
-    zd.setZonelessTime(getCal().getTimeInMillis());
-    return zd;
-  }
-
-  /**
-   * Converts this literal to a
-   * {@link org.apache.calcite.util.ZonelessTime} object.
-   */
-  protected ZonelessTime getTime() {
-    ZonelessTime zt = new ZonelessTime();
-    zt.setZonelessTime(getCal().getTimeInMillis());
-    return zt;
-  }
-
-  /**
-   * Converts this literal to a
-   * {@link org.apache.calcite.util.ZonelessTimestamp} object.
-   */
-  protected ZonelessTimestamp getTimestamp() {
-    ZonelessTimestamp zt = new ZonelessTimestamp();
-    zt.setZonelessTime(getCal().getTimeInMillis());
-    return zt;
-  }
 }
 
 // End SqlAbstractDateTimeLiteral.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/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 9a7aed9..1a757f6 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDateLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDateLiteral.java
@@ -16,13 +16,11 @@
  */
 package org.apache.calcite.sql;
 
-import org.apache.calcite.avatica.util.DateTimeUtils;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.SqlTypeName;
-
-import java.util.Calendar;
+import org.apache.calcite.util.DateString;
 
 /**
  * A SQL literal representing a DATE value, such as <code>DATE
@@ -33,21 +31,22 @@ import java.util.Calendar;
 public class SqlDateLiteral extends SqlAbstractDateTimeLiteral {
   //~ Constructors -----------------------------------------------------------
 
-  SqlDateLiteral(Calendar d, SqlParserPos pos) {
-    super(d, false, SqlTypeName.DATE, 0, DateTimeUtils.DATE_FORMAT_STRING, pos);
-  }
-
-  SqlDateLiteral(Calendar d, String format, SqlParserPos pos) {
-    super(d, false, SqlTypeName.DATE, 0, format, pos);
+  SqlDateLiteral(DateString d, SqlParserPos pos) {
+    super(d, false, SqlTypeName.DATE, 0, pos);
   }
 
   //~ Methods ----------------------------------------------------------------
 
+  /** Converts this literal to a {@link DateString}. */
+  protected DateString getDate() {
+    return (DateString) value;
+  }
+
   public SqlNode clone(SqlParserPos pos) {
-    return new SqlDateLiteral((Calendar) value, pos);
+    return new SqlDateLiteral((DateString) value, pos);
   }
 
-  public String toString() {
+  @Override public String toString() {
     return "DATE '" + toFormattedString() + "'";
   }
 
@@ -55,7 +54,7 @@ public class SqlDateLiteral extends SqlAbstractDateTimeLiteral {
    * Returns e.g. '1969-07-21'.
    */
   public String toFormattedString() {
-    return getDate().toString(formatString);
+    return getDate().toString();
   }
 
   public RelDataType createSqlType(RelDataTypeFactory typeFactory) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java b/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
index 228abfc..4fcafb9 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
@@ -31,8 +31,11 @@ import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
 import org.apache.calcite.util.BitString;
+import org.apache.calcite.util.DateString;
 import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.NlsString;
+import org.apache.calcite.util.TimeString;
+import org.apache.calcite.util.TimestampString;
 import org.apache.calcite.util.Util;
 
 import java.math.BigDecimal;
@@ -195,9 +198,11 @@ public class SqlLiteral extends SqlNode {
     case DOUBLE:
       return value instanceof BigDecimal;
     case DATE:
+      return value instanceof DateString;
     case TIME:
+      return value instanceof TimeString;
     case TIMESTAMP:
-      return value instanceof Calendar;
+      return value instanceof TimestampString;
     case INTERVAL_YEAR:
     case INTERVAL_YEAR_MONTH:
     case INTERVAL_MONTH:
@@ -251,6 +256,61 @@ public class SqlLiteral extends SqlNode {
     return value;
   }
 
+  public <T> T getValueAs(Class<T> clazz) {
+    if (clazz.isInstance(value)) {
+      return clazz.cast(value);
+    }
+    switch (typeName) {
+    case DATE:
+      if (clazz == Calendar.class) {
+        return clazz.cast(((DateString) value).toCalendar());
+      }
+      break;
+    case TIME:
+      if (clazz == Calendar.class) {
+        return clazz.cast(((TimeString) value).toCalendar());
+      }
+      break;
+    case TIMESTAMP:
+      if (clazz == Calendar.class) {
+        return clazz.cast(((TimestampString) value).toCalendar());
+      }
+      break;
+    case INTERVAL_YEAR:
+    case INTERVAL_YEAR_MONTH:
+    case INTERVAL_MONTH:
+      if (clazz == Long.class) {
+        final SqlIntervalLiteral.IntervalValue valMonth =
+            (SqlIntervalLiteral.IntervalValue) value;
+        return clazz.cast(valMonth.getSign()
+            * SqlParserUtil.intervalToMonths(valMonth));
+      } else if (clazz == BigDecimal.class) {
+        return clazz.cast(BigDecimal.valueOf(getValueAs(Long.class)));
+      }
+      break;
+    case INTERVAL_DAY:
+    case INTERVAL_DAY_HOUR:
+    case INTERVAL_DAY_MINUTE:
+    case INTERVAL_DAY_SECOND:
+    case INTERVAL_HOUR:
+    case INTERVAL_HOUR_MINUTE:
+    case INTERVAL_HOUR_SECOND:
+    case INTERVAL_MINUTE:
+    case INTERVAL_MINUTE_SECOND:
+    case INTERVAL_SECOND:
+      if (clazz == Long.class) {
+        final SqlIntervalLiteral.IntervalValue valTime =
+            (SqlIntervalLiteral.IntervalValue) value;
+        return clazz.cast(valTime.getSign()
+            * SqlParserUtil.intervalToMillis(valTime));
+      } else if (clazz == BigDecimal.class) {
+        return clazz.cast(BigDecimal.valueOf(getValueAs(Long.class)));
+      }
+      break;
+    }
+    throw new AssertionError("cannot cast " + value + " as " + clazz);
+  }
+
   /** Returns the value as a symbol. */
   @Deprecated // to be removed before 2.0
   public <E extends Enum<E>> E symbolValue_() {
@@ -363,12 +423,12 @@ public class SqlLiteral extends SqlNode {
     if (node instanceof SqlLiteral) {
       SqlLiteral literal = (SqlLiteral) node;
       assert SqlTypeUtil.inCharFamily(literal.getTypeName());
-      return literal.toValue();
+      return literal.value.toString();
     } else if (SqlUtil.isLiteralChain(node)) {
       final SqlLiteral literal =
           SqlLiteralChainOperator.concatenateOperands((SqlCall) node);
       assert SqlTypeUtil.inCharFamily(literal.getTypeName());
-      return literal.toValue();
+      return literal.value.toString();
     } else if (node instanceof SqlCall
         && ((SqlCall) node).getOperator() == SqlStdOperatorTable.CAST) {
       //noinspection deprecation
@@ -680,24 +740,48 @@ public class SqlLiteral extends SqlNode {
     }
   }
 
+  @Deprecated // to be removed before 2.0
   public static SqlDateLiteral createDate(
       Calendar calendar,
       SqlParserPos pos) {
-    return new SqlDateLiteral(calendar, pos);
+    return createDate(DateString.fromCalendarFields(calendar), pos);
+  }
+
+  public static SqlDateLiteral createDate(
+      DateString date,
+      SqlParserPos pos) {
+    return new SqlDateLiteral(date, pos);
   }
 
+  @Deprecated // to be removed before 2.0
   public static SqlTimestampLiteral createTimestamp(
       Calendar calendar,
       int precision,
       SqlParserPos pos) {
-    return new SqlTimestampLiteral(calendar, precision, false, pos);
+    return createTimestamp(TimestampString.fromCalendarFields(calendar),
+        precision, pos);
+  }
+
+  public static SqlTimestampLiteral createTimestamp(
+      TimestampString ts,
+      int precision,
+      SqlParserPos pos) {
+    return new SqlTimestampLiteral(ts, precision, false, pos);
   }
 
+  @Deprecated // to be removed before 2.0
   public static SqlTimeLiteral createTime(
       Calendar calendar,
       int precision,
       SqlParserPos pos) {
-    return new SqlTimeLiteral(calendar, precision, false, pos);
+    return createTime(TimeString.fromCalendarFields(calendar), precision, pos);
+  }
+
+  public static SqlTimeLiteral createTime(
+      TimeString t,
+      int precision,
+      SqlParserPos pos) {
+    return new SqlTimeLiteral(t, precision, false, pos);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/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 79166c3..4327e2a 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlTimeLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlTimeLiteral.java
@@ -16,14 +16,12 @@
  */
 package org.apache.calcite.sql;
 
-import org.apache.calcite.avatica.util.DateTimeUtils;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.TimeString;
 
 import com.google.common.base.Preconditions;
 
-import java.util.Calendar;
-
 /**
  * A SQL literal representing a TIME value, for example <code>TIME
  * '14:33:44.567'</code>.
@@ -33,33 +31,21 @@ import java.util.Calendar;
 public class SqlTimeLiteral extends SqlAbstractDateTimeLiteral {
   //~ Constructors -----------------------------------------------------------
 
-  SqlTimeLiteral(
-      Calendar t,
-      int precision,
-      boolean hasTimeZone,
-      SqlParserPos pos) {
-    this(t, precision, hasTimeZone, DateTimeUtils.TIME_FORMAT_STRING, pos);
-  }
-
-  SqlTimeLiteral(
-      Calendar t,
-      int precision,
-      boolean hasTimeZone,
-      String format,
+  SqlTimeLiteral(TimeString t, int precision, boolean hasTimeZone,
       SqlParserPos pos) {
-    super(t, hasTimeZone, SqlTypeName.TIME, precision, format, pos);
-    Preconditions.checkArgument(this.precision >= 0 && this.precision <= 3);
+    super(t, hasTimeZone, SqlTypeName.TIME, precision, pos);
+    Preconditions.checkArgument(this.precision >= 0);
   }
 
   //~ Methods ----------------------------------------------------------------
 
+  /** Converts this literal to a {@link TimeString}. */
+  protected TimeString getTime() {
+    return (TimeString) value;
+  }
+
   public SqlNode clone(SqlParserPos pos) {
-    return new SqlTimeLiteral(
-        (Calendar) value,
-        precision,
-        hasTimeZone,
-        formatString,
-        pos);
+    return new SqlTimeLiteral((TimeString) value, precision, hasTimeZone, pos);
   }
 
   public String toString() {
@@ -70,19 +56,7 @@ public class SqlTimeLiteral extends SqlAbstractDateTimeLiteral {
    * Returns e.g. '03:05:67.456'.
    */
   public String toFormattedString() {
-    String result = getTime().toString(formatString);
-    final Calendar cal = getCal();
-    if (precision > 0) {
-      // get the millisecond count.  millisecond => at most 3 digits.
-      String digits = Long.toString(cal.getTimeInMillis());
-      result =
-          result + "."
-          + digits.substring(digits.length() - 3,
-              digits.length() - 3 + precision);
-    } else {
-      assert 0 == cal.get(Calendar.MILLISECOND);
-    }
-    return result;
+    return getTime().toString(precision);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/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 a620595..cc659d5 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlTimestampLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlTimestampLiteral.java
@@ -16,14 +16,12 @@
  */
 package org.apache.calcite.sql;
 
-import org.apache.calcite.avatica.util.DateTimeUtils;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.TimestampString;
 
 import com.google.common.base.Preconditions;
 
-import java.util.Calendar;
-
 /**
  * A SQL literal representing a TIMESTAMP value, for example <code>TIMESTAMP
  * '1969-07-21 03:15 GMT'</code>.
@@ -33,34 +31,17 @@ import java.util.Calendar;
 public class SqlTimestampLiteral extends SqlAbstractDateTimeLiteral {
   //~ Constructors -----------------------------------------------------------
 
-  public SqlTimestampLiteral(
-      Calendar cal,
-      int precision,
-      boolean hasTimeZone,
-      SqlParserPos pos) {
-    this(cal, precision, hasTimeZone, DateTimeUtils.TIMESTAMP_FORMAT_STRING,
-        pos);
-  }
-
-  public SqlTimestampLiteral(
-      Calendar cal,
-      int precision,
-      boolean hasTimeZone,
-      String format,
-      SqlParserPos pos) {
-    super(cal, hasTimeZone, SqlTypeName.TIMESTAMP, precision, format, pos);
-    Preconditions.checkArgument(this.precision >= 0 && this.precision <= 3);
+  SqlTimestampLiteral(TimestampString ts, int precision,
+      boolean hasTimeZone, SqlParserPos pos) {
+    super(ts, hasTimeZone, SqlTypeName.TIMESTAMP, precision, pos);
+    Preconditions.checkArgument(this.precision >= 0);
   }
 
   //~ Methods ----------------------------------------------------------------
 
   public SqlNode clone(SqlParserPos pos) {
-    return new SqlTimestampLiteral(
-        (Calendar) value,
-        precision,
-        hasTimeZone,
-        formatString,
-        pos);
+    return new SqlTimestampLiteral((TimestampString) value, precision,
+        hasTimeZone, pos);
   }
 
   public String toString() {
@@ -71,21 +52,11 @@ public class SqlTimestampLiteral extends SqlAbstractDateTimeLiteral {
    * Returns e.g. '03:05:67.456'.
    */
   public String toFormattedString() {
-    String result = getTimestamp().toString(formatString);
-    final Calendar cal = getCal();
+    TimestampString ts = getTimestamp();
     if (precision > 0) {
-      assert precision <= 3;
-
-      // get the millisecond count.  millisecond => at most 3 digits.
-      String digits = Long.toString(cal.getTimeInMillis());
-      result =
-          result + "."
-          + digits.substring(digits.length() - 3,
-              digits.length() - 3 + precision);
-    } else {
-      assert 0 == cal.get(Calendar.MILLISECOND);
+      ts = ts.round(precision);
     }
-    return result;
+    return ts.toString(precision);
   }
 
   public void unparse(

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/main/java/org/apache/calcite/sql/parser/SqlParserUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/parser/SqlParserUtil.java b/core/src/main/java/org/apache/calcite/sql/parser/SqlParserUtil.java
index d4b7237..e82841a 100644
--- a/core/src/main/java/org/apache/calcite/sql/parser/SqlParserUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/parser/SqlParserUtil.java
@@ -17,10 +17,12 @@
 package org.apache.calcite.sql.parser;
 
 import org.apache.calcite.avatica.util.Casing;
+import org.apache.calcite.avatica.util.DateTimeUtils;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.runtime.CalciteContextException;
 import org.apache.calcite.runtime.PredicateImpl;
 import org.apache.calcite.sql.SqlBinaryOperator;
+import org.apache.calcite.sql.SqlDateLiteral;
 import org.apache.calcite.sql.SqlIntervalLiteral;
 import org.apache.calcite.sql.SqlIntervalQualifier;
 import org.apache.calcite.sql.SqlKind;
@@ -32,9 +34,15 @@ import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.SqlPostfixOperator;
 import org.apache.calcite.sql.SqlPrefixOperator;
 import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlTimeLiteral;
+import org.apache.calcite.sql.SqlTimestampLiteral;
+import org.apache.calcite.sql.SqlUtil;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.util.DateString;
 import org.apache.calcite.util.PrecedenceClimbingParser;
 import org.apache.calcite.util.SaffronProperties;
+import org.apache.calcite.util.TimeString;
+import org.apache.calcite.util.TimestampString;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.trace.CalciteTrace;
 
@@ -46,8 +54,10 @@ import org.slf4j.Logger;
 import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.nio.charset.Charset;
+import java.text.DateFormat;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
+import java.util.Calendar;
 import java.util.List;
 import java.util.Locale;
 import java.util.StringTokenizer;
@@ -129,6 +139,63 @@ public final class SqlParserUtil {
     return java.sql.Timestamp.valueOf(s);
   }
 
+  public static SqlDateLiteral parseDateLiteral(String s, SqlParserPos pos) {
+    final String dateStr = parseString(s);
+    final Calendar cal =
+        DateTimeUtils.parseDateFormat(dateStr, Format.PER_THREAD.get().date,
+            DateTimeUtils.UTC_ZONE);
+    if (cal == null) {
+      throw SqlUtil.newContextException(pos,
+          RESOURCE.illegalLiteral("DATE", s,
+              RESOURCE.badFormat(DateTimeUtils.DATE_FORMAT_STRING).str()));
+    }
+    final DateString d = DateString.fromCalendarFields(cal);
+    return SqlLiteral.createDate(d, pos);
+  }
+
+  public static SqlTimeLiteral parseTimeLiteral(String s, SqlParserPos pos) {
+    final String dateStr = parseString(s);
+    final DateTimeUtils.PrecisionTime pt =
+        DateTimeUtils.parsePrecisionDateTimeLiteral(dateStr,
+            Format.PER_THREAD.get().time, DateTimeUtils.UTC_ZONE, -1);
+    if (pt == null) {
+      throw SqlUtil.newContextException(pos,
+          RESOURCE.illegalLiteral("TIME", s,
+              RESOURCE.badFormat(DateTimeUtils.TIME_FORMAT_STRING).str()));
+    }
+    final TimeString t = TimeString.fromCalendarFields(pt.getCalendar())
+        .withFraction(pt.getFraction());
+    return SqlLiteral.createTime(t, pt.getPrecision(), pos);
+  }
+
+  public static SqlTimestampLiteral parseTimestampLiteral(String s,
+      SqlParserPos pos) {
+    final String dateStr = parseString(s);
+    final DateTimeUtils.PrecisionTime pt =
+        DateTimeUtils.parsePrecisionDateTimeLiteral(dateStr,
+            Format.PER_THREAD.get().timestamp, DateTimeUtils.UTC_ZONE, -1);
+    if (pt == null) {
+      throw SqlUtil.newContextException(pos,
+          RESOURCE.illegalLiteral("TIMESTAMP", s,
+              RESOURCE.badFormat(DateTimeUtils.TIMESTAMP_FORMAT_STRING).str()));
+    }
+    final TimestampString ts =
+        TimestampString.fromCalendarFields(pt.getCalendar())
+            .withFraction(pt.getFraction());
+    return SqlLiteral.createTimestamp(ts, pt.getPrecision(), pos);
+  }
+
+  public static SqlIntervalLiteral parseIntervalLiteral(SqlParserPos pos,
+      int sign, String s, SqlIntervalQualifier intervalQualifier) {
+    final String intervalStr = parseString(s);
+    if (intervalStr.equals("")) {
+      throw SqlUtil.newContextException(pos,
+          RESOURCE.illegalIntervalLiteral(s + " "
+              + intervalQualifier.toString(), pos.toString()));
+    }
+    return SqlLiteral.createInterval(sign, intervalStr, intervalQualifier, pos);
+  }
+
   /**
    * Checks if the date/time format is valid
    *
@@ -875,6 +942,24 @@ public final class SqlParserUtil {
       SqlParserUtil.replaceSublist(list, start, end, e);
     }
   }
+
+  /** Pre-initialized {@link DateFormat} objects, to be used within the current
+   * thread, because {@code DateFormat} is not thread-safe. */
+  private static class Format {
+    private static final ThreadLocal<Format> PER_THREAD =
+        new ThreadLocal<Format>() {
+          @Override protected Format initialValue() {
+            return new Format();
+          }
+        };
+    final DateFormat timestamp =
+        new SimpleDateFormat(DateTimeUtils.TIMESTAMP_FORMAT_STRING,
+            Locale.ROOT);
+    final DateFormat time =
+        new SimpleDateFormat(DateTimeUtils.TIME_FORMAT_STRING, Locale.ROOT);
+    final DateFormat date =
+        new SimpleDateFormat(DateTimeUtils.DATE_FORMAT_STRING, Locale.ROOT);
+  }
 }
 
 // End SqlParserUtil.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFactoryImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFactoryImpl.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFactoryImpl.java
index bbe0801..9f7dc51 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFactoryImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFactoryImpl.java
@@ -56,6 +56,10 @@ public class SqlTypeFactoryImpl extends RelDataTypeFactoryImpl {
   public RelDataType createSqlType(
       SqlTypeName typeName,
       int precision) {
+    final int maxPrecision = typeSystem.getMaxPrecision(typeName);
+    if (maxPrecision >= 0 && precision > maxPrecision) {
+      precision = maxPrecision;
+    }
     if (typeName.allowsScale()) {
       return createSqlType(typeName, precision, typeName.getDefaultScale());
     }
@@ -75,6 +79,10 @@ public class SqlTypeFactoryImpl extends RelDataTypeFactoryImpl {
     assertBasic(typeName);
     assert (precision >= 0)
         || (precision == RelDataType.PRECISION_NOT_SPECIFIED);
+    final int maxPrecision = typeSystem.getMaxPrecision(typeName);
+    if (maxPrecision >= 0 && precision > maxPrecision) {
+      precision = maxPrecision;
+    }
     RelDataType newType =
         new BasicSqlType(typeSystem, typeName, precision, scale);
     newType = SqlTypeUtil.addCharsetAndCollation(newType, this);

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java
index f34ea11..f1b4e6a 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java
@@ -19,6 +19,9 @@ package org.apache.calcite.sql.type;
 import org.apache.calcite.avatica.util.TimeUnit;
 import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.util.DateString;
+import org.apache.calcite.util.TimeString;
+import org.apache.calcite.util.TimestampString;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
@@ -911,11 +914,17 @@ public enum SqlTypeName {
     case BINARY:
       return SqlLiteral.createBinaryString((byte[]) o, pos);
     case DATE:
-      return SqlLiteral.createDate((Calendar) o, pos);
+      return SqlLiteral.createDate(o instanceof Calendar
+          ? DateString.fromCalendarFields((Calendar) o)
+          : (DateString) o, pos);
     case TIME:
-      return SqlLiteral.createTime((Calendar) o, 0 /* todo */, pos);
+      return SqlLiteral.createTime(o instanceof Calendar
+          ? TimeString.fromCalendarFields((Calendar) o)
+          : (TimeString) o, 0 /* todo */, pos);
     case TIMESTAMP:
-      return SqlLiteral.createTimestamp((Calendar) o, 0 /* todo */, pos);
+      return SqlLiteral.createTimestamp(o instanceof Calendar
+          ? TimestampString.fromCalendarFields((Calendar) o)
+          : (TimestampString) o, 0 /* todo */, pos);
     default:
       throw Util.unexpected(this);
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
index b8cddb0..46cd711 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
@@ -2821,7 +2821,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     case DATE:
     case TIME:
     case TIMESTAMP:
-      Calendar calendar = (Calendar) literal.getValue();
+      Calendar calendar = literal.getValueAs(Calendar.class);
       final int year = calendar.get(Calendar.YEAR);
       final int era = calendar.get(Calendar.ERA);
       if (year < 1 || era == GregorianCalendar.BC || year > 9999) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/main/java/org/apache/calcite/sql2rel/SqlNodeToRexConverterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/SqlNodeToRexConverterImpl.java b/core/src/main/java/org/apache/calcite/sql2rel/SqlNodeToRexConverterImpl.java
index 3f7b5cb..9f398d6 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlNodeToRexConverterImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlNodeToRexConverterImpl.java
@@ -31,13 +31,15 @@ import org.apache.calcite.sql.SqlTimestampLiteral;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.util.BitString;
+import org.apache.calcite.util.DateString;
 import org.apache.calcite.util.NlsString;
+import org.apache.calcite.util.TimeString;
+import org.apache.calcite.util.TimestampString;
 import org.apache.calcite.util.Util;
 
 import com.google.common.base.Preconditions;
 
 import java.math.BigDecimal;
-import java.util.Calendar;
 
 /**
  * Standard implementation of {@link SqlNodeToRexConverter}.
@@ -80,8 +82,7 @@ public class SqlNodeToRexConverterImpl implements SqlNodeToRexConverter {
     RexBuilder rexBuilder = cx.getRexBuilder();
     RelDataTypeFactory typeFactory = cx.getTypeFactory();
     SqlValidator validator = cx.getValidator();
-    final Object value = literal.getValue();
-    if (value == null) {
+    if (literal.getValue() == null) {
       // Since there is no eq. RexLiteral of SqlLiteral.Unknown we
       // treat it as a cast(null as boolean)
       RelDataType type;
@@ -102,23 +103,23 @@ public class SqlNodeToRexConverterImpl implements SqlNodeToRexConverter {
 
     switch (literal.getTypeName()) {
     case DECIMAL:
-
       // exact number
-      BigDecimal bd = (BigDecimal) value;
+      BigDecimal bd = literal.getValueAs(BigDecimal.class);
       return rexBuilder.makeExactLiteral(
           bd,
           literal.createSqlType(typeFactory));
-    case DOUBLE:
 
+    case DOUBLE:
       // approximate type
       // TODO:  preserve fixed-point precision and large integers
-      return rexBuilder.makeApproxLiteral((BigDecimal) value);
+      return rexBuilder.makeApproxLiteral(literal.getValueAs(BigDecimal.class));
+
     case CHAR:
-      return rexBuilder.makeCharLiteral((NlsString) value);
+      return rexBuilder.makeCharLiteral(literal.getValueAs(NlsString.class));
     case BOOLEAN:
-      return rexBuilder.makeLiteral(((Boolean) value).booleanValue());
+      return rexBuilder.makeLiteral(literal.getValueAs(Boolean.class));
     case BINARY:
-      bitString = (BitString) value;
+      bitString = literal.getValueAs(BitString.class);
       Preconditions.checkArgument((bitString.getBitCount() % 8) == 0,
           "incomplete octet");
 
@@ -127,17 +128,17 @@ public class SqlNodeToRexConverterImpl implements SqlNodeToRexConverter {
       ByteString byteString = new ByteString(bitString.getAsByteArray());
       return rexBuilder.makeBinaryLiteral(byteString);
     case SYMBOL:
-      return rexBuilder.makeFlag((Enum) value);
+      return rexBuilder.makeFlag(literal.getValueAs(Enum.class));
     case TIMESTAMP:
       return rexBuilder.makeTimestampLiteral(
-          (Calendar) value,
+          literal.getValueAs(TimestampString.class),
           ((SqlTimestampLiteral) literal).getPrec());
     case TIME:
       return rexBuilder.makeTimeLiteral(
-          (Calendar) value,
+          literal.getValueAs(TimeString.class),
           ((SqlTimeLiteral) literal).getPrec());
     case DATE:
-      return rexBuilder.makeDateLiteral((Calendar) value);
+      return rexBuilder.makeDateLiteral(literal.getValueAs(DateString.class));
 
     case INTERVAL_YEAR:
     case INTERVAL_YEAR_MONTH:
@@ -153,10 +154,10 @@ public class SqlNodeToRexConverterImpl implements SqlNodeToRexConverter {
     case INTERVAL_MINUTE_SECOND:
     case INTERVAL_SECOND:
       SqlIntervalQualifier sqlIntervalQualifier =
-          ((SqlIntervalLiteral.IntervalValue) value).getIntervalQualifier();
-      l = (long) SqlLiteral.value(literal);
+          literal.getValueAs(SqlIntervalLiteral.IntervalValue.class)
+              .getIntervalQualifier();
       return rexBuilder.makeIntervalLiteral(
-          BigDecimal.valueOf(l),
+          literal.getValueAs(BigDecimal.class),
           sqlIntervalQualifier);
     default:
       throw Util.unexpected(literal.getTypeName());

http://git-wip-us.apache.org/repos/asf/calcite/blob/205af813/core/src/main/java/org/apache/calcite/util/BasicDatetime.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/BasicDatetime.java b/core/src/main/java/org/apache/calcite/util/BasicDatetime.java
deleted file mode 100644
index 0d38bc0..0000000
--- a/core/src/main/java/org/apache/calcite/util/BasicDatetime.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.calcite.util;
-
-import java.util.TimeZone;
-
-/**
- * BasicDatetime is an interface for dates, times, or timestamps that can be
- * assigned from a long value. The value to be assigned may either be a zoneless
- * time, or it may be a zoned time.
- *
- * <p>A zoneless time is based on milliseconds. It may contain date and/or time
- * components as follows:
- *
- * <pre>
- * The time component = value % milliseconds in a day
- * The date component = value / milliseconds in a day
- * </pre>
- *
- * If a date component is specified, it is relative to the epoch (1970-01-01).
- *
- * <p>A zoned time represents a time that was created in a particular time zone.
- * It may contain date and/or time components that are valid when interpreted
- * relative to a specified time zone, according to a
- * {@link java.util.Calendar Calendar}. Jdbc types, such as
- * {@link java.sql.Date} typically contain zoned times.
- */
-public interface BasicDatetime {
-  //~ Methods ----------------------------------------------------------------
-
-  /**
-   * Gets the internal value of this datetime
-   */
-  long getTime();
-
-  /**
-   * Sets this datetime via a zoneless time value. See class comments for more
-   * information.
-   */
-  void setZonelessTime(long value);
-
-  /**
-   * Sets this datetime via a zoned time value. See class comments for more
-   * information.
-   */
-  void setZonedTime(long value, TimeZone zone);
-}
-
-// End BasicDatetime.java


[5/5] calcite git commit: Following [CALCITE-1798] implement date-time FLOOR for more JDBC dialects (Chris Baynes)

Posted by jh...@apache.org.
Following [CALCITE-1798] implement date-time FLOOR for more JDBC dialects (Chris Baynes)

Add support for MySQL and Microsoft SQL Server;
also fix a bug in the PostgreSQL and Oracle implementations.

Close apache/calcite#458


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

Branch: refs/heads/master
Commit: 946b24f4dc163dee018de90b2dcf7082bb892cfe
Parents: 205af81
Author: Chris Baynes <bi...@gmail.com>
Authored: Mon May 29 10:45:50 2017 +0200
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed May 31 08:17:28 2017 -0700

----------------------------------------------------------------------
 .../calcite/sql/fun/SqlFloorFunction.java       | 171 +++++++++++++++++--
 .../rel/rel2sql/RelToSqlConverterTest.java      |  53 ++++++
 2 files changed, 211 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/946b24f4/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 9e0e5ba..02b4abe 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
@@ -23,6 +23,7 @@ import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlOperatorBinding;
 import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
@@ -78,28 +79,51 @@ public class SqlFloorFunction extends SqlMonotonicUnaryFunction {
     }
 
     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);
+
+    switch (writer.getDialect().getDatabaseProduct()) {
     case ORACLE:
+      replaceTimeUnitOperand(call, timeUnit.name(), timeUnitNode.getParserPosition());
       unparseDatetimeFunction(writer, call, "TRUNC", true);
       break;
     case HSQLDB:
-      // translate timeUnit literal
-      SqlLiteral node = call.operand(1);
-      String translatedLit =
-          convertToHsqlDb((TimeUnitRange) node.getValue());
-      SqlLiteral newNode = SqlLiteral.createCharString(
-          translatedLit, null, node.getParserPosition());
-      call.setOperand(1, newNode);
-
+      String translatedLit = convertToHsqlDb(timeUnit);
+      replaceTimeUnitOperand(call, translatedLit, timeUnitNode.getParserPosition());
       unparseDatetimeFunction(writer, call, "TRUNC", true);
       break;
     case POSTGRESQL:
+      replaceTimeUnitOperand(call, timeUnit.name(), timeUnitNode.getParserPosition());
       unparseDatetimeFunction(writer, call, "DATE_TRUNC", false);
       break;
+    case MSSQL:
+      unparseDatetimeMssql(writer, call);
+      break;
+    case MYSQL:
+      unparseDatetimeMysql(writer, call);
+      break;
     default:
       unparseDatetimeDefault(writer, call);
     }
   }
 
+  private void replaceTimeUnitOperand(SqlCall call, String literal, SqlParserPos pos) {
+    SqlLiteral literalNode = SqlLiteral.createCharString(literal, null, pos);
+    call.setOperand(1, 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);
@@ -108,18 +132,84 @@ public class SqlFloorFunction extends SqlMonotonicUnaryFunction {
     writer.endFunCall(frame);
   }
 
+  /**
+   * Most dialects that natively support datetime floor will use this.
+   * In those cases the call will look like TRUNC(datetime, 'year').
+   *
+   * @param writer SqlWriter
+   * @param call SqlCall
+   * @param funName Name of the sql function to call
+   * @param datetimeFirst Specify the order of the datetime &amp; timeUnit
+   * arguments
+   */
   private void unparseDatetimeFunction(SqlWriter writer, SqlCall call,
       String funName, Boolean datetimeFirst) {
     final SqlWriter.Frame frame = writer.startFunCall(funName);
-    Integer firstOp = datetimeFirst ? 0 : 1;
-    Integer secondOp = datetimeFirst ? 1 : 0;
-
-    call.operand(firstOp).unparse(writer, 0, 0);
+    int firstOpIndex = datetimeFirst ? 0 : 1;
+    int secondOpIndex = datetimeFirst ? 1 : 0;
+    call.operand(firstOpIndex).unparse(writer, 0, 0);
     writer.sep(",", true);
-    call.operand(secondOp).unparse(writer, 0, 0);
+    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);
+  }
+
   private static String convertToHsqlDb(TimeUnitRange unit) {
     switch (unit) {
     case YEAR:
@@ -141,6 +231,61 @@ public class SqlFloorFunction extends SqlMonotonicUnaryFunction {
         + 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;
+    }
+
+    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 SqlFloorFunction.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/946b24f4/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 4e4db08..80957c0 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
@@ -690,6 +690,59 @@ public class RelToSqlConverterTest {
         .ok(expected);
   }
 
+  @Test public void testFloorPostgres() {
+    String query = "SELECT floor(\"hire_date\" TO MINUTE) FROM \"employee\"";
+    String expected = "SELECT DATE_TRUNC('MINUTE', \"hire_date\")\nFROM \"foodmart\".\"employee\"";
+    sql(query)
+        .dialect(DatabaseProduct.POSTGRESQL.getDialect())
+        .ok(expected);
+  }
+
+  @Test public void testFloorOracle() {
+    String query = "SELECT floor(\"hire_date\" TO MINUTE) FROM \"employee\"";
+    String expected = "SELECT TRUNC(\"hire_date\", 'MINUTE')\nFROM \"foodmart\".\"employee\"";
+    sql(query)
+        .dialect(DatabaseProduct.ORACLE.getDialect())
+        .ok(expected);
+  }
+
+  @Test public void testFloorMssqlWeek() {
+    String query = "SELECT floor(\"hire_date\" TO WEEK) FROM \"employee\"";
+    String expected = "SELECT CONVERT(DATETIME, CONVERT(VARCHAR(10), "
+        + "DATEADD(day, - (6 + DATEPART(weekday, [hire_date] )) % 7, [hire_date] ), 126))\n"
+        + "FROM [foodmart].[employee]";
+    sql(query)
+        .dialect(DatabaseProduct.MSSQL.getDialect())
+        .ok(expected);
+  }
+
+  @Test public void testFloorMssqlMonth() {
+    String query = "SELECT floor(\"hire_date\" TO MONTH) FROM \"employee\"";
+    String expected = "SELECT CONVERT(DATETIME, CONVERT(VARCHAR(7), [hire_date] , 126)+'-01')\n"
+        + "FROM [foodmart].[employee]";
+    sql(query)
+        .dialect(DatabaseProduct.MSSQL.getDialect())
+        .ok(expected);
+  }
+
+  @Test public void testFloorMysqlMonth() {
+    String query = "SELECT floor(\"hire_date\" TO MONTH) FROM \"employee\"";
+    String expected = "SELECT DATE_FORMAT(`hire_date`, '%Y-%m-01')\n"
+        + "FROM `foodmart`.`employee`";
+    sql(query)
+        .dialect(DatabaseProduct.MYSQL.getDialect())
+        .ok(expected);
+  }
+
+  @Test public void testFloorMysqlWeek() {
+    String query = "SELECT floor(\"hire_date\" TO WEEK) FROM \"employee\"";
+    String expected = "SELECT STR_TO_DATE(DATE_FORMAT(`hire_date` , '%x%v-1'), '%x%v-%w')\n"
+        + "FROM `foodmart`.`employee`";
+    sql(query)
+        .dialect(DatabaseProduct.MYSQL.getDialect())
+        .ok(expected);
+  }
+
   @Test public void testMatchRecognizePatternExpression() {
     String sql = "select *\n"
         + "  from \"product\" match_recognize\n"