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 2014/12/09 22:49:46 UTC

[2/8] incubator-calcite git commit: [CALCITE-93] Calcite RPC server; [CALCITE-94] Remote JDBC driver

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/core/src/main/java/org/apache/calcite/runtime/EnumeratorCursor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/EnumeratorCursor.java b/core/src/main/java/org/apache/calcite/runtime/EnumeratorCursor.java
index ec6795b..740c54f 100644
--- a/core/src/main/java/org/apache/calcite/runtime/EnumeratorCursor.java
+++ b/core/src/main/java/org/apache/calcite/runtime/EnumeratorCursor.java
@@ -16,10 +16,11 @@
  */
 package org.apache.calcite.runtime;
 
+import org.apache.calcite.avatica.util.PositionedCursor;
 import org.apache.calcite.linq4j.Enumerator;
 
 /**
- * Implementation of {@link org.apache.calcite.avatica.Cursor} on top of an
+ * Implementation of {@link org.apache.calcite.avatica.util.Cursor} on top of an
  * {@link org.apache.calcite.linq4j.Enumerator} that
  * returns a record for each row. The returned record is cached to avoid
  * multiple computations of current row.
@@ -30,9 +31,8 @@ import org.apache.calcite.linq4j.Enumerator;
  *
  * @param <T> Element type
  */
-public abstract class EnumeratorCursor<T> extends AbstractCursor {
+public abstract class EnumeratorCursor<T> extends PositionedCursor<T> {
   private final Enumerator<T> enumerator;
-  private T current;
 
   /**
    * Creates a {@code EnumeratorCursor}
@@ -42,27 +42,17 @@ public abstract class EnumeratorCursor<T> extends AbstractCursor {
     this.enumerator = enumerator;
   }
 
+  protected T current() {
+    return enumerator.current();
+  }
+
   public boolean next() {
-    if (enumerator.moveNext()) {
-      current = enumerator.current();
-      return true;
-    }
-    current = null;
-    return false;
+    return enumerator.moveNext();
   }
 
   public void close() {
-    current = null;
     enumerator.close();
   }
-
-  /**
-   * Returns current row.
-   * @return current row
-   */
-  protected T current() {
-    return current;
-  }
 }
 
 // End EnumeratorCursor.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/core/src/main/java/org/apache/calcite/runtime/ObjectEnumeratorCursor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/ObjectEnumeratorCursor.java b/core/src/main/java/org/apache/calcite/runtime/ObjectEnumeratorCursor.java
index f73d7d6..73cc9a1 100644
--- a/core/src/main/java/org/apache/calcite/runtime/ObjectEnumeratorCursor.java
+++ b/core/src/main/java/org/apache/calcite/runtime/ObjectEnumeratorCursor.java
@@ -16,14 +16,15 @@
  */
 package org.apache.calcite.runtime;
 
+import org.apache.calcite.avatica.util.PositionedCursor;
 import org.apache.calcite.linq4j.Enumerator;
 
 /**
- * Implementation of {@link org.apache.calcite.avatica.Cursor} on top of an
+ * Implementation of {@link org.apache.calcite.avatica.util.Cursor} on top of an
  * {@link org.apache.calcite.linq4j.Enumerator} that
  * returns an {@link Object} for each row.
  */
-public class ObjectEnumeratorCursor extends AbstractCursor {
+public class ObjectEnumeratorCursor extends PositionedCursor<Object> {
   private final Enumerator<Object> enumerator;
 
   /**
@@ -36,7 +37,11 @@ public class ObjectEnumeratorCursor extends AbstractCursor {
   }
 
   protected Getter createGetter(int ordinal) {
-    return new ObjectEnumeratorGetter(ordinal);
+    return new ObjectGetter(ordinal);
+  }
+
+  protected Object current() {
+    return enumerator.current();
   }
 
   public boolean next() {
@@ -46,21 +51,6 @@ public class ObjectEnumeratorCursor extends AbstractCursor {
   public void close() {
     enumerator.close();
   }
-
-  /** Implementation of {@link Getter} for records that consist of a single
-   * field. Each record is represented as an object, and the value of the sole
-   * field is that object. */
-  class ObjectEnumeratorGetter extends AbstractGetter {
-    public ObjectEnumeratorGetter(int field) {
-      assert field == 0;
-    }
-
-    public Object getObject() {
-      Object o = enumerator.current();
-      wasNull[0] = o == null;
-      return o;
-    }
-  }
 }
 
 // End ObjectEnumeratorCursor.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/core/src/main/java/org/apache/calcite/runtime/RecordEnumeratorCursor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/RecordEnumeratorCursor.java b/core/src/main/java/org/apache/calcite/runtime/RecordEnumeratorCursor.java
index 0e922bd..e02bc44 100644
--- a/core/src/main/java/org/apache/calcite/runtime/RecordEnumeratorCursor.java
+++ b/core/src/main/java/org/apache/calcite/runtime/RecordEnumeratorCursor.java
@@ -18,10 +18,8 @@ package org.apache.calcite.runtime;
 
 import org.apache.calcite.linq4j.Enumerator;
 
-import java.lang.reflect.Field;
-
 /**
- * Implementation of {@link org.apache.calcite.avatica.Cursor} on top of an
+ * Implementation of {@link org.apache.calcite.avatica.util.Cursor} on top of an
  * {@link org.apache.calcite.linq4j.Enumerator} that
  * returns a record for each row. The record is a synthetic class whose fields
  * are all public.
@@ -45,27 +43,7 @@ public class RecordEnumeratorCursor<E> extends EnumeratorCursor<E> {
   }
 
   protected Getter createGetter(int ordinal) {
-    return new RecordEnumeratorGetter(clazz.getFields()[ordinal]);
-  }
-
-  /** Implementation of {@link Getter} that reads fields via reflection. */
-  class RecordEnumeratorGetter extends AbstractGetter {
-    protected final Field field;
-
-    public RecordEnumeratorGetter(Field field) {
-      this.field = field;
-    }
-
-    public Object getObject() {
-      Object o;
-      try {
-        o = field.get(current());
-      } catch (IllegalAccessException e) {
-        throw new RuntimeException(e);
-      }
-      wasNull[0] = o == null;
-      return o;
-    }
+    return new FieldGetter(clazz.getFields()[ordinal]);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/core/src/main/java/org/apache/calcite/runtime/Spacer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/Spacer.java b/core/src/main/java/org/apache/calcite/runtime/Spacer.java
deleted file mode 100644
index e028716..0000000
--- a/core/src/main/java/org/apache/calcite/runtime/Spacer.java
+++ /dev/null
@@ -1,80 +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.runtime;
-
-/**
- * Efficiently writes strings of spaces.
- */
-public class Spacer {
-  private int n;
-
-  /** Creates a Spacer with zero spaces. */
-  public Spacer() {
-    this(0);
-  }
-
-  /** Creates a Spacer with a given number of spaces. */
-  public Spacer(int n) {
-    set(n);
-  }
-
-  /** Sets the current number of spaces. */
-  public Spacer set(int n) {
-    this.n = n;
-    return this;
-  }
-
-  /** Returns the current number of spaces. */
-  public int get() {
-    return n;
-  }
-
-  /** Increases the current number of spaces by {@code n}. */
-  public Spacer add(int n) {
-    return set(this.n + n);
-  }
-
-  /** Reduces the current number of spaces by {@code n}. */
-  public Spacer subtract(int n) {
-    return set(this.n - n);
-  }
-
-  /** Returns a string of the current number of spaces. */
-  public String toString() {
-    return Spaces.of(n);
-  }
-
-  /** Appends current number of spaces to a {@link StringBuilder}. */
-  public StringBuilder spaces(StringBuilder buf) {
-    return Spaces.append(buf, n);
-  }
-
-  /** Returns a string that is padded on the right with spaces to the current
-   * length. */
-  public String padRight(String string) {
-    Spaces.padRight(string, n);
-    final int x = n - string.length();
-    if (x <= 0) {
-      return string;
-    }
-    // Replacing StringBuffer with String would hurt performance.
-    //noinspection StringBufferReplaceableByString
-    return Spaces.append(new StringBuilder(string), x).toString();
-  }
-}
-
-// End Spacer.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/core/src/main/java/org/apache/calcite/runtime/Spaces.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/Spaces.java b/core/src/main/java/org/apache/calcite/runtime/Spaces.java
deleted file mode 100644
index 9921579..0000000
--- a/core/src/main/java/org/apache/calcite/runtime/Spaces.java
+++ /dev/null
@@ -1,185 +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.runtime;
-
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.util.AbstractList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.concurrent.CopyOnWriteArrayList;
-
-/** Utilities for creating strings of spaces. */
-public class Spaces {
-  /** It doesn't look like this list is ever updated. But it is - when a call to
-   * to {@link SpaceList#get} causes an {@link IndexOutOfBoundsException}. */
-  @SuppressWarnings("MismatchedQueryAndUpdateOfCollection")
-  private static final List<String> SPACE_LIST = new SpaceList();
-
-  /** The longest possible string of spaces. Fine as long as you don't try
-   * to print it.
-   *
-   * <p>Use with {@link StringBuilder#append(CharSequence, int, int)} to
-   * append spaces without doing memory allocation.</p>
-   */
-  public static final CharSequence MAX = sequence(Integer.MAX_VALUE);
-
-  // Utility class. Do not instantiate.
-  private Spaces() {}
-
-  /** Creates a sequence of {@code n} spaces. */
-  public static CharSequence sequence(int n) {
-    return new SpaceString(n);
-  }
-
-  /** Returns a string of {@code n} spaces. */
-  public static String of(int n) {
-    return SPACE_LIST.get(n);
-  }
-
-  /** Appends {@code n} spaces to an {@link Appendable}. */
-  public static Appendable append(Appendable buf, int n) throws IOException {
-    buf.append(MAX, 0, n);
-    return buf;
-  }
-
-  /** Appends {@code n} spaces to a {@link PrintWriter}. */
-  public static PrintWriter append(PrintWriter pw, int n) {
-    pw.append(MAX, 0, n);
-    return pw;
-  }
-
-  /** Appends {@code n} spaces to a {@link StringWriter}. */
-  public static StringWriter append(StringWriter pw, int n) {
-    pw.append(MAX, 0, n);
-    return pw;
-  }
-
-  /** Appends {@code n} spaces to a {@link StringBuilder}. */
-  public static StringBuilder append(StringBuilder buf, int n) {
-    buf.append(MAX, 0, n);
-    return buf;
-  }
-
-  /** Appends {@code n} spaces to a {@link StringBuffer}. */
-  public static StringBuffer append(StringBuffer buf, int n) {
-    buf.append(MAX, 0, n);
-    return buf;
-  }
-
-  /** Returns a string that is padded on the right with spaces to the given
-   * length. */
-  public static String padRight(String string, int n) {
-    final int x = n - string.length();
-    if (x <= 0) {
-      return string;
-    }
-    // Replacing StringBuffer with String would hurt performance.
-    //noinspection StringBufferReplaceableByString
-    return append(new StringBuilder(string), x).toString();
-  }
-
-  /** Returns a string that is padded on the left with spaces to the given
-   * length. */
-  public static String padLeft(String string, int n) {
-    final int x = n - string.length();
-    if (x <= 0) {
-      return string;
-    }
-    // Replacing StringBuffer with String would hurt performance.
-    //noinspection StringBufferReplaceableByString
-    return append(new StringBuilder(), x).append(string).toString();
-  }
-
-  /** A string of spaces. */
-  private static class SpaceString implements CharSequence {
-    private final int length;
-
-    private SpaceString(int length) {
-      this.length = length;
-    }
-
-    // Do not override equals and hashCode to be like String. CharSequence does
-    // not require it.
-
-    @SuppressWarnings("NullableProblems")
-    @Override public String toString() {
-      return of(length);
-    }
-
-    public int length() {
-      return length;
-    }
-
-    public char charAt(int index) {
-      return ' ';
-    }
-
-    public CharSequence subSequence(int start, int end) {
-      return new SpaceString(end - start);
-    }
-  }
-
-  /** List whose {@code i}th entry is a string consisting of {@code i} spaces.
-   * It populates itself the first time you ask for a particular string, and
-   * caches the result. */
-  private static class SpaceList extends CopyOnWriteArrayList<String> {
-    @Override public String get(int index) {
-      for (;;) {
-        try {
-          return super.get(index);
-        } catch (IndexOutOfBoundsException e) {
-          if (index < 0) {
-            throw e;
-          }
-          populate(Math.max(16, index + 1));
-        }
-      }
-    }
-
-    /**
-     * Populates this list with all prefix strings of a given string. All
-     * of the prefix strings share the same backing array of chars.
-     */
-    private synchronized void populate(int newSize) {
-      final int size = size();
-      if (newSize <= size) {
-        return;
-      }
-      final char[] chars = new char[newSize];
-      Arrays.fill(chars, ' ');
-      final int length = newSize - size;
-      final int offset = size;
-
-      // addAll is much more efficient than repeated add for
-      // CopyOnWriteArrayList
-      addAll(
-          new AbstractList<String>() {
-            public String get(int index) {
-              return new String(chars, 0, offset + index);
-            }
-
-            public int size() {
-              return length;
-            }
-          });
-    }
-  }
-}
-
-// End Spaces.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
index a213ac3..11b359b 100644
--- a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
+++ b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
@@ -17,14 +17,14 @@
 package org.apache.calcite.runtime;
 
 import org.apache.calcite.DataContext;
-import org.apache.calcite.avatica.ByteString;
+import org.apache.calcite.avatica.util.ByteString;
+import org.apache.calcite.avatica.util.DateTimeUtils;
 import org.apache.calcite.linq4j.Enumerable;
 import org.apache.calcite.linq4j.Linq4j;
 import org.apache.calcite.linq4j.function.Deterministic;
 import org.apache.calcite.linq4j.function.Function1;
 import org.apache.calcite.linq4j.function.NonDeterministic;
 import org.apache.calcite.linq4j.tree.Primitive;
-import org.apache.calcite.util.DateTimeUtil;
 
 import java.math.BigDecimal;
 import java.math.BigInteger;
@@ -55,9 +55,6 @@ public class SqlFunctions {
   private static final DecimalFormat DOUBLE_FORMAT =
       new DecimalFormat("0.0E0");
 
-  /** The julian date of the epoch, 1970-01-01. */
-  public static final int EPOCH_JULIAN = 2440588;
-
   private static final TimeZone LOCAL_TZ = TimeZone.getDefault();
 
   private static final Function1<List<Object>, Enumerable<Object>>
@@ -913,7 +910,7 @@ public class SqlFunctions {
   }
 
   public static int toInt(java.util.Date v, TimeZone timeZone) {
-    return (int) (toLong(v, timeZone)  / DateTimeUtil.MILLIS_PER_DAY);
+    return (int) (toLong(v, timeZone)  / DateTimeUtils.MILLIS_PER_DAY);
   }
 
   public static Integer toIntOptional(java.util.Date v) {
@@ -931,7 +928,7 @@ public class SqlFunctions {
   }
 
   public static int toInt(java.sql.Time v) {
-    return (int) (toLong(v) % DateTimeUtil.MILLIS_PER_DAY);
+    return (int) (toLong(v) % DateTimeUtils.MILLIS_PER_DAY);
   }
 
   public static Integer toIntOptional(java.sql.Time v) {
@@ -1067,16 +1064,6 @@ public class SqlFunctions {
     return s.indexOf(seek) + 1;
   }
 
-  /** Cheap, unsafe, long power. power(2, 3) returns 8. */
-  public static long powerX(long a, long b) {
-    long x = 1;
-    while (b > 0) {
-      x *= a;
-      --b;
-    }
-    return x;
-  }
-
   /** Helper for rounding. Truncate(12345, 1000) returns 12000. */
   public static long round(long v, long x) {
     return truncate(v + x / 2, x);
@@ -1105,42 +1092,6 @@ public class SqlFunctions {
     return v - remainder;
   }
 
-  /** Helper for CAST({timestamp} AS VARCHAR(n)). */
-  public static String unixTimestampToString(long timestamp) {
-    final StringBuilder buf = new StringBuilder(17);
-    int date = (int) (timestamp / DateTimeUtil.MILLIS_PER_DAY);
-    int time = (int) (timestamp % DateTimeUtil.MILLIS_PER_DAY);
-    if (time < 0) {
-      --date;
-      time += DateTimeUtil.MILLIS_PER_DAY;
-    }
-    unixDateToString(buf, date);
-    buf.append(' ');
-    unixTimeToString(buf, time);
-    return buf.toString();
-  }
-
-  /** Helper for CAST({timestamp} AS VARCHAR(n)). */
-  public static String unixTimeToString(int time) {
-    final StringBuilder buf = new StringBuilder(8);
-    unixTimeToString(buf, time);
-    return buf.toString();
-  }
-
-  private static void unixTimeToString(StringBuilder buf, int time) {
-    int h = time / 3600000;
-    int time2 = time % 3600000;
-    int m = time2 / 60000;
-    int time3 = time2 % 60000;
-    int s = time3 / 1000;
-    int ms = time3 % 1000;
-    int2(buf, h);
-    buf.append(':');
-    int2(buf, m);
-    buf.append(':');
-    int2(buf, s);
-  }
-
   /** SQL {@code CURRENT_TIMESTAMP} function. */
   @NonDeterministic
   public static long currentTimestamp(DataContext root) {
@@ -1151,9 +1102,9 @@ public class SqlFunctions {
   /** SQL {@code CURRENT_TIME} function. */
   @NonDeterministic
   public static int currentTime(DataContext root) {
-    int time = (int) (currentTimestamp(root) % DateTimeUtil.MILLIS_PER_DAY);
+    int time = (int) (currentTimestamp(root) % DateTimeUtils.MILLIS_PER_DAY);
     if (time < 0) {
-      time += DateTimeUtil.MILLIS_PER_DAY;
+      time += DateTimeUtils.MILLIS_PER_DAY;
     }
     return time;
   }
@@ -1162,8 +1113,8 @@ public class SqlFunctions {
   @NonDeterministic
   public static int currentDate(DataContext root) {
     final long timestamp = currentTimestamp(root);
-    int date = (int) (timestamp / DateTimeUtil.MILLIS_PER_DAY);
-    final int time = (int) (timestamp % DateTimeUtil.MILLIS_PER_DAY);
+    int date = (int) (timestamp / DateTimeUtils.MILLIS_PER_DAY);
+    final int time = (int) (timestamp % DateTimeUtils.MILLIS_PER_DAY);
     if (time < 0) {
       --date;
     }
@@ -1180,418 +1131,7 @@ public class SqlFunctions {
   /** SQL {@code LOCAL_TIME} function. */
   @NonDeterministic
   public static int localTime(DataContext root) {
-    return (int) (localTimestamp(root) % DateTimeUtil.MILLIS_PER_DAY);
-  }
-
-  private static void int2(StringBuilder buf, int i) {
-    buf.append((char) ('0' + (i / 10) % 10));
-    buf.append((char) ('0' + i % 10));
-  }
-
-  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));
-  }
-
-  public static int dateStringToUnixDate(String s) {
-    int hyphen1 = s.indexOf('-');
-    int y;
-    int m;
-    int d;
-    if (hyphen1 < 0) {
-      y = Integer.parseInt(s.trim());
-      m = 1;
-      d = 1;
-    } else {
-      y = Integer.parseInt(s.substring(0, hyphen1).trim());
-      final int hyphen2 = s.indexOf('-', hyphen1 + 1);
-      if (hyphen2 < 0) {
-        m = Integer.parseInt(s.substring(hyphen1 + 1).trim());
-        d = 1;
-      } else {
-        m = Integer.parseInt(s.substring(hyphen1 + 1, hyphen2).trim());
-        d = Integer.parseInt(s.substring(hyphen2 + 1).trim());
-      }
-    }
-    return ymdToUnixDate(y, m, d);
-  }
-
-  public static int timeStringToUnixDate(String v) {
-    return timeStringToUnixDate(v, 0);
-  }
-
-  public static int timeStringToUnixDate(String v, int start) {
-    final int colon1 = v.indexOf(':', start);
-    int hour;
-    int minute;
-    int second;
-    int milli;
-    if (colon1 < 0) {
-      hour = Integer.parseInt(v.trim());
-      minute = 1;
-      second = 1;
-      milli = 0;
-    } else {
-      hour = Integer.parseInt(v.substring(start, colon1).trim());
-      final int colon2 = v.indexOf(':', colon1 + 1);
-      if (colon2 < 0) {
-        minute = Integer.parseInt(v.substring(colon1 + 1).trim());
-        second = 1;
-        milli = 0;
-      } else {
-        minute = Integer.parseInt(v.substring(colon1 + 1, colon2).trim());
-        int dot = v.indexOf('.', colon2);
-        if (dot < 0) {
-          second = Integer.parseInt(v.substring(colon2 + 1).trim());
-          milli = 0;
-        } else {
-          second = Integer.parseInt(v.substring(colon2 + 1, dot).trim());
-          milli = Integer.parseInt(v.substring(dot + 1).trim());
-        }
-      }
-    }
-    return hour * (int) DateTimeUtil.MILLIS_PER_HOUR
-        + minute * (int) DateTimeUtil.MILLIS_PER_MINUTE
-        + second * (int) DateTimeUtil.MILLIS_PER_SECOND
-        + milli;
-  }
-
-  public static long timestampStringToUnixDate(String s) {
-    final long d;
-    final long t;
-    s = s.trim();
-    int space = s.indexOf(' ');
-    if (space >= 0) {
-      d = dateStringToUnixDate(s.substring(0, space));
-      t = timeStringToUnixDate(s, space + 1);
-    } else {
-      d = dateStringToUnixDate(s);
-      t = 0;
-    }
-    return d * DateTimeUtil.MILLIS_PER_DAY + t;
-  }
-
-  /** Helper for CAST({date} AS VARCHAR(n)). */
-  public static String unixDateToString(int date) {
-    final StringBuilder buf = new StringBuilder(10);
-    unixDateToString(buf, date);
-    return buf.toString();
-  }
-
-  private static void unixDateToString(StringBuilder buf, int date) {
-    julianToString(buf, date + EPOCH_JULIAN);
-  }
-
-  private static void julianToString(StringBuilder buf, int julian) {
-    // this shifts the epoch back to astronomical year -4800 instead of the
-    // start of the Christian era in year AD 1 of the proleptic Gregorian
-    // calendar.
-    int j = julian + 32044;
-    int g = j / 146097;
-    int dg = j % 146097;
-    int c = (dg / 36524 + 1) * 3 / 4;
-    int dc = dg - c * 36524;
-    int b = dc / 1461;
-    int db = dc % 1461;
-    int a = (db / 365 + 1) * 3 / 4;
-    int da = db - a * 365;
-
-    // integer number of full years elapsed since March 1, 4801 BC
-    int y = g * 400 + c * 100 + b * 4 + a;
-    // integer number of full months elapsed since the last March 1
-    int m = (da * 5 + 308) / 153 - 2;
-    // number of days elapsed since day 1 of the month
-    int d = da - (m + 4) * 153 / 5 + 122;
-    int year = y - 4800 + (m + 2) / 12;
-    int month = (m + 2) % 12 + 1;
-    int day = d + 1;
-    int4(buf, year);
-    buf.append('-');
-    int2(buf, month);
-    buf.append('-');
-    int2(buf, day);
-  }
-
-  public static long unixDateExtract(TimeUnitRange range, long date) {
-    return julianExtract(range, (int) date + EPOCH_JULIAN);
-  }
-
-  private static int julianExtract(TimeUnitRange range, int julian) {
-    // this shifts the epoch back to astronomical year -4800 instead of the
-    // start of the Christian era in year AD 1 of the proleptic Gregorian
-    // calendar.
-    int j = julian + 32044;
-    int g = j / 146097;
-    int dg = j % 146097;
-    int c = (dg / 36524 + 1) * 3 / 4;
-    int dc = dg - c * 36524;
-    int b = dc / 1461;
-    int db = dc % 1461;
-    int a = (db / 365 + 1) * 3 / 4;
-    int da = db - a * 365;
-
-    // integer number of full years elapsed since March 1, 4801 BC
-    int y = g * 400 + c * 100 + b * 4 + a;
-    // integer number of full months elapsed since the last March 1
-    int m = (da * 5 + 308) / 153 - 2;
-    // number of days elapsed since day 1 of the month
-    int d = da - (m + 4) * 153 / 5 + 122;
-    int year = y - 4800 + (m + 2) / 12;
-    int month = (m + 2) % 12 + 1;
-    int day = d + 1;
-    switch (range) {
-    case YEAR:
-      return year;
-    case MONTH:
-      return month;
-    case DAY:
-      return day;
-    default:
-      throw new AssertionError(range);
-    }
-  }
-
-  public static int ymdToUnixDate(int year, int month, int day) {
-    final int julian = ymdToJulian(year, month, day);
-    return julian - EPOCH_JULIAN;
-  }
-
-  public static int ymdToJulian(int year, int month, int day) {
-    int a = (14 - month) / 12;
-    int y = year + 4800 - a;
-    int m = month + 12 * a - 3;
-    int j = day + (153 * m + 2) / 5
-        + 365 * y
-        + y / 4
-        - y / 100
-        + y / 400
-        - 32045;
-    if (j < 2299161) {
-      j = day + (153 * m + 2) / 5 + 365 * y + y / 4 - 32083;
-    }
-    return j;
-  }
-
-  public static String intervalYearMonthToString(int v, TimeUnitRange range) {
-    final StringBuilder buf = new StringBuilder();
-    if (v >= 0) {
-      buf.append('+');
-    } else {
-      buf.append('-');
-      v = -v;
-    }
-    final int y;
-    final int m;
-    switch (range) {
-    case YEAR:
-      v = roundUp(v, 12);
-      y = v / 12;
-      buf.append(y);
-      break;
-    case YEAR_TO_MONTH:
-      y = v / 12;
-      buf.append(y);
-      buf.append('-');
-      m = v % 12;
-      number(buf, m, 2);
-      break;
-    case MONTH:
-      m = v;
-      buf.append(m);
-      break;
-    default:
-      throw new AssertionError(range);
-    }
-    return buf.toString();
-  }
-
-  private static StringBuilder number(StringBuilder buf, int v, int n) {
-    for (int k = digitCount(v); k < n; k++) {
-      buf.append('0');
-    }
-    return buf.append(v);
-  }
-
-  public static int digitCount(int v) {
-    for (int n = 1;; n++) {
-      v /= 10;
-      if (v == 0) {
-        return n;
-      }
-    }
-  }
-
-  public static String intervalDayTimeToString(long v, TimeUnitRange range,
-      int scale) {
-    final StringBuilder buf = new StringBuilder();
-    if (v >= 0) {
-      buf.append('+');
-    } else {
-      buf.append('-');
-      v = -v;
-    }
-    final long ms;
-    final long s;
-    final long m;
-    final long h;
-    final long d;
-    switch (range) {
-    case DAY_TO_SECOND:
-      v = roundUp(v, powerX(10, 3 - scale));
-      ms = v % 1000;
-      v /= 1000;
-      s = v % 60;
-      v /= 60;
-      m = v % 60;
-      v /= 60;
-      h = v % 24;
-      v /= 24;
-      d = v;
-      buf.append((int) d);
-      buf.append(' ');
-      number(buf, (int) h, 2);
-      buf.append(':');
-      number(buf, (int) m, 2);
-      buf.append(':');
-      number(buf, (int) s, 2);
-      fraction(buf, scale, ms);
-      break;
-    case DAY_TO_MINUTE:
-      v = roundUp(v, 1000 * 60);
-      v /= 1000;
-      v /= 60;
-      m = v % 60;
-      v /= 60;
-      h = v % 24;
-      v /= 24;
-      d = v;
-      buf.append((int) d);
-      buf.append(' ');
-      number(buf, (int) h, 2);
-      buf.append(':');
-      number(buf, (int) m, 2);
-      break;
-    case DAY_TO_HOUR:
-      v = roundUp(v, 1000 * 60 * 60);
-      v /= 1000;
-      v /= 60;
-      v /= 60;
-      h = v % 24;
-      v /= 24;
-      d = v;
-      buf.append((int) d);
-      buf.append(' ');
-      number(buf, (int) h, 2);
-      break;
-    case DAY:
-      v = roundUp(v, 1000 * 60 * 60 * 24);
-      d = v / (1000 * 60 * 60 * 24);
-      buf.append((int) d);
-      break;
-    case HOUR:
-      v = roundUp(v, 1000 * 60 * 60);
-      v /= 1000;
-      v /= 60;
-      v /= 60;
-      h = v;
-      buf.append((int) h);
-      break;
-    case HOUR_TO_MINUTE:
-      v = roundUp(v, 1000 * 60);
-      v /= 1000;
-      v /= 60;
-      m = v % 60;
-      v /= 60;
-      h = v;
-      buf.append((int) h);
-      buf.append(':');
-      number(buf, (int) m, 2);
-      break;
-    case HOUR_TO_SECOND:
-      v = roundUp(v, powerX(10, 3 - scale));
-      ms = v % 1000;
-      v /= 1000;
-      s = v % 60;
-      v /= 60;
-      m = v % 60;
-      v /= 60;
-      h = v;
-      buf.append((int) h);
-      buf.append(':');
-      number(buf, (int) m, 2);
-      buf.append(':');
-      number(buf, (int) s, 2);
-      fraction(buf, scale, ms);
-      break;
-    case MINUTE_TO_SECOND:
-      v = roundUp(v, powerX(10, 3 - scale));
-      ms = v % 1000;
-      v /= 1000;
-      s = v % 60;
-      v /= 60;
-      m = v;
-      buf.append((int) m);
-      buf.append(':');
-      number(buf, (int) s, 2);
-      fraction(buf, scale, ms);
-      break;
-    case MINUTE:
-      v = roundUp(v, 1000 * 60);
-      v /= 1000;
-      v /= 60;
-      m = v;
-      buf.append((int) m);
-      break;
-    case SECOND:
-      v = roundUp(v, powerX(10, 3 - scale));
-      ms = v % 1000;
-      v /= 1000;
-      s = v;
-      buf.append((int) s);
-      fraction(buf, scale, ms);
-      break;
-    default:
-      throw new AssertionError(range);
-    }
-    return buf.toString();
-  }
-
-  /**
-   * Rounds a dividend to the nearest divisor.
-   * For example roundUp(31, 10) yields 30; roundUp(37, 10) yields 40.
-   * @param dividend Number to be divided
-   * @param divisor Number to divide by
-   * @return Rounded dividend
-   */
-  private static long roundUp(long dividend, long divisor) {
-    long remainder = dividend % divisor;
-    dividend -= remainder;
-    if (remainder * 2 > divisor) {
-      dividend += divisor;
-    }
-    return dividend;
-  }
-
-  private static int roundUp(int dividend, int divisor) {
-    int remainder = dividend % divisor;
-    dividend -= remainder;
-    if (remainder * 2 > divisor) {
-      dividend += divisor;
-    }
-    return dividend;
-  }
-
-  private static void fraction(StringBuilder buf, int scale, long ms) {
-    if (scale > 0) {
-      buf.append('.');
-      long v1 = scale == 3 ? ms
-          : scale == 2 ? ms / 10
-          : scale == 1 ? ms / 100
-            : 0;
-      number(buf, (int) v1, scale);
-    }
+    return (int) (localTimestamp(root) % DateTimeUtils.MILLIS_PER_DAY);
   }
 
   /** Helper for "array element reference". Caller has already ensured that
@@ -1673,29 +1213,6 @@ public class SqlFunctions {
     return (Function1<List<E>, Enumerable<E>>) (Function1) LIST_AS_ENUMERABLE;
   }
 
-  /** A range of time units. The first is more significant than the
-   * other (e.g. year-to-day) or the same as the other
-   * (e.g. month). */
-  public enum TimeUnitRange {
-    YEAR,
-    YEAR_TO_MONTH,
-    MONTH,
-    DAY,
-    DAY_TO_HOUR,
-    DAY_TO_MINUTE,
-    DAY_TO_SECOND,
-    HOUR,
-    HOUR_TO_MINUTE,
-    HOUR_TO_SECOND,
-    MINUTE,
-    MINUTE_TO_SECOND,
-    SECOND;
-
-    /** Whether this is in the YEAR-TO-MONTH family of intervals. */
-    public boolean monthly() {
-      return ordinal() <= MONTH.ordinal();
-    }
-  }
 }
 
 // End SqlFunctions.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/core/src/main/java/org/apache/calcite/schema/Schemas.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/Schemas.java b/core/src/main/java/org/apache/calcite/schema/Schemas.java
index 42df429..4172aab 100644
--- a/core/src/main/java/org/apache/calcite/schema/Schemas.java
+++ b/core/src/main/java/org/apache/calcite/schema/Schemas.java
@@ -291,7 +291,7 @@ public final class Schemas {
   }
 
   /** Prepares a SQL query for execution. For use within Calcite only. */
-  public static CalcitePrepare.PrepareResult<Object> prepare(
+  public static CalcitePrepare.CalciteSignature<Object> prepare(
       final CalciteConnection connection, final CalciteSchema schema,
       final List<String> schemaPath, final String sql,
       final ImmutableMap<CalciteConnectionProperty, String> map) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/core/src/main/java/org/apache/calcite/server/CalciteServer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/server/CalciteServer.java b/core/src/main/java/org/apache/calcite/server/CalciteServer.java
index ade037d..b5e8b8e 100644
--- a/core/src/main/java/org/apache/calcite/server/CalciteServer.java
+++ b/core/src/main/java/org/apache/calcite/server/CalciteServer.java
@@ -16,6 +16,9 @@
  */
 package org.apache.calcite.server;
 
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.jdbc.CalciteConnection;
+
 /**
  * Server.
  *
@@ -23,9 +26,11 @@ package org.apache.calcite.server;
  * management facilities.
  */
 public interface CalciteServer {
-  void removeStatement(CalciteServerStatement calciteServerStatement);
+  void removeStatement(Meta.StatementHandle h);
+
+  void addStatement(CalciteConnection connection, Meta.StatementHandle h);
 
-  void addStatement(CalciteServerStatement calciteServerStatement);
+  CalciteServerStatement getStatement(Meta.StatementHandle h);
 }
 
 // End CalciteServer.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/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 9486c58..9a7aed9 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDateLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDateLiteral.java
@@ -16,11 +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 org.apache.calcite.util.DateTimeUtil;
 
 import java.util.Calendar;
 
@@ -34,7 +34,7 @@ public class SqlDateLiteral extends SqlAbstractDateTimeLiteral {
   //~ Constructors -----------------------------------------------------------
 
   SqlDateLiteral(Calendar d, SqlParserPos pos) {
-    super(d, false, SqlTypeName.DATE, 0, DateTimeUtil.DATE_FORMAT_STRING, pos);
+    super(d, false, SqlTypeName.DATE, 0, DateTimeUtils.DATE_FORMAT_STRING, pos);
   }
 
   SqlDateLiteral(Calendar d, String format, SqlParserPos pos) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java b/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java
index 1d036a7..089cbf6 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java
@@ -16,24 +16,21 @@
  */
 package org.apache.calcite.sql;
 
+import org.apache.calcite.avatica.util.TimeUnit;
+import org.apache.calcite.avatica.util.TimeUnitRange;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.runtime.CalciteContextException;
-import org.apache.calcite.runtime.SqlFunctions;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.util.SqlVisitor;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
-import org.apache.calcite.util.DateTimeUtil;
-import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
 import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableMap;
 
 import java.math.BigDecimal;
-import java.util.Map;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -94,120 +91,10 @@ public class SqlIntervalQualifier extends SqlNode {
   private static final BigDecimal INT_MAX_VALUE_PLUS_ONE =
       BigDecimal.valueOf(Integer.MAX_VALUE).add(BigDecimal.ONE);
 
-  //~ Enums ------------------------------------------------------------------
-
-  /**
-   * Enumeration of time units used to construct an interval.
-   */
-  public enum TimeUnit implements SqlLiteral.SqlSymbol {
-    YEAR(true, ' ', 12 /* months */, null),
-    MONTH(true, '-', 1 /* months */, BigDecimal.valueOf(12)),
-    DAY(false, '-', DateTimeUtil.MILLIS_PER_DAY, null),
-    HOUR(false, ' ', DateTimeUtil.MILLIS_PER_HOUR, BigDecimal.valueOf(24)),
-    MINUTE(false, ':', DateTimeUtil.MILLIS_PER_MINUTE, BigDecimal.valueOf(60)),
-    SECOND(false, ':', DateTimeUtil.MILLIS_PER_SECOND, BigDecimal.valueOf(60));
-
-    public final boolean yearMonth;
-    public final char separator;
-    public final long multiplier;
-    private final BigDecimal limit;
-
-    private static final TimeUnit[] CACHED_VALUES = values();
-
-    private TimeUnit(
-        boolean yearMonth,
-        char separator,
-        long multiplier,
-        BigDecimal limit) {
-      this.yearMonth = yearMonth;
-      this.separator = separator;
-      this.multiplier = multiplier;
-      this.limit = limit;
-    }
-
-    /**
-     * Returns the TimeUnit associated with an ordinal. The value returned
-     * is null if the ordinal is not a member of the TimeUnit enumeration.
-     */
-    public static TimeUnit getValue(int ordinal) {
-      return ordinal < 0 || ordinal >= CACHED_VALUES.length
-          ? null
-          : CACHED_VALUES[ordinal];
-    }
-
-    public static final String GET_VALUE_METHOD_NAME = "getValue";
-
-    /**
-     * Returns whether a given value is valid for a field of this time unit.
-     *
-     * @param field Field value
-     * @return Whether value
-     */
-    public boolean isValidValue(BigDecimal field) {
-      return field.compareTo(ZERO) >= 0
-          && (limit == null
-          || field.compareTo(limit) < 0);
-    }
-  }
-
-  /** Range of time units. */
-  private enum TimeUnitRange {
-    YEAR(TimeUnit.YEAR, null),
-    YEAR_TO_MONTH(TimeUnit.YEAR, TimeUnit.MONTH),
-    MONTH(TimeUnit.MONTH, null),
-    DAY(TimeUnit.DAY, null),
-    DAY_TO_HOUR(TimeUnit.DAY, TimeUnit.HOUR),
-    DAY_TO_MINUTE(TimeUnit.DAY, TimeUnit.MINUTE),
-    DAY_TO_SECOND(TimeUnit.DAY, TimeUnit.SECOND),
-    HOUR(TimeUnit.HOUR, null),
-    HOUR_TO_MINUTE(TimeUnit.HOUR, TimeUnit.MINUTE),
-    HOUR_TO_SECOND(TimeUnit.HOUR, TimeUnit.SECOND),
-    MINUTE(TimeUnit.MINUTE, null),
-    MINUTE_TO_SECOND(TimeUnit.MINUTE, TimeUnit.SECOND),
-    SECOND(TimeUnit.SECOND, null);
-
-    private final TimeUnit startUnit;
-    private final TimeUnit endUnit;
-    private static final Map<Pair<TimeUnit, TimeUnit>, TimeUnitRange> MAP;
-
-    static {
-      ImmutableMap.Builder<Pair<TimeUnit, TimeUnit>, TimeUnitRange> builder =
-          ImmutableMap.builder();
-      for (TimeUnitRange value : values()) {
-        builder.put(Pair.of(value.startUnit, value.endUnit), value);
-      }
-      MAP = builder.build();
-    }
-
-    /**
-     * Creates a TimeUnitRange.
-     *
-     * @param startUnit Start time unit
-     * @param endUnit   End time unit
-     */
-    TimeUnitRange(TimeUnit startUnit, TimeUnit endUnit) {
-      assert startUnit != null;
-      this.startUnit = startUnit;
-      this.endUnit = endUnit;
-    }
-
-    /**
-     * Returns a TimeUnitRange with a given start and end unit.
-     *
-     * @param startUnit Start unit
-     * @param endUnit   End unit
-     * @return Time unit range, or null if not valid
-     */
-    public static TimeUnitRange of(
-        TimeUnit startUnit, TimeUnit endUnit) {
-      return MAP.get(new Pair<TimeUnit, TimeUnit>(startUnit, endUnit));
-    }
-  }
-
   //~ Instance fields --------------------------------------------------------
 
   private final int startPrecision;
-  private final TimeUnitRange timeUnitRange;
+  public final TimeUnitRange timeUnitRange;
   private final int fractionalSecondPrecision;
 
   //~ Constructors -----------------------------------------------------------
@@ -245,10 +132,6 @@ public class SqlIntervalQualifier extends SqlNode {
         : SqlTypeName.INTERVAL_DAY_TIME;
   }
 
-  public SqlFunctions.TimeUnitRange foo() {
-    return SqlFunctions.TimeUnitRange.valueOf(timeUnitRange.name());
-  }
-
   public void validate(
       SqlValidator validator,
       SqlValidatorScope scope) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/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 de2b32a..3990a8a 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
@@ -16,6 +16,7 @@
  */
 package org.apache.calcite.sql;
 
+import org.apache.calcite.avatica.util.TimeUnitRange;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.fun.SqlLiteralChainOperator;
@@ -200,11 +201,10 @@ public class SqlLiteral extends SqlNode {
       return value instanceof BitString;
     case CHAR:
       return value instanceof NlsString;
-
     case SYMBOL:
-
       return (value instanceof SqlSymbol)
-          || (value instanceof SqlSampleSpec);
+          || (value instanceof SqlSampleSpec)
+          || (value instanceof TimeUnitRange);
     case MULTISET:
       return true;
     case INTEGER: // not allowed -- use Decimal

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/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 f80990e..31572e1 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlTimeLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlTimeLiteral.java
@@ -16,9 +16,9 @@
  */
 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.DateTimeUtil;
 
 import java.util.Calendar;
 
@@ -40,7 +40,7 @@ public class SqlTimeLiteral extends SqlAbstractDateTimeLiteral {
         t,
         hasTZ,
         SqlTypeName.TIME,
-        precision, DateTimeUtil.TIME_FORMAT_STRING,
+        precision, DateTimeUtils.TIME_FORMAT_STRING,
         pos);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/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 f5db1d8..af31b42 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlTimestampLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlTimestampLiteral.java
@@ -16,9 +16,9 @@
  */
 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.DateTimeUtil;
 
 import java.util.Calendar;
 
@@ -40,7 +40,7 @@ public class SqlTimestampLiteral extends SqlAbstractDateTimeLiteral {
         cal,
         hasTimeZone,
         SqlTypeName.TIMESTAMP,
-        precision, DateTimeUtil.TIMESTAMP_FORMAT_STRING,
+        precision, DateTimeUtils.TIMESTAMP_FORMAT_STRING,
         pos);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/core/src/main/java/org/apache/calcite/sql/parser/SqlAbstractParserImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/parser/SqlAbstractParserImpl.java b/core/src/main/java/org/apache/calcite/sql/parser/SqlAbstractParserImpl.java
index b08278c..6876447 100644
--- a/core/src/main/java/org/apache/calcite/sql/parser/SqlAbstractParserImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/parser/SqlAbstractParserImpl.java
@@ -16,7 +16,7 @@
  */
 package org.apache.calcite.sql.parser;
 
-import org.apache.calcite.avatica.Casing;
+import org.apache.calcite.avatica.util.Casing;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlIdentifier;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/core/src/main/java/org/apache/calcite/sql/parser/SqlParser.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/parser/SqlParser.java b/core/src/main/java/org/apache/calcite/sql/parser/SqlParser.java
index 7c0660e..1cf71fa 100644
--- a/core/src/main/java/org/apache/calcite/sql/parser/SqlParser.java
+++ b/core/src/main/java/org/apache/calcite/sql/parser/SqlParser.java
@@ -16,8 +16,8 @@
  */
 package org.apache.calcite.sql.parser;
 
-import org.apache.calcite.avatica.Casing;
-import org.apache.calcite.avatica.Quoting;
+import org.apache.calcite.avatica.util.Casing;
+import org.apache.calcite.avatica.util.Quoting;
 import org.apache.calcite.config.Lex;
 import org.apache.calcite.runtime.CalciteContextException;
 import org.apache.calcite.sql.SqlNode;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/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 b184a86..c48ab66 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
@@ -16,7 +16,7 @@
  */
 package org.apache.calcite.sql.parser;
 
-import org.apache.calcite.avatica.Casing;
+import org.apache.calcite.avatica.util.Casing;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.runtime.CalciteContextException;
 import org.apache.calcite.sql.SqlBinaryOperator;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/core/src/main/java/org/apache/calcite/sql/pretty/SqlPrettyWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/pretty/SqlPrettyWriter.java b/core/src/main/java/org/apache/calcite/sql/pretty/SqlPrettyWriter.java
index d064e33..4098d52 100644
--- a/core/src/main/java/org/apache/calcite/sql/pretty/SqlPrettyWriter.java
+++ b/core/src/main/java/org/apache/calcite/sql/pretty/SqlPrettyWriter.java
@@ -16,7 +16,7 @@
  */
 package org.apache.calcite.sql.pretty;
 
-import org.apache.calcite.runtime.Spaces;
+import org.apache.calcite.avatica.util.Spaces;
 import org.apache.calcite.sql.SqlDialect;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlWriter;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/core/src/main/java/org/apache/calcite/sql/type/IntervalSqlType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/IntervalSqlType.java b/core/src/main/java/org/apache/calcite/sql/type/IntervalSqlType.java
index 73188b4..f2083d0 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/IntervalSqlType.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/IntervalSqlType.java
@@ -16,6 +16,7 @@
  */
 package org.apache.calcite.sql.type;
 
+import org.apache.calcite.avatica.util.TimeUnit;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactoryImpl;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
@@ -88,13 +89,13 @@ public class IntervalSqlType extends AbstractSqlType {
     assert this.intervalQualifier.isYearMonth()
         == that.intervalQualifier.isYearMonth();
     boolean nullable = isNullable || that.isNullable;
-    SqlIntervalQualifier.TimeUnit thisStart =
+    TimeUnit thisStart =
         this.intervalQualifier.getStartUnit();
-    SqlIntervalQualifier.TimeUnit thisEnd =
+    TimeUnit thisEnd =
         this.intervalQualifier.getEndUnit();
-    SqlIntervalQualifier.TimeUnit thatStart =
+    TimeUnit thatStart =
         that.intervalQualifier.getStartUnit();
-    SqlIntervalQualifier.TimeUnit thatEnd =
+    TimeUnit thatEnd =
         that.intervalQualifier.getEndUnit();
 
     assert null != thisStart;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/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 0391923..a7f6880 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
@@ -16,10 +16,10 @@
  */
 package org.apache.calcite.sql.type;
 
+import org.apache.calcite.avatica.util.DateTimeUtils;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.parser.SqlParserPos;
-import org.apache.calcite.util.DateTimeUtil;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
@@ -543,7 +543,7 @@ public enum SqlTypeName {
       return bytes;
 
     case DATE:
-      calendar = Calendar.getInstance(DateTimeUtil.GMT_ZONE);
+      calendar = Calendar.getInstance(DateTimeUtils.GMT_ZONE);
       switch (limit) {
       case ZERO:
 
@@ -591,7 +591,7 @@ public enum SqlTypeName {
       if (beyond) {
         return null; // invalid values are impossible to represent
       }
-      calendar = Calendar.getInstance(DateTimeUtil.GMT_ZONE);
+      calendar = Calendar.getInstance(DateTimeUtils.GMT_ZONE);
       switch (limit) {
       case ZERO:
 
@@ -616,7 +616,7 @@ public enum SqlTypeName {
       return calendar;
 
     case TIMESTAMP:
-      calendar = Calendar.getInstance(DateTimeUtil.GMT_ZONE);
+      calendar = Calendar.getInstance(DateTimeUtils.GMT_ZONE);
       switch (limit) {
       case ZERO:
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/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 3504f29..07e0fce 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlNodeToRexConverterImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlNodeToRexConverterImpl.java
@@ -16,7 +16,7 @@
  */
 package org.apache.calcite.sql2rel;
 
-import org.apache.calcite.avatica.ByteString;
+import org.apache.calcite.avatica.util.ByteString;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rex.RexBuilder;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java b/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
index 74fc92e..987b99b 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
@@ -16,6 +16,8 @@
  */
 package org.apache.calcite.sql2rel;
 
+import org.apache.calcite.avatica.util.DateTimeUtils;
+import org.apache.calcite.avatica.util.TimeUnit;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
@@ -59,7 +61,6 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.util.DateTimeUtil;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
@@ -521,7 +522,7 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
     RexNode res = rexBuilder.makeReinterpretCast(
         resType, exprs.get(1), rexBuilder.makeLiteral(false));
 
-    final SqlIntervalQualifier.TimeUnit unit =
+    final TimeUnit unit =
         ((SqlIntervalQualifier) operands.get(0)).getStartUnit();
     final SqlTypeName sqlTypeName = exprs.get(1).getType().getSqlTypeName();
     switch (unit) {
@@ -533,7 +534,7 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
       case INTERVAL_DAY_TIME:
         break;
       case TIMESTAMP:
-        res = divide(rexBuilder, res, DateTimeUtil.MILLIS_PER_DAY);
+        res = divide(rexBuilder, res, DateTimeUtils.MILLIS_PER_DAY);
         // fall through
       case DATE:
         return rexBuilder.makeCall(resType, SqlStdOperatorTable.EXTRACT_DATE,
@@ -548,20 +549,20 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
     return res;
   }
 
-  private static long getFactor(SqlIntervalQualifier.TimeUnit unit) {
+  private static long getFactor(TimeUnit unit) {
     switch (unit) {
     case DAY:
       return 1;
     case HOUR:
-      return SqlIntervalQualifier.TimeUnit.DAY.multiplier;
+      return TimeUnit.DAY.multiplier;
     case MINUTE:
-      return SqlIntervalQualifier.TimeUnit.HOUR.multiplier;
+      return TimeUnit.HOUR.multiplier;
     case SECOND:
-      return SqlIntervalQualifier.TimeUnit.MINUTE.multiplier;
+      return TimeUnit.MINUTE.multiplier;
     case YEAR:
       return 1;
     case MONTH:
-      return SqlIntervalQualifier.TimeUnit.YEAR.multiplier;
+      return TimeUnit.YEAR.multiplier;
     default:
       throw Util.unexpected(unit);
     }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/core/src/main/java/org/apache/calcite/tools/Frameworks.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/tools/Frameworks.java b/core/src/main/java/org/apache/calcite/tools/Frameworks.java
index a8ba4b4..a9efcc0 100644
--- a/core/src/main/java/org/apache/calcite/tools/Frameworks.java
+++ b/core/src/main/java/org/apache/calcite/tools/Frameworks.java
@@ -17,7 +17,6 @@
 package org.apache.calcite.tools;
 
 import org.apache.calcite.config.CalciteConnectionProperty;
-import org.apache.calcite.jdbc.CalciteConnection;
 import org.apache.calcite.jdbc.CalciteSchema;
 import org.apache.calcite.plan.Context;
 import org.apache.calcite.plan.RelOptCluster;
@@ -140,10 +139,8 @@ public class Frameworks {
       }
       Connection connection =
           DriverManager.getConnection("jdbc:calcite:", info);
-      CalciteConnection calciteConnection =
-          connection.unwrap(CalciteConnection.class);
       final CalciteServerStatement statement =
-          calciteConnection.createStatement()
+          connection.createStatement()
               .unwrap(CalciteServerStatement.class);
       return new CalcitePrepareImpl().perform(statement, action);
     } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
index 98e0007..be15c2e 100644
--- a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
+++ b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
@@ -19,6 +19,8 @@ package org.apache.calcite.util;
 import org.apache.calcite.DataContext;
 import org.apache.calcite.adapter.java.ReflectiveSchema;
 import org.apache.calcite.adapter.jdbc.JdbcSchema;
+import org.apache.calcite.avatica.util.DateTimeUtils;
+import org.apache.calcite.avatica.util.TimeUnitRange;
 import org.apache.calcite.interpreter.Row;
 import org.apache.calcite.linq4j.AbstractEnumerable;
 import org.apache.calcite.linq4j.Enumerable;
@@ -229,20 +231,20 @@ public enum BuiltInMethod {
   MODIFIABLE_TABLE_GET_MODIFIABLE_COLLECTION(ModifiableTable.class,
       "getModifiableCollection"),
   STRING_TO_BOOLEAN(SqlFunctions.class, "toBoolean", String.class),
-  STRING_TO_DATE(SqlFunctions.class, "dateStringToUnixDate", String.class),
-  STRING_TO_TIME(SqlFunctions.class, "timeStringToUnixDate", String.class),
-  STRING_TO_TIMESTAMP(SqlFunctions.class, "timestampStringToUnixDate",
+  STRING_TO_DATE(DateTimeUtils.class, "dateStringToUnixDate", String.class),
+  STRING_TO_TIME(DateTimeUtils.class, "timeStringToUnixDate", String.class),
+  STRING_TO_TIMESTAMP(DateTimeUtils.class, "timestampStringToUnixDate",
       String.class),
-  UNIX_DATE_TO_STRING(SqlFunctions.class, "unixDateToString", int.class),
-  UNIX_TIME_TO_STRING(SqlFunctions.class, "unixTimeToString", int.class),
-  UNIX_TIMESTAMP_TO_STRING(SqlFunctions.class, "unixTimestampToString",
+  UNIX_DATE_TO_STRING(DateTimeUtils.class, "unixDateToString", int.class),
+  UNIX_TIME_TO_STRING(DateTimeUtils.class, "unixTimeToString", int.class),
+  UNIX_TIMESTAMP_TO_STRING(DateTimeUtils.class, "unixTimestampToString",
       long.class),
-  INTERVAL_YEAR_MONTH_TO_STRING(SqlFunctions.class, "intervalYearMonthToString",
-      int.class, SqlFunctions.TimeUnitRange.class),
-  INTERVAL_DAY_TIME_TO_STRING(SqlFunctions.class, "intervalDayTimeToString",
-      long.class, SqlFunctions.TimeUnitRange.class, int.class),
-  UNIX_DATE_EXTRACT(SqlFunctions.class, "unixDateExtract",
-      SqlFunctions.TimeUnitRange.class, long.class),
+  INTERVAL_YEAR_MONTH_TO_STRING(DateTimeUtils.class,
+      "intervalYearMonthToString", int.class, TimeUnitRange.class),
+  INTERVAL_DAY_TIME_TO_STRING(DateTimeUtils.class, "intervalDayTimeToString",
+      long.class, TimeUnitRange.class, int.class),
+  UNIX_DATE_EXTRACT(DateTimeUtils.class, "unixDateExtract",
+      TimeUnitRange.class, long.class),
   CURRENT_TIMESTAMP(SqlFunctions.class, "currentTimestamp", DataContext.class),
   CURRENT_TIME(SqlFunctions.class, "currentTime", DataContext.class),
   CURRENT_DATE(SqlFunctions.class, "currentDate", DataContext.class),

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/core/src/main/java/org/apache/calcite/util/Compatible.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Compatible.java b/core/src/main/java/org/apache/calcite/util/Compatible.java
index 41b5123..95a696b 100644
--- a/core/src/main/java/org/apache/calcite/util/Compatible.java
+++ b/core/src/main/java/org/apache/calcite/util/Compatible.java
@@ -24,6 +24,7 @@ import com.google.common.collect.Maps;
 import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.Method;
 import java.lang.reflect.Proxy;
+import java.sql.Connection;
 import java.util.Map;
 import java.util.NavigableMap;
 import java.util.NavigableSet;
@@ -52,6 +53,12 @@ public interface Compatible {
    * immutable. */
   <K, V> NavigableMap<K, V> immutableNavigableMap(NavigableMap<K, V> map);
 
+  /** Calls {@link java.sql.Connection}{@code .setSchema(String)}.
+   *
+   * <p>This method is available in JDK 1.7 and above, and in
+   * {@link org.apache.calcite.jdbc.CalciteConnection} in all JDK versions. */
+  void setSchema(Connection connection, String schema);
+
   /** Creates the implementation of Compatible suitable for the
    * current environment. */
   class Factory {
@@ -87,6 +94,13 @@ public interface Compatible {
                 ImmutableSortedMap sortedMap = ImmutableSortedMap.copyOf(map);
                 return CompatibleGuava11.navigableMap(sortedMap);
               }
+              if (method.getName().equals("setSchema")) {
+                Connection connection = (Connection) args[0];
+                String schema = (String) args[1];
+                final Method method1 =
+                    connection.getClass().getMethod("setSchema", String.class);
+                return method1.invoke(connection, schema);
+              }
               return null;
             }
           });

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/core/src/main/java/org/apache/calcite/util/DateTimeUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/DateTimeUtil.java b/core/src/main/java/org/apache/calcite/util/DateTimeUtil.java
deleted file mode 100644
index 82b8665..0000000
--- a/core/src/main/java/org/apache/calcite/util/DateTimeUtil.java
+++ /dev/null
@@ -1,286 +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.text.NumberFormat;
-import java.text.ParsePosition;
-import java.text.SimpleDateFormat;
-import java.util.Calendar;
-import java.util.TimeZone;
-
-/**
- * Utility functions for datetime types: date, time, timestamp. Refactored from
- * SqlParserUtil because they are required by the Jdbc driver. TODO: review
- * methods for performance. Due to allocations required, it may be preferable to
- * introduce a "formatter" with the required state.
- */
-public class DateTimeUtil {
-  private DateTimeUtil() {}
-
-  //~ Static fields/initializers ---------------------------------------------
-
-  /** The SimpleDateFormat string for ISO dates, "yyyy-MM-dd". */
-  public static final String DATE_FORMAT_STRING = "yyyy-MM-dd";
-
-  /** The SimpleDateFormat string for ISO times, "HH:mm:ss". */
-  public static final String TIME_FORMAT_STRING = "HH:mm:ss";
-
-  /** The SimpleDateFormat string for ISO timestamps, "yyyy-MM-dd HH:mm:ss". */
-  public static final String TIMESTAMP_FORMAT_STRING =
-      DATE_FORMAT_STRING + " " + TIME_FORMAT_STRING;
-
-  /** The GMT time zone. */
-  public static final TimeZone GMT_ZONE = TimeZone.getTimeZone("GMT");
-
-  /** The Java default time zone. */
-  public static final TimeZone DEFAULT_ZONE = TimeZone.getDefault();
-
-  /**
-   * The number of milliseconds in a second.
-   */
-  public static final long MILLIS_PER_SECOND = 1000L;
-
-  /**
-   * The number of milliseconds in a minute.
-   */
-  public static final long MILLIS_PER_MINUTE = 60000L;
-
-  /**
-   * The number of milliseconds in an hour.
-   */
-  public static final long MILLIS_PER_HOUR = 3600000L; // = 60 * 60 * 1000
-
-  /**
-   * The number of milliseconds in a day.
-   *
-   * <p>This is the modulo 'mask' used when converting
-   * TIMESTAMP values to DATE and TIME values.
-   */
-  public static final long MILLIS_PER_DAY = 86400000; // = 24 * 60 * 60 * 1000
-
-  /**
-   * Calendar set to the epoch (1970-01-01 00:00:00 UTC). Useful for
-   * initializing other values. Calendars are not immutable, so be careful not
-   * to screw up this object for everyone else.
-   */
-  public static final Calendar ZERO_CALENDAR;
-
-  static {
-    ZERO_CALENDAR = Calendar.getInstance(DateTimeUtil.GMT_ZONE);
-    ZERO_CALENDAR.setTimeInMillis(0);
-  }
-
-  /**
-   * Calendar set to local time.
-   */
-  private static final Calendar LOCAL_CALENDAR = Calendar.getInstance();
-
-  //~ Methods ----------------------------------------------------------------
-
-  /**
-   * Parses a string using {@link SimpleDateFormat} and a given pattern. This
-   * method parses a string at the specified parse position and if successful,
-   * updates the parse position to the index after the last character used.
-   * The parsing is strict and requires months to be less than 12, days to be
-   * less than 31, etc.
-   *
-   * @param s       string to be parsed
-   * @param pattern {@link SimpleDateFormat}  pattern
-   * @param tz      time zone in which to interpret string. Defaults to the Java
-   *                default time zone
-   * @param pp      position to start parsing from
-   * @return a Calendar initialized with the parsed value, or null if parsing
-   * failed. If returned, the Calendar is configured to the GMT time zone.
-   * @pre pattern != null
-   */
-  private static Calendar parseDateFormat(
-      String s,
-      String pattern,
-      TimeZone tz,
-      ParsePosition pp) {
-    assert pattern != null;
-    SimpleDateFormat df = new SimpleDateFormat(pattern);
-    if (tz == null) {
-      tz = DEFAULT_ZONE;
-    }
-    Calendar ret = Calendar.getInstance(tz);
-    df.setCalendar(ret);
-    df.setLenient(false);
-
-    java.util.Date d = df.parse(s, pp);
-    if (null == d) {
-      return null;
-    }
-    ret.setTime(d);
-    ret.setTimeZone(GMT_ZONE);
-    return ret;
-  }
-
-  /**
-   * Parses a string using {@link SimpleDateFormat} and a given pattern. The
-   * entire string must match the pattern specified.
-   *
-   * @param s       string to be parsed
-   * @param pattern {@link SimpleDateFormat}  pattern
-   * @param tz      time zone in which to interpret string. Defaults to the Java
-   *                default time zone
-   * @return a Calendar initialized with the parsed value, or null if parsing
-   * failed. If returned, the Calendar is configured to the GMT time zone.
-   */
-  public static Calendar parseDateFormat(
-      String s,
-      String pattern,
-      TimeZone tz) {
-    assert pattern != null;
-    ParsePosition pp = new ParsePosition(0);
-    Calendar ret = parseDateFormat(s, pattern, tz, pp);
-    if (pp.getIndex() != s.length()) {
-      // Didn't consume entire string - not good
-      return null;
-    }
-    return ret;
-  }
-
-  /**
-   * Parses a string using {@link SimpleDateFormat} and a given pattern, and
-   * if present, parses a fractional seconds component. The fractional seconds
-   * component must begin with a decimal point ('.') followed by numeric
-   * digits. The precision is rounded to a maximum of 3 digits of fractional
-   * seconds precision (to obtain milliseconds).
-   *
-   * @param s       string to be parsed
-   * @param pattern {@link SimpleDateFormat}  pattern
-   * @param tz      time zone in which to interpret string. Defaults to the
-   *                local time zone
-   * @return a {@link DateTimeUtil.PrecisionTime PrecisionTime} initialized
-   * with the parsed value, or null if parsing failed. The PrecisionTime
-   * contains a GMT Calendar and a precision.
-   */
-  public static PrecisionTime parsePrecisionDateTimeLiteral(
-      String s,
-      String pattern,
-      TimeZone tz) {
-    assert pattern != null;
-    ParsePosition pp = new ParsePosition(0);
-    Calendar cal = parseDateFormat(s, pattern, tz, pp);
-    if (cal == null) {
-      return null; // Invalid date/time format
-    }
-
-    // Note: the Java SimpleDateFormat 'S' treats any number after
-    // the decimal as milliseconds. That means 12:00:00.9 has 9
-    // milliseconds and 12:00:00.9999 has 9999 milliseconds.
-    int p = 0;
-    if (pp.getIndex() < s.length()) {
-      // Check to see if rest is decimal portion
-      if (s.charAt(pp.getIndex()) != '.') {
-        return null;
-      }
-
-      // Skip decimal sign
-      pp.setIndex(pp.getIndex() + 1);
-
-      // Parse decimal portion
-      if (pp.getIndex() < s.length()) {
-        String secFraction = s.substring(pp.getIndex());
-        if (!secFraction.matches("\\d+")) {
-          return null;
-        }
-        NumberFormat nf = NumberFormat.getIntegerInstance();
-        Number num = nf.parse(s, pp);
-        if ((num == null) || (pp.getIndex() != s.length())) {
-          // Invalid decimal portion
-          return null;
-        }
-
-        // Determine precision - only support prec 3 or lower
-        // (milliseconds) Higher precisions are quietly rounded away
-        p = Math.min(
-            3,
-            secFraction.length());
-
-        // Calculate milliseconds
-        int ms =
-            (int) Math.round(
-                num.longValue()
-                * Math.pow(10, 3 - secFraction.length()));
-        cal.add(Calendar.MILLISECOND, ms);
-      }
-    }
-
-    assert pp.getIndex() == s.length();
-    PrecisionTime ret = new PrecisionTime(cal, p);
-    return ret;
-  }
-
-  /**
-   * Gets the active time zone based on a Calendar argument
-   */
-  public static TimeZone getTimeZone(Calendar cal) {
-    if (cal == null) {
-      return DEFAULT_ZONE;
-    }
-    return cal.getTimeZone();
-  }
-
-  /**
-   * Checks if the date/time format is valid
-   *
-   * @param pattern {@link SimpleDateFormat}  pattern
-   * @throws IllegalArgumentException if the given pattern is invalid
-   */
-  public static void checkDateFormat(String pattern) {
-    new SimpleDateFormat(pattern);
-  }
-
-  /**
-   * Creates a new date formatter with Farrago specific options. Farrago
-   * parsing is strict and does not allow values such as day 0, month 13, etc.
-   *
-   * @param format {@link SimpleDateFormat}  pattern
-   */
-  public static SimpleDateFormat newDateFormat(String format) {
-    SimpleDateFormat sdf = new SimpleDateFormat(format);
-    sdf.setLenient(false);
-    return sdf;
-  }
-
-  //~ Inner Classes ----------------------------------------------------------
-
-  /**
-   * Helper class for {@link DateTimeUtil#parsePrecisionDateTimeLiteral}
-   */
-  public static class PrecisionTime {
-    private final Calendar cal;
-    private final int precision;
-
-    public PrecisionTime(Calendar cal, int precision) {
-      this.cal = cal;
-      this.precision = precision;
-    }
-
-    public Calendar getCalendar() {
-      return cal;
-    }
-
-    public int getPrecision() {
-      return precision;
-    }
-  }
-}
-
-// End DateTimeUtil.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/core/src/main/java/org/apache/calcite/util/JsonBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/JsonBuilder.java b/core/src/main/java/org/apache/calcite/util/JsonBuilder.java
index 378fd63..fcf4d30 100644
--- a/core/src/main/java/org/apache/calcite/util/JsonBuilder.java
+++ b/core/src/main/java/org/apache/calcite/util/JsonBuilder.java
@@ -16,7 +16,7 @@
  */
 package org.apache.calcite.util;
 
-import org.apache.calcite.runtime.Spaces;
+import org.apache.calcite.avatica.util.Spaces;
 
 import java.util.ArrayList;
 import java.util.LinkedHashMap;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/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
index 9648327..0d5fc16 100644
--- a/core/src/main/java/org/apache/calcite/util/ZonelessDate.java
+++ b/core/src/main/java/org/apache/calcite/util/ZonelessDate.java
@@ -16,6 +16,8 @@
  */
 package org.apache.calcite.util;
 
+import org.apache.calcite.avatica.util.DateTimeUtils;
+
 import java.sql.Date;
 import java.text.DateFormat;
 import java.util.Calendar;
@@ -60,7 +62,7 @@ public class ZonelessDate extends ZonelessDatetime {
 
   // implement ZonelessDatetime
   public Object toJdbcObject() {
-    return new Date(getJdbcDate(DateTimeUtil.DEFAULT_ZONE));
+    return new Date(getJdbcDate(DateTimeUtils.DEFAULT_ZONE));
   }
 
   /**
@@ -70,7 +72,7 @@ public class ZonelessDate extends ZonelessDatetime {
    * @return the formatted date string
    */
   public String toString() {
-    Date jdbcDate = getTempDate(getJdbcDate(DateTimeUtil.DEFAULT_ZONE));
+    Date jdbcDate = getTempDate(getJdbcDate(DateTimeUtils.DEFAULT_ZONE));
     return jdbcDate.toString();
   }
 
@@ -95,7 +97,7 @@ public class ZonelessDate extends ZonelessDatetime {
    * @return the parsed date, or null if parsing failed
    */
   public static ZonelessDate parse(String s) {
-    return parse(s, DateTimeUtil.DATE_FORMAT_STRING);
+    return parse(s, DateTimeUtils.DATE_FORMAT_STRING);
   }
 
   /**
@@ -108,10 +110,7 @@ public class ZonelessDate extends ZonelessDatetime {
    */
   public static ZonelessDate parse(String s, String format) {
     Calendar cal =
-        DateTimeUtil.parseDateFormat(
-            s,
-            format,
-            DateTimeUtil.GMT_ZONE);
+        DateTimeUtils.parseDateFormat(s, format, DateTimeUtils.GMT_ZONE);
     if (cal == null) {
       return null;
     }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/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
index abe8165..13e956b 100644
--- a/core/src/main/java/org/apache/calcite/util/ZonelessDatetime.java
+++ b/core/src/main/java/org/apache/calcite/util/ZonelessDatetime.java
@@ -16,6 +16,8 @@
  */
 package org.apache.calcite.util;
 
+import org.apache.calcite.avatica.util.DateTimeUtils;
+
 import java.io.Serializable;
 import java.text.DateFormat;
 import java.util.Calendar;
@@ -71,9 +73,9 @@ public abstract class ZonelessDatetime implements BasicDatetime, Serializable {
     // 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 % DateTimeUtil.MILLIS_PER_DAY;
+    long timePart = internalTime % DateTimeUtils.MILLIS_PER_DAY;
     if (timePart < 0) {
-      timePart += DateTimeUtil.MILLIS_PER_DAY;
+      timePart += DateTimeUtils.MILLIS_PER_DAY;
     }
     return timePart;
   }
@@ -117,7 +119,7 @@ public abstract class ZonelessDatetime implements BasicDatetime, Serializable {
    * @param zone time zone in which to generate a time value for
    */
   public long getJdbcDate(TimeZone zone) {
-    Calendar cal = getCalendar(DateTimeUtil.GMT_ZONE);
+    Calendar cal = getCalendar(DateTimeUtils.GMT_ZONE);
     cal.setTimeInMillis(getDateValue());
 
     int year = cal.get(Calendar.YEAR);
@@ -137,7 +139,7 @@ public abstract class ZonelessDatetime implements BasicDatetime, Serializable {
    * @param zone time zone in which to generate a time value for
    */
   public long getJdbcTimestamp(TimeZone zone) {
-    Calendar cal = getCalendar(DateTimeUtil.GMT_ZONE);
+    Calendar cal = getCalendar(DateTimeUtils.GMT_ZONE);
     cal.setTimeInMillis(internalTime);
 
     int year = cal.get(Calendar.YEAR);
@@ -186,8 +188,8 @@ public abstract class ZonelessDatetime implements BasicDatetime, Serializable {
     if ((tempFormatter != null) && lastFormat.equals(format)) {
       return tempFormatter;
     }
-    tempFormatter = DateTimeUtil.newDateFormat(format);
-    tempFormatter.setTimeZone(DateTimeUtil.GMT_ZONE);
+    tempFormatter = DateTimeUtils.newDateFormat(format);
+    tempFormatter.setTimeZone(DateTimeUtils.GMT_ZONE);
     lastFormat = format;
     return tempFormatter;
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/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
index d55cb70..28a788c 100644
--- a/core/src/main/java/org/apache/calcite/util/ZonelessTime.java
+++ b/core/src/main/java/org/apache/calcite/util/ZonelessTime.java
@@ -16,6 +16,8 @@
  */
 package org.apache.calcite.util;
 
+import org.apache.calcite.avatica.util.DateTimeUtils;
+
 import java.sql.Time;
 import java.text.DateFormat;
 import java.text.SimpleDateFormat;
@@ -76,7 +78,7 @@ public class ZonelessTime extends ZonelessDatetime {
 
   // implement ZonelessDatetime
   public Object toJdbcObject() {
-    return new Time(getJdbcTime(DateTimeUtil.DEFAULT_ZONE));
+    return new Time(getJdbcTime(DateTimeUtils.DEFAULT_ZONE));
   }
 
   /**
@@ -87,7 +89,7 @@ public class ZonelessTime extends ZonelessDatetime {
    * as specified by the SQL standard.
    */
   public long getJdbcTimestamp(TimeZone zone) {
-    Calendar cal = getCalendar(DateTimeUtil.GMT_ZONE);
+    Calendar cal = getCalendar(DateTimeUtils.GMT_ZONE);
     cal.setTimeInMillis(getTime());
     int hour = cal.get(Calendar.HOUR_OF_DAY);
     int minute = cal.get(Calendar.MINUTE);
@@ -110,7 +112,7 @@ public class ZonelessTime extends ZonelessDatetime {
    * @return the formatted time string
    */
   public String toString() {
-    Time jdbcTime = getTempTime(getJdbcTime(DateTimeUtil.DEFAULT_ZONE));
+    Time jdbcTime = getTempTime(getJdbcTime(DateTimeUtils.DEFAULT_ZONE));
     return jdbcTime.toString();
   }
 
@@ -134,7 +136,7 @@ public class ZonelessTime extends ZonelessDatetime {
    * @return the parsed time, or null if parsing failed
    */
   public static ZonelessTime parse(String s) {
-    return parse(s, DateTimeUtil.TIME_FORMAT_STRING);
+    return parse(s, DateTimeUtils.TIME_FORMAT_STRING);
   }
 
   /**
@@ -145,11 +147,10 @@ public class ZonelessTime extends ZonelessDatetime {
    * @return the parsed time, or null if parsing failed
    */
   public static ZonelessTime parse(String s, String format) {
-    DateTimeUtil.PrecisionTime pt =
-        DateTimeUtil.parsePrecisionDateTimeLiteral(
-            s,
+    DateTimeUtils.PrecisionTime pt =
+        DateTimeUtils.parsePrecisionDateTimeLiteral(s,
             format,
-            DateTimeUtil.GMT_ZONE);
+            DateTimeUtils.GMT_ZONE);
     if (pt == null) {
       return null;
     }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/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
index 94a6b76..40a7281 100644
--- a/core/src/main/java/org/apache/calcite/util/ZonelessTimestamp.java
+++ b/core/src/main/java/org/apache/calcite/util/ZonelessTimestamp.java
@@ -16,6 +16,8 @@
  */
 package org.apache.calcite.util;
 
+import org.apache.calcite.avatica.util.DateTimeUtils;
+
 import java.sql.Timestamp;
 import java.text.DateFormat;
 
@@ -62,7 +64,7 @@ public class ZonelessTimestamp extends ZonelessDatetime {
 
   // implement ZonelessDatetime
   public Object toJdbcObject() {
-    return new Timestamp(getJdbcTimestamp(DateTimeUtil.DEFAULT_ZONE));
+    return new Timestamp(getJdbcTimestamp(DateTimeUtils.DEFAULT_ZONE));
   }
 
   /**
@@ -78,7 +80,7 @@ public class ZonelessTimestamp extends ZonelessDatetime {
    */
   public String toString() {
     Timestamp ts =
-        getTempTimestamp(getJdbcTimestamp(DateTimeUtil.DEFAULT_ZONE));
+        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"
@@ -110,7 +112,7 @@ public class ZonelessTimestamp extends ZonelessDatetime {
    * @return the parsed time, or null if parsing failed
    */
   public static ZonelessTimestamp parse(String s) {
-    return parse(s, DateTimeUtil.TIMESTAMP_FORMAT_STRING);
+    return parse(s, DateTimeUtils.TIMESTAMP_FORMAT_STRING);
   }
 
   /**
@@ -125,11 +127,10 @@ public class ZonelessTimestamp extends ZonelessDatetime {
    * @return the parsed timestamp, or null if parsing failed
    */
   public static ZonelessTimestamp parse(String s, String format) {
-    DateTimeUtil.PrecisionTime pt =
-        DateTimeUtil.parsePrecisionDateTimeLiteral(
-            s,
+    DateTimeUtils.PrecisionTime pt =
+        DateTimeUtils.parsePrecisionDateTimeLiteral(s,
             format,
-            DateTimeUtil.GMT_ZONE);
+            DateTimeUtils.GMT_ZONE);
     if (pt == null) {
       return null;
     }