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 2015/04/03 15:02:34 UTC

[01/10] incubator-calcite git commit: [CALCITE-651] In JdbcMeta, convert property definitions to an enum (Nick Dimiduk)

Repository: incubator-calcite
Updated Branches:
  refs/heads/master 3b55c35a5 -> 30ae0513c


[CALCITE-651] In JdbcMeta, convert property definitions to an enum (Nick Dimiduk)

Close apache/incubator-calcite#68


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

Branch: refs/heads/master
Commit: 17d7a8e3dea6e4868a5851ec46bf400b9caf0038
Parents: 3b55c35
Author: Nick Dimiduk <nd...@gmail.com>
Authored: Tue Mar 31 14:59:51 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Apr 3 01:08:41 2015 -0700

----------------------------------------------------------------------
 .../apache/calcite/avatica/jdbc/JdbcMeta.java   | 159 ++++++++++++-------
 1 file changed, 104 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/17d7a8e3/avatica/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java b/avatica/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
index 627a0d7..90e8c2a 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
@@ -87,55 +87,94 @@ public class JdbcMeta implements Meta {
     SQL_TYPE_TO_JAVA_TYPE.put(Types.ARRAY, Array.class);
   }
 
-  //
-  // Constants for connection cache settings.
-  //
+  /** A "magic column" descriptor used to return statement execute update count via a
+   * {@link Meta.MetaResultSet}. */
+  private static final ColumnMetaData UPDATE_COL = new ColumnMetaData(1, false, false, false,
+      false, ResultSetMetaData.columnNoNulls, true, 10, "u", "u", null, 15, 15, "update_result",
+      "avatica_internal", ColumnMetaData.scalar(Types.INTEGER, "INTEGER",
+        ColumnMetaData.Rep.INTEGER), true, false, false, "java.lang.Integer");
 
   private static final String CONN_CACHE_KEY_BASE = "avatica.connectioncache";
-  /** JDBC connection property for setting connection cache concurrency level. */
-  public static final String CONN_CACHE_CONCURRENCY_KEY =
-      CONN_CACHE_KEY_BASE + ".concurrency";
-  public static final String DEFAULT_CONN_CACHE_CONCURRENCY_LEVEL = "10";
-  /** JDBC connection property for setting connection cache initial capacity. */
-  public static final String CONN_CACHE_INITIAL_CAPACITY_KEY =
-      CONN_CACHE_KEY_BASE + ".initialcapacity";
-  public static final String DEFAULT_CONN_CACHE_INITIAL_CAPACITY = "100";
-  /** JDBC connection property for setting connection cache maximum capacity. */
-  public static final String CONN_CACHE_MAX_CAPACITY_KEY =
-      CONN_CACHE_KEY_BASE + ".maxcapacity";
-  public static final String DEFAULT_CONN_CACHE_MAX_CAPACITY = "1000";
-  /** JDBC connection property for setting connection cache expiration duration. */
-  public static final String CONN_CACHE_EXPIRY_DURATION_KEY =
-      CONN_CACHE_KEY_BASE + ".expirydiration";
-  public static final String DEFAULT_CONN_CACHE_EXPIRY_DURATION = "10";
-  /** JDBC connection property for setting connection cache expiration unit. */
-  public static final String CONN_CACHE_EXPIRY_UNIT_KEY = CONN_CACHE_KEY_BASE + ".expiryunit";
-  public static final String DEFAULT_CONN_CACHE_EXPIRY_UNIT = TimeUnit.MINUTES.name();
-
-  //
-  // Constants for statement cache settings.
-  //
+
+  /** Configurable connection cache settings. */
+  public enum ConnectionCacheSettings {
+    /** JDBC connection property for setting connection cache concurrency level. */
+    CONCURRENCY_LEVEL(CONN_CACHE_KEY_BASE + ".concurrency", "10"),
+
+    /** JDBC connection property for setting connection cache initial capacity. */
+    INITIAL_CAPACITY(CONN_CACHE_KEY_BASE + ".initialcapacity", "100"),
+
+    /** JDBC connection property for setting connection cache maximum capacity. */
+    MAX_CAPACITY(CONN_CACHE_KEY_BASE + ".maxcapacity", "1000"),
+
+    /** JDBC connection property for setting connection cache expiration duration. */
+    EXPIRY_DURATION(CONN_CACHE_KEY_BASE + ".expirydiration", "10"),
+
+    /** JDBC connection property for setting connection cache expiration unit. */
+    EXPIRY_UNIT(CONN_CACHE_KEY_BASE + ".expiryunit", TimeUnit.MINUTES.name());
+
+    private final String key;
+    private final String defaultValue;
+
+    private ConnectionCacheSettings(String key, String defaultValue) {
+      this.key = key;
+      this.defaultValue = defaultValue;
+    }
+
+    /** The configuration key for specifying this setting. */
+    public String key() {
+      return key;
+    }
+
+    /** The default value for this setting. */
+    public String defaultValue() {
+      return defaultValue;
+    }
+  }
 
   private static final String STMT_CACHE_KEY_BASE = "avatica.statementcache";
-  /** JDBC connection property for setting connection cache concurrency level. */
-  public static final String STMT_CACHE_CONCURRENCY_KEY =
-      STMT_CACHE_KEY_BASE + ".concurrency";
-  public static final String DEFAULT_STMT_CACHE_CONCURRENCY_LEVEL = "100";
-  /** JDBC connection property for setting connection cache initial capacity. */
-  public static final String STMT_CACHE_INITIAL_CAPACITY_KEY =
-      STMT_CACHE_KEY_BASE + ".initialcapacity";
-  public static final String DEFAULT_STMT_CACHE_INITIAL_CAPACITY = "1000";
-  /** JDBC connection property for setting connection cache maximum capacity. */
-  public static final String STMT_CACHE_MAX_CAPACITY_KEY =
-      STMT_CACHE_KEY_BASE + ".maxcapacity";
-  public static final String DEFAULT_STMT_CACHE_MAX_CAPACITY = "10000";
-  /** JDBC connection property for setting connection cache expiration duration. */
-  public static final String STMT_CACHE_EXPIRY_DURATION_KEY =
-      STMT_CACHE_KEY_BASE + ".expirydiration";
-  public static final String DEFAULT_STMT_CACHE_EXPIRY_DURATION = "5";
-  /** JDBC connection property for setting connection cache expiration unit. */
-  public static final String STMT_CACHE_EXPIRY_UNIT_KEY = STMT_CACHE_KEY_BASE + ".expiryunit";
-  public static final String DEFAULT_STMT_CACHE_EXPIRY_UNIT = TimeUnit.MINUTES.name();
+
+  /** Configurable statement cache settings. */
+  public enum StatementCacheSettings {
+    /** JDBC connection property for setting connection cache concurrency level. */
+    CONCURRENCY_LEVEL(STMT_CACHE_KEY_BASE + ".concurrency", "100"),
+
+    /** JDBC connection property for setting connection cache initial capacity. */
+    INITIAL_CAPACITY(STMT_CACHE_KEY_BASE + ".initialcapacity", "1000"),
+
+    /** JDBC connection property for setting connection cache maximum capacity. */
+    MAX_CAPACITY(STMT_CACHE_KEY_BASE + ".maxcapacity", "10000"),
+
+    /** JDBC connection property for setting connection cache expiration duration.
+     *
+     * <p>Used in conjunction with {@link #EXPIRY_UNIT}.</p>
+     */
+    EXPIRY_DURATION(STMT_CACHE_KEY_BASE + ".expirydiration", "5"),
+
+    /** JDBC connection property for setting connection cache expiration unit.
+     *
+     * <p>Used in conjunction with {@link #EXPIRY_DURATION}.</p>
+     */
+    EXPIRY_UNIT(STMT_CACHE_KEY_BASE + ".expiryunit", TimeUnit.MINUTES.name());
+
+    private final String key;
+    private final String defaultValue;
+
+    private StatementCacheSettings(String key, String defaultValue) {
+      this.key = key;
+      this.defaultValue = defaultValue;
+    }
+
+    /** The configuration key for specifying this setting. */
+    public String key() {
+      return key;
+    }
+
+    /** The default value for this setting. */
+    public String defaultValue() {
+      return defaultValue;
+    }
+  }
 
   private static final String DEFAULT_CONN_ID =
       UUID.fromString("00000000-0000-0000-0000-000000000000").toString();
@@ -294,15 +333,20 @@ public class JdbcMeta implements Meta {
     this.connection = DriverManager.getConnection(url, info);
 
     int concurrencyLevel = Integer.parseInt(
-        info.getProperty(CONN_CACHE_CONCURRENCY_KEY, DEFAULT_CONN_CACHE_CONCURRENCY_LEVEL));
+        info.getProperty(ConnectionCacheSettings.CONCURRENCY_LEVEL.key(),
+            ConnectionCacheSettings.CONCURRENCY_LEVEL.defaultValue()));
     int initialCapacity = Integer.parseInt(
-        info.getProperty(CONN_CACHE_INITIAL_CAPACITY_KEY, DEFAULT_CONN_CACHE_INITIAL_CAPACITY));
+        info.getProperty(ConnectionCacheSettings.INITIAL_CAPACITY.key(),
+            ConnectionCacheSettings.INITIAL_CAPACITY.defaultValue()));
     long maxCapacity = Long.parseLong(
-        info.getProperty(CONN_CACHE_MAX_CAPACITY_KEY, DEFAULT_CONN_CACHE_MAX_CAPACITY));
+        info.getProperty(ConnectionCacheSettings.MAX_CAPACITY.key(),
+            ConnectionCacheSettings.MAX_CAPACITY.defaultValue()));
     long connectionExpiryDuration = Long.parseLong(
-        info.getProperty(CONN_CACHE_EXPIRY_DURATION_KEY, DEFAULT_CONN_CACHE_EXPIRY_DURATION));
+        info.getProperty(ConnectionCacheSettings.EXPIRY_DURATION.key(),
+            ConnectionCacheSettings.EXPIRY_DURATION.defaultValue()));
     TimeUnit connectionExpiryUnit = TimeUnit.valueOf(
-        info.getProperty(CONN_CACHE_EXPIRY_UNIT_KEY, DEFAULT_CONN_CACHE_EXPIRY_UNIT));
+        info.getProperty(ConnectionCacheSettings.EXPIRY_UNIT.key(),
+            ConnectionCacheSettings.EXPIRY_UNIT.defaultValue()));
     this.connectionCache = CacheBuilder.newBuilder()
         .concurrencyLevel(concurrencyLevel)
         .initialCapacity(initialCapacity)
@@ -315,15 +359,20 @@ public class JdbcMeta implements Meta {
     }
 
     concurrencyLevel = Integer.parseInt(
-        info.getProperty(STMT_CACHE_CONCURRENCY_KEY, DEFAULT_STMT_CACHE_CONCURRENCY_LEVEL));
+        info.getProperty(StatementCacheSettings.CONCURRENCY_LEVEL.key(),
+            StatementCacheSettings.CONCURRENCY_LEVEL.defaultValue()));
     initialCapacity = Integer.parseInt(
-        info.getProperty(STMT_CACHE_INITIAL_CAPACITY_KEY, DEFAULT_STMT_CACHE_INITIAL_CAPACITY));
+        info.getProperty(StatementCacheSettings.INITIAL_CAPACITY.key(),
+            StatementCacheSettings.INITIAL_CAPACITY.defaultValue()));
     maxCapacity = Long.parseLong(
-        info.getProperty(STMT_CACHE_MAX_CAPACITY_KEY, DEFAULT_STMT_CACHE_MAX_CAPACITY));
+        info.getProperty(StatementCacheSettings.MAX_CAPACITY.key(),
+            StatementCacheSettings.MAX_CAPACITY.defaultValue()));
     connectionExpiryDuration = Long.parseLong(
-        info.getProperty(STMT_CACHE_EXPIRY_DURATION_KEY, DEFAULT_STMT_CACHE_EXPIRY_DURATION));
+        info.getProperty(StatementCacheSettings.EXPIRY_DURATION.key(),
+            StatementCacheSettings.EXPIRY_DURATION.defaultValue()));
     connectionExpiryUnit = TimeUnit.valueOf(
-        info.getProperty(STMT_CACHE_EXPIRY_UNIT_KEY, DEFAULT_STMT_CACHE_EXPIRY_UNIT));
+        info.getProperty(StatementCacheSettings.EXPIRY_UNIT.key(),
+            StatementCacheSettings.EXPIRY_UNIT.defaultValue()));
     this.statementCache = CacheBuilder.newBuilder()
         .concurrencyLevel(concurrencyLevel)
         .initialCapacity(initialCapacity)


[05/10] incubator-calcite git commit: [CALCITE-660] Improve Avatica date support

Posted by jh...@apache.org.
[CALCITE-660] Improve Avatica date support


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

Branch: refs/heads/master
Commit: 5648083f017a1cf28e223f2230c59c20cd713a4b
Parents: 304eb70
Author: Julian Hyde <jh...@apache.org>
Authored: Thu Apr 2 10:24:52 2015 -0400
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Apr 3 01:10:05 2015 -0700

----------------------------------------------------------------------
 .../apache/calcite/avatica/jdbc/JdbcMeta.java   |  14 ++-
 .../calcite/avatica/jdbc/JdbcResultSet.java     |  17 +++
 .../calcite/avatica/RemoteDriverTest.java       |  34 ++++++
 .../calcite/avatica/AvaticaParameter.java       |  95 ++++++++++++++-
 .../calcite/avatica/AvaticaResultSet.java       |   4 +-
 .../apache/calcite/avatica/ColumnMetaData.java  |  14 +--
 .../calcite/avatica/remote/JsonService.java     |  65 +++++++++-
 .../calcite/avatica/remote/LocalService.java    |  42 ++-----
 .../calcite/avatica/util/AbstractCursor.java    | 120 +++++++++++--------
 9 files changed, 306 insertions(+), 99 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5648083f/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
----------------------------------------------------------------------
diff --git a/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java b/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
index b4b6211..698d7a7 100644
--- a/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
+++ b/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
@@ -195,9 +195,21 @@ public class JdbcMeta implements Meta {
     for (int i = 1; i <= metaData.getColumnCount(); i++) {
       final Type javaType =
           SQL_TYPE_TO_JAVA_TYPE.get(metaData.getColumnType(i));
+      final ColumnMetaData.Rep rep;
+      switch (metaData.getColumnType(i)) {
+      case Types.DATE:
+      case Types.TIME:
+        rep = ColumnMetaData.Rep.INTEGER;
+        break;
+      case Types.TIMESTAMP:
+        rep = ColumnMetaData.Rep.LONG;
+        break;
+      default:
+        rep = ColumnMetaData.Rep.of(javaType);
+      }
       ColumnMetaData.AvaticaType t =
           ColumnMetaData.scalar(metaData.getColumnType(i),
-              metaData.getColumnTypeName(i), ColumnMetaData.Rep.of(javaType));
+              metaData.getColumnTypeName(i), rep);
       ColumnMetaData md =
           new ColumnMetaData(i - 1, metaData.isAutoIncrement(i),
               metaData.isCaseSensitive(i), metaData.isSearchable(i),

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5648083f/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcResultSet.java
----------------------------------------------------------------------
diff --git a/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcResultSet.java b/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcResultSet.java
index 827f31d..407b91e 100644
--- a/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcResultSet.java
+++ b/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcResultSet.java
@@ -17,10 +17,14 @@
 package org.apache.calcite.avatica.jdbc;
 
 import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.avatica.util.DateTimeUtils;
 
+import java.sql.Date;
 import java.sql.ResultSet;
 import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
 import java.sql.Types;
 import java.util.ArrayList;
 import java.util.List;
@@ -97,6 +101,19 @@ class JdbcResultSet extends Meta.MetaResultSet {
     case Types.REAL:
       final float aFloat = resultSet.getFloat(j + 1);
       return aFloat == 0D && resultSet.wasNull() ? null : aFloat;
+    case Types.DATE:
+      final Date aDate = resultSet.getDate(j + 1);
+      return aDate == null
+          ? null
+          : (int) (aDate.getTime() / DateTimeUtils.MILLIS_PER_DAY);
+    case Types.TIME:
+      final Time aTime = resultSet.getTime(j + 1);
+      return aTime == null
+          ? null
+          : (int) (aTime.getTime() % DateTimeUtils.MILLIS_PER_DAY);
+    case Types.TIMESTAMP:
+      final Timestamp aTimestamp = resultSet.getTimestamp(j + 1);
+      return aTimestamp == null ? null : aTimestamp.getTime();
     default:
       return resultSet.getObject(j + 1);
     }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5648083f/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
----------------------------------------------------------------------
diff --git a/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java b/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
index c24a7fd..1a5316c 100644
--- a/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
+++ b/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
@@ -215,6 +215,40 @@ public class RemoteDriverTest {
     }
   }
 
+  @Test public void testTypeHandling() throws Exception {
+    final String query = "select * from EMP";
+    try (Connection cannon =
+             DriverManager.getConnection(CONNECTION_SPEC.url,
+                 CONNECTION_SPEC.username, CONNECTION_SPEC.password);
+        Connection underTest = ljs();
+        Statement s1 = cannon.createStatement();
+        Statement s2 = underTest.createStatement()) {
+      assertTrue(s1.execute(query));
+      assertTrue(s2.execute(query));
+      try (ResultSet rs1 = s1.getResultSet();
+          ResultSet rs2 = s2.getResultSet()) {
+        assertEquals(rs1.getMetaData().getColumnCount(), rs2.getMetaData().getColumnCount());
+        int colCount = rs1.getMetaData().getColumnCount();
+        while (rs1.next() && rs2.next()) {
+          for (int i = 0; i < colCount; i++) {
+            Object o1 = rs1.getObject(i + 1);
+            Object o2 = rs2.getObject(i + 1);
+            if (o1 instanceof Integer && o2 instanceof Short) {
+              // Hsqldb returns Integer for short columns; we prefer Short
+              o1 = ((Number) o1).shortValue();
+            }
+            if (o1 instanceof Integer && o2 instanceof Byte) {
+              // Hsqldb returns Integer for tinyint columns; we prefer Byte
+              o1 = ((Number) o1).byteValue();
+            }
+            assertEquals(o1, o2);
+          }
+        }
+        assertEquals(rs1.next(), rs2.next());
+      }
+    }
+  }
+
   @Test public void testStatementLifecycle() throws Exception {
     try (AvaticaConnection connection = (AvaticaConnection) ljs()) {
       Map<Integer, AvaticaStatement> clientMap = connection.statementMap;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5648083f/avatica/src/main/java/org/apache/calcite/avatica/AvaticaParameter.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaParameter.java b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaParameter.java
index 8cbe5e4..8013a3b 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaParameter.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaParameter.java
@@ -17,6 +17,7 @@
 package org.apache.calcite.avatica;
 
 import org.apache.calcite.avatica.util.ByteString;
+import org.apache.calcite.avatica.util.Cursor;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
@@ -33,6 +34,7 @@ import java.sql.Date;
 import java.sql.NClob;
 import java.sql.Ref;
 import java.sql.RowId;
+import java.sql.SQLException;
 import java.sql.SQLXML;
 import java.sql.Time;
 import java.sql.Timestamp;
@@ -301,6 +303,97 @@ public class AvaticaParameter {
     }
   }
 
+  /** Similar logic to {@link #setObject}. */
+  public static Object get(Cursor.Accessor accessor, int targetSqlType,
+      Calendar localCalendar) throws SQLException {
+    switch (targetSqlType) {
+    case Types.CLOB:
+    case Types.DATALINK:
+    case Types.NCLOB:
+    case Types.REF:
+    case Types.SQLXML:
+    case Types.STRUCT:
+      throw notImplemented();
+    case Types.ARRAY:
+      return accessor.getArray();
+    case Types.BIGINT:
+      final long aLong = accessor.getLong();
+      if (aLong == 0 && accessor.wasNull()) {
+        return null;
+      }
+      return aLong;
+    case Types.BINARY:
+    case Types.LONGVARBINARY:
+    case Types.VARBINARY:
+      return accessor.getBytes();
+    case Types.BIT:
+    case Types.BOOLEAN:
+      final boolean aBoolean = accessor.getBoolean();
+      if (!aBoolean && accessor.wasNull()) {
+        return null;
+      }
+      return aBoolean;
+    case Types.BLOB:
+      return accessor.getBlob();
+    case Types.DATE:
+      return accessor.getDate(localCalendar);
+    case Types.DECIMAL:
+    case Types.NUMERIC:
+      return accessor.getBigDecimal();
+    case Types.DISTINCT:
+      throw notImplemented();
+    case Types.DOUBLE:
+    case Types.FLOAT: // yes really; SQL FLOAT is up to 8 bytes
+      final double aDouble = accessor.getDouble();
+      if (aDouble == 0 && accessor.wasNull()) {
+        return null;
+      }
+      return aDouble;
+    case Types.INTEGER:
+      final int anInt = accessor.getInt();
+      if (anInt == 0 && accessor.wasNull()) {
+        return null;
+      }
+      return anInt;
+    case Types.JAVA_OBJECT:
+    case Types.OTHER:
+      return accessor.getObject();
+    case Types.LONGNVARCHAR:
+    case Types.LONGVARCHAR:
+    case Types.NVARCHAR:
+    case Types.VARCHAR:
+    case Types.CHAR:
+    case Types.NCHAR:
+      return accessor.getString();
+    case Types.REAL:
+      final float aFloat = accessor.getFloat();
+      if (aFloat == 0 && accessor.wasNull()) {
+        return null;
+      }
+      return aFloat;
+    case Types.ROWID:
+      throw notImplemented();
+    case Types.SMALLINT:
+      final short aShort = accessor.getShort();
+      if (aShort == 0 && accessor.wasNull()) {
+        return null;
+      }
+      return aShort;
+    case Types.TIME:
+      return accessor.getTime(localCalendar);
+    case Types.TIMESTAMP:
+      return accessor.getTimestamp(localCalendar);
+    case Types.TINYINT:
+      final byte aByte = accessor.getByte();
+      if (aByte == 0 && accessor.wasNull()) {
+        return null;
+      }
+      return aByte;
+    default:
+      throw notImplemented();
+    }
+  }
+
   public void setObject(Object[] slots, int index, Object x) {
     slots[index] = wrap(x);
   }
@@ -354,7 +447,7 @@ public class AvaticaParameter {
     throw unsupportedCast(x.getClass(), Array.class);
   }
 
-  private static BigDecimal toBigDecimal(Object x) {
+  public static BigDecimal toBigDecimal(Object x) {
     if (x instanceof BigDecimal) {
       return (BigDecimal) x;
     } else if (x instanceof BigInteger) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5648083f/avatica/src/main/java/org/apache/calcite/avatica/AvaticaResultSet.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaResultSet.java b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaResultSet.java
index fca227f..6f528b3 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaResultSet.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaResultSet.java
@@ -375,7 +375,9 @@ public class AvaticaResultSet implements ResultSet, ArrayImpl.Factory {
   }
 
   public Object getObject(int columnIndex) throws SQLException {
-    return getAccessor(columnIndex).getObject();
+    final Cursor.Accessor accessor = getAccessor(columnIndex);
+    final ColumnMetaData metaData = columnMetaDataList.get(columnIndex - 1);
+    return AvaticaParameter.get(accessor, metaData.type.id, localCalendar);
   }
 
   public Object getObject(String columnLabel) throws SQLException {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5648083f/avatica/src/main/java/org/apache/calcite/avatica/ColumnMetaData.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/ColumnMetaData.java b/avatica/src/main/java/org/apache/calcite/avatica/ColumnMetaData.java
index 0a4f90a..ed03710 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/ColumnMetaData.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/ColumnMetaData.java
@@ -31,6 +31,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 
 /**
  * Metadata for a column.
@@ -158,10 +159,10 @@ public class ColumnMetaData {
         type.columnClassName());
   }
 
-  public ColumnMetaData setTypeId(int typeId) {
+  public ColumnMetaData setRep(Rep rep) {
     return new ColumnMetaData(ordinal, autoIncrement, caseSensitive, searchable,
         currency, nullable, signed, displaySize, label, columnName, schemaName,
-        precision, scale, tableName, catalogName, type.setId(typeId), readOnly,
+        precision, scale, tableName, catalogName, type.setRep(rep), readOnly,
         writable, definitelyWritable, columnClassName);
   }
 
@@ -233,16 +234,15 @@ public class ColumnMetaData {
 
     protected AvaticaType(int id, String name, Rep rep) {
       this.id = id;
-      this.name = name;
-      this.rep = rep;
-      assert rep != null;
+      this.name = Objects.requireNonNull(name);
+      this.rep = Objects.requireNonNull(rep);
     }
 
     public String columnClassName() {
       return SqlType.valueOf(id).clazz.getName();
     }
 
-    public AvaticaType setId(int rep) {
+    public AvaticaType setRep(Rep rep) {
       throw new UnsupportedOperationException();
     }
   }
@@ -256,7 +256,7 @@ public class ColumnMetaData {
       super(id, name, rep);
     }
 
-    @Override public AvaticaType setId(int id) {
+    @Override public AvaticaType setRep(Rep rep) {
       return new ScalarType(id, name, rep);
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5648083f/avatica/src/main/java/org/apache/calcite/avatica/remote/JsonService.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/remote/JsonService.java b/avatica/src/main/java/org/apache/calcite/avatica/remote/JsonService.java
index d8b94c7..2dd349c 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/remote/JsonService.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/JsonService.java
@@ -16,11 +16,16 @@
  */
 package org.apache.calcite.avatica.remote;
 
+import org.apache.calcite.avatica.ColumnMetaData;
+import org.apache.calcite.avatica.Meta;
+
 import com.fasterxml.jackson.core.JsonParser;
 import com.fasterxml.jackson.databind.ObjectMapper;
 
 import java.io.IOException;
 import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.List;
 
 /**
  * Implementation of {@link org.apache.calcite.avatica.remote.Service}
@@ -41,6 +46,62 @@ public abstract class JsonService implements Service {
    * responses to and from the peer service. */
   public abstract String apply(String request);
 
+  /** Modifies a signature, changing the type of columns within it. */
+  private static Meta.Signature fangle(Meta.Signature signature) {
+    final List<ColumnMetaData> columns = new ArrayList<>();
+    int changeCount = 0;
+    for (ColumnMetaData column : signature.columns) {
+      switch (column.type.rep) {
+      case BYTE:
+      case PRIMITIVE_BYTE:
+      case DOUBLE:
+      case PRIMITIVE_DOUBLE:
+      case FLOAT:
+      case PRIMITIVE_FLOAT:
+      case INTEGER:
+      case PRIMITIVE_INT:
+      case SHORT:
+      case PRIMITIVE_SHORT:
+      case LONG:
+      case PRIMITIVE_LONG:
+        column = column.setRep(ColumnMetaData.Rep.OBJECT);
+        ++changeCount;
+      }
+      columns.add(column);
+    }
+    if (changeCount == 0) {
+      return signature;
+    }
+    return new Meta.Signature(columns, signature.sql,
+        signature.parameters, signature.internalParameters,
+        signature.cursorFactory);
+  }
+
+  private static PrepareResponse fangle(PrepareResponse response) {
+    final Meta.StatementHandle statement = fangle(response.statement);
+    if (statement == response.statement) {
+      return response;
+    }
+    return new PrepareResponse(statement);
+  }
+
+  private static Meta.StatementHandle fangle(Meta.StatementHandle h) {
+    final Meta.Signature signature = fangle(h.signature);
+    if (signature == h.signature) {
+      return h;
+    }
+    return new Meta.StatementHandle(h.connectionId, h.id, signature);
+  }
+
+  private static ResultSetResponse fangle(ResultSetResponse r) {
+    final Meta.Signature signature = fangle(r.signature);
+    if (signature == r.signature) {
+      return r;
+    }
+    return new ResultSetResponse(r.connectionId, r.statementId, r.ownStatement,
+        signature, r.firstFrame);
+  }
+
   //@VisibleForTesting
   protected static <T> T decode(String response, Class<T> valueType)
       throws IOException {
@@ -100,7 +161,7 @@ public abstract class JsonService implements Service {
 
   public PrepareResponse apply(PrepareRequest request) {
     try {
-      return decode(apply(encode(request)), PrepareResponse.class);
+      return fangle(decode(apply(encode(request)), PrepareResponse.class));
     } catch (IOException e) {
       throw handle(e);
     }
@@ -108,7 +169,7 @@ public abstract class JsonService implements Service {
 
   public ResultSetResponse apply(PrepareAndExecuteRequest request) {
     try {
-      return decode(apply(encode(request)), ResultSetResponse.class);
+      return fangle(decode(apply(encode(request)), ResultSetResponse.class));
     } catch (IOException e) {
       throw handle(e);
     }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5648083f/avatica/src/main/java/org/apache/calcite/avatica/remote/LocalService.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/remote/LocalService.java b/avatica/src/main/java/org/apache/calcite/avatica/remote/LocalService.java
index 5bbfa36..2bdb1bd 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/remote/LocalService.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/LocalService.java
@@ -16,11 +16,9 @@
  */
 package org.apache.calcite.avatica.remote;
 
-import org.apache.calcite.avatica.ColumnMetaData;
 import org.apache.calcite.avatica.Meta;
 import org.apache.calcite.avatica.MetaImpl;
 
-import java.sql.Types;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
@@ -30,8 +28,6 @@ import java.util.List;
  */
 public class LocalService implements Service {
   final Meta meta;
-  /** Whether output is going to JSON. */
-  private final boolean json = true;
 
   public LocalService(Meta meta) {
     this.meta = meta;
@@ -41,7 +37,7 @@ public class LocalService implements Service {
     if (iterable instanceof List) {
       return (List<E>) iterable;
     }
-    final List<E> rowList = new ArrayList<E>();
+    final List<E> rowList = new ArrayList<>();
     for (E row : iterable) {
       rowList.add(row);
     }
@@ -99,8 +95,10 @@ public class LocalService implements Service {
 
   public ResultSetResponse apply(TablesRequest request) {
     final Meta.MetaResultSet resultSet =
-        meta.getTables(request.catalog, Meta.Pat.of(request.schemaPattern),
-            Meta.Pat.of(request.tableNamePattern), request.typeList);
+        meta.getTables(request.catalog,
+            Meta.Pat.of(request.schemaPattern),
+            Meta.Pat.of(request.tableNamePattern),
+            request.typeList);
     return toResponse(resultSet);
   }
 
@@ -123,32 +121,6 @@ public class LocalService implements Service {
         new Meta.ConnectionHandle(request.connectionId);
     final Meta.StatementHandle h =
         meta.prepare(ch, request.sql, request.maxRowCount);
-    if (json) {
-      Meta.Signature signature = h.signature;
-      final List<ColumnMetaData> columns = new ArrayList<>();
-      for (ColumnMetaData column : signature.columns) {
-        switch (column.type.rep) {
-        case BYTE:
-        case PRIMITIVE_BYTE:
-        case DOUBLE:
-        case PRIMITIVE_DOUBLE:
-        case FLOAT:
-        case PRIMITIVE_FLOAT:
-        case INTEGER:
-        case PRIMITIVE_INT:
-        case SHORT:
-        case PRIMITIVE_SHORT:
-        case LONG:
-        case PRIMITIVE_LONG:
-          column = column.setTypeId(Types.NUMERIC);
-        }
-        columns.add(column);
-      }
-      signature = new Meta.Signature(columns, signature.sql,
-          signature.parameters, signature.internalParameters,
-          signature.cursorFactory);
-      h.signature = signature;
-    }
     return new PrepareResponse(h);
   }
 
@@ -179,7 +151,9 @@ public class LocalService implements Service {
     final Meta.StatementHandle h = new Meta.StatementHandle(
         request.connectionId, request.statementId, null);
     final Meta.Frame frame =
-        meta.fetch(h, request.parameterValues, request.offset,
+        meta.fetch(h,
+            request.parameterValues,
+            request.offset,
             request.fetchMaxRowCount);
     return new FetchResponse(frame);
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/5648083f/avatica/src/main/java/org/apache/calcite/avatica/util/AbstractCursor.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/util/AbstractCursor.java b/avatica/src/main/java/org/apache/calcite/avatica/util/AbstractCursor.java
index fec143e..e1c9f75 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/util/AbstractCursor.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/util/AbstractCursor.java
@@ -16,13 +16,13 @@
  */
 package org.apache.calcite.avatica.util;
 
+import org.apache.calcite.avatica.AvaticaParameter;
 import org.apache.calcite.avatica.AvaticaUtils;
 import org.apache.calcite.avatica.ColumnMetaData;
 
 import java.io.InputStream;
 import java.io.Reader;
 import java.math.BigDecimal;
-import java.math.BigInteger;
 import java.math.RoundingMode;
 import java.net.URL;
 import java.sql.Array;
@@ -64,7 +64,7 @@ public abstract class AbstractCursor implements Cursor {
 
   public List<Accessor> createAccessors(List<ColumnMetaData> types,
       Calendar localCalendar, ArrayImpl.Factory factory) {
-    List<Accessor> accessors = new ArrayList<Accessor>();
+    List<Accessor> accessors = new ArrayList<>();
     for (ColumnMetaData type : types) {
       accessors.add(
           createAccessor(type, accessors.size(), localCalendar, factory));
@@ -77,6 +77,21 @@ public abstract class AbstractCursor implements Cursor {
     // Create an accessor appropriate to the underlying type; the accessor
     // can convert to any type in the same family.
     Getter getter = createGetter(ordinal);
+    switch (columnMetaData.type.rep) {
+    case OBJECT:
+      switch (columnMetaData.type.id) {
+      case Types.TINYINT:
+      case Types.SMALLINT:
+      case Types.INTEGER:
+      case Types.BIGINT:
+      case Types.REAL:
+      case Types.FLOAT:
+      case Types.DOUBLE:
+      case Types.NUMERIC:
+      case Types.DECIMAL:
+        return new NumberAccessor(getter, columnMetaData.scale);
+      }
+    }
     switch (columnMetaData.type.id) {
     case Types.TINYINT:
       return new ByteAccessor(getter);
@@ -93,8 +108,6 @@ public abstract class AbstractCursor implements Cursor {
     case Types.FLOAT:
     case Types.DOUBLE:
       return new DoubleAccessor(getter);
-    case Types.NUMERIC:
-      return new NumberAccessor(getter);
     case Types.DECIMAL:
       return new BigDecimalAccessor(getter);
     case Types.CHAR:
@@ -114,7 +127,8 @@ public abstract class AbstractCursor implements Cursor {
       switch (columnMetaData.type.rep) {
       case PRIMITIVE_INT:
       case INTEGER:
-        return new DateFromIntAccessor(getter, localCalendar);
+      case OBJECT:
+        return new DateFromNumberAccessor(getter, localCalendar);
       case JAVA_SQL_DATE:
         return new DateAccessor(getter, localCalendar);
       default:
@@ -124,7 +138,7 @@ public abstract class AbstractCursor implements Cursor {
       switch (columnMetaData.type.rep) {
       case PRIMITIVE_INT:
       case INTEGER:
-        return new TimeFromIntAccessor(getter, localCalendar);
+        return new TimeFromNumberAccessor(getter, localCalendar);
       case JAVA_SQL_TIME:
         return new TimeAccessor(getter, localCalendar);
       default:
@@ -134,7 +148,7 @@ public abstract class AbstractCursor implements Cursor {
       switch (columnMetaData.type.rep) {
       case PRIMITIVE_LONG:
       case LONG:
-        return new TimestampFromLongAccessor(getter, localCalendar);
+        return new TimestampFromNumberAccessor(getter, localCalendar);
       case JAVA_SQL_TIMESTAMP:
         return new TimestampAccessor(getter, localCalendar);
       case JAVA_UTIL_DATE:
@@ -629,29 +643,28 @@ public abstract class AbstractCursor implements Cursor {
    * So you cannot be sure that the source and target type are the same.
    */
   private static class NumberAccessor extends BigNumberAccessor {
-    public NumberAccessor(Getter getter) {
+    private final int scale;
+
+    public NumberAccessor(Getter getter, int scale) {
       super(getter);
+      this.scale = scale;
     }
 
     protected Number getNumber() {
-      return (Number) getObject();
+      return (Number) super.getObject();
     }
 
     public BigDecimal getBigDecimal(int scale) {
-      return getBigDecimal();
+      Number n = getNumber();
+      if (n == null) {
+        return null;
+      }
+      return AvaticaParameter.toBigDecimal(n)
+          .setScale(scale, BigDecimal.ROUND_UNNECESSARY);
     }
 
     public BigDecimal getBigDecimal() {
-      Number n = getNumber();
-      return n == null ? null
-          : n instanceof BigDecimal ? (BigDecimal) n
-          : n instanceof BigInteger ? new BigDecimal((BigInteger) n)
-          : n instanceof Long ? new BigDecimal((long) n)
-          : n instanceof Integer ? new BigDecimal((int) n)
-          : n instanceof Short ? new BigDecimal((short) n)
-          : n instanceof Byte ? new BigDecimal((byte) n)
-          : n instanceof Double ? new BigDecimal((double) n)
-          : new BigDecimal((float) n);
+      return getBigDecimal(scale);
     }
   }
 
@@ -731,11 +744,11 @@ public abstract class AbstractCursor implements Cursor {
    * in its default representation {@code int};
    * corresponds to {@link java.sql.Types#DATE}.
    */
-  private static class DateFromIntAccessor extends IntAccessor {
+  private static class DateFromNumberAccessor extends NumberAccessor {
     private final Calendar localCalendar;
 
-    public DateFromIntAccessor(Getter getter, Calendar localCalendar) {
-      super(getter);
+    public DateFromNumberAccessor(Getter getter, Calendar localCalendar) {
+      super(getter, 0);
       this.localCalendar = localCalendar;
     }
 
@@ -744,27 +757,28 @@ public abstract class AbstractCursor implements Cursor {
     }
 
     @Override public Date getDate(Calendar calendar) {
-      final int v = getInt();
-      if (v == 0 && getter.wasNull()) {
+      final Number v = getNumber();
+      if (v == null) {
         return null;
       }
-      return longToDate((long) v * DateTimeUtils.MILLIS_PER_DAY, calendar);
+      return longToDate(v.longValue() * DateTimeUtils.MILLIS_PER_DAY, calendar);
     }
 
     @Override public Timestamp getTimestamp(Calendar calendar) {
-      final int v = getInt();
-      if (v == 0 && getter.wasNull()) {
+      final Number v = getNumber();
+      if (v == null) {
         return null;
       }
-      return longToTimestamp((long) v * DateTimeUtils.MILLIS_PER_DAY, calendar);
+      return longToTimestamp(v.longValue() * DateTimeUtils.MILLIS_PER_DAY,
+          calendar);
     }
 
     @Override public String getString() {
-      final int v = getInt();
-      if (v == 0 && wasNull()) {
+      final Number v = getNumber();
+      if (v == null) {
         return null;
       }
-      return dateAsString(v, null);
+      return dateAsString(v.intValue(), null);
     }
   }
 
@@ -773,11 +787,11 @@ public abstract class AbstractCursor implements Cursor {
    * in its default representation {@code int};
    * corresponds to {@link java.sql.Types#TIME}.
    */
-  private static class TimeFromIntAccessor extends IntAccessor {
+  private static class TimeFromNumberAccessor extends NumberAccessor {
     private final Calendar localCalendar;
 
-    public TimeFromIntAccessor(Getter getter, Calendar localCalendar) {
-      super(getter);
+    public TimeFromNumberAccessor(Getter getter, Calendar localCalendar) {
+      super(getter, 0);
       this.localCalendar = localCalendar;
     }
 
@@ -786,27 +800,27 @@ public abstract class AbstractCursor implements Cursor {
     }
 
     @Override public Time getTime(Calendar calendar) {
-      final int v = getInt();
-      if (v == 0 && wasNull()) {
+      final Number v = getNumber();
+      if (v == null) {
         return null;
       }
-      return intToTime(v, calendar);
+      return intToTime(v.intValue(), calendar);
     }
 
     @Override public Timestamp getTimestamp(Calendar calendar) {
-      final long v = getLong();
-      if (v == 0 && wasNull()) {
+      final Number v = getNumber();
+      if (v == null) {
         return null;
       }
-      return longToTimestamp(v, calendar);
+      return longToTimestamp(v.longValue(), calendar);
     }
 
     @Override public String getString() {
-      final int v = getInt();
-      if (v == 0 && wasNull()) {
+      final Number v = getNumber();
+      if (v == null) {
         return null;
       }
-      return timeAsString(v, null);
+      return timeAsString(v.intValue(), null);
     }
   }
 
@@ -815,11 +829,11 @@ public abstract class AbstractCursor implements Cursor {
    * in its default representation {@code long};
    * corresponds to {@link java.sql.Types#TIMESTAMP}.
    */
-  private static class TimestampFromLongAccessor extends LongAccessor {
+  private static class TimestampFromNumberAccessor extends NumberAccessor {
     private final Calendar localCalendar;
 
-    public TimestampFromLongAccessor(Getter getter, Calendar localCalendar) {
-      super(getter);
+    public TimestampFromNumberAccessor(Getter getter, Calendar localCalendar) {
+      super(getter, 0);
       this.localCalendar = localCalendar;
     }
 
@@ -828,19 +842,19 @@ public abstract class AbstractCursor implements Cursor {
     }
 
     @Override public Timestamp getTimestamp(Calendar calendar) {
-      final long v = getLong();
-      if (v == 0 && wasNull()) {
+      final Number v = getNumber();
+      if (v == null) {
         return null;
       }
-      return longToTimestamp(v, calendar);
+      return longToTimestamp(v.longValue(), calendar);
     }
 
     @Override public String getString() {
-      final long v = getLong();
-      if (v == 0L && wasNull()) {
+      final Number v = getNumber();
+      if (v == null) {
         return null;
       }
-      return timestampAsString(v, null);
+      return timestampAsString(v.longValue(), null);
     }
   }
 


[08/10] incubator-calcite git commit: [CALCITE-613] Implicitly convert character values in comparisons

Posted by jh...@apache.org.
[CALCITE-613] Implicitly convert character values in comparisons

Add SqlOperandTypeChecker.getConsistency()

Make "scott" data set available in Quidem (.oq) tests

For "precision" of TIME and TIMESTAMP columns read from JDBC, use "scale" not "size"


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

Branch: refs/heads/master
Commit: 272e604082bbaf27a93e5c4f04a2eb1c951a99f4
Parents: 5648083
Author: Julian Hyde <jh...@apache.org>
Authored: Wed Mar 25 11:29:27 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Apr 3 01:10:06 2015 -0700

----------------------------------------------------------------------
 .../apache/calcite/adapter/jdbc/JdbcSchema.java |  22 +++-
 .../java/org/apache/calcite/rex/RexUtil.java    |  24 ++++
 .../org/apache/calcite/sql/SqlTimeLiteral.java  |  18 +--
 .../apache/calcite/sql/SqlTimestampLiteral.java |  33 +----
 .../calcite/sql/fun/SqlBetweenOperator.java     |   5 +-
 .../sql/type/AssignableOperandTypeChecker.java  |   7 +-
 .../sql/type/ComparableOperandTypeChecker.java  |  18 ++-
 .../sql/type/CompositeOperandTypeChecker.java   |   4 +
 .../sql/type/FamilyOperandTypeChecker.java      |   8 +-
 .../sql/type/LiteralOperandTypeChecker.java     |   4 +
 .../sql/type/MultisetOperandTypeChecker.java    |   4 +
 .../apache/calcite/sql/type/OperandTypes.java   |  21 ++-
 .../sql/type/SameOperandTypeChecker.java        |   5 +-
 .../sql/type/SetopOperandTypeChecker.java       |   6 +-
 .../calcite/sql/type/SqlOperandTypeChecker.java |  15 +++
 .../apache/calcite/sql/type/SqlTypeUtil.java    |  30 +++++
 .../calcite/sql2rel/SqlToRelConverter.java      |   7 +-
 .../sql2rel/StandardConvertletTable.java        | 128 ++++++++++++-------
 .../org/apache/calcite/test/CalciteAssert.java  |  27 +++-
 .../java/org/apache/calcite/test/JdbcTest.java  |   5 +
 .../apache/calcite/test/SqlValidatorTest.java   |  97 +++++++++-----
 core/src/test/resources/sql/misc.oq             | 114 +++++++++++++++++
 22 files changed, 461 insertions(+), 141 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/272e6040/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcSchema.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcSchema.java
index 7db25ae..de97237 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcSchema.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcSchema.java
@@ -16,6 +16,7 @@
  */
 package org.apache.calcite.adapter.jdbc;
 
+import org.apache.calcite.avatica.SqlType;
 import org.apache.calcite.linq4j.tree.Expression;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
@@ -128,8 +129,8 @@ public class JdbcSchema implements Schema {
     }
     String jdbcCatalog = (String) operand.get("jdbcCatalog");
     String jdbcSchema = (String) operand.get("jdbcSchema");
-    return JdbcSchema.create(parentSchema, name, dataSource, jdbcCatalog,
-        jdbcSchema);
+    return JdbcSchema.create(
+        parentSchema, name, dataSource, jdbcCatalog, jdbcSchema);
   }
 
   /** Returns a suitable SQL dialect for the given data source. */
@@ -256,10 +257,21 @@ public class JdbcSchema implements Schema {
       final String columnName = resultSet.getString(4);
       final int dataType = resultSet.getInt(5);
       final String typeString = resultSet.getString(6);
-      final int size = resultSet.getInt(7);
-      final int scale = resultSet.getInt(9);
+      final int precision;
+      final int scale;
+      switch (SqlType.valueOf(dataType)) {
+      case TIMESTAMP:
+      case TIME:
+        precision = resultSet.getInt(9); // SCALE
+        scale = 0;
+        break;
+      default:
+        precision = resultSet.getInt(7); // SIZE
+        scale = resultSet.getInt(9); // SCALE
+        break;
+      }
       RelDataType sqlType =
-          sqlType(typeFactory, dataType, size, scale, typeString);
+          sqlType(typeFactory, dataType, precision, scale, typeString);
       boolean nullable = resultSet.getBoolean(11);
       fieldInfo.add(columnName, sqlType).nullable(nullable);
     }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/272e6040/core/src/main/java/org/apache/calcite/rex/RexUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexUtil.java b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
index 240f895..d470a4f 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexUtil.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
@@ -23,6 +23,7 @@ import org.apache.calcite.rel.RelCollations;
 import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeFamily;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlKind;
@@ -75,6 +76,20 @@ public class RexUtil {
         }
       };
 
+  private static final Function<RexNode, RelDataType> TYPE_FN =
+      new Function<RexNode, RelDataType>() {
+        public RelDataType apply(RexNode input) {
+          return input.getType();
+        }
+      };
+
+  private static final Function<RelDataType, RelDataTypeFamily> FAMILY_FN =
+      new Function<RelDataType, RelDataTypeFamily>() {
+        public RelDataTypeFamily apply(RelDataType input) {
+          return input.getFamily();
+        }
+      };
+
   private RexUtil() {
   }
 
@@ -1132,6 +1147,15 @@ public class RexUtil {
     }.apply(nodes);
   }
 
+  /** Transforms a list of expressions into a list of their types. */
+  public static List<RelDataType> types(List<? extends RexNode> nodes) {
+    return Lists.transform(nodes, TYPE_FN);
+  }
+
+  public static List<RelDataTypeFamily> families(List<RelDataType> types) {
+    return Lists.transform(types, FAMILY_FN);
+  }
+
   //~ Inner Classes ----------------------------------------------------------
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/272e6040/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 31572e1..79166c3 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlTimeLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlTimeLiteral.java
@@ -20,6 +20,8 @@ import org.apache.calcite.avatica.util.DateTimeUtils;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.SqlTypeName;
 
+import com.google.common.base.Preconditions;
+
 import java.util.Calendar;
 
 /**
@@ -34,23 +36,19 @@ public class SqlTimeLiteral extends SqlAbstractDateTimeLiteral {
   SqlTimeLiteral(
       Calendar t,
       int precision,
-      boolean hasTZ,
+      boolean hasTimeZone,
       SqlParserPos pos) {
-    super(
-        t,
-        hasTZ,
-        SqlTypeName.TIME,
-        precision, DateTimeUtils.TIME_FORMAT_STRING,
-        pos);
+    this(t, precision, hasTimeZone, DateTimeUtils.TIME_FORMAT_STRING, pos);
   }
 
   SqlTimeLiteral(
       Calendar t,
       int precision,
-      boolean hasTZ,
+      boolean hasTimeZone,
       String format,
       SqlParserPos pos) {
-    super(t, hasTZ, SqlTypeName.TIME, precision, format, pos);
+    super(t, hasTimeZone, SqlTypeName.TIME, precision, format, pos);
+    Preconditions.checkArgument(this.precision >= 0 && this.precision <= 3);
   }
 
   //~ Methods ----------------------------------------------------------------
@@ -75,8 +73,6 @@ public class SqlTimeLiteral extends SqlAbstractDateTimeLiteral {
     String result = getTime().toString(formatString);
     final Calendar cal = getCal();
     if (precision > 0) {
-      assert precision <= 3;
-
       // get the millisecond count.  millisecond => at most 3 digits.
       String digits = Long.toString(cal.getTimeInMillis());
       result =

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/272e6040/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 af31b42..a620595 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlTimestampLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlTimestampLiteral.java
@@ -20,6 +20,8 @@ import org.apache.calcite.avatica.util.DateTimeUtils;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.SqlTypeName;
 
+import com.google.common.base.Preconditions;
+
 import java.util.Calendar;
 
 /**
@@ -36,11 +38,7 @@ public class SqlTimestampLiteral extends SqlAbstractDateTimeLiteral {
       int precision,
       boolean hasTimeZone,
       SqlParserPos pos) {
-    super(
-        cal,
-        hasTimeZone,
-        SqlTypeName.TIMESTAMP,
-        precision, DateTimeUtils.TIMESTAMP_FORMAT_STRING,
+    this(cal, precision, hasTimeZone, DateTimeUtils.TIMESTAMP_FORMAT_STRING,
         pos);
   }
 
@@ -50,33 +48,12 @@ public class SqlTimestampLiteral extends SqlAbstractDateTimeLiteral {
       boolean hasTimeZone,
       String format,
       SqlParserPos pos) {
-    super(
-        cal, hasTimeZone, SqlTypeName.TIMESTAMP, precision,
-        format, pos);
+    super(cal, hasTimeZone, SqlTypeName.TIMESTAMP, precision, format, pos);
+    Preconditions.checkArgument(this.precision >= 0 && this.precision <= 3);
   }
 
   //~ Methods ----------------------------------------------------------------
 
-/*
-  /**
-   * Converts this literal to a {@link java.sql.Timestamp} object.
-   o/
-  public Timestamp getTimestamp() {
-    return new Timestamp(getCal().getTimeInMillis());
-  }
-*/
-
-/*
-  /**
-   * Converts this literal to a {@link java.sql.Time} object.
-   o/
-  public Time getTime() {
-    long millis = getCal().getTimeInMillis();
-    int tzOffset = Calendar.getInstance().getTimeZone().getOffset(millis);
-    return new Time(millis - tzOffset);
-  }
-*/
-
   public SqlNode clone(SqlParserPos pos) {
     return new SqlTimestampLiteral(
         (Calendar) value,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/272e6040/core/src/main/java/org/apache/calcite/sql/fun/SqlBetweenOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlBetweenOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlBetweenOperator.java
index 4b982ad..03b6646 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlBetweenOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlBetweenOperator.java
@@ -83,9 +83,8 @@ public class SqlBetweenOperator extends SqlInfixOperator {
    * Custom operand-type checking strategy.
    */
   private static final SqlOperandTypeChecker OTC_CUSTOM =
-      new ComparableOperandTypeChecker(
-          3,
-          RelDataTypeComparability.ALL);
+      new ComparableOperandTypeChecker(3, RelDataTypeComparability.ALL,
+          SqlOperandTypeChecker.Consistency.COMPARE);
   private static final SqlWriter.FrameType FRAME_TYPE =
       SqlWriter.FrameTypeEnum.create("BETWEEN");
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/272e6040/core/src/main/java/org/apache/calcite/sql/type/AssignableOperandTypeChecker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/AssignableOperandTypeChecker.java b/core/src/main/java/org/apache/calcite/sql/type/AssignableOperandTypeChecker.java
index 3f9f9ba..7ee0a64 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/AssignableOperandTypeChecker.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/AssignableOperandTypeChecker.java
@@ -52,12 +52,10 @@ public class AssignableOperandTypeChecker implements SqlOperandTypeChecker {
 
   //~ Methods ----------------------------------------------------------------
 
-  // implement SqlOperandTypeChecker
   public SqlOperandCountRange getOperandCountRange() {
     return SqlOperandCountRanges.of(paramTypes.size());
   }
 
-  // implement SqlOperandTypeChecker
   public boolean checkOperandTypes(
       SqlCallBinding callBinding,
       boolean throwOnFailure) {
@@ -78,7 +76,6 @@ public class AssignableOperandTypeChecker implements SqlOperandTypeChecker {
     return true;
   }
 
-  // implement SqlOperandTypeChecker
   public String getAllowedSignatures(SqlOperator op, String opName) {
     StringBuilder sb = new StringBuilder();
     sb.append(opName);
@@ -94,6 +91,10 @@ public class AssignableOperandTypeChecker implements SqlOperandTypeChecker {
     sb.append(")");
     return sb.toString();
   }
+
+  public Consistency getConsistency() {
+    return Consistency.NONE;
+  }
 }
 
 // End AssignableOperandTypeChecker.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/272e6040/core/src/main/java/org/apache/calcite/sql/type/ComparableOperandTypeChecker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/ComparableOperandTypeChecker.java b/core/src/main/java/org/apache/calcite/sql/type/ComparableOperandTypeChecker.java
index 454897d..3370b5f 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/ComparableOperandTypeChecker.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/ComparableOperandTypeChecker.java
@@ -23,6 +23,8 @@ import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.SqlOperatorBinding;
 import org.apache.calcite.sql.SqlUtil;
 
+import com.google.common.base.Preconditions;
+
 import java.util.Collections;
 
 /**
@@ -33,14 +35,21 @@ public class ComparableOperandTypeChecker extends SameOperandTypeChecker {
   //~ Instance fields --------------------------------------------------------
 
   private final RelDataTypeComparability requiredComparability;
+  private final Consistency consistency;
 
   //~ Constructors -----------------------------------------------------------
 
-  public ComparableOperandTypeChecker(
-      int nOperands,
+  @Deprecated // to be removed before 2.0
+  public ComparableOperandTypeChecker(int nOperands,
       RelDataTypeComparability requiredComparability) {
+    this(nOperands, requiredComparability, Consistency.NONE);
+  }
+
+  public ComparableOperandTypeChecker(int nOperands,
+      RelDataTypeComparability requiredComparability, Consistency consistency) {
     super(nOperands);
     this.requiredComparability = requiredComparability;
+    this.consistency = Preconditions.checkNotNull(consistency);
   }
 
   //~ Methods ----------------------------------------------------------------
@@ -107,11 +116,14 @@ public class ComparableOperandTypeChecker extends SameOperandTypeChecker {
     return b;
   }
 
-  // implement SqlOperandTypeChecker
   public String getAllowedSignatures(SqlOperator op, String opName) {
     return SqlUtil.getAliasedSignature(op, opName,
         Collections.nCopies(nOperands, "COMPARABLE_TYPE"));
   }
+
+  @Override public Consistency getConsistency() {
+    return consistency;
+  }
 }
 
 // End ComparableOperandTypeChecker.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/272e6040/core/src/main/java/org/apache/calcite/sql/type/CompositeOperandTypeChecker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/CompositeOperandTypeChecker.java b/core/src/main/java/org/apache/calcite/sql/type/CompositeOperandTypeChecker.java
index 712c3c4..6a77c95 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/CompositeOperandTypeChecker.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/CompositeOperandTypeChecker.java
@@ -111,6 +111,10 @@ public class CompositeOperandTypeChecker implements SqlOperandTypeChecker {
     return allowedRules;
   }
 
+  public Consistency getConsistency() {
+    return Consistency.NONE;
+  }
+
   public String getAllowedSignatures(SqlOperator op, String opName) {
     if (allowedSignatures != null) {
       return allowedSignatures;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/272e6040/core/src/main/java/org/apache/calcite/sql/type/FamilyOperandTypeChecker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/FamilyOperandTypeChecker.java b/core/src/main/java/org/apache/calcite/sql/type/FamilyOperandTypeChecker.java
index c421727..4bef483 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/FamilyOperandTypeChecker.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/FamilyOperandTypeChecker.java
@@ -50,7 +50,6 @@ public class FamilyOperandTypeChecker implements SqlSingleOperandTypeChecker {
 
   //~ Methods ----------------------------------------------------------------
 
-  // implement SqlSingleOperandTypeChecker
   public boolean checkSingleOperandType(
       SqlCallBinding callBinding,
       SqlNode node,
@@ -89,7 +88,6 @@ public class FamilyOperandTypeChecker implements SqlSingleOperandTypeChecker {
     return true;
   }
 
-  // implement SqlOperandTypeChecker
   public boolean checkOperandTypes(
       SqlCallBinding callBinding,
       boolean throwOnFailure) {
@@ -111,15 +109,17 @@ public class FamilyOperandTypeChecker implements SqlSingleOperandTypeChecker {
     return true;
   }
 
-  // implement SqlOperandTypeChecker
   public SqlOperandCountRange getOperandCountRange() {
     return SqlOperandCountRanges.of(families.size());
   }
 
-  // implement SqlOperandTypeChecker
   public String getAllowedSignatures(SqlOperator op, String opName) {
     return SqlUtil.getAliasedSignature(op, opName, families);
   }
+
+  public Consistency getConsistency() {
+    return Consistency.NONE;
+  }
 }
 
 // End FamilyOperandTypeChecker.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/272e6040/core/src/main/java/org/apache/calcite/sql/type/LiteralOperandTypeChecker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/LiteralOperandTypeChecker.java b/core/src/main/java/org/apache/calcite/sql/type/LiteralOperandTypeChecker.java
index 83161d1..ac94f14 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/LiteralOperandTypeChecker.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/LiteralOperandTypeChecker.java
@@ -91,6 +91,10 @@ public class LiteralOperandTypeChecker implements SqlSingleOperandTypeChecker {
   public String getAllowedSignatures(SqlOperator op, String opName) {
     return "<LITERAL>";
   }
+
+  public Consistency getConsistency() {
+    return Consistency.NONE;
+  }
 }
 
 // End LiteralOperandTypeChecker.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/272e6040/core/src/main/java/org/apache/calcite/sql/type/MultisetOperandTypeChecker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/MultisetOperandTypeChecker.java b/core/src/main/java/org/apache/calcite/sql/type/MultisetOperandTypeChecker.java
index 5b33fce..2d04c46 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/MultisetOperandTypeChecker.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/MultisetOperandTypeChecker.java
@@ -89,6 +89,10 @@ public class MultisetOperandTypeChecker implements SqlOperandTypeChecker {
   public String getAllowedSignatures(SqlOperator op, String opName) {
     return "<MULTISET> " + opName + " <MULTISET>";
   }
+
+  public Consistency getConsistency() {
+    return Consistency.NONE;
+  }
 }
 
 // End MultisetOperandTypeChecker.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/272e6040/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java b/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java
index 10bccbd..0d57234 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java
@@ -158,6 +158,10 @@ public abstract class OperandTypes {
       public String getAllowedSignatures(SqlOperator op, String opName) {
         return opName + "(...)";
       }
+
+      public Consistency getConsistency() {
+        return Consistency.NONE;
+      }
     };
   }
 
@@ -315,7 +319,8 @@ public abstract class OperandTypes {
    */
   public static final SqlOperandTypeChecker
   COMPARABLE_ORDERED_COMPARABLE_ORDERED =
-      new ComparableOperandTypeChecker(2, RelDataTypeComparability.ALL);
+      new ComparableOperandTypeChecker(2, RelDataTypeComparability.ALL,
+          SqlOperandTypeChecker.Consistency.COMPARE);
 
   /**
    * Operand type-checking strategy where operand type must allow ordered
@@ -323,7 +328,8 @@ public abstract class OperandTypes {
    * functions
    */
   public static final SqlOperandTypeChecker COMPARABLE_ORDERED =
-      new ComparableOperandTypeChecker(1, RelDataTypeComparability.ALL);
+      new ComparableOperandTypeChecker(1, RelDataTypeComparability.ALL,
+          SqlOperandTypeChecker.Consistency.NONE);
 
   /**
    * Operand type-checking strategy where operand types must allow unordered
@@ -331,7 +337,8 @@ public abstract class OperandTypes {
    */
   public static final SqlOperandTypeChecker
   COMPARABLE_UNORDERED_COMPARABLE_UNORDERED =
-      new ComparableOperandTypeChecker(2, RelDataTypeComparability.UNORDERED);
+      new ComparableOperandTypeChecker(2, RelDataTypeComparability.UNORDERED,
+          SqlOperandTypeChecker.Consistency.LEAST_RESTRICTIVE);
 
   /**
    * Operand type-checking strategy where two operands must both be in the
@@ -479,6 +486,10 @@ public abstract class OperandTypes {
         public String getAllowedSignatures(SqlOperator op, String opName) {
           return "UNNEST(<MULTISET>)";
         }
+
+        public Consistency getConsistency() {
+          return Consistency.NONE;
+        }
       };
 
   /** Checker that returns whether a value is a collection (multiset or array)
@@ -539,6 +550,10 @@ public abstract class OperandTypes {
           return SqlUtil.getAliasedSignature(op, opName,
               ImmutableList.of("RECORDTYPE(SINGLE FIELD)"));
         }
+
+        public Consistency getConsistency() {
+          return Consistency.NONE;
+        }
       };
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/272e6040/core/src/main/java/org/apache/calcite/sql/type/SameOperandTypeChecker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SameOperandTypeChecker.java b/core/src/main/java/org/apache/calcite/sql/type/SameOperandTypeChecker.java
index 5f853b8..dbbd393 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SameOperandTypeChecker.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SameOperandTypeChecker.java
@@ -49,7 +49,10 @@ public class SameOperandTypeChecker implements SqlSingleOperandTypeChecker {
 
   //~ Methods ----------------------------------------------------------------
 
-  // implement SqlOperandTypeChecker
+  public Consistency getConsistency() {
+    return Consistency.NONE;
+  }
+
   public boolean checkOperandTypes(
       SqlCallBinding callBinding,
       boolean throwOnFailure) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/272e6040/core/src/main/java/org/apache/calcite/sql/type/SetopOperandTypeChecker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SetopOperandTypeChecker.java b/core/src/main/java/org/apache/calcite/sql/type/SetopOperandTypeChecker.java
index 04c812b..a38b682 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SetopOperandTypeChecker.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SetopOperandTypeChecker.java
@@ -123,7 +123,11 @@ public class SetopOperandTypeChecker implements SqlOperandTypeChecker {
   }
 
   public String getAllowedSignatures(SqlOperator op, String opName) {
-    return "{0} " + opName + " {1}"; // todo: Wael, please review.
+    return "{0} " + opName + " {1}";
+  }
+
+  public Consistency getConsistency() {
+    return Consistency.NONE;
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/272e6040/core/src/main/java/org/apache/calcite/sql/type/SqlOperandTypeChecker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlOperandTypeChecker.java b/core/src/main/java/org/apache/calcite/sql/type/SqlOperandTypeChecker.java
index b6a21f6..5a9eea4 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlOperandTypeChecker.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlOperandTypeChecker.java
@@ -55,6 +55,21 @@ public interface SqlOperandTypeChecker {
    * @return generated string
    */
   String getAllowedSignatures(SqlOperator op, String opName);
+
+  /** Returns the strategy for making the arguments have consistency types. */
+  Consistency getConsistency();
+
+  /** Strategy used to make arguments consistent. */
+  enum Consistency {
+    /** Do not try to make arguments consistent. */
+    NONE,
+    /** Make arguments of consistent type using comparison semantics.
+     * Character values are implicitly converted to numeric, date-time, interval
+     * or boolean. */
+    COMPARE,
+    /** Convert all arguments to the least restrictive type. */
+    LEAST_RESTRICTIVE
+  }
 }
 
 // End SqlOperandTypeChecker.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/272e6040/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
index 39c9ded..658acd4 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
@@ -1217,6 +1217,36 @@ public abstract class SqlTypeUtil {
       return true;
     }
 
+    // We can implicitly convert from character to date
+    if (family1 == SqlTypeFamily.CHARACTER
+        && canConvertStringInCompare(family2)
+        || family2 == SqlTypeFamily.CHARACTER
+        && canConvertStringInCompare(family1)) {
+      return true;
+    }
+
+    return false;
+  }
+
+  /** Returns whether a character data type can be implicitly converted to a
+   * given family in a compare operation. */
+  private static boolean canConvertStringInCompare(RelDataTypeFamily family) {
+    if (family instanceof SqlTypeFamily) {
+      SqlTypeFamily sqlTypeFamily = (SqlTypeFamily) family;
+      switch (sqlTypeFamily) {
+      case DATE:
+      case TIME:
+      case TIMESTAMP:
+      case INTERVAL_DAY_TIME:
+      case INTERVAL_YEAR_MONTH:
+      case NUMERIC:
+      case APPROXIMATE_NUMERIC:
+      case EXACT_NUMERIC:
+      case INTEGER:
+      case BOOLEAN:
+        return true;
+      }
+    }
     return false;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/272e6040/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
index 7fadfa0..4247083 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -1308,7 +1308,8 @@ public class SqlToRelConverter {
             rexBuilder.makeCall(
                 SqlStdOperatorTable.EQUALS,
                 leftKeys.get(0),
-                bb.convertExpression(rightVals));
+                rexBuilder.ensureType(leftKeys.get(0).getType(),
+                    bb.convertExpression(rightVals), true));
       } else {
         assert rightVals instanceof SqlCall;
         final SqlBasicCall call = (SqlBasicCall) rightVals;
@@ -1322,7 +1323,9 @@ public class SqlToRelConverter {
                     new Function<Pair<RexNode, SqlNode>, RexNode>() {
                       public RexNode apply(Pair<RexNode, SqlNode> pair) {
                         return rexBuilder.makeCall(SqlStdOperatorTable.EQUALS,
-                            pair.left, bb.convertExpression(pair.right));
+                            pair.left,
+                            rexBuilder.ensureType(pair.left.getType(),
+                                bb.convertExpression(pair.right), true));
                       }
                     }),
                 false);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/272e6040/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 850d798..8bf28e0 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
@@ -22,6 +22,7 @@ import org.apache.calcite.avatica.util.TimeUnitRange;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeFamily;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexCall;
 import org.apache.calcite.rex.RexCallBinding;
@@ -62,17 +63,20 @@ import org.apache.calcite.sql.fun.SqlRowOperator;
 import org.apache.calcite.sql.fun.SqlSequenceValueOperator;
 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.SqlOperandTypeChecker;
+import org.apache.calcite.sql.type.SqlTypeFamily;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
 
 import java.math.BigDecimal;
 import java.math.MathContext;
-import java.util.AbstractList;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Set;
 
 /**
  * Standard implementation of {@link SqlRexConvertletTable}.
@@ -513,7 +517,8 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
       SqlCall call) {
     final RexBuilder rexBuilder = cx.getRexBuilder();
     final List<SqlNode> operands = call.getOperandList();
-    final List<RexNode> exprs = convertExpressionList(cx, operands);
+    final List<RexNode> exprs = convertExpressionList(cx, operands,
+        SqlOperandTypeChecker.Consistency.NONE);
 
     // TODO: Will need to use decimal type for seconds with precision
     RelDataType resType =
@@ -626,7 +631,8 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
     // Rewrite datetime minus
     final RexBuilder rexBuilder = cx.getRexBuilder();
     final List<SqlNode> operands = call.getOperandList();
-    final List<RexNode> exprs = convertExpressionList(cx, operands);
+    final List<RexNode> exprs = convertExpressionList(cx, operands,
+        SqlOperandTypeChecker.Consistency.NONE);
 
     // TODO: Handle year month interval (represented in months)
     for (RexNode expr : exprs) {
@@ -668,7 +674,8 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
       SqlFunction fun,
       SqlCall call) {
     final List<SqlNode> operands = call.getOperandList();
-    final List<RexNode> exprs = convertExpressionList(cx, operands);
+    final List<RexNode> exprs = convertExpressionList(cx, operands,
+        SqlOperandTypeChecker.Consistency.NONE);
     if (fun.getFunctionType() == SqlFunctionCategory.USER_DEFINED_CONSTRUCTOR) {
       return makeConstructorCall(cx, fun, exprs);
     }
@@ -703,7 +710,8 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
     if (call.isCountStar()) {
       exprs = ImmutableList.of();
     } else {
-      exprs = convertExpressionList(cx, operands);
+      exprs = convertExpressionList(cx, operands,
+          SqlOperandTypeChecker.Consistency.NONE);
     }
     RelDataType returnType =
         cx.getValidator().getValidatedNodeTypeIfKnown(call);
@@ -769,11 +777,12 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
       SqlOperator op) {
     final List<SqlNode> operands = call.getOperandList();
     final RexBuilder rexBuilder = cx.getRexBuilder();
-    final List<RexNode> exprs = convertExpressionList(cx, operands);
-    if (op.getOperandTypeChecker()
-        == OperandTypes.COMPARABLE_UNORDERED_COMPARABLE_UNORDERED) {
-      ensureSameType(cx, exprs);
-    }
+    final SqlOperandTypeChecker.Consistency consistency =
+        op.getOperandTypeChecker() == null
+            ? SqlOperandTypeChecker.Consistency.NONE
+            : op.getOperandTypeChecker().getConsistency();
+    final List<RexNode> exprs =
+        convertExpressionList(cx, operands, consistency);
     RelDataType type = rexBuilder.deriveReturnType(op, exprs);
     return rexBuilder.makeCall(type, op, RexUtil.flatten(exprs, op));
   }
@@ -793,35 +802,68 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
     return list;
   }
 
-  private void ensureSameType(SqlRexContext cx, final List<RexNode> exprs) {
-    RelDataType type =
-        cx.getTypeFactory().leastRestrictive(
-            new AbstractList<RelDataType>() {
-              public RelDataType get(int index) {
-                return exprs.get(index).getType();
-              }
-
-              public int size() {
-                return exprs.size();
-              }
-            });
-    for (int i = 0; i < exprs.size(); i++) {
-      // REVIEW: assigning to a list that may be immutable?
-      exprs.set(
-          i, cx.getRexBuilder().ensureType(type, exprs.get(i), true));
-    }
-  }
-
-  private static List<RexNode> convertExpressionList(
-      SqlRexContext cx,
-      List<SqlNode> nodes) {
-    final ArrayList<RexNode> exprs = new ArrayList<RexNode>();
+  private static List<RexNode> convertExpressionList(SqlRexContext cx,
+      List<SqlNode> nodes, SqlOperandTypeChecker.Consistency consistency) {
+    final List<RexNode> exprs = Lists.newArrayList();
     for (SqlNode node : nodes) {
       exprs.add(cx.convertExpression(node));
     }
+    if (exprs.size() > 1) {
+      final RelDataType type =
+          consistentType(cx, consistency, RexUtil.types(exprs));
+      if (type != null) {
+        final List<RexNode> oldExprs = Lists.newArrayList(exprs);
+        exprs.clear();
+        for (RexNode expr : oldExprs) {
+          exprs.add(cx.getRexBuilder().ensureType(type, expr, true));
+        }
+      }
+    }
     return exprs;
   }
 
+  private static RelDataType consistentType(SqlRexContext cx,
+      SqlOperandTypeChecker.Consistency consistency, List<RelDataType> types) {
+    switch (consistency) {
+    case COMPARE:
+      final Set<RelDataTypeFamily> families =
+          Sets.newHashSet(RexUtil.families(types));
+      if (families.size() < 2) {
+        // All arguments are of same family. No need for explicit casts.
+        return null;
+      }
+      final List<RelDataType> nonCharacterTypes = Lists.newArrayList();
+      for (RelDataType type : types) {
+        if (type.getFamily() != SqlTypeFamily.CHARACTER) {
+          nonCharacterTypes.add(type);
+        }
+      }
+      if (!nonCharacterTypes.isEmpty()) {
+        final int typeCount = types.size();
+        types = nonCharacterTypes;
+        if (nonCharacterTypes.size() < typeCount) {
+          final RelDataTypeFamily family =
+              nonCharacterTypes.get(0).getFamily();
+          if (family instanceof SqlTypeFamily) {
+            // The character arguments might be larger than the numeric
+            // argument. Give ourselves some headroom.
+            switch ((SqlTypeFamily) family) {
+            case INTEGER:
+            case NUMERIC:
+              nonCharacterTypes.add(
+                  cx.getTypeFactory().createSqlType(SqlTypeName.BIGINT));
+            }
+          }
+        }
+      }
+      // fall through
+    case LEAST_RESTRICTIVE:
+      return cx.getTypeFactory().leastRestrictive(types);
+    default:
+      return null;
+    }
+  }
+
   private RexNode convertPlus(SqlRexContext cx, SqlCall call) {
     final RexNode rex = convertCall(cx, call);
     switch (rex.getType().getSqlTypeName()) {
@@ -853,20 +895,17 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
       SqlRexContext cx,
       SqlBetweenOperator op,
       SqlCall call) {
-    final SqlNode value = call.operand(SqlBetweenOperator.VALUE_OPERAND);
-    RexNode x = cx.convertExpression(value);
-    final SqlBetweenOperator.Flag symmetric = op.flag;
-    final SqlNode lower = call.operand(SqlBetweenOperator.LOWER_OPERAND);
-    RexNode y = cx.convertExpression(lower);
-    final SqlNode upper = call.operand(SqlBetweenOperator.UPPER_OPERAND);
-    RexNode z = cx.convertExpression(upper);
+    final List<RexNode> list =
+        convertExpressionList(cx, call.getOperandList(),
+            op.getOperandTypeChecker().getConsistency());
+    final RexNode x = list.get(SqlBetweenOperator.VALUE_OPERAND);
+    final RexNode y = list.get(SqlBetweenOperator.LOWER_OPERAND);
+    final RexNode z = list.get(SqlBetweenOperator.UPPER_OPERAND);
 
     final RexBuilder rexBuilder = cx.getRexBuilder();
     RexNode ge1 =
         rexBuilder.makeCall(
-            SqlStdOperatorTable.GREATER_THAN_OR_EQUAL,
-            x,
-            y);
+            SqlStdOperatorTable.GREATER_THAN_OR_EQUAL, x, y);
     RexNode le1 =
         rexBuilder.makeCall(
             SqlStdOperatorTable.LESS_THAN_OR_EQUAL,
@@ -879,6 +918,7 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
             le1);
 
     RexNode res;
+    final SqlBetweenOperator.Flag symmetric = op.flag;
     switch (symmetric) {
     case ASYMMETRIC:
       res = and1;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/272e6040/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 298f8d6..9ebb85a 100644
--- a/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
+++ b/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
@@ -629,13 +629,22 @@ public class CalciteAssert {
 
   public static SchemaPlus addSchema(SchemaPlus rootSchema, SchemaSpec schema) {
     SchemaPlus foodmart;
+    SchemaPlus jdbcScott;
+    final ConnectionSpec cs;
+    final DataSource dataSource;
     switch (schema) {
     case REFLECTIVE_FOODMART:
       return rootSchema.add("foodmart",
           new ReflectiveSchema(new JdbcTest.FoodmartSchema()));
+    case JDBC_SCOTT:
+      cs = DatabaseInstance.HSQLDB.scott;
+      dataSource = JdbcSchema.dataSource(cs.url, cs.driver, cs.username,
+          cs.password);
+      return rootSchema.add("jdbc_scott",
+          JdbcSchema.create(rootSchema, "jdbc_scott", dataSource, null, null));
     case JDBC_FOODMART:
-      final ConnectionSpec cs = DB.foodmart;
-      final DataSource dataSource =
+      cs = DB.foodmart;
+      dataSource =
           JdbcSchema.dataSource(cs.url, cs.driver, cs.username, cs.password);
       return rootSchema.add("foodmart",
           JdbcSchema.create(rootSchema, "foodmart", dataSource, null,
@@ -655,6 +664,13 @@ public class CalciteAssert {
                   + "join \"foodmart\".\"product_class\" as pc on p.\"product_class_id\" = pc.\"product_class_id\"",
               true));
       return foodmart;
+    case SCOTT:
+      jdbcScott = rootSchema.getSubSchema("jdbc_scott");
+      if (jdbcScott == null) {
+        jdbcScott =
+            CalciteAssert.addSchema(rootSchema, SchemaSpec.JDBC_SCOTT);
+      }
+      return rootSchema.add("scott", new CloneSchema(jdbcScott));
     case CLONE_FOODMART:
       foodmart = rootSchema.getSubSchema("foodmart");
       if (foodmart == null) {
@@ -747,6 +763,8 @@ public class CalciteAssert {
         return with(SchemaSpec.CLONE_FOODMART);
       case JDBC_FOODMART_WITH_LATTICE:
         return with(SchemaSpec.JDBC_FOODMART_WITH_LATTICE);
+      case SCOTT:
+        return with(SchemaSpec.SCOTT);
       default:
         throw Util.unexpected(config);
       }
@@ -1378,6 +1396,9 @@ public class CalciteAssert {
     /** Configuration that includes the metadata schema. */
     REGULAR_PLUS_METADATA,
 
+    /** Configuration that loads the "scott/tiger" database. */
+    SCOTT,
+
     /** Configuration that loads Spark. */
     SPARK,
   }
@@ -1462,6 +1483,8 @@ public class CalciteAssert {
     CLONE_FOODMART,
     JDBC_FOODMART_WITH_LATTICE,
     HR,
+    JDBC_SCOTT,
+    SCOTT,
     LINGUAL,
     POST
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/272e6040/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 7af8448..9ea3d08 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
@@ -4406,6 +4406,11 @@ public class JdbcTest {
                   .with(CalciteAssert.Config.FOODMART_CLONE)
                   .connect();
             }
+            if (name.equals("scott")) {
+              return CalciteAssert.that()
+                  .with(CalciteAssert.Config.SCOTT)
+                  .connect();
+            }
             if (name.equals("post")) {
               return CalciteAssert.that()
                   .with(CalciteAssert.Config.REGULAR)

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/272e6040/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
index 6718df4..ec3fed1 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -364,24 +364,14 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
   }
 
   @Test public void testEqualNotEqualFails() {
-    checkExpFails(
-        "^''<>1^",
-        "(?s).*Cannot apply '<>' to arguments of type '<CHAR.0.> <> <INTEGER>'.*");
-    checkExpFails(
-        "^'1'>=1^",
-        "(?s).*Cannot apply '>=' to arguments of type '<CHAR.1.> >= <INTEGER>'.*");
-    checkExpFails(
-        "^1<>n'abc'^",
-        "(?s).*Cannot apply '<>' to arguments of type '<INTEGER> <> <CHAR.3.>'.*");
-    checkExpFails(
-        "^''=.1^",
-        "(?s).*Cannot apply '=' to arguments of type '<CHAR.0.> = <DECIMAL.1..1.>'.*");
+    checkExp("''<>1"); // compare CHAR, INTEGER ok; implicitly convert CHAR
+    checkExp("'1'>=1");
+    checkExp("1<>n'abc'"); // compare INTEGER, NCHAR ok
+    checkExp("''=.1"); // compare CHAR, DECIMAL ok
     checkExpFails(
         "^true<>1e-1^",
         "(?s).*Cannot apply '<>' to arguments of type '<BOOLEAN> <> <DOUBLE>'.*");
-    checkExpFails(
-        "^false=''^",
-        "(?s).*Cannot apply '=' to arguments of type '<BOOLEAN> = <CHAR.0.>'.*");
+    checkExp("false=''"); // compare BOOLEAN, CHAR ok
     checkExpFails(
         "^x'a4'=0.01^",
         "(?s).*Cannot apply '=' to arguments of type '<BINARY.1.> = <DECIMAL.3, 2.>'.*");
@@ -628,8 +618,8 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
   @Test public void testBetween() {
     checkExp("1 between 2 and 3");
     checkExp("'a' between 'b' and 'c'");
-    checkWholeExpFails(
-        "'' between 2 and 3",
+    checkExp("'' between 2 and 3"); // can implicitly convert CHAR to INTEGER
+    checkWholeExpFails("date '2012-02-03' between 2 and 3",
         "(?s).*Cannot apply 'BETWEEN' to arguments of type.*");
   }
 
@@ -4486,7 +4476,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
    *
    * <p>See also
    * <a href="https://issues.apache.org/jira/browse/CALCITE-546">[CALCITE-546]
-   * "Allow table, column and field called '*'"</a> (not yet fixed).
+   * Allow table, column and field called '*'</a> (not yet fixed).
    */
   @Test public void testStarInFromFails() {
     sql("select emp.empno AS x from ^sales.*^")
@@ -4570,8 +4560,8 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     checkExpFails(
         "1 in ^((2), (3,4))^",
         ERR_IN_VALUES_INCOMPATIBLE);
-    checkExpFails(
-        "false and ^1 in ('b', 'c')^",
+    checkExp("false and ^1 in ('b', 'c')^");
+    checkExpFails("false and ^1 in (date '2012-01-02', date '2012-01-04')^",
         ERR_IN_OPERANDS_INCOMPATIBLE);
     checkExpFails(
         "1 > 5 ^or (1, 2) in (3, 4)^",
@@ -5149,10 +5139,15 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
   }
 
   @Test public void testNaturalJoinIncompatibleDatatype() {
-    checkFails(
-        "select * from emp natural ^join^\n"
+    checkFails("select *\n"
+            + "from (select ename as name, hiredate as deptno from emp)\n"
+            + "natural ^join^\n"
             + "(select deptno, name as sal from dept)",
-        "Column 'SAL' matched using NATURAL keyword or USING clause has incompatible types: cannot compare 'INTEGER' to 'VARCHAR\\(10\\)'");
+        "Column 'DEPTNO' matched using NATURAL keyword or USING clause has incompatible types: cannot compare 'TIMESTAMP\\(0\\)' to 'INTEGER'");
+
+    // INTEGER and VARCHAR are comparable: VARCHAR implicit converts to INTEGER
+    check("select * from emp natural ^join^\n"
+            + "(select deptno, name as sal from dept)");
 
     // make sal occur more than once on rhs, it is ignored and therefore
     // there is no error about incompatible types
@@ -5161,9 +5156,14 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
   }
 
   @Test public void testJoinUsingIncompatibleDatatype() {
-    checkFails(
-        "select * from emp join (select deptno, name as sal from dept) using (deptno, ^sal^)",
-        "Column 'SAL' matched using NATURAL keyword or USING clause has incompatible types: cannot compare 'INTEGER' to 'VARCHAR\\(10\\)'");
+    checkFails("select *\n"
+            + "from (select ename as name, hiredate as deptno from emp)\n"
+            + "join (select deptno, name as sal from dept) using (^deptno^, sal)",
+        "Column 'DEPTNO' matched using NATURAL keyword or USING clause has incompatible types: cannot compare 'TIMESTAMP\\(0\\)' to 'INTEGER'");
+
+    // INTEGER and VARCHAR are comparable: VARCHAR implicit converts to INTEGER
+    check("select * from emp\n"
+        + "join (select deptno, name as sal from dept) using (deptno, sal)");
   }
 
   @Test public void testJoinUsingInvalidColsFails() {
@@ -5624,7 +5624,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
   }
 
   /** Test case for
-   * <a href="https://issues.apache.org/jira/browse/CALCITE-633">[CALCITE-633],
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-633">[CALCITE-633]
    * WITH ... ORDER BY cannot find table</a>. */
   @Test public void testWithOrder() {
     sql("with e as (select * from emp)\n"
@@ -6183,6 +6183,41 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         "(?s).*Cannot apply '=' to arguments of type '<INTERVAL MONTH> = <INTERVAL DAY>'.*");
   }
 
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-613">[CALCITE-613]
+   * Implicitly convert strings in comparisons</a>. */
+  @Test public void testDateCompare() {
+    // can convert character value to date, time, timestamp, interval
+    // provided it is on one side of a comparison operator (=, <, >, BETWEEN)
+    checkExpType("date '2015-03-17' < '2015-03-18'", "BOOLEAN NOT NULL");
+    checkExpType("date '2015-03-17' > '2015-03-18'", "BOOLEAN NOT NULL");
+    checkExpType("date '2015-03-17' = '2015-03-18'", "BOOLEAN NOT NULL");
+    checkExpType("'2015-03-17' < date '2015-03-18'", "BOOLEAN NOT NULL");
+    checkExpType("date '2015-03-17' between '2015-03-16' and '2015-03-19'",
+        "BOOLEAN NOT NULL");
+    checkExpType("date '2015-03-17' between '2015-03-16' and '2015-03'||'-19'",
+        "BOOLEAN NOT NULL");
+    checkExpType("'2015-03-17' between date '2015-03-16' and date '2015-03-19'",
+        "BOOLEAN NOT NULL");
+    checkExpType("date '2015-03-17' between date '2015-03-16' and '2015-03-19'",
+        "BOOLEAN NOT NULL");
+    checkExpType("date '2015-03-17' between '2015-03-16' and date '2015-03-19'",
+        "BOOLEAN NOT NULL");
+    checkExpType("time '12:34:56' < '12:34:57'", "BOOLEAN NOT NULL");
+    checkExpType("timestamp '2015-03-17 12:34:56' < '2015-03-17 12:34:57'",
+        "BOOLEAN NOT NULL");
+    checkExpType("interval '2' hour < '2:30'", "BOOLEAN NOT NULL");
+
+    // can convert to exact and approximate numeric
+    checkExpType("123 > '72'", "BOOLEAN NOT NULL");
+    checkExpType("12.3 > '7.2'", "BOOLEAN NOT NULL");
+
+    // can convert to boolean
+    checkExpType("true = 'true'", "BOOLEAN NOT NULL");
+    checkExpFails("^true and 'true'^",
+        "Cannot apply 'AND' to arguments of type '<BOOLEAN> AND <CHAR\\(4\\)>'\\..*");
+  }
+
   @Test public void testOverlaps() {
     checkExpType(
         "(date '1-2-3', date '1-2-3') overlaps (date '1-2-3', date '1-2-3')",
@@ -6549,8 +6584,8 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
   }
 
   /** Test case for
-   * <a href="https://issues.apache.org/jira/browse/CALCITE-xxx">CALCITE-xxx,
-   * "Unexpected upper-casing of keywords when using java lexer"</a>. */
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-497">[CALCITE-497]
+   * Support optional qualifier for column name references</a>. */
   @Test public void testRecordTypeElided() {
     checkResultType(
         "SELECT contact.x, contact.coord.y FROM customer.contact",
@@ -6811,8 +6846,8 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
   }
 
   /** Test case for
-   * <a href="https://issues.apache.org/jira/browse/CALCITE-145">CALCITE-145,
-   * "Unexpected upper-casing of keywords when using java lexer"</a>. */
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-145">[CALCITE-145]
+   * Unexpected upper-casing of keywords when using java lexer</a>. */
   @Test public void testLexJavaKeyword() {
     final SqlTester tester1 = tester.withLex(Lex.JAVA);
     tester1.checkResultType(

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/272e6040/core/src/test/resources/sql/misc.oq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/misc.oq b/core/src/test/resources/sql/misc.oq
index 7b8ccb7..63422cf 100644
--- a/core/src/test/resources/sql/misc.oq
+++ b/core/src/test/resources/sql/misc.oq
@@ -826,4 +826,118 @@ select distinct gender, sum(deptno) as s from emp group by gender;
 #Expression 'DEPTNO' is not being grouped
 #!error
 
+!use scott
+
+# [CALCITE-613] Implicitly convert strings in comparisons
+select * from "scott".emp where hiredate < '1981-01-02';
++-------+-------+-------+------+------------+--------+------+--------+
+| EMPNO | ENAME | JOB   | MGR  | HIREDATE   | SAL    | COMM | DEPTNO |
++-------+-------+-------+------+------------+--------+------+--------+
+|  7369 | SMITH | CLERK | 7902 | 1980-12-17 | 800.00 |      |     20 |
++-------+-------+-------+------+------------+--------+------+--------+
+(1 row)
+
+!ok
+EnumerableCalc(expr#0..7=[{inputs}], expr#8=['1981-01-02'], expr#9=[CAST($t8):DATE NOT NULL], expr#10=[<($t4, $t9)], proj#0..7=[{exprs}], $condition=[$t10])
+  EnumerableTableScan(table=[[scott, EMP]])
+!plan
+select * from "scott".emp where '1981-01-02' > hiredate;
++-------+-------+-------+------+------------+--------+------+--------+
+| EMPNO | ENAME | JOB   | MGR  | HIREDATE   | SAL    | COMM | DEPTNO |
++-------+-------+-------+------+------------+--------+------+--------+
+|  7369 | SMITH | CLERK | 7902 | 1980-12-17 | 800.00 |      |     20 |
++-------+-------+-------+------+------------+--------+------+--------+
+(1 row)
+
+!ok
+select * from "scott".emp where hiredate between '1981-01-02' and '1981-06-01';
++-------+-------+----------+------+------------+---------+--------+--------+
+| EMPNO | ENAME | JOB      | MGR  | HIREDATE   | SAL     | COMM   | DEPTNO |
++-------+-------+----------+------+------------+---------+--------+--------+
+|  7499 | ALLEN | SALESMAN | 7698 | 1981-02-20 | 1600.00 | 300.00 |     30 |
+|  7521 | WARD  | SALESMAN | 7698 | 1981-02-22 | 1250.00 | 500.00 |     30 |
+|  7566 | JONES | MANAGER  | 7839 | 1981-02-04 | 2975.00 |        |     20 |
+|  7698 | BLAKE | MANAGER  | 7839 | 1981-01-05 | 2850.00 |        |     30 |
++-------+-------+----------+------+------------+---------+--------+--------+
+(4 rows)
+
+!ok
+select * from "scott".emp where hiredate > '1986-01-02';
++-------+-------+---------+------+------------+---------+------+--------+
+| EMPNO | ENAME | JOB     | MGR  | HIREDATE   | SAL     | COMM | DEPTNO |
++-------+-------+---------+------+------------+---------+------+--------+
+|  7788 | SCOTT | ANALYST | 7566 | 1987-04-19 | 3000.00 |      |     20 |
+|  7876 | ADAMS | CLERK   | 7788 | 1987-05-23 | 1100.00 |      |     20 |
++-------+-------+---------+------+------------+---------+------+--------+
+(2 rows)
+
+!ok
+select * from "scott".emp where '1986-01-02' < hiredate;
++-------+-------+---------+------+------------+---------+------+--------+
+| EMPNO | ENAME | JOB     | MGR  | HIREDATE   | SAL     | COMM | DEPTNO |
++-------+-------+---------+------+------------+---------+------+--------+
+|  7788 | SCOTT | ANALYST | 7566 | 1987-04-19 | 3000.00 |      |     20 |
+|  7876 | ADAMS | CLERK   | 7788 | 1987-05-23 | 1100.00 |      |     20 |
++-------+-------+---------+------+------------+---------+------+--------+
+(2 rows)
+
+!ok
+select * from "scott".emp where '1986-' || '01-02' < hiredate;
++-------+-------+---------+------+------------+---------+------+--------+
+| EMPNO | ENAME | JOB     | MGR  | HIREDATE   | SAL     | COMM | DEPTNO |
++-------+-------+---------+------+------------+---------+------+--------+
+|  7788 | SCOTT | ANALYST | 7566 | 1987-04-19 | 3000.00 |      |     20 |
+|  7876 | ADAMS | CLERK   | 7788 | 1987-05-23 | 1100.00 |      |     20 |
++-------+-------+---------+------+------------+---------+------+--------+
+(2 rows)
+
+!ok
+select * from "scott".emp where sal < '1100';
++-------+-------+-------+------+------------+--------+------+--------+
+| EMPNO | ENAME | JOB   | MGR  | HIREDATE   | SAL    | COMM | DEPTNO |
++-------+-------+-------+------+------------+--------+------+--------+
+|  7369 | SMITH | CLERK | 7902 | 1980-12-17 | 800.00 |      |     20 |
+|  7900 | JAMES | CLERK | 7698 | 1981-12-03 | 950.00 |      |     30 |
++-------+-------+-------+------+------------+--------+------+--------+
+(2 rows)
+
+!ok
+select * from "scott".emp where empno in ('7369', '7876');
++-------+-------+-------+------+------------+---------+------+--------+
+| EMPNO | ENAME | JOB   | MGR  | HIREDATE   | SAL     | COMM | DEPTNO |
++-------+-------+-------+------+------------+---------+------+--------+
+|  7369 | SMITH | CLERK | 7902 | 1980-12-17 |  800.00 |      |     20 |
+|  7876 | ADAMS | CLERK | 7788 | 1987-05-23 | 1100.00 |      |     20 |
++-------+-------+-------+------+------------+---------+------+--------+
+(2 rows)
+
+!ok
+select * from "scott".emp where empno between '7500' and '07600';
++-------+-------+----------+------+------------+---------+--------+--------+
+| EMPNO | ENAME | JOB      | MGR  | HIREDATE   | SAL     | COMM   | DEPTNO |
++-------+-------+----------+------+------------+---------+--------+--------+
+|  7521 | WARD  | SALESMAN | 7698 | 1981-02-22 | 1250.00 | 500.00 |     30 |
+|  7566 | JONES | MANAGER  | 7839 | 1981-02-04 | 2975.00 |        |     20 |
++-------+-------+----------+------+------------+---------+--------+--------+
+(2 rows)
+
+!ok
+select * from "scott".emp where deptno between '7369' and '7876';
++-------+-------+-----+-----+----------+-----+------+--------+
+| EMPNO | ENAME | JOB | MGR | HIREDATE | SAL | COMM | DEPTNO |
++-------+-------+-----+-----+----------+-----+------+--------+
++-------+-------+-----+-----+----------+-----+------+--------+
+(0 rows)
+
+!ok
+select * from "scott".emp where '7369' between empno and '7876';
++-------+-------+-------+------+------------+--------+------+--------+
+| EMPNO | ENAME | JOB   | MGR  | HIREDATE   | SAL    | COMM | DEPTNO |
++-------+-------+-------+------+------------+--------+------+--------+
+|  7369 | SMITH | CLERK | 7902 | 1980-12-17 | 800.00 |      |     20 |
++-------+-------+-------+------+------------+--------+------+--------+
+(1 row)
+
+!ok
+
 # End misc.oq


[09/10] incubator-calcite git commit: [CALCITE-646] AvaticaStatement execute method broken over remote JDBC (Yeong Wei and Julian Hyde)

Posted by jh...@apache.org.
[CALCITE-646] AvaticaStatement execute method broken over remote JDBC (Yeong Wei and Julian Hyde)


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

Branch: refs/heads/master
Commit: 0d80fd25315bc53f71622415c8bb3d364e634507
Parents: 272e604
Author: Julian Hyde <jh...@apache.org>
Authored: Thu Apr 2 23:50:27 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Apr 3 01:10:06 2015 -0700

----------------------------------------------------------------------
 .../apache/calcite/avatica/jdbc/JdbcMeta.java   |  28 ++---
 .../calcite/avatica/jdbc/JdbcResultSet.java     |   2 +-
 .../calcite/avatica/RemoteDriverTest.java       | 105 +++++++++++++++----
 .../calcite/avatica/AvaticaConnection.java      |  28 ++---
 .../calcite/avatica/AvaticaStatement.java       |  79 +++++++-------
 .../java/org/apache/calcite/avatica/Meta.java   |  50 +++++++--
 .../org/apache/calcite/avatica/MetaImpl.java    |   2 +-
 .../calcite/avatica/remote/JsonService.java     |  47 ++++++---
 .../calcite/avatica/remote/LocalService.java    |  19 +++-
 .../calcite/avatica/remote/MockJsonService.java |   8 +-
 .../calcite/avatica/remote/RemoteMeta.java      |  23 +++-
 .../apache/calcite/avatica/remote/Service.java  |  28 ++++-
 .../apache/calcite/jdbc/CalciteMetaImpl.java    |  16 +--
 .../calcite/jdbc/CalciteRemoteDriverTest.java   |  17 +++
 14 files changed, 311 insertions(+), 141 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/0d80fd25/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
----------------------------------------------------------------------
diff --git a/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java b/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
index 698d7a7..f9d783d 100644
--- a/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
+++ b/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
@@ -88,13 +88,6 @@ public class JdbcMeta implements Meta {
     SQL_TYPE_TO_JAVA_TYPE.put(Types.ARRAY, Array.class);
   }
 
-  /** A "magic column" descriptor used to return statement execute update count via a
-   * {@link Meta.MetaResultSet}. */
-  private static final ColumnMetaData UPDATE_COL = new ColumnMetaData(1, false, false, false,
-      false, ResultSetMetaData.columnNoNulls, true, 10, "u", "u", null, 15, 15, "update_result",
-      "avatica_internal", ColumnMetaData.scalar(Types.INTEGER, "INTEGER",
-        ColumnMetaData.Rep.INTEGER), true, false, false, "java.lang.Integer");
-
   private static final String CONN_CACHE_KEY_BASE = "avatica.connectioncache";
 
   /** Configurable connection cache settings. */
@@ -751,7 +744,7 @@ public class JdbcMeta implements Meta {
     }
   }
 
-  public MetaResultSet prepareAndExecute(ConnectionHandle ch, String sql,
+  public ExecuteResult prepareAndExecute(ConnectionHandle ch, String sql,
       int maxRowCount, PrepareCallback callback) {
     try {
       final Connection connection = getConnection(ch.id);
@@ -762,27 +755,20 @@ public class JdbcMeta implements Meta {
       boolean ret = statement.execute();
       info.resultSet = statement.getResultSet();
       assert ret || info.resultSet == null;
-      final MetaResultSet mrs;
+      final List<MetaResultSet> resultSets = new ArrayList<>();
       if (info.resultSet == null) {
-        // build a non-JDBC result that contains the update count
-        int updateCount = statement.getUpdateCount();
-        List<ColumnMetaData> columns = new ArrayList<>(1);
-        columns.add(UPDATE_COL);
-        List<Object> val = new ArrayList<>();
-        val.add(new Object[] { updateCount });
-        final Signature signature =
-            new Signature(columns, sql, null, null, CursorFactory.ARRAY);
-        final Frame frame = new Frame(0, true, val);
-        mrs = new MetaResultSet(ch.id, id, true, signature, frame);
+        // Create a special result set that just carries update count
+        resultSets.add(
+            MetaResultSet.count(ch.id, id, statement.getUpdateCount()));
       } else {
-        mrs = JdbcResultSet.create(ch.id, id, info.resultSet);
+        resultSets.add(JdbcResultSet.create(ch.id, id, info.resultSet));
       }
       if (LOG.isTraceEnabled()) {
         StatementHandle h = new StatementHandle(ch.id, id, null);
         LOG.trace("prepAndExec statement " + h);
       }
       // TODO: review client to ensure statementId is updated when appropriate
-      return mrs;
+      return new ExecuteResult(resultSets);
     } catch (SQLException e) {
       throw propagate(e);
     }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/0d80fd25/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcResultSet.java
----------------------------------------------------------------------
diff --git a/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcResultSet.java b/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcResultSet.java
index 407b91e..bcc2745 100644
--- a/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcResultSet.java
+++ b/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcResultSet.java
@@ -36,7 +36,7 @@ import java.util.List;
 class JdbcResultSet extends Meta.MetaResultSet {
   protected JdbcResultSet(String connectionId, int statementId,
       boolean ownStatement, Meta.Signature signature, Meta.Frame firstFrame) {
-    super(connectionId, statementId, ownStatement, signature, firstFrame);
+    super(connectionId, statementId, ownStatement, signature, firstFrame, -1);
   }
 
   /** Creates a result set. */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/0d80fd25/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
----------------------------------------------------------------------
diff --git a/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java b/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
index 1a5316c..f23e2fc 100644
--- a/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
+++ b/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
@@ -45,6 +45,7 @@ import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -202,16 +203,95 @@ public class RemoteDriverTest {
     final String insert = "insert into TEST_TABLE values(1, 'foo')";
     final String update = "update TEST_TABLE set msg='bar' where id=1";
     try (Connection connection = ljs();
-        Statement statement = connection.createStatement()) {
-      boolean ret;
+        Statement statement = connection.createStatement();
+        PreparedStatement pstmt = connection.prepareStatement("values 1")) {
+      // drop
       assertFalse(statement.execute(drop));
       assertEquals(0, statement.getUpdateCount());
+      assertNull(statement.getResultSet());
+      try {
+        final ResultSet rs = statement.executeQuery(drop);
+        fail("expected error, got " + rs);
+      } catch (SQLException e) {
+        assertThat(e.getMessage(),
+            equalTo("Statement did not return a result set"));
+      }
+      assertEquals(0, statement.executeUpdate(drop));
+      assertEquals(0, statement.getUpdateCount());
+      assertNull(statement.getResultSet());
+
+      // create
       assertFalse(statement.execute(create));
       assertEquals(0, statement.getUpdateCount());
+      assertNull(statement.getResultSet());
+      assertFalse(statement.execute(drop)); // tidy up
+      try {
+        final ResultSet rs = statement.executeQuery(create);
+        fail("expected error, got " + rs);
+      } catch (SQLException e) {
+        assertThat(e.getMessage(),
+            equalTo("Statement did not return a result set"));
+      }
+      assertFalse(statement.execute(drop)); // tidy up
+      assertEquals(0, statement.executeUpdate(create));
+      assertEquals(0, statement.getUpdateCount());
+      assertNull(statement.getResultSet());
+
+      // insert
       assertFalse(statement.execute(insert));
       assertEquals(1, statement.getUpdateCount());
+      assertNull(statement.getResultSet());
+      try {
+        final ResultSet rs = statement.executeQuery(insert);
+        fail("expected error, got " + rs);
+      } catch (SQLException e) {
+        assertThat(e.getMessage(),
+            equalTo("Statement did not return a result set"));
+      }
+      assertEquals(1, statement.executeUpdate(insert));
+      assertEquals(1, statement.getUpdateCount());
+      assertNull(statement.getResultSet());
+
+      // update
       assertFalse(statement.execute(update));
-      assertEquals(0, statement.executeUpdate(drop));
+      assertEquals(3, statement.getUpdateCount());
+      assertNull(statement.getResultSet());
+      try {
+        final ResultSet rs = statement.executeQuery(update);
+        fail("expected error, got " + rs);
+      } catch (SQLException e) {
+        assertThat(e.getMessage(),
+            equalTo("Statement did not return a result set"));
+      }
+      assertEquals(3, statement.executeUpdate(update));
+      assertEquals(3, statement.getUpdateCount());
+      assertNull(statement.getResultSet());
+
+      final String[] messages = {
+        "Cannot call executeQuery(String) on prepared or callable statement",
+        "Cannot call execute(String) on prepared or callable statement",
+        "Cannot call executeUpdate(String) on prepared or callable statement",
+      };
+      for (String sql : new String[]{drop, create, insert, update}) {
+        for (int i = 0; i <= 2; i++) {
+          try {
+            Object o;
+            switch (i) {
+            case 0:
+              o = pstmt.executeQuery(sql);
+              break;
+            case 1:
+              o = pstmt.execute(sql);
+              break;
+            default:
+              o = pstmt.executeUpdate(sql);
+            }
+            fail("expected error, got " + o);
+          } catch (SQLException e) {
+            assertThat(e.getMessage(), equalTo(messages[i]));
+          }
+        }
+      }
     }
   }
 
@@ -294,25 +374,6 @@ public class RemoteDriverTest {
         0, connectionMap.size());
   }
 
-  private void checkStatementExecuteQuery(Connection connection)
-      throws SQLException {
-    final Statement statement = connection.createStatement();
-    final ResultSet resultSet =
-        statement.executeQuery("select * from (\n"
-            + "  values (1, 'a'), (null, 'b'), (3, 'c')) as t (c1, c2)");
-    final ResultSetMetaData metaData = resultSet.getMetaData();
-    assertEquals(2, metaData.getColumnCount());
-    assertEquals("C1", metaData.getColumnName(1));
-    assertEquals("C2", metaData.getColumnName(2));
-    assertTrue(resultSet.next());
-    assertTrue(resultSet.next());
-    assertTrue(resultSet.next());
-    assertFalse(resultSet.next());
-    resultSet.close();
-    statement.close();
-    connection.close();
-  }
-
   @Test public void testPrepareBindExecuteFetch() throws Exception {
     checkPrepareBindExecuteFetch(ljs());
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/0d80fd25/avatica/src/main/java/org/apache/calcite/avatica/AvaticaConnection.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaConnection.java b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaConnection.java
index 2c66d1e..f6bb25a 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaConnection.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaConnection.java
@@ -432,10 +432,10 @@ public abstract class AvaticaConnection implements Connection {
     return statement.openResultSet;
   }
 
-  protected ResultSet prepareAndExecuteInternal(
+  protected Meta.ExecuteResult prepareAndExecuteInternal(
       final AvaticaStatement statement, String sql, int maxRowCount)
       throws SQLException {
-    Meta.MetaResultSet x = meta.prepareAndExecute(handle, sql, maxRowCount,
+    final Meta.PrepareCallback callback =
         new Meta.PrepareCallback() {
           public Object getMonitor() {
             return statement;
@@ -454,20 +454,24 @@ public abstract class AvaticaConnection implements Connection {
             }
           }
 
-          public void assign(Meta.Signature signature, Meta.Frame firstFrame)
-              throws SQLException {
-            final TimeZone timeZone = getTimeZone();
-            statement.openResultSet =
-                factory.newResultSet(statement, signature, timeZone,
-                    firstFrame);
+          public void assign(Meta.Signature signature, Meta.Frame firstFrame,
+              int updateCount) throws SQLException {
+            if (updateCount != -1) {
+              statement.updateCount = updateCount;
+            } else {
+              final TimeZone timeZone = getTimeZone();
+              statement.openResultSet = factory.newResultSet(statement,
+                  signature, timeZone, firstFrame);
+            }
           }
 
           public void execute() throws SQLException {
-            statement.openResultSet.execute();
+            if (statement.openResultSet != null) {
+              statement.openResultSet.execute();
+            }
           }
-        });
-    assert statement.openResultSet != null;
-    return statement.openResultSet;
+        };
+    return meta.prepareAndExecute(handle, sql, maxRowCount, callback);
   }
 
   protected ResultSet createResultSet(Meta.MetaResultSet metaResultSet)

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/0d80fd25/avatica/src/main/java/org/apache/calcite/avatica/AvaticaStatement.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaStatement.java b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaStatement.java
index 1a7e5ec..8deb643 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaStatement.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaStatement.java
@@ -16,8 +16,9 @@
  */
 package org.apache.calcite.avatica;
 
+import java.sql.CallableStatement;
+import java.sql.PreparedStatement;
 import java.sql.ResultSet;
-import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
 import java.sql.SQLWarning;
 import java.sql.Statement;
@@ -49,6 +50,9 @@ public abstract class AvaticaStatement
    */
   protected AvaticaResultSet openResultSet;
 
+  /** Current update count. Same lifecycle as {@link #openResultSet}. */
+  protected int updateCount;
+
   private int queryTimeoutMillis;
   final int resultSetType;
   final int resultSetConcurrency;
@@ -56,7 +60,6 @@ public abstract class AvaticaStatement
   private int fetchSize;
   private int fetchDirection;
   protected int maxRowCount = 0;
-  private int updateCount = -1;
 
   /**
    * Creates an AvaticaStatement.
@@ -88,37 +91,47 @@ public abstract class AvaticaStatement
     return handle.id;
   }
 
-  // implement Statement
-
-  public boolean execute(String sql) throws SQLException {
-    ResultSet resultSet = executeQuery(sql);
-    ResultSetMetaData md = resultSet.getMetaData();
-    // hackish, but be sure we're looking at an update count result, not user data
-    if (md.getCatalogName(1).equalsIgnoreCase("avatica_internal")
-        && md.getTableName(1).equalsIgnoreCase("update_result")
-        && md.getColumnCount() == 1
-        && md.getColumnName(1).equalsIgnoreCase("u")) {
-      if (!resultSet.next()) {
-        throw new SQLException("expected one row, got zero");
-      }
-      this.updateCount = resultSet.getInt(1);
-      if (resultSet.next()) {
-        throw new SQLException("expected one row, got two or more");
-      }
-      resultSet.close();
-      return false;
-    } else {
-      return !resultSet.isClosed();
+  private void checkNotPreparedOrCallable(String s) throws SQLException {
+    if (this instanceof PreparedStatement
+        || this instanceof CallableStatement) {
+      throw connection.helper.createException("Cannot call " + s
+          + " on prepared or callable statement");
     }
   }
 
-  public ResultSet executeQuery(String sql) throws SQLException {
+  protected void executeInternal(String sql) throws SQLException {
     // reset previous state before moving forward.
     this.updateCount = -1;
     try {
       // In JDBC, maxRowCount = 0 means no limit; in prepare it means LIMIT 0
       final int maxRowCount1 = maxRowCount <= 0 ? -1 : maxRowCount;
-      return connection.prepareAndExecuteInternal(this, sql, maxRowCount1);
+      Meta.ExecuteResult x =
+          connection.prepareAndExecuteInternal(this, sql, maxRowCount1);
+    } catch (RuntimeException e) {
+      throw connection.helper.createException(
+          "error while executing SQL \"" + sql + "\": " + e.getMessage(), e);
+    }
+  }
+
+  // implement Statement
+
+  public boolean execute(String sql) throws SQLException {
+    checkNotPreparedOrCallable("execute(String)");
+    executeInternal(sql);
+    // Result set is null for DML or DDL.
+    // Result set is closed if user cancelled the query.
+    return openResultSet != null && !openResultSet.isClosed();
+  }
+
+  public ResultSet executeQuery(String sql) throws SQLException {
+    checkNotPreparedOrCallable("executeQuery(String)");
+    try {
+      executeInternal(sql);
+      if (openResultSet == null) {
+        throw connection.helper.createException(
+            "Statement did not return a result set");
+      }
+      return openResultSet;
     } catch (RuntimeException e) {
       throw connection.helper.createException(
         "error while executing SQL \"" + sql + "\": " + e.getMessage(), e);
@@ -126,19 +139,9 @@ public abstract class AvaticaStatement
   }
 
   public int executeUpdate(String sql) throws SQLException {
-    ResultSet resultSet = executeQuery(sql);
-    if (resultSet.getMetaData().getColumnCount() != 1) {
-      throw new SQLException("expected one result column");
-    }
-    if (!resultSet.next()) {
-      throw new SQLException("expected one row, got zero");
-    }
-    this.updateCount = resultSet.getInt(1);
-    if (resultSet.next()) {
-      throw new SQLException("expected one row, got two or more");
-    }
-    resultSet.close();
-    return this.updateCount;
+    checkNotPreparedOrCallable("executeUpdate(String)");
+    executeInternal(sql);
+    return updateCount;
   }
 
   public synchronized void close() throws SQLException {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/0d80fd25/avatica/src/main/java/org/apache/calcite/avatica/Meta.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/Meta.java b/avatica/src/main/java/org/apache/calcite/avatica/Meta.java
index 49fc3a5..cadcd17 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/Meta.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/Meta.java
@@ -168,9 +168,11 @@ public interface Meta {
    * @param sql SQL query
    * @param maxRowCount Negative for no limit (different meaning than JDBC)
    * @param callback Callback to lock, clear and assign cursor
-   * @return MetaResultSet containing statement ID and first frame of data
+   *
+   * @return Result containing statement ID, and if a query, a result set and
+   * first frame of data
    */
-  MetaResultSet prepareAndExecute(ConnectionHandle ch, String sql,
+  ExecuteResult prepareAndExecute(ConnectionHandle ch, String sql,
       int maxRowCount, PrepareCallback callback);
 
   /** Returns a frame of rows.
@@ -234,21 +236,53 @@ public interface Meta {
     }
   }
 
-  /** Meta data from which a result set can be constructed. */
+  /** Response from execute.
+   *
+   * <p>Typically a query will have a result set and rowCount = -1;
+   * a DML statement will have a rowCount and no result sets.
+   */
+  class ExecuteResult {
+    public final List<MetaResultSet> resultSets;
+
+    public ExecuteResult(List<MetaResultSet> resultSets) {
+      this.resultSets = resultSets;
+    }
+  }
+
+  /** Meta data from which a result set can be constructed.
+   *
+   * <p>If {@code updateCount} is not -1, the result is just a count. A result
+   * set cannot be constructed. */
   class MetaResultSet {
     public final String connectionId;
     public final int statementId;
     public final boolean ownStatement;
     public final Frame firstFrame;
     public final Signature signature;
+    public final int updateCount;
 
-    public MetaResultSet(String connectionId, int statementId,
-        boolean ownStatement, Signature signature, Frame firstFrame) {
-      this.signature = Objects.requireNonNull(signature);
+    protected MetaResultSet(String connectionId, int statementId,
+        boolean ownStatement, Signature signature, Frame firstFrame,
+        int updateCount) {
+      this.signature = signature;
       this.connectionId = connectionId;
       this.statementId = statementId;
       this.ownStatement = ownStatement;
-      this.firstFrame = firstFrame; // may be null
+      this.firstFrame = firstFrame; // may be null even if signature is not null
+      this.updateCount = updateCount;
+    }
+
+    public static MetaResultSet create(String connectionId, int statementId,
+        boolean ownStatement, Signature signature, Frame firstFrame) {
+      return new MetaResultSet(connectionId, statementId, ownStatement,
+          Objects.requireNonNull(signature), firstFrame, -1);
+    }
+
+    public static MetaResultSet count(String connectionId, int statementId,
+        int updateCount) {
+      assert updateCount >= 0;
+      return new MetaResultSet(connectionId, statementId, false, null, null,
+          updateCount);
     }
   }
 
@@ -555,7 +589,7 @@ public interface Meta {
   interface PrepareCallback {
     Object getMonitor();
     void clear() throws SQLException;
-    void assign(Signature signature, Frame firstFrame)
+    void assign(Signature signature, Frame firstFrame, int updateCount)
         throws SQLException;
     void execute() throws SQLException;
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/0d80fd25/avatica/src/main/java/org/apache/calcite/avatica/MetaImpl.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/MetaImpl.java b/avatica/src/main/java/org/apache/calcite/avatica/MetaImpl.java
index b8820e8..aa21194 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/MetaImpl.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/MetaImpl.java
@@ -243,7 +243,7 @@ public abstract class MetaImpl implements Meta {
       final Signature signature =
           new Signature(columns, "", Collections.<AvaticaParameter>emptyList(),
               internalParameters, cursorFactory);
-      return new MetaResultSet(connection.id, statement.getId(), true,
+      return MetaResultSet.create(connection.id, statement.getId(), true,
           signature, firstFrame);
     } catch (SQLException e) {
       throw new RuntimeException(e);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/0d80fd25/avatica/src/main/java/org/apache/calcite/avatica/remote/JsonService.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/remote/JsonService.java b/avatica/src/main/java/org/apache/calcite/avatica/remote/JsonService.java
index 2dd349c..d8e22e6 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/remote/JsonService.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/JsonService.java
@@ -46,8 +46,11 @@ public abstract class JsonService implements Service {
    * responses to and from the peer service. */
   public abstract String apply(String request);
 
-  /** Modifies a signature, changing the type of columns within it. */
-  private static Meta.Signature fangle(Meta.Signature signature) {
+  /** Modifies a signature, changing the representation of numeric columns
+   * within it. This deals with the fact that JSON transmits a small long value,
+   * or a float which is a whole number, as an integer. Thus the accessors need
+   * be prepared to accept any numeric type. */
+  private static Meta.Signature finagle(Meta.Signature signature) {
     final List<ColumnMetaData> columns = new ArrayList<>();
     int changeCount = 0;
     for (ColumnMetaData column : signature.columns) {
@@ -77,29 +80,49 @@ public abstract class JsonService implements Service {
         signature.cursorFactory);
   }
 
-  private static PrepareResponse fangle(PrepareResponse response) {
-    final Meta.StatementHandle statement = fangle(response.statement);
+  private static PrepareResponse finagle(PrepareResponse response) {
+    final Meta.StatementHandle statement = finagle(response.statement);
     if (statement == response.statement) {
       return response;
     }
     return new PrepareResponse(statement);
   }
 
-  private static Meta.StatementHandle fangle(Meta.StatementHandle h) {
-    final Meta.Signature signature = fangle(h.signature);
+  private static Meta.StatementHandle finagle(Meta.StatementHandle h) {
+    final Meta.Signature signature = finagle(h.signature);
     if (signature == h.signature) {
       return h;
     }
     return new Meta.StatementHandle(h.connectionId, h.id, signature);
   }
 
-  private static ResultSetResponse fangle(ResultSetResponse r) {
-    final Meta.Signature signature = fangle(r.signature);
+  private static ResultSetResponse finagle(ResultSetResponse r) {
+    if (r.updateCount != -1) {
+      assert r.signature == null;
+      return r;
+    }
+    final Meta.Signature signature = finagle(r.signature);
     if (signature == r.signature) {
       return r;
     }
     return new ResultSetResponse(r.connectionId, r.statementId, r.ownStatement,
-        signature, r.firstFrame);
+        signature, r.firstFrame, r.updateCount);
+  }
+
+  private static ExecuteResponse finagle(ExecuteResponse r) {
+    final List<ResultSetResponse> results = new ArrayList<>();
+    int changeCount = 0;
+    for (ResultSetResponse result : r.results) {
+      ResultSetResponse result2 = finagle(result);
+      if (result2 != result) {
+        ++changeCount;
+      }
+      results.add(result2);
+    }
+    if (changeCount == 0) {
+      return r;
+    }
+    return new ExecuteResponse(results);
   }
 
   //@VisibleForTesting
@@ -161,15 +184,15 @@ public abstract class JsonService implements Service {
 
   public PrepareResponse apply(PrepareRequest request) {
     try {
-      return fangle(decode(apply(encode(request)), PrepareResponse.class));
+      return finagle(decode(apply(encode(request)), PrepareResponse.class));
     } catch (IOException e) {
       throw handle(e);
     }
   }
 
-  public ResultSetResponse apply(PrepareAndExecuteRequest request) {
+  public ExecuteResponse apply(PrepareAndExecuteRequest request) {
     try {
-      return fangle(decode(apply(encode(request)), ResultSetResponse.class));
+      return finagle(decode(apply(encode(request)), ExecuteResponse.class));
     } catch (IOException e) {
       throw handle(e);
     }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/0d80fd25/avatica/src/main/java/org/apache/calcite/avatica/remote/LocalService.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/remote/LocalService.java b/avatica/src/main/java/org/apache/calcite/avatica/remote/LocalService.java
index 2bdb1bd..aad40e1 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/remote/LocalService.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/LocalService.java
@@ -46,6 +46,11 @@ public class LocalService implements Service {
 
   /** Converts a result set (not serializable) into a serializable response. */
   public ResultSetResponse toResponse(Meta.MetaResultSet resultSet) {
+    if (resultSet.updateCount != -1) {
+      return new ResultSetResponse(resultSet.connectionId,
+          resultSet.statementId, resultSet.ownStatement, null, null,
+          resultSet.updateCount);
+    }
     Meta.CursorFactory cursorFactory = resultSet.signature.cursorFactory;
     final List<Object> list;
     if (resultSet.firstFrame != null) {
@@ -70,7 +75,7 @@ public class LocalService implements Service {
       signature = signature.setCursorFactory(cursorFactory);
     }
     return new ResultSetResponse(resultSet.connectionId, resultSet.statementId,
-        resultSet.ownStatement, signature, new Meta.Frame(0, true, list));
+        resultSet.ownStatement, signature, new Meta.Frame(0, true, list), -1);
   }
 
   private List<List<Object>> list2(Meta.MetaResultSet resultSet) {
@@ -124,10 +129,10 @@ public class LocalService implements Service {
     return new PrepareResponse(h);
   }
 
-  public ResultSetResponse apply(PrepareAndExecuteRequest request) {
+  public ExecuteResponse apply(PrepareAndExecuteRequest request) {
     final Meta.ConnectionHandle ch =
         new Meta.ConnectionHandle(request.connectionId);
-    final Meta.MetaResultSet resultSet =
+    final Meta.ExecuteResult executeResult =
         meta.prepareAndExecute(ch, request.sql, request.maxRowCount,
             new Meta.PrepareCallback() {
               @Override public Object getMonitor() {
@@ -138,13 +143,17 @@ public class LocalService implements Service {
               }
 
               @Override public void assign(Meta.Signature signature,
-                  Meta.Frame firstFrame) {
+                  Meta.Frame firstFrame, int updateCount) {
               }
 
               @Override public void execute() {
               }
             });
-    return toResponse(resultSet);
+    final List<ResultSetResponse> results = new ArrayList<>();
+    for (Meta.MetaResultSet metaResultSet : executeResult.resultSets) {
+      results.add(toResponse(metaResultSet));
+    }
+    return new ExecuteResponse(results);
   }
 
   public FetchResponse apply(FetchRequest request) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/0d80fd25/avatica/src/main/java/org/apache/calcite/avatica/remote/MockJsonService.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/remote/MockJsonService.java b/avatica/src/main/java/org/apache/calcite/avatica/remote/MockJsonService.java
index 02cb191..7cc5420 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/remote/MockJsonService.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/MockJsonService.java
@@ -64,21 +64,21 @@ public class MockJsonService extends JsonService {
       try {
         map1.put(
             "{\"request\":\"getSchemas\",\"catalog\":null,\"schemaPattern\":{\"s\":null}}",
-            "{\"response\":\"resultSet\", firstFrame: {offset: 0, done: true, rows: []}}");
+            "{\"response\":\"resultSet\", updateCount: -1, firstFrame: {offset: 0, done: true, rows: []}}");
         map1.put(
             JsonService.encode(new SchemasRequest(null, null)),
-            "{\"response\":\"resultSet\", firstFrame: {offset: 0, done: true, rows: []}}");
+            "{\"response\":\"resultSet\", updateCount: -1, firstFrame: {offset: 0, done: true, rows: []}}");
         map1.put(
             JsonService.encode(
                 new TablesRequest(null, null, null, Arrays.<String>asList())),
-            "{\"response\":\"resultSet\", firstFrame: {offset: 0, done: true, rows: []}}");
+            "{\"response\":\"resultSet\", updateCount: -1, firstFrame: {offset: 0, done: true, rows: []}}");
         map1.put(
             "{\"request\":\"createStatement\",\"connectionId\":0}",
             "{\"response\":\"createStatement\",\"id\":0}");
         map1.put(
             "{\"request\":\"prepareAndExecute\",\"statementId\":0,"
                 + "\"sql\":\"select * from (\\n  values (1, 'a'), (null, 'b'), (3, 'c')) as t (c1, c2)\",\"maxRowCount\":-1}",
-            "{\"response\":\"resultSet\",\"signature\": {\n"
+            "{\"response\":\"resultSet\", updateCount: -1, \"signature\": {\n"
                 + " \"columns\": [\n"
                 + "   {\"columnName\": \"C1\", \"type\": {type: \"scalar\", id: 4, rep: \"INTEGER\"}},\n"
                 + "   {\"columnName\": \"C2\", \"type\": {type: \"scalar\", id: 12, rep: \"STRING\"}}\n"

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/0d80fd25/avatica/src/main/java/org/apache/calcite/avatica/remote/RemoteMeta.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/remote/RemoteMeta.java b/avatica/src/main/java/org/apache/calcite/avatica/remote/RemoteMeta.java
index 93e5541..9a3eab6 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/remote/RemoteMeta.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/RemoteMeta.java
@@ -24,6 +24,7 @@ import org.apache.calcite.avatica.Meta;
 import org.apache.calcite.avatica.MetaImpl;
 
 import java.sql.SQLException;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
@@ -43,6 +44,10 @@ class RemoteMeta extends MetaImpl {
 
   private MetaResultSet toResultSet(Class clazz,
       Service.ResultSetResponse response) {
+    if (response.updateCount != -1) {
+      return MetaResultSet.count(response.connectionId, response.statementId,
+          response.updateCount);
+    }
     Signature signature0 = response.signature;
     if (signature0 == null) {
       final List<ColumnMetaData> columns =
@@ -52,7 +57,7 @@ class RemoteMeta extends MetaImpl {
       signature0 = Signature.create(columns,
           "?", Collections.<AvaticaParameter>emptyList(), CursorFactory.ARRAY);
     }
-    return new MetaResultSet(response.connectionId, response.statementId,
+    return MetaResultSet.create(response.connectionId, response.statementId,
         response.ownStatement, signature0, response.firstFrame);
   }
 
@@ -142,19 +147,27 @@ class RemoteMeta extends MetaImpl {
     return response.statement;
   }
 
-  @Override public MetaResultSet prepareAndExecute(ConnectionHandle ch,
+  @Override public ExecuteResult prepareAndExecute(ConnectionHandle ch,
       String sql, int maxRowCount, PrepareCallback callback) {
     connectionSync(ch, new ConnectionPropertiesImpl()); // sync connection state if necessary
-    final Service.ResultSetResponse response;
+    final Service.ExecuteResponse response;
     try {
       synchronized (callback.getMonitor()) {
         callback.clear();
         response = service.apply(
             new Service.PrepareAndExecuteRequest(ch.id, sql, maxRowCount));
-        callback.assign(response.signature, response.firstFrame);
+        if (response.results.size() > 0) {
+          final Service.ResultSetResponse result = response.results.get(0);
+          callback.assign(result.signature, result.firstFrame,
+              result.updateCount);
+        }
       }
       callback.execute();
-      return toResultSet(null, response);
+      List<MetaResultSet> metaResultSets = new ArrayList<>();
+      for (Service.ResultSetResponse result : response.results) {
+        metaResultSets.add(toResultSet(null, result));
+      }
+      return new ExecuteResult(metaResultSets);
     } catch (SQLException e) {
       throw new RuntimeException(e);
     }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/0d80fd25/avatica/src/main/java/org/apache/calcite/avatica/remote/Service.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/remote/Service.java b/avatica/src/main/java/org/apache/calcite/avatica/remote/Service.java
index c85e53a..430450d 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/remote/Service.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/Service.java
@@ -36,7 +36,7 @@ public interface Service {
   ResultSetResponse apply(TableTypesRequest request);
   ResultSetResponse apply(ColumnsRequest request);
   PrepareResponse apply(PrepareRequest request);
-  ResultSetResponse apply(PrepareAndExecuteRequest request);
+  ExecuteResponse apply(PrepareAndExecuteRequest request);
   FetchResponse apply(FetchRequest request);
   CreateStatementResponse apply(CreateStatementRequest request);
   CloseStatementResponse apply(CloseStatementRequest request);
@@ -148,8 +148,7 @@ public interface Service {
   /** Request for
    * {@link Meta#getTableTypes()}. */
   class TableTypesRequest extends Request {
-
-    ResultSetResponse accept(Service service) {
+    @Override ResultSetResponse accept(Service service) {
       return service.apply(this);
     }
   }
@@ -181,6 +180,10 @@ public interface Service {
 
   /** Response that contains a result set.
    *
+   * <p>Regular result sets have {@code updateCount} -1;
+   * any other value means a dummy result set that is just a count, and has
+   * no signature and no other data.
+   *
    * <p>Several types of request, including
    * {@link org.apache.calcite.avatica.Meta#getCatalogs()} and
    * {@link org.apache.calcite.avatica.Meta#getSchemas(String, org.apache.calcite.avatica.Meta.Pat)}
@@ -193,6 +196,7 @@ public interface Service {
     public final boolean ownStatement;
     public final Meta.Signature signature;
     public final Meta.Frame firstFrame;
+    public final int updateCount;
 
     @JsonCreator
     public ResultSetResponse(
@@ -200,12 +204,14 @@ public interface Service {
         @JsonProperty("statementId") int statementId,
         @JsonProperty("ownStatement") boolean ownStatement,
         @JsonProperty("signature") Meta.Signature signature,
-        @JsonProperty("firstFrame") Meta.Frame firstFrame) {
+        @JsonProperty("firstFrame") Meta.Frame firstFrame,
+        @JsonProperty("updateCount") int updateCount) {
       this.connectionId = connectionId;
       this.statementId = statementId;
       this.ownStatement = ownStatement;
       this.signature = signature;
       this.firstFrame = firstFrame;
+      this.updateCount = updateCount;
     }
   }
 
@@ -226,11 +232,23 @@ public interface Service {
       this.maxRowCount = maxRowCount;
     }
 
-    @Override ResultSetResponse accept(Service service) {
+    @Override ExecuteResponse accept(Service service) {
       return service.apply(this);
     }
   }
 
+  /** Response to a
+   * {@link org.apache.calcite.avatica.remote.Service.PrepareAndExecuteRequest}. */
+  class ExecuteResponse extends Response {
+    public final List<ResultSetResponse> results;
+
+    @JsonCreator
+    public ExecuteResponse(
+        @JsonProperty("resultSets") List<ResultSetResponse> results) {
+      this.results = results;
+    }
+  }
+
   /** Request for
    * {@link org.apache.calcite.avatica.Meta#prepare(org.apache.calcite.avatica.Meta.ConnectionHandle, String, int)}. */
   class PrepareRequest extends Request {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/0d80fd25/core/src/main/java/org/apache/calcite/jdbc/CalciteMetaImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/CalciteMetaImpl.java b/core/src/main/java/org/apache/calcite/jdbc/CalciteMetaImpl.java
index 2a9c4ad..90c711e 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/CalciteMetaImpl.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/CalciteMetaImpl.java
@@ -150,9 +150,9 @@ public class CalciteMetaImpl extends MetaImpl {
 
   private <E> MetaResultSet createResultSet(Enumerable<E> enumerable,
       Class clazz, String... names) {
-    final List<ColumnMetaData> columns = new ArrayList<ColumnMetaData>();
-    final List<Field> fields = new ArrayList<Field>();
-    final List<String> fieldNames = new ArrayList<String>();
+    final List<ColumnMetaData> columns = new ArrayList<>();
+    final List<Field> fields = new ArrayList<>();
+    final List<String> fieldNames = new ArrayList<>();
     for (String name : names) {
       final int index = fields.size();
       final String fieldName = AvaticaUtils.toCamelCase(name);
@@ -196,7 +196,7 @@ public class CalciteMetaImpl extends MetaImpl {
               return Linq4j.asEnumerable(firstFrame.rows);
             }
           };
-      return new MetaResultSet(connection.id, statement.getId(), true,
+      return MetaResultSet.create(connection.id, statement.getId(), true,
           signature, firstFrame);
     } catch (SQLException e) {
       throw new RuntimeException(e);
@@ -476,7 +476,7 @@ public class CalciteMetaImpl extends MetaImpl {
     return h;
   }
 
-  @Override public MetaResultSet prepareAndExecute(ConnectionHandle ch,
+  @Override public ExecuteResult prepareAndExecute(ConnectionHandle ch,
       String sql, int maxRowCount, PrepareCallback callback) {
     final CalcitePrepare.CalciteSignature<Object> signature;
     final StatementHandle h = createStatement(ch);
@@ -488,10 +488,12 @@ public class CalciteMetaImpl extends MetaImpl {
             calciteConnection.server.getStatement(h);
         signature = calciteConnection.parseQuery(sql,
             statement.createPrepareContext(), maxRowCount);
-        callback.assign(signature, null);
+        callback.assign(signature, null, -1);
       }
       callback.execute();
-      return new MetaResultSet(h.connectionId, h.id, false, signature, null);
+      final MetaResultSet metaResultSet =
+          MetaResultSet.create(h.connectionId, h.id, false, signature, null);
+      return new ExecuteResult(ImmutableList.of(metaResultSet));
     } catch (SQLException e) {
       throw new RuntimeException(e);
     }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/0d80fd25/core/src/test/java/org/apache/calcite/jdbc/CalciteRemoteDriverTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/jdbc/CalciteRemoteDriverTest.java b/core/src/test/java/org/apache/calcite/jdbc/CalciteRemoteDriverTest.java
index 05f54b7..9521201 100644
--- a/core/src/test/java/org/apache/calcite/jdbc/CalciteRemoteDriverTest.java
+++ b/core/src/test/java/org/apache/calcite/jdbc/CalciteRemoteDriverTest.java
@@ -381,6 +381,23 @@ public class CalciteRemoteDriverTest {
     }
   }
 
+  /** Checks {@link Statement#execute} on a query over a remote connection.
+   *
+   * <p>Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-646">[CALCITE-646]
+   * AvaticaStatement execute method broken over remote JDBC</a>. */
+  @Test public void testRemoteStatementExecute() throws Exception {
+    final Statement statement = remoteConnection.createStatement();
+    final boolean status = statement.execute("values (1, 2), (3, 4), (5, 6)");
+    final ResultSet resultSet = statement.getResultSet();
+    int n = 0;
+    while (resultSet.next()) {
+      ++n;
+    }
+    assertThat(n, equalTo(3));
+
+  }
+
   /** A bunch of sample values of various types. */
   private static final List<Object> SAMPLE_VALUES =
       ImmutableList.<Object>of(false, true,


[03/10] incubator-calcite git commit: [CALCITE-652] Move server pieces of avatica into avatica-server (Nick Dimiduk)

Posted by jh...@apache.org.
[CALCITE-652] Move server pieces of avatica into avatica-server (Nick Dimiduk)

Close apache/incubator-calcite#69


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

Branch: refs/heads/master
Commit: fa9bdc4ac8b7d0106ebf5fa875cd9edc00d47865
Parents: 17d7a8e
Author: Nick Dimiduk <nd...@gmail.com>
Authored: Tue Mar 31 16:04:43 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Apr 3 01:09:29 2015 -0700

----------------------------------------------------------------------
 avatica-server/pom.xml                          |  47 +-
 .../apache/calcite/avatica/jdbc/JdbcMeta.java   | 765 +++++++++++++++++++
 .../calcite/avatica/jdbc/JdbcResultSet.java     | 106 +++
 .../calcite/avatica/jdbc/package-info.java      |  22 +
 .../calcite/avatica/RemoteDriverTest.java       | 431 +++++++++++
 .../calcite/avatica/test/AvaticaSuite.java      |  36 +
 avatica/pom.xml                                 |  35 +-
 .../apache/calcite/avatica/jdbc/JdbcMeta.java   | 765 -------------------
 .../calcite/avatica/jdbc/JdbcResultSet.java     | 106 ---
 .../calcite/avatica/jdbc/package-info.java      |  22 -
 .../calcite/avatica/test/AvaticaSuite.java      |  33 -
 .../calcite/avatica/test/RemoteDriverTest.java  | 435 -----------
 pom.xml                                         |  41 +-
 13 files changed, 1461 insertions(+), 1383 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/fa9bdc4a/avatica-server/pom.xml
----------------------------------------------------------------------
diff --git a/avatica-server/pom.xml b/avatica-server/pom.xml
index 4b35978..2da32ec 100644
--- a/avatica-server/pom.xml
+++ b/avatica-server/pom.xml
@@ -40,11 +40,56 @@ limitations under the License.
       <groupId>org.apache.calcite</groupId>
       <artifactId>calcite-avatica</artifactId>
     </dependency>
-
+    <dependency>
+      <groupId>javax.servlet</groupId>
+      <artifactId>javax.servlet-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>commons-logging</groupId>
+      <artifactId>commons-logging</artifactId>
+    </dependency>
     <dependency>
       <groupId>org.eclipse.jetty</groupId>
       <artifactId>jetty-server</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-util</artifactId>
+    </dependency>
+
+    <!-- test dependencies -->
+    <dependency>
+      <groupId>org.apache.calcite</groupId>
+      <artifactId>calcite-avatica</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <!-- Used in RemoteDriverTest but dependency not detected by maven-dependency-plugin:2.8:analyze -->
+      <groupId>net.hydromatic</groupId>
+      <artifactId>scott-data-hsqldb</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-core</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <!-- Used in RemoteDriverTest but dependency not detected by maven-dependency-plugin:2.8:analyze -->
+      <groupId>org.hsqldb</groupId>
+      <artifactId>hsqldb</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/fa9bdc4a/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
----------------------------------------------------------------------
diff --git a/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java b/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
new file mode 100644
index 0000000..90e8c2a
--- /dev/null
+++ b/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
@@ -0,0 +1,765 @@
+/*
+ * 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.avatica.jdbc;
+
+import org.apache.calcite.avatica.AvaticaParameter;
+import org.apache.calcite.avatica.ColumnMetaData;
+import org.apache.calcite.avatica.Meta;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.RemovalListener;
+import com.google.common.cache.RemovalNotification;
+
+import java.lang.reflect.Array;
+import java.lang.reflect.Type;
+import java.math.BigDecimal;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ParameterMetaData;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+/** Implementation of {@link Meta} upon an existing JDBC data source. */
+public class JdbcMeta implements Meta {
+
+  private static final Log LOG = LogFactory.getLog(JdbcMeta.class);
+
+  /**
+   * JDBC Types Mapped to Java Types
+   *
+   * @see <a href="https://docs.oracle.com/javase/1.5.0/docs/guide/jdbc/getstart/mapping.html#1051555">JDBC Types Mapped to Java Types</a>
+   */
+  protected static final Map<Integer, Type> SQL_TYPE_TO_JAVA_TYPE =
+      new HashMap<>();
+  static {
+    SQL_TYPE_TO_JAVA_TYPE.put(Types.CHAR, String.class);
+    SQL_TYPE_TO_JAVA_TYPE.put(Types.VARCHAR, String.class);
+    SQL_TYPE_TO_JAVA_TYPE.put(Types.LONGNVARCHAR, String.class);
+    SQL_TYPE_TO_JAVA_TYPE.put(Types.NUMERIC, BigDecimal.class);
+    SQL_TYPE_TO_JAVA_TYPE.put(Types.DECIMAL, BigDecimal.class);
+    SQL_TYPE_TO_JAVA_TYPE.put(Types.BIT, Boolean.TYPE);
+    SQL_TYPE_TO_JAVA_TYPE.put(Types.TINYINT, Byte.TYPE);
+    SQL_TYPE_TO_JAVA_TYPE.put(Types.SMALLINT, Short.TYPE);
+    SQL_TYPE_TO_JAVA_TYPE.put(Types.INTEGER, Integer.TYPE);
+    SQL_TYPE_TO_JAVA_TYPE.put(Types.BIGINT, Long.TYPE);
+    SQL_TYPE_TO_JAVA_TYPE.put(Types.REAL, Float.TYPE);
+    SQL_TYPE_TO_JAVA_TYPE.put(Types.FLOAT, Double.TYPE);
+    SQL_TYPE_TO_JAVA_TYPE.put(Types.DOUBLE, Double.TYPE);
+    SQL_TYPE_TO_JAVA_TYPE.put(Types.BINARY, byte[].class);
+    SQL_TYPE_TO_JAVA_TYPE.put(Types.VARBINARY, byte[].class);
+    SQL_TYPE_TO_JAVA_TYPE.put(Types.LONGVARBINARY, byte[].class);
+    SQL_TYPE_TO_JAVA_TYPE.put(Types.DATE, java.sql.Date.class);
+    SQL_TYPE_TO_JAVA_TYPE.put(Types.TIME, java.sql.Time.class);
+    SQL_TYPE_TO_JAVA_TYPE.put(Types.TIMESTAMP, java.sql.Timestamp.class);
+    //put(Types.CLOB, Clob);
+    //put(Types.BLOB, Blob);
+    SQL_TYPE_TO_JAVA_TYPE.put(Types.ARRAY, Array.class);
+  }
+
+  /** A "magic column" descriptor used to return statement execute update count via a
+   * {@link Meta.MetaResultSet}. */
+  private static final ColumnMetaData UPDATE_COL = new ColumnMetaData(1, false, false, false,
+      false, ResultSetMetaData.columnNoNulls, true, 10, "u", "u", null, 15, 15, "update_result",
+      "avatica_internal", ColumnMetaData.scalar(Types.INTEGER, "INTEGER",
+        ColumnMetaData.Rep.INTEGER), true, false, false, "java.lang.Integer");
+
+  private static final String CONN_CACHE_KEY_BASE = "avatica.connectioncache";
+
+  /** Configurable connection cache settings. */
+  public enum ConnectionCacheSettings {
+    /** JDBC connection property for setting connection cache concurrency level. */
+    CONCURRENCY_LEVEL(CONN_CACHE_KEY_BASE + ".concurrency", "10"),
+
+    /** JDBC connection property for setting connection cache initial capacity. */
+    INITIAL_CAPACITY(CONN_CACHE_KEY_BASE + ".initialcapacity", "100"),
+
+    /** JDBC connection property for setting connection cache maximum capacity. */
+    MAX_CAPACITY(CONN_CACHE_KEY_BASE + ".maxcapacity", "1000"),
+
+    /** JDBC connection property for setting connection cache expiration duration. */
+    EXPIRY_DURATION(CONN_CACHE_KEY_BASE + ".expirydiration", "10"),
+
+    /** JDBC connection property for setting connection cache expiration unit. */
+    EXPIRY_UNIT(CONN_CACHE_KEY_BASE + ".expiryunit", TimeUnit.MINUTES.name());
+
+    private final String key;
+    private final String defaultValue;
+
+    private ConnectionCacheSettings(String key, String defaultValue) {
+      this.key = key;
+      this.defaultValue = defaultValue;
+    }
+
+    /** The configuration key for specifying this setting. */
+    public String key() {
+      return key;
+    }
+
+    /** The default value for this setting. */
+    public String defaultValue() {
+      return defaultValue;
+    }
+  }
+
+  private static final String STMT_CACHE_KEY_BASE = "avatica.statementcache";
+
+  /** Configurable statement cache settings. */
+  public enum StatementCacheSettings {
+    /** JDBC connection property for setting connection cache concurrency level. */
+    CONCURRENCY_LEVEL(STMT_CACHE_KEY_BASE + ".concurrency", "100"),
+
+    /** JDBC connection property for setting connection cache initial capacity. */
+    INITIAL_CAPACITY(STMT_CACHE_KEY_BASE + ".initialcapacity", "1000"),
+
+    /** JDBC connection property for setting connection cache maximum capacity. */
+    MAX_CAPACITY(STMT_CACHE_KEY_BASE + ".maxcapacity", "10000"),
+
+    /** JDBC connection property for setting connection cache expiration duration.
+     *
+     * <p>Used in conjunction with {@link #EXPIRY_UNIT}.</p>
+     */
+    EXPIRY_DURATION(STMT_CACHE_KEY_BASE + ".expirydiration", "5"),
+
+    /** JDBC connection property for setting connection cache expiration unit.
+     *
+     * <p>Used in conjunction with {@link #EXPIRY_DURATION}.</p>
+     */
+    EXPIRY_UNIT(STMT_CACHE_KEY_BASE + ".expiryunit", TimeUnit.MINUTES.name());
+
+    private final String key;
+    private final String defaultValue;
+
+    private StatementCacheSettings(String key, String defaultValue) {
+      this.key = key;
+      this.defaultValue = defaultValue;
+    }
+
+    /** The configuration key for specifying this setting. */
+    public String key() {
+      return key;
+    }
+
+    /** The default value for this setting. */
+    public String defaultValue() {
+      return defaultValue;
+    }
+  }
+
+  private static final String DEFAULT_CONN_ID =
+      UUID.fromString("00000000-0000-0000-0000-000000000000").toString();
+
+  private final String url;
+  private final Properties info;
+  private final Connection connection; // TODO: remove default connection
+  private final Cache<String, Connection> connectionCache;
+  private final Cache<Integer, StatementInfo> statementCache;
+
+  /**
+   * Convert from JDBC metadata to Avatica columns.
+   */
+  protected static List<ColumnMetaData>
+  columns(ResultSetMetaData metaData) throws SQLException {
+    final List<ColumnMetaData> columns = new ArrayList<>();
+    for (int i = 1; i <= metaData.getColumnCount(); i++) {
+      final Type javaType =
+          SQL_TYPE_TO_JAVA_TYPE.get(metaData.getColumnType(i));
+      ColumnMetaData.AvaticaType t =
+          ColumnMetaData.scalar(metaData.getColumnType(i),
+              metaData.getColumnTypeName(i), ColumnMetaData.Rep.of(javaType));
+      ColumnMetaData md =
+          new ColumnMetaData(i - 1, metaData.isAutoIncrement(i),
+              metaData.isCaseSensitive(i), metaData.isSearchable(i),
+              metaData.isCurrency(i), metaData.isNullable(i),
+              metaData.isSigned(i), metaData.getColumnDisplaySize(i),
+              metaData.getColumnLabel(i), metaData.getColumnName(i),
+              metaData.getSchemaName(i), metaData.getPrecision(i),
+              metaData.getScale(i), metaData.getTableName(i),
+              metaData.getCatalogName(i), t, metaData.isReadOnly(i),
+              metaData.isWritable(i), metaData.isDefinitelyWritable(i),
+              metaData.getColumnClassName(i));
+      columns.add(md);
+    }
+    return columns;
+  }
+
+  /**
+   * Converts from JDBC metadata to AvaticaParameters
+   */
+  protected static List<AvaticaParameter> parameters(ParameterMetaData metaData)
+      throws SQLException {
+    if (metaData == null) {
+      return Collections.emptyList();
+    }
+    final List<AvaticaParameter> params = new ArrayList<>();
+    for (int i = 1; i <= metaData.getParameterCount(); i++) {
+      params.add(
+          new AvaticaParameter(metaData.isSigned(i), metaData.getPrecision(i),
+              metaData.getScale(i), metaData.getParameterType(i),
+              metaData.getParameterTypeName(i),
+              metaData.getParameterClassName(i), "?" + i));
+    }
+    return params;
+  }
+
+  protected static Signature signature(ResultSetMetaData metaData,
+      ParameterMetaData parameterMetaData, String sql) throws  SQLException {
+    return new Signature(columns(metaData), sql, parameters(parameterMetaData),
+        null, CursorFactory.LIST /* LIST because JdbcResultSet#frame */);
+  }
+
+  protected static Signature signature(ResultSetMetaData metaData)
+      throws SQLException {
+    return signature(metaData, null, null);
+  }
+
+  /** Callback for {@link #connectionCache} member expiration. */
+  private class ConnectionExpiryHandler
+      implements RemovalListener<String, Connection> {
+
+    public void onRemoval(RemovalNotification<String, Connection> notification) {
+      String connectionId = notification.getKey();
+      Connection doomed = notification.getValue();
+      // is String.equals() more efficient?
+      if (notification.getValue() == connection) {
+        return;
+      }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Expiring connection " + connectionId + " because "
+            + notification.getCause());
+      }
+      try {
+        if (doomed != null) {
+          doomed.close();
+        }
+      } catch (Throwable t) {
+        LOG.info("Exception thrown while expiring connection " + connectionId, t);
+      }
+    }
+  }
+
+  /** Callback for {@link #statementCache} member expiration. */
+  private class StatementExpiryHandler
+      implements RemovalListener<Integer, StatementInfo> {
+    public void onRemoval(RemovalNotification<Integer, StatementInfo> notification) {
+      Integer stmtId = notification.getKey();
+      StatementInfo doomed = notification.getValue();
+      if (doomed == null) {
+        // log/throw?
+        return;
+      }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Expiring statement " + stmtId + " because "
+            + notification.getCause());
+      }
+      try {
+        if (doomed.resultSet != null) {
+          doomed.resultSet.close();
+        }
+        if (doomed.statement != null) {
+          doomed.statement.close();
+        }
+      } catch (Throwable t) {
+        LOG.info("Exception thrown while expiring statement " + stmtId);
+      }
+    }
+  }
+
+  /**
+   * @param url a database url of the form
+   *  <code> jdbc:<em>subprotocol</em>:<em>subname</em></code>
+   */
+  public JdbcMeta(String url) throws SQLException {
+    this(url, new Properties());
+  }
+
+  /**
+   * @param url a database url of the form
+   * <code>jdbc:<em>subprotocol</em>:<em>subname</em></code>
+   * @param user the database user on whose behalf the connection is being
+   *   made
+   * @param password the user's password
+   */
+  public JdbcMeta(final String url, final String user, final String password)
+      throws SQLException {
+    this(url, new Properties() {
+      {
+        put("user", user);
+        put("password", password);
+      }
+    });
+  }
+
+  /**
+   * @param url a database url of the form
+   * <code> jdbc:<em>subprotocol</em>:<em>subname</em></code>
+   * @param info a list of arbitrary string tag/value pairs as
+   * connection arguments; normally at least a "user" and
+   * "password" property should be included
+   */
+  public JdbcMeta(String url, Properties info) throws SQLException {
+    this.url = url;
+    this.info = info;
+    this.connection = DriverManager.getConnection(url, info);
+
+    int concurrencyLevel = Integer.parseInt(
+        info.getProperty(ConnectionCacheSettings.CONCURRENCY_LEVEL.key(),
+            ConnectionCacheSettings.CONCURRENCY_LEVEL.defaultValue()));
+    int initialCapacity = Integer.parseInt(
+        info.getProperty(ConnectionCacheSettings.INITIAL_CAPACITY.key(),
+            ConnectionCacheSettings.INITIAL_CAPACITY.defaultValue()));
+    long maxCapacity = Long.parseLong(
+        info.getProperty(ConnectionCacheSettings.MAX_CAPACITY.key(),
+            ConnectionCacheSettings.MAX_CAPACITY.defaultValue()));
+    long connectionExpiryDuration = Long.parseLong(
+        info.getProperty(ConnectionCacheSettings.EXPIRY_DURATION.key(),
+            ConnectionCacheSettings.EXPIRY_DURATION.defaultValue()));
+    TimeUnit connectionExpiryUnit = TimeUnit.valueOf(
+        info.getProperty(ConnectionCacheSettings.EXPIRY_UNIT.key(),
+            ConnectionCacheSettings.EXPIRY_UNIT.defaultValue()));
+    this.connectionCache = CacheBuilder.newBuilder()
+        .concurrencyLevel(concurrencyLevel)
+        .initialCapacity(initialCapacity)
+        .maximumSize(maxCapacity)
+        .expireAfterAccess(connectionExpiryDuration, connectionExpiryUnit)
+        .removalListener(new ConnectionExpiryHandler())
+        .build();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("instantiated connection cache: " + connectionCache.stats());
+    }
+
+    concurrencyLevel = Integer.parseInt(
+        info.getProperty(StatementCacheSettings.CONCURRENCY_LEVEL.key(),
+            StatementCacheSettings.CONCURRENCY_LEVEL.defaultValue()));
+    initialCapacity = Integer.parseInt(
+        info.getProperty(StatementCacheSettings.INITIAL_CAPACITY.key(),
+            StatementCacheSettings.INITIAL_CAPACITY.defaultValue()));
+    maxCapacity = Long.parseLong(
+        info.getProperty(StatementCacheSettings.MAX_CAPACITY.key(),
+            StatementCacheSettings.MAX_CAPACITY.defaultValue()));
+    connectionExpiryDuration = Long.parseLong(
+        info.getProperty(StatementCacheSettings.EXPIRY_DURATION.key(),
+            StatementCacheSettings.EXPIRY_DURATION.defaultValue()));
+    connectionExpiryUnit = TimeUnit.valueOf(
+        info.getProperty(StatementCacheSettings.EXPIRY_UNIT.key(),
+            StatementCacheSettings.EXPIRY_UNIT.defaultValue()));
+    this.statementCache = CacheBuilder.newBuilder()
+        .concurrencyLevel(concurrencyLevel)
+        .initialCapacity(initialCapacity)
+        .maximumSize(maxCapacity)
+        .expireAfterAccess(connectionExpiryDuration, connectionExpiryUnit)
+        .removalListener(new StatementExpiryHandler())
+        .build();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("instantiated statement cache: " + statementCache.stats());
+    }
+  }
+
+  public String getSqlKeywords() {
+    try {
+      return connection.getMetaData().getSQLKeywords();
+    } catch (SQLException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public String getNumericFunctions() {
+    try {
+      return connection.getMetaData().getNumericFunctions();
+    } catch (SQLException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public String getStringFunctions() {
+    return null;
+  }
+
+  public String getSystemFunctions() {
+    return null;
+  }
+
+  public String getTimeDateFunctions() {
+    return null;
+  }
+
+  public MetaResultSet getTables(String catalog, Pat schemaPattern,
+      Pat tableNamePattern, List<String> typeList) {
+    try {
+      String[] types = new String[typeList == null ? 0 : typeList.size()];
+      return JdbcResultSet.create(DEFAULT_CONN_ID, -1,
+          connection.getMetaData().getTables(catalog, schemaPattern.s,
+              tableNamePattern.s,
+              typeList == null ? types : typeList.toArray(types)));
+    } catch (SQLException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public MetaResultSet getColumns(String catalog, Pat schemaPattern,
+      Pat tableNamePattern, Pat columnNamePattern) {
+    try {
+      return JdbcResultSet.create(DEFAULT_CONN_ID, -1,
+          connection.getMetaData().getColumns(catalog, schemaPattern.s,
+              tableNamePattern.s, columnNamePattern.s));
+    } catch (SQLException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public MetaResultSet getSchemas(String catalog, Pat schemaPattern) {
+    try {
+      return JdbcResultSet.create(DEFAULT_CONN_ID, -1,
+          connection.getMetaData().getSchemas(catalog, schemaPattern.s));
+    } catch (SQLException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public MetaResultSet getCatalogs() {
+    try {
+      return JdbcResultSet.create(DEFAULT_CONN_ID, -1,
+          connection.getMetaData().getCatalogs());
+    } catch (SQLException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public MetaResultSet getTableTypes() {
+    try {
+      return JdbcResultSet.create(DEFAULT_CONN_ID, -1,
+          connection.getMetaData().getTableTypes());
+    } catch (SQLException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public MetaResultSet getProcedures(String catalog, Pat schemaPattern,
+      Pat procedureNamePattern) {
+    try {
+      return JdbcResultSet.create(DEFAULT_CONN_ID, -1,
+          connection.getMetaData().getProcedures(catalog, schemaPattern.s,
+              procedureNamePattern.s));
+    } catch (SQLException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public MetaResultSet getProcedureColumns(String catalog, Pat schemaPattern,
+      Pat procedureNamePattern, Pat columnNamePattern) {
+    try {
+      return JdbcResultSet.create(DEFAULT_CONN_ID, -1,
+          connection.getMetaData().getProcedureColumns(catalog,
+              schemaPattern.s, procedureNamePattern.s, columnNamePattern.s));
+    } catch (SQLException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public MetaResultSet getColumnPrivileges(String catalog, String schema,
+      String table, Pat columnNamePattern) {
+    try {
+      return JdbcResultSet.create(DEFAULT_CONN_ID, -1,
+          connection.getMetaData().getColumnPrivileges(catalog, schema,
+              table, columnNamePattern.s));
+    } catch (SQLException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public MetaResultSet getTablePrivileges(String catalog, Pat schemaPattern,
+      Pat tableNamePattern) {
+    try {
+      return JdbcResultSet.create(DEFAULT_CONN_ID, -1,
+          connection.getMetaData().getTablePrivileges(catalog,
+              schemaPattern.s, tableNamePattern.s));
+    } catch (SQLException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public MetaResultSet getBestRowIdentifier(String catalog, String schema,
+      String table, int scope, boolean nullable) {
+    try {
+      return JdbcResultSet.create(DEFAULT_CONN_ID, -1,
+          connection.getMetaData().getBestRowIdentifier(catalog, schema,
+              table, scope, nullable));
+    } catch (SQLException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public MetaResultSet getVersionColumns(String catalog, String schema,
+      String table) {
+    try {
+      return JdbcResultSet.create(DEFAULT_CONN_ID, -1,
+          connection.getMetaData().getVersionColumns(catalog, schema, table));
+    } catch (SQLException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public MetaResultSet getPrimaryKeys(String catalog, String schema,
+      String table) {
+    try {
+      return JdbcResultSet.create(DEFAULT_CONN_ID, -1,
+          connection.getMetaData().getPrimaryKeys(catalog, schema, table));
+    } catch (SQLException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public MetaResultSet getImportedKeys(String catalog, String schema,
+      String table) {
+    return null;
+  }
+
+  public MetaResultSet getExportedKeys(String catalog, String schema,
+      String table) {
+    return null;
+  }
+
+  public MetaResultSet getCrossReference(String parentCatalog,
+      String parentSchema, String parentTable, String foreignCatalog,
+      String foreignSchema, String foreignTable) {
+    return null;
+  }
+
+  public MetaResultSet getTypeInfo() {
+    return null;
+  }
+
+  public MetaResultSet getIndexInfo(String catalog, String schema, String table,
+      boolean unique, boolean approximate) {
+    return null;
+  }
+
+  public MetaResultSet getUDTs(String catalog, Pat schemaPattern,
+      Pat typeNamePattern, int[] types) {
+    return null;
+  }
+
+  public MetaResultSet getSuperTypes(String catalog, Pat schemaPattern,
+      Pat typeNamePattern) {
+    return null;
+  }
+
+  public MetaResultSet getSuperTables(String catalog, Pat schemaPattern,
+      Pat tableNamePattern) {
+    return null;
+  }
+
+  public MetaResultSet getAttributes(String catalog, Pat schemaPattern,
+      Pat typeNamePattern, Pat attributeNamePattern) {
+    return null;
+  }
+
+  public MetaResultSet getClientInfoProperties() {
+    return null;
+  }
+
+  public MetaResultSet getFunctions(String catalog, Pat schemaPattern,
+      Pat functionNamePattern) {
+    return null;
+  }
+
+  public MetaResultSet getFunctionColumns(String catalog, Pat schemaPattern,
+      Pat functionNamePattern, Pat columnNamePattern) {
+    return null;
+  }
+
+  public MetaResultSet getPseudoColumns(String catalog, Pat schemaPattern,
+      Pat tableNamePattern, Pat columnNamePattern) {
+    return null;
+  }
+
+  public Iterable<Object> createIterable(StatementHandle handle,
+      Signature signature, List<Object> parameterValues, Frame firstFrame) {
+    return null;
+  }
+
+  protected Connection getConnection(String id) throws SQLException {
+    Connection conn = connectionCache.getIfPresent(id);
+    if (conn == null) {
+      conn = DriverManager.getConnection(url, info);
+      connectionCache.put(id, conn);
+    }
+    return conn;
+  }
+
+  public StatementHandle createStatement(ConnectionHandle ch) {
+    try {
+      final Connection conn = getConnection(ch.id);
+      final Statement statement = conn.createStatement();
+      final int id = System.identityHashCode(statement);
+      statementCache.put(id, new StatementInfo(statement));
+      StatementHandle h = new StatementHandle(ch.id, id, null);
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("created statement " + h);
+      }
+      return h;
+    } catch (SQLException e) {
+      throw propagate(e);
+    }
+  }
+
+  @Override public void closeStatement(StatementHandle h) {
+    StatementInfo info = statementCache.getIfPresent(h.id);
+    if (info == null || info.statement == null) {
+      LOG.debug("client requested close unknown statement " + h);
+      return;
+    }
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("closing statement " + h);
+    }
+    try {
+      if (info.resultSet != null) {
+        info.resultSet.close();
+      }
+      info.statement.close();
+    } catch (SQLException e) {
+      throw propagate(e);
+    } finally {
+      statementCache.invalidate(h.id);
+    }
+  }
+
+  @Override public void closeConnection(ConnectionHandle ch) {
+    Connection conn = connectionCache.getIfPresent(ch.id);
+    if (conn == null) {
+      LOG.debug("client requested close unknown connection " + ch);
+      return;
+    }
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("closing connection " + ch);
+    }
+    try {
+      conn.close();
+    } catch (SQLException e) {
+      throw propagate(e);
+    } finally {
+      connectionCache.invalidate(ch.id);
+    }
+  }
+
+  private RuntimeException propagate(Throwable e) {
+    if (e instanceof RuntimeException) {
+      throw (RuntimeException) e;
+    } else if (e instanceof Error) {
+      throw (Error) e;
+    } else {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public StatementHandle prepare(ConnectionHandle ch, String sql,
+      int maxRowCount) {
+    try {
+      final Connection conn = getConnection(ch.id);
+      final PreparedStatement statement = conn.prepareStatement(sql);
+      final int id = System.identityHashCode(statement);
+      statementCache.put(id, new StatementInfo(statement));
+      StatementHandle h = new StatementHandle(ch.id, id,
+          signature(statement.getMetaData(), statement.getParameterMetaData(),
+              sql));
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("prepared statement " + h);
+      }
+      return h;
+    } catch (SQLException e) {
+      throw propagate(e);
+    }
+  }
+
+  public MetaResultSet prepareAndExecute(ConnectionHandle ch, String sql,
+      int maxRowCount, PrepareCallback callback) {
+    try {
+      final Connection connection = getConnection(ch.id);
+      final PreparedStatement statement = connection.prepareStatement(sql);
+      final int id = System.identityHashCode(statement);
+      final StatementInfo info = new StatementInfo(statement);
+      statementCache.put(id, info);
+      info.resultSet = statement.executeQuery();
+      MetaResultSet mrs = JdbcResultSet.create(ch.id, id, info.resultSet);
+      if (LOG.isTraceEnabled()) {
+        StatementHandle h = new StatementHandle(ch.id, id, null);
+        LOG.trace("prepAndExec statement " + h);
+      }
+      return mrs;
+    } catch (SQLException e) {
+      throw propagate(e);
+    }
+  }
+
+  public Frame fetch(StatementHandle h, List<Object> parameterValues,
+      int offset, int fetchMaxRowCount) {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("fetching " + h + " offset:" + offset + " fetchMaxRowCount:" + fetchMaxRowCount);
+    }
+    try {
+      final StatementInfo statementInfo = Objects.requireNonNull(
+          statementCache.getIfPresent(h.id),
+          "Statement not found, potentially expired. " + h);
+      if (statementInfo.resultSet == null || parameterValues != null) {
+        if (statementInfo.resultSet != null) {
+          statementInfo.resultSet.close();
+        }
+        final PreparedStatement preparedStatement =
+            (PreparedStatement) statementInfo.statement;
+        if (parameterValues != null) {
+          for (int i = 0; i < parameterValues.size(); i++) {
+            Object o = parameterValues.get(i);
+            preparedStatement.setObject(i + 1, o);
+          }
+        }
+        statementInfo.resultSet = preparedStatement.executeQuery();
+      }
+      return JdbcResultSet.frame(statementInfo.resultSet, offset,
+          fetchMaxRowCount);
+    } catch (SQLException e) {
+      throw propagate(e);
+    }
+  }
+
+  /** All we know about a statement. */
+  private static class StatementInfo {
+    final Statement statement; // sometimes a PreparedStatement
+    ResultSet resultSet;
+
+    private StatementInfo(Statement statement) {
+      this.statement = Objects.requireNonNull(statement);
+    }
+  }
+}
+
+// End JdbcMeta.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/fa9bdc4a/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcResultSet.java
----------------------------------------------------------------------
diff --git a/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcResultSet.java b/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcResultSet.java
new file mode 100644
index 0000000..827f31d
--- /dev/null
+++ b/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcResultSet.java
@@ -0,0 +1,106 @@
+/*
+ * 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.avatica.jdbc;
+
+import org.apache.calcite.avatica.Meta;
+
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.List;
+
+/** Implementation of {@link org.apache.calcite.avatica.Meta.MetaResultSet}
+ *  upon a JDBC {@link java.sql.ResultSet}.
+ *
+ *  @see org.apache.calcite.avatica.jdbc.JdbcMeta */
+class JdbcResultSet extends Meta.MetaResultSet {
+  protected JdbcResultSet(String connectionId, int statementId,
+      boolean ownStatement, Meta.Signature signature, Meta.Frame firstFrame) {
+    super(connectionId, statementId, ownStatement, signature, firstFrame);
+  }
+
+  /** Creates a result set. */
+  public static JdbcResultSet create(String connectionId, int statementId,
+      ResultSet resultSet) {
+    try {
+      Meta.Signature sig = JdbcMeta.signature(resultSet.getMetaData());
+      final Meta.Frame firstFrame = frame(resultSet, 0, -1);
+      resultSet.close();
+      return new JdbcResultSet(connectionId, statementId, true, sig,
+          firstFrame);
+    } catch (SQLException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /** Creates a frame containing a given number or unlimited number of rows
+   * from a result set. */
+  static Meta.Frame frame(ResultSet resultSet, int offset,
+      int fetchMaxRowCount) throws SQLException {
+    final ResultSetMetaData metaData = resultSet.getMetaData();
+    final int columnCount = metaData.getColumnCount();
+    final int[] types = new int[columnCount];
+    for (int i = 0; i < types.length; i++) {
+      types[i] = metaData.getColumnType(i + 1);
+    }
+    final List<Object> rows = new ArrayList<>();
+    boolean done = false;
+    for (int i = 0; fetchMaxRowCount < 0 || i < fetchMaxRowCount; i++) {
+      if (!resultSet.next()) {
+        done = true;
+        break;
+      }
+      Object[] columns = new Object[columnCount];
+      for (int j = 0; j < columnCount; j++) {
+        columns[j] = getValue(resultSet, types[j], j);
+      }
+      rows.add(columns);
+    }
+    return new Meta.Frame(offset, done, rows);
+  }
+
+  private static Object getValue(ResultSet resultSet, int type, int j)
+      throws SQLException {
+    switch (type) {
+    case Types.BIGINT:
+      final long aLong = resultSet.getLong(j + 1);
+      return aLong == 0 && resultSet.wasNull() ? null : aLong;
+    case Types.INTEGER:
+      final int anInt = resultSet.getInt(j + 1);
+      return anInt == 0 && resultSet.wasNull() ? null : anInt;
+    case Types.SMALLINT:
+      final short aShort = resultSet.getShort(j + 1);
+      return aShort == 0 && resultSet.wasNull() ? null : aShort;
+    case Types.TINYINT:
+      final byte aByte = resultSet.getByte(j + 1);
+      return aByte == 0 && resultSet.wasNull() ? null : aByte;
+    case Types.DOUBLE:
+    case Types.FLOAT:
+      final double aDouble = resultSet.getDouble(j + 1);
+      return aDouble == 0D && resultSet.wasNull() ? null : aDouble;
+    case Types.REAL:
+      final float aFloat = resultSet.getFloat(j + 1);
+      return aFloat == 0D && resultSet.wasNull() ? null : aFloat;
+    default:
+      return resultSet.getObject(j + 1);
+    }
+  }
+}
+
+// End JdbcResultSet.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/fa9bdc4a/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/package-info.java
----------------------------------------------------------------------
diff --git a/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/package-info.java b/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/package-info.java
new file mode 100644
index 0000000..8b8fb76
--- /dev/null
+++ b/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/** Implements an Avatica provider on top of an existing JDBC data source. */
+package org.apache.calcite.avatica.jdbc;
+
+
+// End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/fa9bdc4a/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
----------------------------------------------------------------------
diff --git a/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java b/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
new file mode 100644
index 0000000..a2ab87a
--- /dev/null
+++ b/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
@@ -0,0 +1,431 @@
+/*
+ * 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.avatica;
+
+import org.apache.calcite.avatica.jdbc.JdbcMeta;
+import org.apache.calcite.avatica.remote.LocalJsonService;
+import org.apache.calcite.avatica.remote.LocalService;
+import org.apache.calcite.avatica.remote.MockJsonService;
+import org.apache.calcite.avatica.remote.Service;
+
+import com.google.common.cache.Cache;
+
+import net.hydromatic.scott.data.hsqldb.ScottHsqldb;
+
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.lang.reflect.Field;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ParameterMetaData;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Map;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Unit test for Avatica Remote JDBC driver.
+ */
+public class RemoteDriverTest {
+  public static final String MJS =
+      MockJsonService.Factory.class.getName();
+
+  public static final String LJS =
+      LocalJdbcServiceFactory.class.getName();
+
+  public static final String QRJS =
+      QuasiRemoteJdbcServiceFactory.class.getName();
+
+  private static final ConnectionSpec CONNECTION_SPEC = ConnectionSpec.HSQLDB;
+
+  private Connection mjs() throws SQLException {
+    return DriverManager.getConnection("jdbc:avatica:remote:factory=" + MJS);
+  }
+
+  private Connection ljs() throws SQLException {
+    return DriverManager.getConnection("jdbc:avatica:remote:factory=" + QRJS);
+  }
+
+  @Before
+  public void before() throws Exception {
+    QuasiRemoteJdbcServiceFactory.initService();
+  }
+
+  @Test public void testRegister() throws Exception {
+    final Connection connection =
+        DriverManager.getConnection("jdbc:avatica:remote:");
+    assertThat(connection.isClosed(), is(false));
+    connection.close();
+    assertThat(connection.isClosed(), is(true));
+  }
+
+  @Test public void testSchemas() throws Exception {
+    final Connection connection = mjs();
+    final ResultSet resultSet =
+        connection.getMetaData().getSchemas(null, null);
+    assertFalse(resultSet.next());
+    final ResultSetMetaData metaData = resultSet.getMetaData();
+    assertTrue(metaData.getColumnCount() >= 2);
+    assertEquals("TABLE_CATALOG", metaData.getColumnName(1));
+    assertEquals("TABLE_SCHEM", metaData.getColumnName(2));
+    resultSet.close();
+    connection.close();
+  }
+
+  @Test public void testTables() throws Exception {
+    final Connection connection = mjs();
+    final ResultSet resultSet =
+        connection.getMetaData().getTables(null, null, null, new String[0]);
+    assertFalse(resultSet.next());
+    final ResultSetMetaData metaData = resultSet.getMetaData();
+    assertTrue(metaData.getColumnCount() >= 3);
+    assertEquals("TABLE_CAT", metaData.getColumnName(1));
+    assertEquals("TABLE_SCHEM", metaData.getColumnName(2));
+    assertEquals("TABLE_NAME", metaData.getColumnName(3));
+    resultSet.close();
+    connection.close();
+  }
+
+  @Ignore
+  @Test public void testNoFactory() throws Exception {
+    final Connection connection =
+        DriverManager.getConnection("jdbc:avatica:remote:");
+    assertThat(connection.isClosed(), is(false));
+    final ResultSet resultSet = connection.getMetaData().getSchemas();
+    assertFalse(resultSet.next());
+    final ResultSetMetaData metaData = resultSet.getMetaData();
+    assertEquals(2, metaData.getColumnCount());
+    assertEquals("TABLE_SCHEM", metaData.getColumnName(1));
+    assertEquals("TABLE_CATALOG", metaData.getColumnName(2));
+    resultSet.close();
+    connection.close();
+    assertThat(connection.isClosed(), is(true));
+  }
+
+  @Ignore
+  @Test public void testCatalogsMock() throws Exception {
+    final Connection connection = mjs();
+    assertThat(connection.isClosed(), is(false));
+    final ResultSet resultSet = connection.getMetaData().getSchemas();
+    assertFalse(resultSet.next());
+    final ResultSetMetaData metaData = resultSet.getMetaData();
+    assertEquals(2, metaData.getColumnCount());
+    assertEquals("TABLE_SCHEM", metaData.getColumnName(1));
+    assertEquals("TABLE_CATALOG", metaData.getColumnName(2));
+    resultSet.close();
+    connection.close();
+    assertThat(connection.isClosed(), is(true));
+  }
+
+  @Test public void testStatementExecuteQueryLocal() throws Exception {
+    checkStatementExecuteQuery(ljs(), false);
+  }
+
+  @Ignore
+  @Test public void testStatementExecuteQueryMock() throws Exception {
+    checkStatementExecuteQuery(mjs(), false);
+  }
+
+  @Ignore
+  @Test public void testPrepareExecuteQueryLocal() throws Exception {
+    checkStatementExecuteQuery(ljs(), true);
+  }
+
+  @Ignore
+  @Test public void testPrepareExecuteQueryMock() throws Exception {
+    checkStatementExecuteQuery(mjs(), true);
+  }
+
+  private void checkStatementExecuteQuery(Connection connection,
+      boolean prepare) throws SQLException {
+    final String sql = "select * from (\n"
+        + "  values (1, 'a'), (null, 'b'), (3, 'c')) as t (c1, c2)";
+    final Statement statement;
+    final ResultSet resultSet;
+    final ParameterMetaData parameterMetaData;
+    if (prepare) {
+      final PreparedStatement ps = connection.prepareStatement(sql);
+      statement = ps;
+      parameterMetaData = ps.getParameterMetaData();
+      resultSet = ps.executeQuery();
+    } else {
+      statement = connection.createStatement();
+      parameterMetaData = null;
+      resultSet = statement.executeQuery(sql);
+    }
+    if (parameterMetaData != null) {
+      assertThat(parameterMetaData.getParameterCount(), equalTo(2));
+    }
+    final ResultSetMetaData metaData = resultSet.getMetaData();
+    assertEquals(2, metaData.getColumnCount());
+    assertEquals("C1", metaData.getColumnName(1));
+    assertEquals("C2", metaData.getColumnName(2));
+    assertTrue(resultSet.next());
+    assertTrue(resultSet.next());
+    assertTrue(resultSet.next());
+    assertFalse(resultSet.next());
+    resultSet.close();
+    statement.close();
+    connection.close();
+  }
+
+  @Test public void testStatementLifecycle() throws Exception {
+    try (AvaticaConnection connection = (AvaticaConnection) ljs()) {
+      Map<Integer, AvaticaStatement> clientMap = connection.statementMap;
+      Cache<Integer, Object> serverMap =
+          QuasiRemoteJdbcServiceFactory.getRemoteStatementMap(connection);
+      assertEquals(0, clientMap.size());
+      assertEquals(0, serverMap.size());
+      Statement stmt = connection.createStatement();
+      assertEquals(1, clientMap.size());
+      assertEquals(1, serverMap.size());
+      stmt.close();
+      assertEquals(0, clientMap.size());
+      assertEquals(0, serverMap.size());
+    }
+  }
+
+  @Test public void testConnectionIsolation() throws Exception {
+    final String sql = "select * from (values (1, 'a'))";
+    Connection conn1 = ljs();
+    Connection conn2 = ljs();
+    Cache<String, Connection> connectionMap =
+        QuasiRemoteJdbcServiceFactory.getRemoteConnectionMap(
+            (AvaticaConnection) conn1);
+    assertEquals("connection cache should start empty",
+        0, connectionMap.size());
+    PreparedStatement conn1stmt1 = conn1.prepareStatement(sql);
+    assertEquals(
+        "statement creation implicitly creates a connection server-side",
+        1, connectionMap.size());
+    PreparedStatement conn2stmt1 = conn2.prepareStatement(sql);
+    assertEquals(
+        "statement creation implicitly creates a connection server-side",
+        2, connectionMap.size());
+    AvaticaPreparedStatement s1 = (AvaticaPreparedStatement) conn1stmt1;
+    AvaticaPreparedStatement s2 = (AvaticaPreparedStatement) conn2stmt1;
+    assertFalse("connection id's should be unique",
+        s1.handle.connectionId.equalsIgnoreCase(s2.handle.connectionId));
+    conn2.close();
+    assertEquals("closing a connection closes the server-side connection",
+        1, connectionMap.size());
+    conn1.close();
+    assertEquals("closing a connection closes the server-side connection",
+        0, connectionMap.size());
+  }
+
+  private void checkStatementExecuteQuery(Connection connection)
+      throws SQLException {
+    final Statement statement = connection.createStatement();
+    final ResultSet resultSet =
+        statement.executeQuery("select * from (\n"
+            + "  values (1, 'a'), (null, 'b'), (3, 'c')) as t (c1, c2)");
+    final ResultSetMetaData metaData = resultSet.getMetaData();
+    assertEquals(2, metaData.getColumnCount());
+    assertEquals("C1", metaData.getColumnName(1));
+    assertEquals("C2", metaData.getColumnName(2));
+    assertTrue(resultSet.next());
+    assertTrue(resultSet.next());
+    assertTrue(resultSet.next());
+    assertFalse(resultSet.next());
+    resultSet.close();
+    statement.close();
+    connection.close();
+  }
+
+  @Test public void testPrepareBindExecuteFetch() throws Exception {
+    checkPrepareBindExecuteFetch(ljs());
+  }
+
+  private void checkPrepareBindExecuteFetch(Connection connection)
+      throws SQLException {
+    final String sql = "select cast(? as integer) * 3 as c, 'x' as x\n"
+        + "from (values (1, 'a'))";
+    final PreparedStatement ps =
+        connection.prepareStatement(sql);
+    final ResultSetMetaData metaData = ps.getMetaData();
+    assertEquals(2, metaData.getColumnCount());
+    assertEquals("C", metaData.getColumnName(1));
+    assertEquals("X", metaData.getColumnName(2));
+    try {
+      final ResultSet resultSet = ps.executeQuery();
+      fail("expected error, got " + resultSet);
+    } catch (SQLException e) {
+      assertThat(e.getMessage(),
+          equalTo("exception while executing query: unbound parameter"));
+    }
+
+    final ParameterMetaData parameterMetaData = ps.getParameterMetaData();
+    assertThat(parameterMetaData.getParameterCount(), equalTo(1));
+
+    ps.setInt(1, 10);
+    final ResultSet resultSet = ps.executeQuery();
+    assertTrue(resultSet.next());
+    assertThat(resultSet.getInt(1), equalTo(30));
+    assertFalse(resultSet.next());
+    resultSet.close();
+
+    ps.setInt(1, 20);
+    final ResultSet resultSet2 = ps.executeQuery();
+    assertFalse(resultSet2.isClosed());
+    assertTrue(resultSet2.next());
+    assertThat(resultSet2.getInt(1), equalTo(60));
+    assertThat(resultSet2.wasNull(), is(false));
+    assertFalse(resultSet2.next());
+    resultSet2.close();
+
+    ps.setObject(1, null);
+    final ResultSet resultSet3 = ps.executeQuery();
+    assertTrue(resultSet3.next());
+    assertThat(resultSet3.getInt(1), equalTo(0));
+    assertThat(resultSet3.wasNull(), is(true));
+    assertFalse(resultSet3.next());
+    resultSet3.close();
+
+    ps.close();
+    connection.close();
+  }
+
+  /**
+   * Factory that creates a service based on a local JDBC connection.
+   */
+  public static class LocalJdbcServiceFactory implements Service.Factory {
+    @Override public Service create(AvaticaConnection connection) {
+      try {
+        return new LocalService(
+            new JdbcMeta(CONNECTION_SPEC.url, CONNECTION_SPEC.username,
+                CONNECTION_SPEC.password));
+      } catch (SQLException e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+
+  /**
+   * Factory that creates a service based on a local JDBC connection.
+   */
+  public static class QuasiRemoteJdbcServiceFactory implements Service.Factory {
+
+    /** a singleton instance that is recreated for each test */
+    private static Service service;
+
+    static void initService() {
+      try {
+        final JdbcMeta jdbcMeta = new JdbcMeta(CONNECTION_SPEC.url,
+            CONNECTION_SPEC.username, CONNECTION_SPEC.password);
+        final LocalService localService = new LocalService(jdbcMeta);
+        service = new LocalJsonService(localService);
+      } catch (SQLException e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    @Override public Service create(AvaticaConnection connection) {
+      assert service != null;
+      return service;
+    }
+
+    /**
+     * Reach into the guts of a quasi-remote connection and pull out the
+     * statement map from the other side.
+     * TODO: refactor tests to replace reflection with package-local access
+     */
+    static Cache<Integer, Object>
+    getRemoteStatementMap(AvaticaConnection connection) throws Exception {
+      Field metaF = AvaticaConnection.class.getDeclaredField("meta");
+      metaF.setAccessible(true);
+      Meta clientMeta = (Meta) metaF.get(connection);
+      Field remoteMetaServiceF = clientMeta.getClass().getDeclaredField("service");
+      remoteMetaServiceF.setAccessible(true);
+      LocalJsonService remoteMetaService = (LocalJsonService) remoteMetaServiceF.get(clientMeta);
+      Field remoteMetaServiceServiceF = remoteMetaService.getClass().getDeclaredField("service");
+      remoteMetaServiceServiceF.setAccessible(true);
+      LocalService remoteMetaServiceService =
+          (LocalService) remoteMetaServiceServiceF.get(remoteMetaService);
+      Field remoteMetaServiceServiceMetaF =
+          remoteMetaServiceService.getClass().getDeclaredField("meta");
+      remoteMetaServiceServiceMetaF.setAccessible(true);
+      JdbcMeta serverMeta = (JdbcMeta) remoteMetaServiceServiceMetaF.get(remoteMetaServiceService);
+      Field jdbcMetaStatementMapF = JdbcMeta.class.getDeclaredField("statementCache");
+      jdbcMetaStatementMapF.setAccessible(true);
+      //noinspection unchecked
+      return (Cache<Integer, Object>) jdbcMetaStatementMapF.get(serverMeta);
+    }
+
+    /**
+     * Reach into the guts of a quasi-remote connection and pull out the
+     * connection map from the other side.
+     * TODO: refactor tests to replace reflection with package-local access
+     */
+    static Cache<String, Connection>
+    getRemoteConnectionMap(AvaticaConnection connection) throws Exception {
+      Field metaF = AvaticaConnection.class.getDeclaredField("meta");
+      metaF.setAccessible(true);
+      Meta clientMeta = (Meta) metaF.get(connection);
+      Field remoteMetaServiceF = clientMeta.getClass().getDeclaredField("service");
+      remoteMetaServiceF.setAccessible(true);
+      LocalJsonService remoteMetaService = (LocalJsonService) remoteMetaServiceF.get(clientMeta);
+      Field remoteMetaServiceServiceF = remoteMetaService.getClass().getDeclaredField("service");
+      remoteMetaServiceServiceF.setAccessible(true);
+      LocalService remoteMetaServiceService =
+          (LocalService) remoteMetaServiceServiceF.get(remoteMetaService);
+      Field remoteMetaServiceServiceMetaF =
+          remoteMetaServiceService.getClass().getDeclaredField("meta");
+      remoteMetaServiceServiceMetaF.setAccessible(true);
+      JdbcMeta serverMeta = (JdbcMeta) remoteMetaServiceServiceMetaF.get(remoteMetaServiceService);
+      Field jdbcMetaConnectionCacheF = JdbcMeta.class.getDeclaredField("connectionCache");
+      jdbcMetaConnectionCacheF.setAccessible(true);
+      //noinspection unchecked
+      return (Cache<String, Connection>) jdbcMetaConnectionCacheF.get(serverMeta);
+    }
+  }
+
+  /** Information necessary to create a JDBC connection. Specify one to run
+   * tests against a different database. (hsqldb is the default.) */
+  public static class ConnectionSpec {
+    public final String url;
+    public final String username;
+    public final String password;
+    public final String driver;
+
+    public ConnectionSpec(String url, String username, String password,
+        String driver) {
+      this.url = url;
+      this.username = username;
+      this.password = password;
+      this.driver = driver;
+    }
+
+    public static final ConnectionSpec HSQLDB =
+        new ConnectionSpec(ScottHsqldb.URI, ScottHsqldb.USER,
+            ScottHsqldb.PASSWORD, "org.hsqldb.jdbcDriver");
+  }
+}
+
+// End RemoteDriverTest.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/fa9bdc4a/avatica-server/src/test/java/org/apache/calcite/avatica/test/AvaticaSuite.java
----------------------------------------------------------------------
diff --git a/avatica-server/src/test/java/org/apache/calcite/avatica/test/AvaticaSuite.java b/avatica-server/src/test/java/org/apache/calcite/avatica/test/AvaticaSuite.java
new file mode 100644
index 0000000..5afba20
--- /dev/null
+++ b/avatica-server/src/test/java/org/apache/calcite/avatica/test/AvaticaSuite.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.avatica.test;
+
+import org.apache.calcite.avatica.RemoteDriverTest;
+
+import org.junit.runner.RunWith;
+
+import org.junit.runners.Suite;
+
+/**
+ * Avatica test suite.
+ */
+@RunWith(Suite.class)
+@Suite.SuiteClasses({
+    ConnectStringParserTest.class,
+    RemoteDriverTest.class
+})
+public class AvaticaSuite {
+}
+
+// End AvaticaSuite.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/fa9bdc4a/avatica/pom.xml
----------------------------------------------------------------------
diff --git a/avatica/pom.xml b/avatica/pom.xml
index 0bacc4a..01cd4e9 100644
--- a/avatica/pom.xml
+++ b/avatica/pom.xml
@@ -38,32 +38,21 @@ limitations under the License.
          or on libraries other than Jackson. -->
     <dependency>
       <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-databind</artifactId>
+      <artifactId>jackson-core</artifactId>
     </dependency>
     <dependency>
-      <groupId>com.google.guava</groupId>
-      <artifactId>guava</artifactId>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-annotations</artifactId>
     </dependency>
     <dependency>
-      <groupId>commons-logging</groupId>
-      <artifactId>commons-logging</artifactId>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
     </dependency>
     <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>
     </dependency>
-    <dependency>
-      <groupId>net.hydromatic</groupId>
-      <artifactId>scott-data-hsqldb</artifactId>
-      <version>0.1</version>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.hsqldb</groupId>
-      <artifactId>hsqldb</artifactId>
-      <scope>test</scope>
-    </dependency>
   </dependencies>
 
   <build>
@@ -94,6 +83,20 @@ limitations under the License.
           </execution>
         </executions>
       </plugin>
+
+      <!-- Produce a tests jar so that avatica-server/pom.xml can reference for suite.
+           TODO: remove after moving over to annotation-based TestSuite definitions. -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
   </build>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/fa9bdc4a/avatica/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java b/avatica/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
deleted file mode 100644
index 90e8c2a..0000000
--- a/avatica/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
+++ /dev/null
@@ -1,765 +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.avatica.jdbc;
-
-import org.apache.calcite.avatica.AvaticaParameter;
-import org.apache.calcite.avatica.ColumnMetaData;
-import org.apache.calcite.avatica.Meta;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.RemovalListener;
-import com.google.common.cache.RemovalNotification;
-
-import java.lang.reflect.Array;
-import java.lang.reflect.Type;
-import java.math.BigDecimal;
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.ParameterMetaData;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.ResultSetMetaData;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.sql.Types;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Properties;
-import java.util.UUID;
-import java.util.concurrent.TimeUnit;
-
-/** Implementation of {@link Meta} upon an existing JDBC data source. */
-public class JdbcMeta implements Meta {
-
-  private static final Log LOG = LogFactory.getLog(JdbcMeta.class);
-
-  /**
-   * JDBC Types Mapped to Java Types
-   *
-   * @see <a href="https://docs.oracle.com/javase/1.5.0/docs/guide/jdbc/getstart/mapping.html#1051555">JDBC Types Mapped to Java Types</a>
-   */
-  protected static final Map<Integer, Type> SQL_TYPE_TO_JAVA_TYPE =
-      new HashMap<>();
-  static {
-    SQL_TYPE_TO_JAVA_TYPE.put(Types.CHAR, String.class);
-    SQL_TYPE_TO_JAVA_TYPE.put(Types.VARCHAR, String.class);
-    SQL_TYPE_TO_JAVA_TYPE.put(Types.LONGNVARCHAR, String.class);
-    SQL_TYPE_TO_JAVA_TYPE.put(Types.NUMERIC, BigDecimal.class);
-    SQL_TYPE_TO_JAVA_TYPE.put(Types.DECIMAL, BigDecimal.class);
-    SQL_TYPE_TO_JAVA_TYPE.put(Types.BIT, Boolean.TYPE);
-    SQL_TYPE_TO_JAVA_TYPE.put(Types.TINYINT, Byte.TYPE);
-    SQL_TYPE_TO_JAVA_TYPE.put(Types.SMALLINT, Short.TYPE);
-    SQL_TYPE_TO_JAVA_TYPE.put(Types.INTEGER, Integer.TYPE);
-    SQL_TYPE_TO_JAVA_TYPE.put(Types.BIGINT, Long.TYPE);
-    SQL_TYPE_TO_JAVA_TYPE.put(Types.REAL, Float.TYPE);
-    SQL_TYPE_TO_JAVA_TYPE.put(Types.FLOAT, Double.TYPE);
-    SQL_TYPE_TO_JAVA_TYPE.put(Types.DOUBLE, Double.TYPE);
-    SQL_TYPE_TO_JAVA_TYPE.put(Types.BINARY, byte[].class);
-    SQL_TYPE_TO_JAVA_TYPE.put(Types.VARBINARY, byte[].class);
-    SQL_TYPE_TO_JAVA_TYPE.put(Types.LONGVARBINARY, byte[].class);
-    SQL_TYPE_TO_JAVA_TYPE.put(Types.DATE, java.sql.Date.class);
-    SQL_TYPE_TO_JAVA_TYPE.put(Types.TIME, java.sql.Time.class);
-    SQL_TYPE_TO_JAVA_TYPE.put(Types.TIMESTAMP, java.sql.Timestamp.class);
-    //put(Types.CLOB, Clob);
-    //put(Types.BLOB, Blob);
-    SQL_TYPE_TO_JAVA_TYPE.put(Types.ARRAY, Array.class);
-  }
-
-  /** A "magic column" descriptor used to return statement execute update count via a
-   * {@link Meta.MetaResultSet}. */
-  private static final ColumnMetaData UPDATE_COL = new ColumnMetaData(1, false, false, false,
-      false, ResultSetMetaData.columnNoNulls, true, 10, "u", "u", null, 15, 15, "update_result",
-      "avatica_internal", ColumnMetaData.scalar(Types.INTEGER, "INTEGER",
-        ColumnMetaData.Rep.INTEGER), true, false, false, "java.lang.Integer");
-
-  private static final String CONN_CACHE_KEY_BASE = "avatica.connectioncache";
-
-  /** Configurable connection cache settings. */
-  public enum ConnectionCacheSettings {
-    /** JDBC connection property for setting connection cache concurrency level. */
-    CONCURRENCY_LEVEL(CONN_CACHE_KEY_BASE + ".concurrency", "10"),
-
-    /** JDBC connection property for setting connection cache initial capacity. */
-    INITIAL_CAPACITY(CONN_CACHE_KEY_BASE + ".initialcapacity", "100"),
-
-    /** JDBC connection property for setting connection cache maximum capacity. */
-    MAX_CAPACITY(CONN_CACHE_KEY_BASE + ".maxcapacity", "1000"),
-
-    /** JDBC connection property for setting connection cache expiration duration. */
-    EXPIRY_DURATION(CONN_CACHE_KEY_BASE + ".expirydiration", "10"),
-
-    /** JDBC connection property for setting connection cache expiration unit. */
-    EXPIRY_UNIT(CONN_CACHE_KEY_BASE + ".expiryunit", TimeUnit.MINUTES.name());
-
-    private final String key;
-    private final String defaultValue;
-
-    private ConnectionCacheSettings(String key, String defaultValue) {
-      this.key = key;
-      this.defaultValue = defaultValue;
-    }
-
-    /** The configuration key for specifying this setting. */
-    public String key() {
-      return key;
-    }
-
-    /** The default value for this setting. */
-    public String defaultValue() {
-      return defaultValue;
-    }
-  }
-
-  private static final String STMT_CACHE_KEY_BASE = "avatica.statementcache";
-
-  /** Configurable statement cache settings. */
-  public enum StatementCacheSettings {
-    /** JDBC connection property for setting connection cache concurrency level. */
-    CONCURRENCY_LEVEL(STMT_CACHE_KEY_BASE + ".concurrency", "100"),
-
-    /** JDBC connection property for setting connection cache initial capacity. */
-    INITIAL_CAPACITY(STMT_CACHE_KEY_BASE + ".initialcapacity", "1000"),
-
-    /** JDBC connection property for setting connection cache maximum capacity. */
-    MAX_CAPACITY(STMT_CACHE_KEY_BASE + ".maxcapacity", "10000"),
-
-    /** JDBC connection property for setting connection cache expiration duration.
-     *
-     * <p>Used in conjunction with {@link #EXPIRY_UNIT}.</p>
-     */
-    EXPIRY_DURATION(STMT_CACHE_KEY_BASE + ".expirydiration", "5"),
-
-    /** JDBC connection property for setting connection cache expiration unit.
-     *
-     * <p>Used in conjunction with {@link #EXPIRY_DURATION}.</p>
-     */
-    EXPIRY_UNIT(STMT_CACHE_KEY_BASE + ".expiryunit", TimeUnit.MINUTES.name());
-
-    private final String key;
-    private final String defaultValue;
-
-    private StatementCacheSettings(String key, String defaultValue) {
-      this.key = key;
-      this.defaultValue = defaultValue;
-    }
-
-    /** The configuration key for specifying this setting. */
-    public String key() {
-      return key;
-    }
-
-    /** The default value for this setting. */
-    public String defaultValue() {
-      return defaultValue;
-    }
-  }
-
-  private static final String DEFAULT_CONN_ID =
-      UUID.fromString("00000000-0000-0000-0000-000000000000").toString();
-
-  private final String url;
-  private final Properties info;
-  private final Connection connection; // TODO: remove default connection
-  private final Cache<String, Connection> connectionCache;
-  private final Cache<Integer, StatementInfo> statementCache;
-
-  /**
-   * Convert from JDBC metadata to Avatica columns.
-   */
-  protected static List<ColumnMetaData>
-  columns(ResultSetMetaData metaData) throws SQLException {
-    final List<ColumnMetaData> columns = new ArrayList<>();
-    for (int i = 1; i <= metaData.getColumnCount(); i++) {
-      final Type javaType =
-          SQL_TYPE_TO_JAVA_TYPE.get(metaData.getColumnType(i));
-      ColumnMetaData.AvaticaType t =
-          ColumnMetaData.scalar(metaData.getColumnType(i),
-              metaData.getColumnTypeName(i), ColumnMetaData.Rep.of(javaType));
-      ColumnMetaData md =
-          new ColumnMetaData(i - 1, metaData.isAutoIncrement(i),
-              metaData.isCaseSensitive(i), metaData.isSearchable(i),
-              metaData.isCurrency(i), metaData.isNullable(i),
-              metaData.isSigned(i), metaData.getColumnDisplaySize(i),
-              metaData.getColumnLabel(i), metaData.getColumnName(i),
-              metaData.getSchemaName(i), metaData.getPrecision(i),
-              metaData.getScale(i), metaData.getTableName(i),
-              metaData.getCatalogName(i), t, metaData.isReadOnly(i),
-              metaData.isWritable(i), metaData.isDefinitelyWritable(i),
-              metaData.getColumnClassName(i));
-      columns.add(md);
-    }
-    return columns;
-  }
-
-  /**
-   * Converts from JDBC metadata to AvaticaParameters
-   */
-  protected static List<AvaticaParameter> parameters(ParameterMetaData metaData)
-      throws SQLException {
-    if (metaData == null) {
-      return Collections.emptyList();
-    }
-    final List<AvaticaParameter> params = new ArrayList<>();
-    for (int i = 1; i <= metaData.getParameterCount(); i++) {
-      params.add(
-          new AvaticaParameter(metaData.isSigned(i), metaData.getPrecision(i),
-              metaData.getScale(i), metaData.getParameterType(i),
-              metaData.getParameterTypeName(i),
-              metaData.getParameterClassName(i), "?" + i));
-    }
-    return params;
-  }
-
-  protected static Signature signature(ResultSetMetaData metaData,
-      ParameterMetaData parameterMetaData, String sql) throws  SQLException {
-    return new Signature(columns(metaData), sql, parameters(parameterMetaData),
-        null, CursorFactory.LIST /* LIST because JdbcResultSet#frame */);
-  }
-
-  protected static Signature signature(ResultSetMetaData metaData)
-      throws SQLException {
-    return signature(metaData, null, null);
-  }
-
-  /** Callback for {@link #connectionCache} member expiration. */
-  private class ConnectionExpiryHandler
-      implements RemovalListener<String, Connection> {
-
-    public void onRemoval(RemovalNotification<String, Connection> notification) {
-      String connectionId = notification.getKey();
-      Connection doomed = notification.getValue();
-      // is String.equals() more efficient?
-      if (notification.getValue() == connection) {
-        return;
-      }
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Expiring connection " + connectionId + " because "
-            + notification.getCause());
-      }
-      try {
-        if (doomed != null) {
-          doomed.close();
-        }
-      } catch (Throwable t) {
-        LOG.info("Exception thrown while expiring connection " + connectionId, t);
-      }
-    }
-  }
-
-  /** Callback for {@link #statementCache} member expiration. */
-  private class StatementExpiryHandler
-      implements RemovalListener<Integer, StatementInfo> {
-    public void onRemoval(RemovalNotification<Integer, StatementInfo> notification) {
-      Integer stmtId = notification.getKey();
-      StatementInfo doomed = notification.getValue();
-      if (doomed == null) {
-        // log/throw?
-        return;
-      }
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Expiring statement " + stmtId + " because "
-            + notification.getCause());
-      }
-      try {
-        if (doomed.resultSet != null) {
-          doomed.resultSet.close();
-        }
-        if (doomed.statement != null) {
-          doomed.statement.close();
-        }
-      } catch (Throwable t) {
-        LOG.info("Exception thrown while expiring statement " + stmtId);
-      }
-    }
-  }
-
-  /**
-   * @param url a database url of the form
-   *  <code> jdbc:<em>subprotocol</em>:<em>subname</em></code>
-   */
-  public JdbcMeta(String url) throws SQLException {
-    this(url, new Properties());
-  }
-
-  /**
-   * @param url a database url of the form
-   * <code>jdbc:<em>subprotocol</em>:<em>subname</em></code>
-   * @param user the database user on whose behalf the connection is being
-   *   made
-   * @param password the user's password
-   */
-  public JdbcMeta(final String url, final String user, final String password)
-      throws SQLException {
-    this(url, new Properties() {
-      {
-        put("user", user);
-        put("password", password);
-      }
-    });
-  }
-
-  /**
-   * @param url a database url of the form
-   * <code> jdbc:<em>subprotocol</em>:<em>subname</em></code>
-   * @param info a list of arbitrary string tag/value pairs as
-   * connection arguments; normally at least a "user" and
-   * "password" property should be included
-   */
-  public JdbcMeta(String url, Properties info) throws SQLException {
-    this.url = url;
-    this.info = info;
-    this.connection = DriverManager.getConnection(url, info);
-
-    int concurrencyLevel = Integer.parseInt(
-        info.getProperty(ConnectionCacheSettings.CONCURRENCY_LEVEL.key(),
-            ConnectionCacheSettings.CONCURRENCY_LEVEL.defaultValue()));
-    int initialCapacity = Integer.parseInt(
-        info.getProperty(ConnectionCacheSettings.INITIAL_CAPACITY.key(),
-            ConnectionCacheSettings.INITIAL_CAPACITY.defaultValue()));
-    long maxCapacity = Long.parseLong(
-        info.getProperty(ConnectionCacheSettings.MAX_CAPACITY.key(),
-            ConnectionCacheSettings.MAX_CAPACITY.defaultValue()));
-    long connectionExpiryDuration = Long.parseLong(
-        info.getProperty(ConnectionCacheSettings.EXPIRY_DURATION.key(),
-            ConnectionCacheSettings.EXPIRY_DURATION.defaultValue()));
-    TimeUnit connectionExpiryUnit = TimeUnit.valueOf(
-        info.getProperty(ConnectionCacheSettings.EXPIRY_UNIT.key(),
-            ConnectionCacheSettings.EXPIRY_UNIT.defaultValue()));
-    this.connectionCache = CacheBuilder.newBuilder()
-        .concurrencyLevel(concurrencyLevel)
-        .initialCapacity(initialCapacity)
-        .maximumSize(maxCapacity)
-        .expireAfterAccess(connectionExpiryDuration, connectionExpiryUnit)
-        .removalListener(new ConnectionExpiryHandler())
-        .build();
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("instantiated connection cache: " + connectionCache.stats());
-    }
-
-    concurrencyLevel = Integer.parseInt(
-        info.getProperty(StatementCacheSettings.CONCURRENCY_LEVEL.key(),
-            StatementCacheSettings.CONCURRENCY_LEVEL.defaultValue()));
-    initialCapacity = Integer.parseInt(
-        info.getProperty(StatementCacheSettings.INITIAL_CAPACITY.key(),
-            StatementCacheSettings.INITIAL_CAPACITY.defaultValue()));
-    maxCapacity = Long.parseLong(
-        info.getProperty(StatementCacheSettings.MAX_CAPACITY.key(),
-            StatementCacheSettings.MAX_CAPACITY.defaultValue()));
-    connectionExpiryDuration = Long.parseLong(
-        info.getProperty(StatementCacheSettings.EXPIRY_DURATION.key(),
-            StatementCacheSettings.EXPIRY_DURATION.defaultValue()));
-    connectionExpiryUnit = TimeUnit.valueOf(
-        info.getProperty(StatementCacheSettings.EXPIRY_UNIT.key(),
-            StatementCacheSettings.EXPIRY_UNIT.defaultValue()));
-    this.statementCache = CacheBuilder.newBuilder()
-        .concurrencyLevel(concurrencyLevel)
-        .initialCapacity(initialCapacity)
-        .maximumSize(maxCapacity)
-        .expireAfterAccess(connectionExpiryDuration, connectionExpiryUnit)
-        .removalListener(new StatementExpiryHandler())
-        .build();
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("instantiated statement cache: " + statementCache.stats());
-    }
-  }
-
-  public String getSqlKeywords() {
-    try {
-      return connection.getMetaData().getSQLKeywords();
-    } catch (SQLException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  public String getNumericFunctions() {
-    try {
-      return connection.getMetaData().getNumericFunctions();
-    } catch (SQLException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  public String getStringFunctions() {
-    return null;
-  }
-
-  public String getSystemFunctions() {
-    return null;
-  }
-
-  public String getTimeDateFunctions() {
-    return null;
-  }
-
-  public MetaResultSet getTables(String catalog, Pat schemaPattern,
-      Pat tableNamePattern, List<String> typeList) {
-    try {
-      String[] types = new String[typeList == null ? 0 : typeList.size()];
-      return JdbcResultSet.create(DEFAULT_CONN_ID, -1,
-          connection.getMetaData().getTables(catalog, schemaPattern.s,
-              tableNamePattern.s,
-              typeList == null ? types : typeList.toArray(types)));
-    } catch (SQLException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  public MetaResultSet getColumns(String catalog, Pat schemaPattern,
-      Pat tableNamePattern, Pat columnNamePattern) {
-    try {
-      return JdbcResultSet.create(DEFAULT_CONN_ID, -1,
-          connection.getMetaData().getColumns(catalog, schemaPattern.s,
-              tableNamePattern.s, columnNamePattern.s));
-    } catch (SQLException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  public MetaResultSet getSchemas(String catalog, Pat schemaPattern) {
-    try {
-      return JdbcResultSet.create(DEFAULT_CONN_ID, -1,
-          connection.getMetaData().getSchemas(catalog, schemaPattern.s));
-    } catch (SQLException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  public MetaResultSet getCatalogs() {
-    try {
-      return JdbcResultSet.create(DEFAULT_CONN_ID, -1,
-          connection.getMetaData().getCatalogs());
-    } catch (SQLException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  public MetaResultSet getTableTypes() {
-    try {
-      return JdbcResultSet.create(DEFAULT_CONN_ID, -1,
-          connection.getMetaData().getTableTypes());
-    } catch (SQLException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  public MetaResultSet getProcedures(String catalog, Pat schemaPattern,
-      Pat procedureNamePattern) {
-    try {
-      return JdbcResultSet.create(DEFAULT_CONN_ID, -1,
-          connection.getMetaData().getProcedures(catalog, schemaPattern.s,
-              procedureNamePattern.s));
-    } catch (SQLException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  public MetaResultSet getProcedureColumns(String catalog, Pat schemaPattern,
-      Pat procedureNamePattern, Pat columnNamePattern) {
-    try {
-      return JdbcResultSet.create(DEFAULT_CONN_ID, -1,
-          connection.getMetaData().getProcedureColumns(catalog,
-              schemaPattern.s, procedureNamePattern.s, columnNamePattern.s));
-    } catch (SQLException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  public MetaResultSet getColumnPrivileges(String catalog, String schema,
-      String table, Pat columnNamePattern) {
-    try {
-      return JdbcResultSet.create(DEFAULT_CONN_ID, -1,
-          connection.getMetaData().getColumnPrivileges(catalog, schema,
-              table, columnNamePattern.s));
-    } catch (SQLException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  public MetaResultSet getTablePrivileges(String catalog, Pat schemaPattern,
-      Pat tableNamePattern) {
-    try {
-      return JdbcResultSet.create(DEFAULT_CONN_ID, -1,
-          connection.getMetaData().getTablePrivileges(catalog,
-              schemaPattern.s, tableNamePattern.s));
-    } catch (SQLException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  public MetaResultSet getBestRowIdentifier(String catalog, String schema,
-      String table, int scope, boolean nullable) {
-    try {
-      return JdbcResultSet.create(DEFAULT_CONN_ID, -1,
-          connection.getMetaData().getBestRowIdentifier(catalog, schema,
-              table, scope, nullable));
-    } catch (SQLException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  public MetaResultSet getVersionColumns(String catalog, String schema,
-      String table) {
-    try {
-      return JdbcResultSet.create(DEFAULT_CONN_ID, -1,
-          connection.getMetaData().getVersionColumns(catalog, schema, table));
-    } catch (SQLException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  public MetaResultSet getPrimaryKeys(String catalog, String schema,
-      String table) {
-    try {
-      return JdbcResultSet.create(DEFAULT_CONN_ID, -1,
-          connection.getMetaData().getPrimaryKeys(catalog, schema, table));
-    } catch (SQLException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  public MetaResultSet getImportedKeys(String catalog, String schema,
-      String table) {
-    return null;
-  }
-
-  public MetaResultSet getExportedKeys(String catalog, String schema,
-      String table) {
-    return null;
-  }
-
-  public MetaResultSet getCrossReference(String parentCatalog,
-      String parentSchema, String parentTable, String foreignCatalog,
-      String foreignSchema, String foreignTable) {
-    return null;
-  }
-
-  public MetaResultSet getTypeInfo() {
-    return null;
-  }
-
-  public MetaResultSet getIndexInfo(String catalog, String schema, String table,
-      boolean unique, boolean approximate) {
-    return null;
-  }
-
-  public MetaResultSet getUDTs(String catalog, Pat schemaPattern,
-      Pat typeNamePattern, int[] types) {
-    return null;
-  }
-
-  public MetaResultSet getSuperTypes(String catalog, Pat schemaPattern,
-      Pat typeNamePattern) {
-    return null;
-  }
-
-  public MetaResultSet getSuperTables(String catalog, Pat schemaPattern,
-      Pat tableNamePattern) {
-    return null;
-  }
-
-  public MetaResultSet getAttributes(String catalog, Pat schemaPattern,
-      Pat typeNamePattern, Pat attributeNamePattern) {
-    return null;
-  }
-
-  public MetaResultSet getClientInfoProperties() {
-    return null;
-  }
-
-  public MetaResultSet getFunctions(String catalog, Pat schemaPattern,
-      Pat functionNamePattern) {
-    return null;
-  }
-
-  public MetaResultSet getFunctionColumns(String catalog, Pat schemaPattern,
-      Pat functionNamePattern, Pat columnNamePattern) {
-    return null;
-  }
-
-  public MetaResultSet getPseudoColumns(String catalog, Pat schemaPattern,
-      Pat tableNamePattern, Pat columnNamePattern) {
-    return null;
-  }
-
-  public Iterable<Object> createIterable(StatementHandle handle,
-      Signature signature, List<Object> parameterValues, Frame firstFrame) {
-    return null;
-  }
-
-  protected Connection getConnection(String id) throws SQLException {
-    Connection conn = connectionCache.getIfPresent(id);
-    if (conn == null) {
-      conn = DriverManager.getConnection(url, info);
-      connectionCache.put(id, conn);
-    }
-    return conn;
-  }
-
-  public StatementHandle createStatement(ConnectionHandle ch) {
-    try {
-      final Connection conn = getConnection(ch.id);
-      final Statement statement = conn.createStatement();
-      final int id = System.identityHashCode(statement);
-      statementCache.put(id, new StatementInfo(statement));
-      StatementHandle h = new StatementHandle(ch.id, id, null);
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("created statement " + h);
-      }
-      return h;
-    } catch (SQLException e) {
-      throw propagate(e);
-    }
-  }
-
-  @Override public void closeStatement(StatementHandle h) {
-    StatementInfo info = statementCache.getIfPresent(h.id);
-    if (info == null || info.statement == null) {
-      LOG.debug("client requested close unknown statement " + h);
-      return;
-    }
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("closing statement " + h);
-    }
-    try {
-      if (info.resultSet != null) {
-        info.resultSet.close();
-      }
-      info.statement.close();
-    } catch (SQLException e) {
-      throw propagate(e);
-    } finally {
-      statementCache.invalidate(h.id);
-    }
-  }
-
-  @Override public void closeConnection(ConnectionHandle ch) {
-    Connection conn = connectionCache.getIfPresent(ch.id);
-    if (conn == null) {
-      LOG.debug("client requested close unknown connection " + ch);
-      return;
-    }
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("closing connection " + ch);
-    }
-    try {
-      conn.close();
-    } catch (SQLException e) {
-      throw propagate(e);
-    } finally {
-      connectionCache.invalidate(ch.id);
-    }
-  }
-
-  private RuntimeException propagate(Throwable e) {
-    if (e instanceof RuntimeException) {
-      throw (RuntimeException) e;
-    } else if (e instanceof Error) {
-      throw (Error) e;
-    } else {
-      throw new RuntimeException(e);
-    }
-  }
-
-  public StatementHandle prepare(ConnectionHandle ch, String sql,
-      int maxRowCount) {
-    try {
-      final Connection conn = getConnection(ch.id);
-      final PreparedStatement statement = conn.prepareStatement(sql);
-      final int id = System.identityHashCode(statement);
-      statementCache.put(id, new StatementInfo(statement));
-      StatementHandle h = new StatementHandle(ch.id, id,
-          signature(statement.getMetaData(), statement.getParameterMetaData(),
-              sql));
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("prepared statement " + h);
-      }
-      return h;
-    } catch (SQLException e) {
-      throw propagate(e);
-    }
-  }
-
-  public MetaResultSet prepareAndExecute(ConnectionHandle ch, String sql,
-      int maxRowCount, PrepareCallback callback) {
-    try {
-      final Connection connection = getConnection(ch.id);
-      final PreparedStatement statement = connection.prepareStatement(sql);
-      final int id = System.identityHashCode(statement);
-      final StatementInfo info = new StatementInfo(statement);
-      statementCache.put(id, info);
-      info.resultSet = statement.executeQuery();
-      MetaResultSet mrs = JdbcResultSet.create(ch.id, id, info.resultSet);
-      if (LOG.isTraceEnabled()) {
-        StatementHandle h = new StatementHandle(ch.id, id, null);
-        LOG.trace("prepAndExec statement " + h);
-      }
-      return mrs;
-    } catch (SQLException e) {
-      throw propagate(e);
-    }
-  }
-
-  public Frame fetch(StatementHandle h, List<Object> parameterValues,
-      int offset, int fetchMaxRowCount) {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("fetching " + h + " offset:" + offset + " fetchMaxRowCount:" + fetchMaxRowCount);
-    }
-    try {
-      final StatementInfo statementInfo = Objects.requireNonNull(
-          statementCache.getIfPresent(h.id),
-          "Statement not found, potentially expired. " + h);
-      if (statementInfo.resultSet == null || parameterValues != null) {
-        if (statementInfo.resultSet != null) {
-          statementInfo.resultSet.close();
-        }
-        final PreparedStatement preparedStatement =
-            (PreparedStatement) statementInfo.statement;
-        if (parameterValues != null) {
-          for (int i = 0; i < parameterValues.size(); i++) {
-            Object o = parameterValues.get(i);
-            preparedStatement.setObject(i + 1, o);
-          }
-        }
-        statementInfo.resultSet = preparedStatement.executeQuery();
-      }
-      return JdbcResultSet.frame(statementInfo.resultSet, offset,
-          fetchMaxRowCount);
-    } catch (SQLException e) {
-      throw propagate(e);
-    }
-  }
-
-  /** All we know about a statement. */
-  private static class StatementInfo {
-    final Statement statement; // sometimes a PreparedStatement
-    ResultSet resultSet;
-
-    private StatementInfo(Statement statement) {
-      this.statement = Objects.requireNonNull(statement);
-    }
-  }
-}
-
-// End JdbcMeta.java


[10/10] incubator-calcite git commit: [CALCITE-659] Missing types in averageTypeValueSize method in RelMdSize (Jesus Camacho Rodriguez)

Posted by jh...@apache.org.
[CALCITE-659] Missing types in averageTypeValueSize method in RelMdSize (Jesus Camacho Rodriguez)

Close apache/incubator-calcite#71


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

Branch: refs/heads/master
Commit: 30ae0513c1c7e9cfb14a8a6b0e60cc37ec499ce7
Parents: 0d80fd2
Author: Jesus Camacho Rodriguez <jc...@hortonworks.com>
Authored: Wed Apr 1 16:39:31 2015 +0100
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Apr 3 01:10:06 2015 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/calcite/rel/metadata/RelMdSize.java  | 7 +++++++
 1 file changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/30ae0513/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 27e8a0c..af60e73 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
@@ -275,6 +275,7 @@ public class RelMdSize {
       return 2d;
     case INTEGER:
     case REAL:
+    case DECIMAL:
     case DATE:
     case TIME:
       return 4d;
@@ -294,6 +295,12 @@ public class RelMdSize {
     case VARCHAR:
       // Even in large (say VARCHAR(2000)) columns most strings are small
       return Math.min((double) type.getPrecision() * BYTES_PER_CHARACTER, 100d);
+    case ROW:
+      Double average = 0.0;
+      for (RelDataTypeField field : type.getFieldList()) {
+        average += averageTypeValueSize(field.getType());
+      }
+      return average;
     default:
       return null;
     }


[07/10] incubator-calcite git commit: [CALCITE-655] Implement "connectionSync" RPC (Nick Dimiduk)

Posted by jh...@apache.org.
[CALCITE-655] Implement "connectionSync" RPC (Nick Dimiduk)


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

Branch: refs/heads/master
Commit: 304eb70c905ed40f9bc2900935f949f5b5880225
Parents: 0ea085b
Author: Nick Dimiduk <nd...@gmail.com>
Authored: Wed Apr 1 16:08:21 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Apr 3 01:10:05 2015 -0700

----------------------------------------------------------------------
 .../apache/calcite/avatica/jdbc/JdbcMeta.java   |  46 ++++-
 .../calcite/avatica/RemoteDriverTest.java       |   2 -
 .../calcite/avatica/remote/RemoteMetaTest.java  | 132 ++++++++++++++
 .../calcite/avatica/AvaticaConnection.java      |  26 ++-
 .../avatica/ConnectionPropertiesImpl.java       | 180 +++++++++++++++++++
 .../java/org/apache/calcite/avatica/Meta.java   |  79 ++++++++
 .../org/apache/calcite/avatica/MetaImpl.java    |  15 ++
 .../calcite/avatica/remote/JsonService.java     |   8 +
 .../calcite/avatica/remote/LocalService.java    |   6 +
 .../calcite/avatica/remote/RemoteMeta.java      |  31 ++++
 .../apache/calcite/avatica/remote/Service.java  |  36 +++-
 .../apache/calcite/jdbc/CalciteMetaImpl.java    |   6 +
 12 files changed, 545 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/304eb70c/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
----------------------------------------------------------------------
diff --git a/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java b/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
index b5fb561..b4b6211 100644
--- a/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
+++ b/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
@@ -18,6 +18,7 @@ package org.apache.calcite.avatica.jdbc;
 
 import org.apache.calcite.avatica.AvaticaParameter;
 import org.apache.calcite.avatica.ColumnMetaData;
+import org.apache.calcite.avatica.ConnectionPropertiesImpl;
 import org.apache.calcite.avatica.Meta;
 
 import org.apache.commons.logging.Log;
@@ -116,7 +117,7 @@ public class JdbcMeta implements Meta {
     private final String key;
     private final String defaultValue;
 
-    private ConnectionCacheSettings(String key, String defaultValue) {
+    ConnectionCacheSettings(String key, String defaultValue) {
       this.key = key;
       this.defaultValue = defaultValue;
     }
@@ -672,6 +673,43 @@ public class JdbcMeta implements Meta {
     }
   }
 
+  protected void apply(Connection conn, ConnectionProperties connProps)
+      throws SQLException {
+    if (connProps.isAutoCommit() != null) {
+      conn.setAutoCommit(connProps.isAutoCommit());
+    }
+    if (connProps.isReadOnly() != null) {
+      conn.setReadOnly(connProps.isReadOnly());
+    }
+    if (connProps.getTransactionIsolation() != null) {
+      conn.setTransactionIsolation(connProps.getTransactionIsolation());
+    }
+    if (connProps.getCatalog() != null) {
+      conn.setCatalog(connProps.getCatalog());
+    }
+    if (connProps.getSchema() != null) {
+      conn.setSchema(connProps.getSchema());
+    }
+  }
+
+  @Override public ConnectionProperties connectionSync(ConnectionHandle ch,
+      ConnectionProperties connProps) {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("syncing properties for connection " + ch);
+    }
+    try {
+      Connection conn = getConnection(ch.id);
+      ConnectionPropertiesImpl props = new ConnectionPropertiesImpl(conn).merge(connProps);
+      if (props.isDirty()) {
+        apply(conn, props);
+        props.setDirty(false);
+      }
+      return props;
+    } catch (SQLException e) {
+      throw propagate(e);
+    }
+  }
+
   private RuntimeException propagate(Throwable e) {
     if (e instanceof RuntimeException) {
       throw (RuntimeException) e;
@@ -720,8 +758,10 @@ public class JdbcMeta implements Meta {
         columns.add(UPDATE_COL);
         List<Object> val = new ArrayList<>();
         val.add(new Object[] { updateCount });
-        mrs = new MetaResultSet(ch.id, id, true, new Signature(columns, sql, null, null,
-            CursorFactory.ARRAY), new Frame(0, true, val));
+        final Signature signature =
+            new Signature(columns, sql, null, null, CursorFactory.ARRAY);
+        final Frame frame = new Frame(0, true, val);
+        mrs = new MetaResultSet(ch.id, id, true, signature, frame);
       } else {
         mrs = JdbcResultSet.create(ch.id, id, info.resultSet);
       }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/304eb70c/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
----------------------------------------------------------------------
diff --git a/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java b/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
index 2b9c111..c24a7fd 100644
--- a/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
+++ b/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
@@ -332,8 +332,6 @@ public class RemoteDriverTest {
     connection.close();
   }
 
-
-
   /**
    * Factory that creates a service based on a local JDBC connection.
    */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/304eb70c/avatica-server/src/test/java/org/apache/calcite/avatica/remote/RemoteMetaTest.java
----------------------------------------------------------------------
diff --git a/avatica-server/src/test/java/org/apache/calcite/avatica/remote/RemoteMetaTest.java b/avatica-server/src/test/java/org/apache/calcite/avatica/remote/RemoteMetaTest.java
new file mode 100644
index 0000000..890a78a
--- /dev/null
+++ b/avatica-server/src/test/java/org/apache/calcite/avatica/remote/RemoteMetaTest.java
@@ -0,0 +1,132 @@
+/*
+ * 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.avatica.remote;
+
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.ConnectionPropertiesImpl;
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.avatica.RemoteDriverTest;
+import org.apache.calcite.avatica.jdbc.JdbcMeta;
+import org.apache.calcite.avatica.server.HttpServer;
+import org.apache.calcite.avatica.server.Main;
+
+import com.google.common.cache.Cache;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.lang.reflect.Field;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/** Tests covering {@link RemoteMeta}. */
+public class RemoteMetaTest {
+  private static final RemoteDriverTest.ConnectionSpec CONNECTION_SPEC =
+      RemoteDriverTest.ConnectionSpec.HSQLDB;
+
+  private static HttpServer start;
+  private static String url;
+
+  /** Factory that provides a JMeta */
+  public static class FullyRemoteJdbcMetaFactory implements Meta.Factory {
+
+    private static JdbcMeta instance = null;
+
+    private static JdbcMeta getInstance() {
+      if (instance == null) {
+        try {
+          instance = new JdbcMeta(CONNECTION_SPEC.url, CONNECTION_SPEC.username,
+              CONNECTION_SPEC.password);
+        } catch (SQLException e) {
+          throw new RuntimeException(e);
+        }
+      }
+      return instance;
+    }
+
+    @Override public Meta create(List<String> args) {
+      return getInstance();
+    }
+  }
+
+  @BeforeClass public static void beforeClass() throws Exception {
+    start = Main.start(new String[] { FullyRemoteJdbcMetaFactory.class.getName() });
+    final int port = start.getPort();
+    url = "jdbc:avatica:remote:url=http://localhost:" + port;
+  }
+
+  @AfterClass public static void afterClass() throws Exception {
+    if (start != null) {
+      start.stop();
+    }
+  }
+
+  private static Meta getMeta(AvaticaConnection conn) throws Exception {
+    Field f = AvaticaConnection.class.getDeclaredField("meta");
+    f.setAccessible(true);
+    return (Meta) f.get(conn);
+  }
+
+  private static Connection getConnection(JdbcMeta m, String id) throws Exception {
+    Field f = JdbcMeta.class.getDeclaredField("connectionCache");
+    f.setAccessible(true);
+    //noinspection unchecked
+    Cache<String, Connection> connectionCache = (Cache<String, Connection>) f.get(m);
+    return connectionCache.getIfPresent(id);
+  }
+
+  @Test public void testRemoteConnectionProperties() throws Exception {
+    try (AvaticaConnection conn = (AvaticaConnection) DriverManager.getConnection(url)) {
+      String id = conn.id;
+      final Map<String, ConnectionPropertiesImpl> m = ((RemoteMeta) getMeta(conn)).propsMap;
+      assertFalse("remote connection map should start ignorant", m.containsKey(id));
+      // force creating a connection object on the remote side.
+      try (final Statement stmt = conn.createStatement()) {
+        assertTrue("creating a statement starts a local object.", m.containsKey(id));
+        assertTrue(stmt.execute("select count(1) from EMP"));
+      }
+      Connection remoteConn = getConnection(FullyRemoteJdbcMetaFactory.getInstance(), id);
+      final boolean defaultRO = remoteConn.isReadOnly();
+      final boolean defaultAutoCommit = remoteConn.getAutoCommit();
+      final String defaultCatalog = remoteConn.getCatalog();
+      final String defaultSchema = remoteConn.getSchema();
+      conn.setReadOnly(!defaultRO);
+      assertTrue("local changes dirty local state", m.get(id).isDirty());
+      assertEquals("remote connection has not been touched", defaultRO, remoteConn.isReadOnly());
+      conn.setAutoCommit(!defaultAutoCommit);
+      assertEquals("remote connection has not been touched",
+          defaultAutoCommit, remoteConn.getAutoCommit());
+
+      // further interaction with the connection will force a sync
+      try (final Statement stmt = conn.createStatement()) {
+        assertEquals(!defaultAutoCommit, remoteConn.getAutoCommit());
+        assertFalse("local values should be clean", m.get(id).isDirty());
+      }
+    }
+  }
+}
+
+// End RemoteMetaTest.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/304eb70c/avatica/src/main/java/org/apache/calcite/avatica/AvaticaConnection.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaConnection.java b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaConnection.java
index f05907f..2c66d1e 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaConnection.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaConnection.java
@@ -49,14 +49,11 @@ import java.util.concurrent.Executor;
  * <p>Abstract to allow newer versions of JDBC to add methods.
  */
 public abstract class AvaticaConnection implements Connection {
+
   protected int statementCount;
-  private boolean autoCommit;
   private boolean closed;
-  private boolean readOnly;
-  private int transactionIsolation;
   private int holdability;
   private int networkTimeout;
-  private String catalog;
 
   public final String id;
   public final Meta.ConnectionHandle handle;
@@ -65,7 +62,6 @@ public abstract class AvaticaConnection implements Connection {
   final String url;
   protected final Properties info;
   protected final Meta meta;
-  private String schema;
   protected final AvaticaDatabaseMetaData metaData;
   public final Helper helper = Helper.INSTANCE;
   public final Map<InternalProperty, Object> properties = new HashMap<>();
@@ -132,11 +128,11 @@ public abstract class AvaticaConnection implements Connection {
   }
 
   public void setAutoCommit(boolean autoCommit) throws SQLException {
-    this.autoCommit = autoCommit;
+    meta.connectionSync(handle, new ConnectionPropertiesImpl().setAutoCommit(autoCommit));
   }
 
   public boolean getAutoCommit() throws SQLException {
-    return autoCommit;
+    return meta.connectionSync(handle, new ConnectionPropertiesImpl()).isAutoCommit();
   }
 
   public void commit() throws SQLException {
@@ -171,27 +167,27 @@ public abstract class AvaticaConnection implements Connection {
   }
 
   public void setReadOnly(boolean readOnly) throws SQLException {
-    this.readOnly = readOnly;
+    meta.connectionSync(handle, new ConnectionPropertiesImpl().setReadOnly(readOnly));
   }
 
   public boolean isReadOnly() throws SQLException {
-    return readOnly;
+    return meta.connectionSync(handle, new ConnectionPropertiesImpl()).isReadOnly();
   }
 
   public void setCatalog(String catalog) throws SQLException {
-    this.catalog = catalog;
+    meta.connectionSync(handle, new ConnectionPropertiesImpl().setCatalog(catalog));
   }
 
   public String getCatalog() {
-    return catalog;
+    return meta.connectionSync(handle, new ConnectionPropertiesImpl()).getCatalog();
   }
 
   public void setTransactionIsolation(int level) throws SQLException {
-    this.transactionIsolation = level;
+    meta.connectionSync(handle, new ConnectionPropertiesImpl().setTransactionIsolation(level));
   }
 
   public int getTransactionIsolation() throws SQLException {
-    return transactionIsolation;
+    return meta.connectionSync(handle, new ConnectionPropertiesImpl()).getTransactionIsolation();
   }
 
   public SQLWarning getWarnings() throws SQLException {
@@ -355,11 +351,11 @@ public abstract class AvaticaConnection implements Connection {
   }
 
   public void setSchema(String schema) throws SQLException {
-    this.schema = schema;
+    meta.connectionSync(handle, new ConnectionPropertiesImpl().setSchema(schema));
   }
 
   public String getSchema() {
-    return schema;
+    return meta.connectionSync(handle, new ConnectionPropertiesImpl()).getSchema();
   }
 
   public void abort(Executor executor) throws SQLException {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/304eb70c/avatica/src/main/java/org/apache/calcite/avatica/ConnectionPropertiesImpl.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/ConnectionPropertiesImpl.java b/avatica/src/main/java/org/apache/calcite/avatica/ConnectionPropertiesImpl.java
new file mode 100644
index 0000000..6086b35
--- /dev/null
+++ b/avatica/src/main/java/org/apache/calcite/avatica/ConnectionPropertiesImpl.java
@@ -0,0 +1,180 @@
+/*
+ * 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.avatica;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.sql.Connection;
+import java.sql.SQLException;
+
+/** Concrete implementation of {@link Meta.ConnectionProperties}. Provides additional state
+ * tracking to enable {@code RemoteMeta} to lazily push changes up to a query server.
+ *
+ * <p>{@code Meta} instances should probably hold authority on the {@code isDirty}
+ * flag because {@code AvaticaConnection} instances have no way of knowing if they're local or
+ * remote.
+ */
+public class ConnectionPropertiesImpl implements Meta.ConnectionProperties {
+  private boolean isDirty = false;
+  private Boolean autoCommit;
+  private Boolean readOnly;
+  private Integer transactionIsolation;
+  private String catalog;
+  private String schema;
+
+  // TODO: replace with Meta.ConnectionProperties$EMPTY instance?
+  public ConnectionPropertiesImpl() {}
+
+  public ConnectionPropertiesImpl(Connection conn) throws SQLException {
+    this(conn.getAutoCommit(), conn.isReadOnly(), conn.getTransactionIsolation(),
+        conn.getCatalog(), conn.getSchema());
+  }
+
+  @JsonCreator
+  public ConnectionPropertiesImpl(
+      @JsonProperty("autoCommit") Boolean autoCommit,
+      @JsonProperty("readOnly") Boolean readOnly,
+      @JsonProperty("transactionIsolation") Integer transactionIsolation,
+      @JsonProperty("catalog") String catalog,
+      @JsonProperty("schema") String schema) {
+    this.autoCommit = autoCommit;
+    this.readOnly = readOnly;
+    this.transactionIsolation = transactionIsolation;
+    this.catalog = catalog;
+    this.schema = schema;
+  }
+
+  public ConnectionPropertiesImpl setDirty(boolean val) {
+    this.isDirty = val;
+    return this;
+  }
+
+  public boolean isDirty() {
+    return this.isDirty;
+  }
+
+  @Override public boolean isEmpty() {
+    return autoCommit == null && readOnly == null && transactionIsolation == null
+        && catalog == null && schema == null;
+  }
+
+  /** Overwrites fields in {@code this} with any non-null fields in {@code that}. Sets
+   * {@code isDirty} if any fields are changed.
+   *
+   * @return {@code this}
+   */
+  @Override public ConnectionPropertiesImpl merge(Meta.ConnectionProperties that) {
+    if (this == that) {
+      return this;
+    }
+    if (that.isAutoCommit() != null && this.autoCommit != that.isAutoCommit()) {
+      this.autoCommit = that.isAutoCommit();
+      this.isDirty = true;
+    }
+    if (that.isReadOnly() != null && this.readOnly != that.isReadOnly()) {
+      this.readOnly = that.isReadOnly();
+      this.isDirty = true;
+    }
+    if (that.getTransactionIsolation() != null
+        && !that.getTransactionIsolation().equals(this.transactionIsolation)) {
+      this.transactionIsolation = that.getTransactionIsolation();
+      this.isDirty = true;
+    }
+    if (that.getCatalog() != null && !that.getCatalog().equalsIgnoreCase(this.catalog)) {
+      this.catalog = that.getCatalog();
+      this.isDirty = true;
+    }
+    if (that.getSchema() != null && !that.getSchema().equalsIgnoreCase(this.schema)) {
+      this.schema = that.getSchema();
+      this.isDirty = true;
+    }
+    return this;
+  }
+
+  /** Sets {@code autoCommit} status and flag as dirty.
+   *
+   * @return {@code this}
+   */
+  @Override public Meta.ConnectionProperties setAutoCommit(boolean val) {
+    this.autoCommit = val;
+    this.isDirty = true;
+    return this;
+  }
+
+  @Override public Boolean isAutoCommit() {
+    return this.autoCommit;
+  }
+
+  /** Sets {@code readOnly} status and flag as dirty.
+   *
+   * @return {@code this}
+   */
+  @Override public Meta.ConnectionProperties setReadOnly(boolean val) {
+    this.readOnly = val;
+    this.isDirty = true;
+    return this;
+  }
+
+  @Override public Boolean isReadOnly() {
+    return this.readOnly;
+  }
+
+  /** Sets {@code transactionIsolation} status and flag as dirty.
+   *
+   * @return {@code this}
+   */
+  @Override public Meta.ConnectionProperties setTransactionIsolation(int val) {
+    this.transactionIsolation = val;
+    this.isDirty = true;
+    return this;
+  }
+
+  public Integer getTransactionIsolation() {
+    return this.transactionIsolation;
+  }
+
+  /** Sets {@code catalog} and flag as dirty.
+   *
+   * @return {@code this}
+   */
+  @Override public Meta.ConnectionProperties setCatalog(String val) {
+    this.catalog = val;
+    this.isDirty = true;
+    return this;
+  }
+
+  @Override public String getCatalog() {
+    return this.catalog;
+  }
+
+  /** Sets {@code schema} and flag as dirty.
+   *
+   * @return {@code this}
+   */
+  @Override public Meta.ConnectionProperties setSchema(String val) {
+    this.schema = val;
+    this.isDirty = true;
+    return this;
+  }
+
+  public String getSchema() {
+    return this.schema;
+  }
+}
+
+// End ConnectionPropertiesImpl.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/304eb70c/avatica/src/main/java/org/apache/calcite/avatica/Meta.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/Meta.java b/avatica/src/main/java/org/apache/calcite/avatica/Meta.java
index ecd3ee3..49fc3a5 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/Meta.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/Meta.java
@@ -19,6 +19,8 @@ package org.apache.calcite.avatica;
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonSubTypes;
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
 
 import java.lang.reflect.Field;
 import java.sql.SQLException;
@@ -203,6 +205,14 @@ public interface Meta {
   /** Close a connection */
   void closeConnection(ConnectionHandle ch);
 
+  /** Sync client and server view of connection properties.
+   *
+   * <p>Note: this interface is considered "experimental" and may undergo further changes as this
+   * functionality is extended to other aspects of state management for
+   * {@link java.sql.Connection}, {@link java.sql.Statement}, and {@link java.sql.ResultSet}.</p>
+   */
+  ConnectionProperties connectionSync(ConnectionHandle ch, ConnectionProperties connProps);
+
   /** Factory to create instances of {@link Meta}. */
   interface Factory {
     Meta create(List<String> args);
@@ -471,6 +481,75 @@ public interface Meta {
     }
   }
 
+  /** A pojo containing various client-settable {@link java.sql.Connection} properties.
+   *
+   * <p>{@code java.lang} types are used here so that {@code null} can be used to indicate
+   * a value has no been set.</p>
+   *
+   * <p>Note: this interface is considered "experimental" and may undergo further changes as this
+   * functionality is extended to other aspects of state management for
+   * {@link java.sql.Connection}, {@link java.sql.Statement}, and {@link java.sql.ResultSet}.</p>
+   */
+  @JsonTypeInfo(
+      use = JsonTypeInfo.Id.NAME,
+      property = "connProps",
+      defaultImpl = ConnectionPropertiesImpl.class)
+  @JsonSubTypes({
+      @JsonSubTypes.Type(value = ConnectionPropertiesImpl.class, name = "connPropsImpl")
+  })
+  interface ConnectionProperties {
+
+    /** Overwrite fields in {@code this} with any non-null fields in {@code that}
+     *
+     * @return {@code this}
+     */
+    ConnectionProperties merge(ConnectionProperties that);
+
+    /** @return {@code true} when no properies have been set, {@code false} otherwise. */
+    @JsonIgnore
+    boolean isEmpty();
+
+    /** Set {@code autoCommit} status.
+     *
+     * @return {@code this}
+     */
+    ConnectionProperties setAutoCommit(boolean val);
+
+    Boolean isAutoCommit();
+
+    /** Set {@code readOnly} status.
+     *
+     * @return {@code this}
+     */
+    ConnectionProperties setReadOnly(boolean val);
+
+    Boolean isReadOnly();
+
+    /** Set {@code transactionIsolation} status.
+     *
+     * @return {@code this}
+     */
+    ConnectionProperties setTransactionIsolation(int val);
+
+    Integer getTransactionIsolation();
+
+    /** Set {@code catalog}.
+     *
+     * @return {@code this}
+     */
+    ConnectionProperties setCatalog(String val);
+
+    String getCatalog();
+
+    /** Set {@code schema}.
+     *
+     * @return {@code this}
+     */
+    ConnectionProperties setSchema(String val);
+
+    String getSchema();
+  }
+
   /** API to put a result set into a statement, being careful to enforce
    * thread-safety and not to overwrite existing open result sets. */
   interface PrepareCallback {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/304eb70c/avatica/src/main/java/org/apache/calcite/avatica/MetaImpl.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/MetaImpl.java b/avatica/src/main/java/org/apache/calcite/avatica/MetaImpl.java
index 3ebff75..b8820e8 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/MetaImpl.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/MetaImpl.java
@@ -52,10 +52,18 @@ import java.util.NoSuchElementException;
  * default metadata methods return empty collections.
  */
 public abstract class MetaImpl implements Meta {
+  /** The {@link AvaticaConnection} backing {@code this}. */
   protected final AvaticaConnection connection;
+  /** Represents the various states specific to {@link #connection}.
+   *
+   * <p>Note: this instance is used recursively with {@link #connection}'s getter and setter
+   * methods.</p>
+   */
+  protected final ConnectionPropertiesImpl connProps;
 
   public MetaImpl(AvaticaConnection connection) {
     this.connection = connection;
+    this.connProps = new ConnectionPropertiesImpl();
   }
 
   /** Uses a {@link org.apache.calcite.avatica.Meta.CursorFactory} to convert
@@ -175,6 +183,13 @@ public abstract class MetaImpl implements Meta {
 //    }
   }
 
+  @Override public ConnectionProperties connectionSync(ConnectionHandle ch,
+      ConnectionProperties connProps) {
+    this.connProps.merge(connProps);
+    this.connProps.setDirty(false);
+    return this.connProps;
+  }
+
   public StatementHandle createStatement(ConnectionHandle ch) {
     return new StatementHandle(ch.id, connection.statementCount++, null);
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/304eb70c/avatica/src/main/java/org/apache/calcite/avatica/remote/JsonService.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/remote/JsonService.java b/avatica/src/main/java/org/apache/calcite/avatica/remote/JsonService.java
index b8d640d..d8b94c7 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/remote/JsonService.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/JsonService.java
@@ -145,6 +145,14 @@ public abstract class JsonService implements Service {
       throw handle(e);
     }
   }
+
+  public ConnectionSyncResponse apply(ConnectionSyncRequest request) {
+    try {
+      return decode(apply(encode(request)), ConnectionSyncResponse.class);
+    } catch (IOException e) {
+      throw handle(e);
+    }
+  }
 }
 
 // End JsonService.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/304eb70c/avatica/src/main/java/org/apache/calcite/avatica/remote/LocalService.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/remote/LocalService.java b/avatica/src/main/java/org/apache/calcite/avatica/remote/LocalService.java
index 842ebd6..5bbfa36 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/remote/LocalService.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/LocalService.java
@@ -201,6 +201,12 @@ public class LocalService implements Service {
     meta.closeConnection(new Meta.ConnectionHandle(request.connectionId));
     return new CloseConnectionResponse();
   }
+
+  public ConnectionSyncResponse apply(ConnectionSyncRequest request) {
+    final Meta.ConnectionProperties connProps =
+        meta.connectionSync(new Meta.ConnectionHandle(request.connectionId), request.connProps);
+    return new ConnectionSyncResponse(connProps);
+  }
 }
 
 // End LocalService.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/304eb70c/avatica/src/main/java/org/apache/calcite/avatica/remote/RemoteMeta.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/remote/RemoteMeta.java b/avatica/src/main/java/org/apache/calcite/avatica/remote/RemoteMeta.java
index 772321c..93e5541 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/remote/RemoteMeta.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/RemoteMeta.java
@@ -19,18 +19,22 @@ package org.apache.calcite.avatica.remote;
 import org.apache.calcite.avatica.AvaticaConnection;
 import org.apache.calcite.avatica.AvaticaParameter;
 import org.apache.calcite.avatica.ColumnMetaData;
+import org.apache.calcite.avatica.ConnectionPropertiesImpl;
 import org.apache.calcite.avatica.Meta;
 import org.apache.calcite.avatica.MetaImpl;
 
 import java.sql.SQLException;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 /**
  * Implementation of {@link Meta} for the remote driver.
  */
 class RemoteMeta extends MetaImpl {
   final Service service;
+  final Map<String, ConnectionPropertiesImpl> propsMap = new HashMap<>();
 
   public RemoteMeta(AvaticaConnection connection, Service service) {
     super(connection);
@@ -53,6 +57,7 @@ class RemoteMeta extends MetaImpl {
   }
 
   @Override public StatementHandle createStatement(ConnectionHandle ch) {
+    connectionSync(ch, new ConnectionPropertiesImpl()); // sync connection state if necessary
     final Service.CreateStatementResponse response =
         service.apply(new Service.CreateStatementRequest(ch.id));
     return new StatementHandle(response.connectionId, response.statementId,
@@ -67,6 +72,30 @@ class RemoteMeta extends MetaImpl {
   @Override public void closeConnection(ConnectionHandle ch) {
     final Service.CloseConnectionResponse response =
         service.apply(new Service.CloseConnectionRequest(ch.id));
+    propsMap.remove(ch.id);
+  }
+
+  @Override public ConnectionProperties connectionSync(ConnectionHandle ch,
+      ConnectionProperties connProps) {
+    ConnectionPropertiesImpl localProps = propsMap.get(ch.id);
+    if (localProps == null) {
+      localProps = new ConnectionPropertiesImpl();
+      localProps.setDirty(true);
+      propsMap.put(ch.id, localProps);
+    }
+
+    // Only make an RPC if necessary. RPC is necessary when we have local changes that need
+    // flushed to the server (be sure to introduce any new changes from connProps before checking
+    // AND when connProps.isEmpty() (meaning, this was a request for a value, not overriding a
+    // value). Otherwise, accumulate the change locally and return immediately.
+    if (localProps.merge(connProps).isDirty() && connProps.isEmpty()) {
+      final Service.ConnectionSyncResponse response = service.apply(
+          new Service.ConnectionSyncRequest(ch.id, localProps));
+      propsMap.put(ch.id, (ConnectionPropertiesImpl) response.connProps);
+      return response.connProps;
+    } else {
+      return localProps;
+    }
   }
 
   @Override public MetaResultSet getCatalogs() {
@@ -107,6 +136,7 @@ class RemoteMeta extends MetaImpl {
 
   @Override public StatementHandle prepare(ConnectionHandle ch, String sql,
       int maxRowCount) {
+    connectionSync(ch, new ConnectionPropertiesImpl()); // sync connection state if necessary
     final Service.PrepareResponse response = service.apply(
         new Service.PrepareRequest(ch.id, sql, maxRowCount));
     return response.statement;
@@ -114,6 +144,7 @@ class RemoteMeta extends MetaImpl {
 
   @Override public MetaResultSet prepareAndExecute(ConnectionHandle ch,
       String sql, int maxRowCount, PrepareCallback callback) {
+    connectionSync(ch, new ConnectionPropertiesImpl()); // sync connection state if necessary
     final Service.ResultSetResponse response;
     try {
       synchronized (callback.getMonitor()) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/304eb70c/avatica/src/main/java/org/apache/calcite/avatica/remote/Service.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/remote/Service.java b/avatica/src/main/java/org/apache/calcite/avatica/remote/Service.java
index c56524d..c85e53a 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/remote/Service.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/Service.java
@@ -41,6 +41,7 @@ public interface Service {
   CreateStatementResponse apply(CreateStatementRequest request);
   CloseStatementResponse apply(CloseStatementRequest request);
   CloseConnectionResponse apply(CloseConnectionRequest request);
+  ConnectionSyncResponse apply(ConnectionSyncRequest request);
 
   /** Factory that creates a {@code Service}. */
   interface Factory {
@@ -68,7 +69,8 @@ public interface Service {
       @JsonSubTypes.Type(value = CloseStatementRequest.class,
           name = "closeStatement"),
       @JsonSubTypes.Type(value = CloseConnectionRequest.class,
-          name = "closeConnection") })
+          name = "closeConnection"),
+      @JsonSubTypes.Type(value = ConnectionSyncRequest.class, name = "connectionSync") })
   abstract class Request {
     abstract Response accept(Service service);
   }
@@ -87,7 +89,8 @@ public interface Service {
       @JsonSubTypes.Type(value = CloseStatementResponse.class,
           name = "closeStatement"),
       @JsonSubTypes.Type(value = CloseConnectionResponse.class,
-          name = "closeConnection") })
+          name = "closeConnection"),
+      @JsonSubTypes.Type(value = ConnectionSyncResponse.class, name = "connectionSync") })
   abstract class Response {
   }
 
@@ -384,6 +387,35 @@ public interface Service {
     @JsonCreator
     public CloseConnectionResponse() {}
   }
+
+  /** Request for {@link Meta#connectionSync(Meta.ConnectionHandle, Meta.ConnectionProperties)}. */
+  class ConnectionSyncRequest extends Request {
+    public final String connectionId;
+    public final Meta.ConnectionProperties connProps;
+
+    @JsonCreator
+    public ConnectionSyncRequest(
+        @JsonProperty("connectionId") String connectionId,
+        @JsonProperty("connProps") Meta.ConnectionProperties connProps) {
+      this.connectionId = connectionId;
+      this.connProps = connProps;
+    }
+
+    @Override ConnectionSyncResponse accept(Service service) {
+      return service.apply(this);
+    }
+  }
+
+  /** Response for
+   * {@link Meta#connectionSync(Meta.ConnectionHandle, Meta.ConnectionProperties)}. */
+  class ConnectionSyncResponse extends Response {
+    public final Meta.ConnectionProperties connProps;
+
+    @JsonCreator
+    public ConnectionSyncResponse(@JsonProperty("connProps") Meta.ConnectionProperties connProps) {
+      this.connProps = connProps;
+    }
+  }
 }
 
 // End Service.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/304eb70c/core/src/main/java/org/apache/calcite/jdbc/CalciteMetaImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/CalciteMetaImpl.java b/core/src/main/java/org/apache/calcite/jdbc/CalciteMetaImpl.java
index 849ef63..2a9c4ad 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/CalciteMetaImpl.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/CalciteMetaImpl.java
@@ -52,6 +52,7 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 
 import java.lang.reflect.Field;
+import java.sql.Connection;
 import java.sql.DatabaseMetaData;
 import java.sql.SQLException;
 import java.util.ArrayList;
@@ -69,6 +70,11 @@ public class CalciteMetaImpl extends MetaImpl {
 
   public CalciteMetaImpl(CalciteConnectionImpl connection) {
     super(connection);
+    this.connProps
+        .setAutoCommit(false)
+        .setReadOnly(false)
+        .setTransactionIsolation(Connection.TRANSACTION_NONE);
+    this.connProps.setDirty(false);
   }
 
   static <T extends Named> Predicate1<T> namedMatcher(final Pat pattern) {


[06/10] incubator-calcite git commit: [CALCITE-654] Tighten up AvaticaStatement#execute semantics (Nick Dimiduk)

Posted by jh...@apache.org.
[CALCITE-654] Tighten up AvaticaStatement#execute semantics (Nick Dimiduk)


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

Branch: refs/heads/master
Commit: 0ea085b7858cbf0e8b250ac238d4f1def1ff20f5
Parents: ba07b09
Author: Nick Dimiduk <nd...@gmail.com>
Authored: Mon Mar 30 17:42:26 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Apr 3 01:10:05 2015 -0700

----------------------------------------------------------------------
 .../apache/calcite/avatica/jdbc/JdbcMeta.java   | 21 +++++++++--
 .../calcite/avatica/RemoteDriverTest.java       | 26 ++++++++++++--
 .../calcite/avatica/AvaticaStatement.java       | 37 +++++++++++++-------
 3 files changed, 66 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/0ea085b7/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
----------------------------------------------------------------------
diff --git a/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java b/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
index 90e8c2a..b5fb561 100644
--- a/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
+++ b/avatica-server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
@@ -708,13 +708,28 @@ public class JdbcMeta implements Meta {
       final PreparedStatement statement = connection.prepareStatement(sql);
       final int id = System.identityHashCode(statement);
       final StatementInfo info = new StatementInfo(statement);
-      statementCache.put(id, info);
-      info.resultSet = statement.executeQuery();
-      MetaResultSet mrs = JdbcResultSet.create(ch.id, id, info.resultSet);
+      statementCache.put(id, info); // TODO: must we retain a statement in all cases?
+      boolean ret = statement.execute();
+      info.resultSet = statement.getResultSet();
+      assert ret || info.resultSet == null;
+      final MetaResultSet mrs;
+      if (info.resultSet == null) {
+        // build a non-JDBC result that contains the update count
+        int updateCount = statement.getUpdateCount();
+        List<ColumnMetaData> columns = new ArrayList<>(1);
+        columns.add(UPDATE_COL);
+        List<Object> val = new ArrayList<>();
+        val.add(new Object[] { updateCount });
+        mrs = new MetaResultSet(ch.id, id, true, new Signature(columns, sql, null, null,
+            CursorFactory.ARRAY), new Frame(0, true, val));
+      } else {
+        mrs = JdbcResultSet.create(ch.id, id, info.resultSet);
+      }
       if (LOG.isTraceEnabled()) {
         StatementHandle h = new StatementHandle(ch.id, id, null);
         LOG.trace("prepAndExec statement " + h);
       }
+      // TODO: review client to ensure statementId is updated when appropriate
       return mrs;
     } catch (SQLException e) {
       throw propagate(e);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/0ea085b7/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
----------------------------------------------------------------------
diff --git a/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java b/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
index a2ab87a..2b9c111 100644
--- a/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
+++ b/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
@@ -152,7 +152,6 @@ public class RemoteDriverTest {
     checkStatementExecuteQuery(mjs(), false);
   }
 
-  @Ignore
   @Test public void testPrepareExecuteQueryLocal() throws Exception {
     checkStatementExecuteQuery(ljs(), true);
   }
@@ -180,7 +179,7 @@ public class RemoteDriverTest {
       resultSet = statement.executeQuery(sql);
     }
     if (parameterMetaData != null) {
-      assertThat(parameterMetaData.getParameterCount(), equalTo(2));
+      assertThat(parameterMetaData.getParameterCount(), equalTo(0));
     }
     final ResultSetMetaData metaData = resultSet.getMetaData();
     assertEquals(2, metaData.getColumnCount());
@@ -195,6 +194,27 @@ public class RemoteDriverTest {
     connection.close();
   }
 
+  @Test public void testCreateInsertUpdateDrop() throws Exception {
+    final String drop = "drop table TEST_TABLE if exists";
+    final String create = "create table TEST_TABLE("
+        + "id int not null, "
+        + "msg varchar(3) not null)";
+    final String insert = "insert into TEST_TABLE values(1, 'foo')";
+    final String update = "update TEST_TABLE set msg='bar' where id=1";
+    try (Connection connection = ljs();
+        Statement statement = connection.createStatement()) {
+      boolean ret;
+      assertFalse(statement.execute(drop));
+      assertEquals(0, statement.getUpdateCount());
+      assertFalse(statement.execute(create));
+      assertEquals(0, statement.getUpdateCount());
+      assertFalse(statement.execute(insert));
+      assertEquals(1, statement.getUpdateCount());
+      assertFalse(statement.execute(update));
+      assertEquals(0, statement.executeUpdate(drop));
+    }
+  }
+
   @Test public void testStatementLifecycle() throws Exception {
     try (AvaticaConnection connection = (AvaticaConnection) ljs()) {
       Map<Integer, AvaticaStatement> clientMap = connection.statementMap;
@@ -312,6 +332,8 @@ public class RemoteDriverTest {
     connection.close();
   }
 
+
+
   /**
    * Factory that creates a service based on a local JDBC connection.
    */

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/0ea085b7/avatica/src/main/java/org/apache/calcite/avatica/AvaticaStatement.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaStatement.java b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaStatement.java
index 8276b07..1a7e5ec 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaStatement.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaStatement.java
@@ -17,6 +17,7 @@
 package org.apache.calcite.avatica;
 
 import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
 import java.sql.SQLWarning;
 import java.sql.Statement;
@@ -55,6 +56,7 @@ public abstract class AvaticaStatement
   private int fetchSize;
   private int fetchDirection;
   protected int maxRowCount = 0;
+  private int updateCount = -1;
 
   /**
    * Creates an AvaticaStatement.
@@ -89,21 +91,30 @@ public abstract class AvaticaStatement
   // implement Statement
 
   public boolean execute(String sql) throws SQLException {
-    try {
-      // In JDBC, maxRowCount = 0 means no limit; in prepare it means LIMIT 0
-      final int maxRowCount1 = maxRowCount <= 0 ? -1 : maxRowCount;
-      ResultSet resultSet =
-          connection.prepareAndExecuteInternal(this, sql, maxRowCount1);
-      if (resultSet.isClosed()) {
-        return false;
+    ResultSet resultSet = executeQuery(sql);
+    ResultSetMetaData md = resultSet.getMetaData();
+    // hackish, but be sure we're looking at an update count result, not user data
+    if (md.getCatalogName(1).equalsIgnoreCase("avatica_internal")
+        && md.getTableName(1).equalsIgnoreCase("update_result")
+        && md.getColumnCount() == 1
+        && md.getColumnName(1).equalsIgnoreCase("u")) {
+      if (!resultSet.next()) {
+        throw new SQLException("expected one row, got zero");
       }
-      return true;
-    } catch (RuntimeException e) {
-      throw connection.helper.createException("while executing SQL: " + sql, e);
+      this.updateCount = resultSet.getInt(1);
+      if (resultSet.next()) {
+        throw new SQLException("expected one row, got two or more");
+      }
+      resultSet.close();
+      return false;
+    } else {
+      return !resultSet.isClosed();
     }
   }
 
   public ResultSet executeQuery(String sql) throws SQLException {
+    // reset previous state before moving forward.
+    this.updateCount = -1;
     try {
       // In JDBC, maxRowCount = 0 means no limit; in prepare it means LIMIT 0
       final int maxRowCount1 = maxRowCount <= 0 ? -1 : maxRowCount;
@@ -122,12 +133,12 @@ public abstract class AvaticaStatement
     if (!resultSet.next()) {
       throw new SQLException("expected one row, got zero");
     }
-    int result = resultSet.getInt(1);
+    this.updateCount = resultSet.getInt(1);
     if (resultSet.next()) {
       throw new SQLException("expected one row, got two or more");
     }
     resultSet.close();
-    return result;
+    return this.updateCount;
   }
 
   public synchronized void close() throws SQLException {
@@ -240,7 +251,7 @@ public abstract class AvaticaStatement
   }
 
   public int getUpdateCount() throws SQLException {
-    return -1;
+    return updateCount;
   }
 
   public boolean getMoreResults() throws SQLException {


[04/10] incubator-calcite git commit: [CALCITE-658] Cleanup dependency usage (Nick Dimiduk)

Posted by jh...@apache.org.
[CALCITE-658] Cleanup dependency usage (Nick Dimiduk)

Close apache/incubator-calcite#70


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

Branch: refs/heads/master
Commit: ba07b09aae2c5e09acfeb3ad77a4a6e4ddfb3ac4
Parents: fa9bdc4
Author: Nick Dimiduk <nd...@gmail.com>
Authored: Tue Mar 31 17:25:53 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Apr 3 01:09:49 2015 -0700

----------------------------------------------------------------------
 avatica-server/pom.xml | 23 +++++++++++++++++++++++
 core/pom.xml           | 41 +++++++++++++++++++++++++++++++++++++++++
 example/csv/pom.xml    | 16 ++++++++++++----
 linq4j/pom.xml         |  5 +++++
 mongodb/pom.xml        | 13 +++++++++----
 plus/pom.xml           |  9 +++++----
 pom.xml                | 22 ++++++++++++++++++++++
 spark/pom.xml          | 44 ++++++++++++++++++++++++++++++++++++++++----
 splunk/pom.xml         |  9 +++++++++
 ubenchmark/pom.xml     | 26 ++++++++++++++++++++++----
 10 files changed, 188 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/ba07b09a/avatica-server/pom.xml
----------------------------------------------------------------------
diff --git a/avatica-server/pom.xml b/avatica-server/pom.xml
index 2da32ec..f45366e 100644
--- a/avatica-server/pom.xml
+++ b/avatica-server/pom.xml
@@ -120,6 +120,29 @@ limitations under the License.
           </execution>
         </executions>
       </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <version>2.10</version>
+        <!-- configurations do not cascade, so all of the definition from
+             ../pom.xml:build:plugin-management:plugins:plugin must be repeated in child poms -->
+        <executions>
+          <execution>
+            <id>analyze</id>
+            <goals>
+              <goal>analyze-only</goal>
+            </goals>
+            <configuration>
+              <failOnWarning>true</failOnWarning>
+              <!-- ignore "unused but declared" warnings -->
+              <ignoredUnusedDeclaredDependencies>
+                <ignoredUnusedDeclaredDependency>net.hydromatic:scott-data-hsqldb</ignoredUnusedDeclaredDependency>
+                <ignoredUnusedDeclaredDependency>org.hsqldb:hsqldb</ignoredUnusedDeclaredDependency>
+              </ignoredUnusedDeclaredDependencies>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
   </build>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/ba07b09a/core/pom.xml
----------------------------------------------------------------------
diff --git a/core/pom.xml b/core/pom.xml
index 5533984..9edb0dc 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -57,6 +57,14 @@ limitations under the License.
     </dependency>
     <dependency>
       <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-annotations</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
       <artifactId>jackson-databind</artifactId>
     </dependency>
     <dependency>
@@ -64,6 +72,10 @@ limitations under the License.
       <artifactId>jsr305</artifactId>
     </dependency>
     <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>
@@ -116,6 +128,11 @@ limitations under the License.
       <scope>test</scope>
     </dependency>
     <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-core</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>org.pentaho</groupId>
       <artifactId>pentaho-aggdesigner-algorithm</artifactId>
     </dependency>
@@ -200,6 +217,30 @@ limitations under the License.
           </execution>
         </executions>
       </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <version>2.10</version>
+        <!-- configurations do not cascade, so all of the definition from
+             ../pom.xml:build:plugin-management:plugins:plugin must be repeated in child poms -->
+        <executions>
+          <execution>
+            <id>analyze</id>
+            <goals>
+              <goal>analyze-only</goal>
+            </goals>
+            <configuration>
+              <failOnWarning>true</failOnWarning>
+              <!-- ignore "unused but declared" warnings -->
+              <ignoredUnusedDeclaredDependencies>
+                <ignoredUnusedDeclaredDependency>net.hydromatic:scott-data-hsqldb</ignoredUnusedDeclaredDependency>
+                <ignoredUnusedDeclaredDependency>net.hydromatic:foodmart-data-hsqldb</ignoredUnusedDeclaredDependency>
+                <ignoredUnusedDeclaredDependency>mysql:mysql-connector-java</ignoredUnusedDeclaredDependency>
+              </ignoredUnusedDeclaredDependencies>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
 
     <resources>

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/ba07b09a/example/csv/pom.xml
----------------------------------------------------------------------
diff --git a/example/csv/pom.xml b/example/csv/pom.xml
index ed6a457..fa2995e 100644
--- a/example/csv/pom.xml
+++ b/example/csv/pom.xml
@@ -37,10 +37,6 @@ limitations under the License.
   <dependencies>
     <dependency>
       <groupId>org.apache.calcite</groupId>
-      <artifactId>calcite-avatica</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.calcite</groupId>
       <artifactId>calcite-core</artifactId>
     </dependency>
     <dependency>
@@ -49,6 +45,18 @@ limitations under the License.
     </dependency>
 
     <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/ba07b09a/linq4j/pom.xml
----------------------------------------------------------------------
diff --git a/linq4j/pom.xml b/linq4j/pom.xml
index 528df06..67e7592 100644
--- a/linq4j/pom.xml
+++ b/linq4j/pom.xml
@@ -40,6 +40,11 @@ limitations under the License.
       <artifactId>guava</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-core</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/ba07b09a/mongodb/pom.xml
----------------------------------------------------------------------
diff --git a/mongodb/pom.xml b/mongodb/pom.xml
index 58ee276..bbed40d 100644
--- a/mongodb/pom.xml
+++ b/mongodb/pom.xml
@@ -38,10 +38,6 @@ limitations under the License.
          in dependencyManagement in the root POM, not here. -->
     <dependency>
       <groupId>org.apache.calcite</groupId>
-      <artifactId>calcite-avatica</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.calcite</groupId>
       <artifactId>calcite-core</artifactId>
       <type>jar</type>
     </dependency>
@@ -57,11 +53,20 @@ limitations under the License.
     </dependency>
 
     <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-core</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>org.mongodb</groupId>
       <artifactId>mongo-java-driver</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/ba07b09a/plus/pom.xml
----------------------------------------------------------------------
diff --git a/plus/pom.xml b/plus/pom.xml
index f951be2..d5b4510 100644
--- a/plus/pom.xml
+++ b/plus/pom.xml
@@ -38,10 +38,6 @@ limitations under the License.
          in dependencyManagement in the root POM, not here. -->
     <dependency>
       <groupId>org.apache.calcite</groupId>
-      <artifactId>calcite-avatica</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.calcite</groupId>
       <artifactId>calcite-core</artifactId>
       <type>jar</type>
     </dependency>
@@ -77,6 +73,11 @@ limitations under the License.
       <artifactId>tpcds</artifactId>
       <type>jar</type>
     </dependency>
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-core</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/ba07b09a/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index cb7217e..be63e3f 100644
--- a/pom.xml
+++ b/pom.xml
@@ -217,6 +217,11 @@ limitations under the License.
         <version>3.2</version>
       </dependency>
       <dependency>
+        <groupId>org.scala-lang</groupId>
+        <artifactId>scala-library</artifactId>
+        <version>2.10.3</version>
+      </dependency>
+      <dependency>
         <groupId>org.codehaus.janino</groupId>
         <artifactId>janino</artifactId>
         <version>2.7.6</version>
@@ -335,6 +340,23 @@ limitations under the License.
         </executions>
       </plugin>
       <plugin>
+        <!-- override default version 2.8 for access to additional config settings -->
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <version>2.10</version>
+        <executions>
+          <execution>
+            <id>analyze</id>
+            <goals>
+              <goal>analyze-only</goal>
+            </goals>
+            <configuration>
+              <failOnWarning>true</failOnWarning>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
         <!-- Override apache parent POM's definition of release
              plugin. If we don't specify gitexe version, git doesn't
              commit during release process. -->

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/ba07b09a/spark/pom.xml
----------------------------------------------------------------------
diff --git a/spark/pom.xml b/spark/pom.xml
index 88370ff..985f465 100644
--- a/spark/pom.xml
+++ b/spark/pom.xml
@@ -37,10 +37,6 @@ limitations under the License.
          in dependencyManagement in the root POM, not here. -->
     <dependency>
       <groupId>org.apache.calcite</groupId>
-      <artifactId>calcite-avatica</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.calcite</groupId>
       <artifactId>calcite-core</artifactId>
     </dependency>
     <dependency>
@@ -55,10 +51,27 @@ limitations under the License.
     </dependency>
 
     <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>org.apache.spark</groupId>
       <artifactId>spark-core_2.10</artifactId>
       <version>0.9.0-incubating</version>
     </dependency>
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-util</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.scala-lang</groupId>
+      <artifactId>scala-library</artifactId>
+    </dependency>
     <!-- Use explicit xerces and xalan. Without them, Hadoop gives
       "Failed to set setXIncludeAware(true)" error. -->
     <dependency>
@@ -126,6 +139,29 @@ limitations under the License.
           </excludes>
         </configuration>
       </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <version>2.10</version>
+        <!-- configurations do not cascade, so all of the definition from
+             ../pom.xml:build:plugin-management:plugins:plugin must be repeated in child poms -->
+        <executions>
+          <execution>
+            <id>analyze</id>
+            <goals>
+              <goal>analyze-only</goal>
+            </goals>
+            <configuration>
+              <failOnWarning>true</failOnWarning>
+              <!-- ignore "unused but declared" warnings -->
+              <ignoredUnusedDeclaredDependencies>
+                <ignoredUnusedDeclaredDependency>xerces:xercesImpl</ignoredUnusedDeclaredDependency>
+                <ignoredUnusedDeclaredDependency>xalan:xalan</ignoredUnusedDeclaredDependency>
+              </ignoredUnusedDeclaredDependencies>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
   </build>
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/ba07b09a/splunk/pom.xml
----------------------------------------------------------------------
diff --git a/splunk/pom.xml b/splunk/pom.xml
index 5f6a805..4f331ca 100644
--- a/splunk/pom.xml
+++ b/splunk/pom.xml
@@ -56,6 +56,10 @@ limitations under the License.
     </dependency>
 
     <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>
@@ -64,6 +68,11 @@ limitations under the License.
       <groupId>net.sf.opencsv</groupId>
       <artifactId>opencsv</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-core</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/ba07b09a/ubenchmark/pom.xml
----------------------------------------------------------------------
diff --git a/ubenchmark/pom.xml b/ubenchmark/pom.xml
index 3994425..fcd4deb 100644
--- a/ubenchmark/pom.xml
+++ b/ubenchmark/pom.xml
@@ -45,10 +45,6 @@ limitations under the License.
       <groupId>org.apache.calcite</groupId>
       <artifactId>calcite-core</artifactId>
     </dependency>
-    <dependency>
-      <groupId>org.apache.calcite</groupId>
-      <artifactId>calcite-linq4j</artifactId>
-    </dependency>
 
     <dependency>
       <groupId>org.openjdk.jmh</groupId>
@@ -107,6 +103,28 @@ limitations under the License.
           </execution>
         </executions>
       </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <version>2.10</version>
+        <!-- configurations do not cascade, so all of the definition from
+             ../pom.xml:build:plugin-management:plugins:plugin must be repeated in child poms -->
+        <executions>
+          <execution>
+            <id>analyze</id>
+            <goals>
+              <goal>analyze-only</goal>
+            </goals>
+            <configuration>
+              <failOnWarning>true</failOnWarning>
+              <!-- ignore "unused but declared" warnings -->
+              <ignoredUnusedDeclaredDependencies>
+                <ignoredUnusedDeclaredDependency>org.openjdk.jmh:jmh-generator-annprocess</ignoredUnusedDeclaredDependency>
+              </ignoredUnusedDeclaredDependencies>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
   </build>
 </project>


[02/10] incubator-calcite git commit: [CALCITE-652] Move server pieces of avatica into avatica-server (Nick Dimiduk)

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/fa9bdc4a/avatica/src/main/java/org/apache/calcite/avatica/jdbc/JdbcResultSet.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/jdbc/JdbcResultSet.java b/avatica/src/main/java/org/apache/calcite/avatica/jdbc/JdbcResultSet.java
deleted file mode 100644
index 827f31d..0000000
--- a/avatica/src/main/java/org/apache/calcite/avatica/jdbc/JdbcResultSet.java
+++ /dev/null
@@ -1,106 +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.avatica.jdbc;
-
-import org.apache.calcite.avatica.Meta;
-
-import java.sql.ResultSet;
-import java.sql.ResultSetMetaData;
-import java.sql.SQLException;
-import java.sql.Types;
-import java.util.ArrayList;
-import java.util.List;
-
-/** Implementation of {@link org.apache.calcite.avatica.Meta.MetaResultSet}
- *  upon a JDBC {@link java.sql.ResultSet}.
- *
- *  @see org.apache.calcite.avatica.jdbc.JdbcMeta */
-class JdbcResultSet extends Meta.MetaResultSet {
-  protected JdbcResultSet(String connectionId, int statementId,
-      boolean ownStatement, Meta.Signature signature, Meta.Frame firstFrame) {
-    super(connectionId, statementId, ownStatement, signature, firstFrame);
-  }
-
-  /** Creates a result set. */
-  public static JdbcResultSet create(String connectionId, int statementId,
-      ResultSet resultSet) {
-    try {
-      Meta.Signature sig = JdbcMeta.signature(resultSet.getMetaData());
-      final Meta.Frame firstFrame = frame(resultSet, 0, -1);
-      resultSet.close();
-      return new JdbcResultSet(connectionId, statementId, true, sig,
-          firstFrame);
-    } catch (SQLException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  /** Creates a frame containing a given number or unlimited number of rows
-   * from a result set. */
-  static Meta.Frame frame(ResultSet resultSet, int offset,
-      int fetchMaxRowCount) throws SQLException {
-    final ResultSetMetaData metaData = resultSet.getMetaData();
-    final int columnCount = metaData.getColumnCount();
-    final int[] types = new int[columnCount];
-    for (int i = 0; i < types.length; i++) {
-      types[i] = metaData.getColumnType(i + 1);
-    }
-    final List<Object> rows = new ArrayList<>();
-    boolean done = false;
-    for (int i = 0; fetchMaxRowCount < 0 || i < fetchMaxRowCount; i++) {
-      if (!resultSet.next()) {
-        done = true;
-        break;
-      }
-      Object[] columns = new Object[columnCount];
-      for (int j = 0; j < columnCount; j++) {
-        columns[j] = getValue(resultSet, types[j], j);
-      }
-      rows.add(columns);
-    }
-    return new Meta.Frame(offset, done, rows);
-  }
-
-  private static Object getValue(ResultSet resultSet, int type, int j)
-      throws SQLException {
-    switch (type) {
-    case Types.BIGINT:
-      final long aLong = resultSet.getLong(j + 1);
-      return aLong == 0 && resultSet.wasNull() ? null : aLong;
-    case Types.INTEGER:
-      final int anInt = resultSet.getInt(j + 1);
-      return anInt == 0 && resultSet.wasNull() ? null : anInt;
-    case Types.SMALLINT:
-      final short aShort = resultSet.getShort(j + 1);
-      return aShort == 0 && resultSet.wasNull() ? null : aShort;
-    case Types.TINYINT:
-      final byte aByte = resultSet.getByte(j + 1);
-      return aByte == 0 && resultSet.wasNull() ? null : aByte;
-    case Types.DOUBLE:
-    case Types.FLOAT:
-      final double aDouble = resultSet.getDouble(j + 1);
-      return aDouble == 0D && resultSet.wasNull() ? null : aDouble;
-    case Types.REAL:
-      final float aFloat = resultSet.getFloat(j + 1);
-      return aFloat == 0D && resultSet.wasNull() ? null : aFloat;
-    default:
-      return resultSet.getObject(j + 1);
-    }
-  }
-}
-
-// End JdbcResultSet.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/fa9bdc4a/avatica/src/main/java/org/apache/calcite/avatica/jdbc/package-info.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/jdbc/package-info.java b/avatica/src/main/java/org/apache/calcite/avatica/jdbc/package-info.java
deleted file mode 100644
index 8b8fb76..0000000
--- a/avatica/src/main/java/org/apache/calcite/avatica/jdbc/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/** Implements an Avatica provider on top of an existing JDBC data source. */
-package org.apache.calcite.avatica.jdbc;
-
-
-// End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/fa9bdc4a/avatica/src/test/java/org/apache/calcite/avatica/test/AvaticaSuite.java
----------------------------------------------------------------------
diff --git a/avatica/src/test/java/org/apache/calcite/avatica/test/AvaticaSuite.java b/avatica/src/test/java/org/apache/calcite/avatica/test/AvaticaSuite.java
deleted file mode 100644
index 3f25225..0000000
--- a/avatica/src/test/java/org/apache/calcite/avatica/test/AvaticaSuite.java
+++ /dev/null
@@ -1,33 +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.avatica.test;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
-
-/**
- * Avatica test suite.
- */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-    ConnectStringParserTest.class,
-    RemoteDriverTest.class
-})
-public class AvaticaSuite {
-}
-
-// End AvaticaSuite.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/fa9bdc4a/avatica/src/test/java/org/apache/calcite/avatica/test/RemoteDriverTest.java
----------------------------------------------------------------------
diff --git a/avatica/src/test/java/org/apache/calcite/avatica/test/RemoteDriverTest.java b/avatica/src/test/java/org/apache/calcite/avatica/test/RemoteDriverTest.java
deleted file mode 100644
index 0f6ab8d..0000000
--- a/avatica/src/test/java/org/apache/calcite/avatica/test/RemoteDriverTest.java
+++ /dev/null
@@ -1,435 +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.avatica.test;
-
-import org.apache.calcite.avatica.AvaticaConnection;
-import org.apache.calcite.avatica.AvaticaPreparedStatement;
-import org.apache.calcite.avatica.AvaticaStatement;
-import org.apache.calcite.avatica.Meta;
-import org.apache.calcite.avatica.jdbc.JdbcMeta;
-import org.apache.calcite.avatica.remote.LocalJsonService;
-import org.apache.calcite.avatica.remote.LocalService;
-import org.apache.calcite.avatica.remote.MockJsonService;
-import org.apache.calcite.avatica.remote.Service;
-
-import com.google.common.cache.Cache;
-
-import net.hydromatic.scott.data.hsqldb.ScottHsqldb;
-
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-
-import java.lang.reflect.Field;
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.ParameterMetaData;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.ResultSetMetaData;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.Map;
-
-import static org.hamcrest.CoreMatchers.equalTo;
-import static org.hamcrest.CoreMatchers.is;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-/**
- * Unit test for Avatica Remote JDBC driver.
- */
-public class RemoteDriverTest {
-  public static final String MJS =
-      MockJsonService.Factory.class.getName();
-
-  public static final String LJS =
-      LocalJdbcServiceFactory.class.getName();
-
-  public static final String QRJS =
-      QuasiRemoteJdbcServiceFactory.class.getName();
-
-  private static final ConnectionSpec CONNECTION_SPEC = ConnectionSpec.HSQLDB;
-
-  private Connection mjs() throws SQLException {
-    return DriverManager.getConnection("jdbc:avatica:remote:factory=" + MJS);
-  }
-
-  private Connection ljs() throws SQLException {
-    return DriverManager.getConnection("jdbc:avatica:remote:factory=" + QRJS);
-  }
-
-  @Before
-  public void before() throws Exception {
-    QuasiRemoteJdbcServiceFactory.initService();
-  }
-
-  @Test public void testRegister() throws Exception {
-    final Connection connection =
-        DriverManager.getConnection("jdbc:avatica:remote:");
-    assertThat(connection.isClosed(), is(false));
-    connection.close();
-    assertThat(connection.isClosed(), is(true));
-  }
-
-  @Test public void testSchemas() throws Exception {
-    final Connection connection = mjs();
-    final ResultSet resultSet =
-        connection.getMetaData().getSchemas(null, null);
-    assertFalse(resultSet.next());
-    final ResultSetMetaData metaData = resultSet.getMetaData();
-    assertTrue(metaData.getColumnCount() >= 2);
-    assertEquals("TABLE_CATALOG", metaData.getColumnName(1));
-    assertEquals("TABLE_SCHEM", metaData.getColumnName(2));
-    resultSet.close();
-    connection.close();
-  }
-
-  @Test public void testTables() throws Exception {
-    final Connection connection = mjs();
-    final ResultSet resultSet =
-        connection.getMetaData().getTables(null, null, null, new String[0]);
-    assertFalse(resultSet.next());
-    final ResultSetMetaData metaData = resultSet.getMetaData();
-    assertTrue(metaData.getColumnCount() >= 3);
-    assertEquals("TABLE_CAT", metaData.getColumnName(1));
-    assertEquals("TABLE_SCHEM", metaData.getColumnName(2));
-    assertEquals("TABLE_NAME", metaData.getColumnName(3));
-    resultSet.close();
-    connection.close();
-  }
-
-  @Ignore
-  @Test public void testNoFactory() throws Exception {
-    final Connection connection =
-        DriverManager.getConnection("jdbc:avatica:remote:");
-    assertThat(connection.isClosed(), is(false));
-    final ResultSet resultSet = connection.getMetaData().getSchemas();
-    assertFalse(resultSet.next());
-    final ResultSetMetaData metaData = resultSet.getMetaData();
-    assertEquals(2, metaData.getColumnCount());
-    assertEquals("TABLE_SCHEM", metaData.getColumnName(1));
-    assertEquals("TABLE_CATALOG", metaData.getColumnName(2));
-    resultSet.close();
-    connection.close();
-    assertThat(connection.isClosed(), is(true));
-  }
-
-  @Ignore
-  @Test public void testCatalogsMock() throws Exception {
-    final Connection connection = mjs();
-    assertThat(connection.isClosed(), is(false));
-    final ResultSet resultSet = connection.getMetaData().getSchemas();
-    assertFalse(resultSet.next());
-    final ResultSetMetaData metaData = resultSet.getMetaData();
-    assertEquals(2, metaData.getColumnCount());
-    assertEquals("TABLE_SCHEM", metaData.getColumnName(1));
-    assertEquals("TABLE_CATALOG", metaData.getColumnName(2));
-    resultSet.close();
-    connection.close();
-    assertThat(connection.isClosed(), is(true));
-  }
-
-  @Test public void testStatementExecuteQueryLocal() throws Exception {
-    checkStatementExecuteQuery(ljs(), false);
-  }
-
-  @Ignore
-  @Test public void testStatementExecuteQueryMock() throws Exception {
-    checkStatementExecuteQuery(mjs(), false);
-  }
-
-  @Ignore
-  @Test public void testPrepareExecuteQueryLocal() throws Exception {
-    checkStatementExecuteQuery(ljs(), true);
-  }
-
-  @Ignore
-  @Test public void testPrepareExecuteQueryMock() throws Exception {
-    checkStatementExecuteQuery(mjs(), true);
-  }
-
-  private void checkStatementExecuteQuery(Connection connection,
-      boolean prepare) throws SQLException {
-    final String sql = "select * from (\n"
-        + "  values (1, 'a'), (null, 'b'), (3, 'c')) as t (c1, c2)";
-    final Statement statement;
-    final ResultSet resultSet;
-    final ParameterMetaData parameterMetaData;
-    if (prepare) {
-      final PreparedStatement ps = connection.prepareStatement(sql);
-      statement = ps;
-      parameterMetaData = ps.getParameterMetaData();
-      resultSet = ps.executeQuery();
-    } else {
-      statement = connection.createStatement();
-      parameterMetaData = null;
-      resultSet = statement.executeQuery(sql);
-    }
-    if (parameterMetaData != null) {
-      assertThat(parameterMetaData.getParameterCount(), equalTo(2));
-    }
-    final ResultSetMetaData metaData = resultSet.getMetaData();
-    assertEquals(2, metaData.getColumnCount());
-    assertEquals("C1", metaData.getColumnName(1));
-    assertEquals("C2", metaData.getColumnName(2));
-    assertTrue(resultSet.next());
-    assertTrue(resultSet.next());
-    assertTrue(resultSet.next());
-    assertFalse(resultSet.next());
-    resultSet.close();
-    statement.close();
-    connection.close();
-  }
-
-  @Test public void testStatementLifecycle() throws Exception {
-    try (AvaticaConnection connection = (AvaticaConnection) ljs()) {
-      Map<Integer, AvaticaStatement> clientMap = connection.statementMap;
-      Cache<Integer, Object> serverMap =
-          QuasiRemoteJdbcServiceFactory.getRemoteStatementMap(connection);
-      assertEquals(0, clientMap.size());
-      assertEquals(0, serverMap.size());
-      Statement stmt = connection.createStatement();
-      assertEquals(1, clientMap.size());
-      assertEquals(1, serverMap.size());
-      stmt.close();
-      assertEquals(0, clientMap.size());
-      assertEquals(0, serverMap.size());
-    }
-  }
-
-  @Test public void testConnectionIsolation() throws Exception {
-    final String sql = "select * from (values (1, 'a'))";
-    Connection conn1 = ljs();
-    Connection conn2 = ljs();
-    Cache<String, Connection> connectionMap =
-        QuasiRemoteJdbcServiceFactory.getRemoteConnectionMap(
-            (AvaticaConnection) conn1);
-    assertEquals("connection cache should start empty",
-        0, connectionMap.size());
-    PreparedStatement conn1stmt1 = conn1.prepareStatement(sql);
-    assertEquals(
-        "statement creation implicitly creates a connection server-side",
-        1, connectionMap.size());
-    PreparedStatement conn2stmt1 = conn2.prepareStatement(sql);
-    assertEquals(
-        "statement creation implicitly creates a connection server-side",
-        2, connectionMap.size());
-    AvaticaPreparedStatement s1 = (AvaticaPreparedStatement) conn1stmt1;
-    AvaticaPreparedStatement s2 = (AvaticaPreparedStatement) conn2stmt1;
-    assertFalse("connection id's should be unique",
-        s1.handle.connectionId.equalsIgnoreCase(s2.handle.connectionId));
-    conn2.close();
-    assertEquals("closing a connection closes the server-side connection",
-        1, connectionMap.size());
-    conn1.close();
-    assertEquals("closing a connection closes the server-side connection",
-        0, connectionMap.size());
-  }
-
-  private void checkStatementExecuteQuery(Connection connection)
-      throws SQLException {
-    final Statement statement = connection.createStatement();
-    final ResultSet resultSet =
-        statement.executeQuery("select * from (\n"
-            + "  values (1, 'a'), (null, 'b'), (3, 'c')) as t (c1, c2)");
-    final ResultSetMetaData metaData = resultSet.getMetaData();
-    assertEquals(2, metaData.getColumnCount());
-    assertEquals("C1", metaData.getColumnName(1));
-    assertEquals("C2", metaData.getColumnName(2));
-    assertTrue(resultSet.next());
-    assertTrue(resultSet.next());
-    assertTrue(resultSet.next());
-    assertFalse(resultSet.next());
-    resultSet.close();
-    statement.close();
-    connection.close();
-  }
-
-  @Test public void testPrepareBindExecuteFetch() throws Exception {
-    checkPrepareBindExecuteFetch(ljs());
-  }
-
-  private void checkPrepareBindExecuteFetch(Connection connection)
-      throws SQLException {
-    final String sql = "select cast(? as integer) * 3 as c, 'x' as x\n"
-        + "from (values (1, 'a'))";
-    final PreparedStatement ps =
-        connection.prepareStatement(sql);
-    final ResultSetMetaData metaData = ps.getMetaData();
-    assertEquals(2, metaData.getColumnCount());
-    assertEquals("C", metaData.getColumnName(1));
-    assertEquals("X", metaData.getColumnName(2));
-    try {
-      final ResultSet resultSet = ps.executeQuery();
-      fail("expected error, got " + resultSet);
-    } catch (SQLException e) {
-      assertThat(e.getMessage(),
-          equalTo("exception while executing query: unbound parameter"));
-    }
-
-    final ParameterMetaData parameterMetaData = ps.getParameterMetaData();
-    assertThat(parameterMetaData.getParameterCount(), equalTo(1));
-
-    ps.setInt(1, 10);
-    final ResultSet resultSet = ps.executeQuery();
-    assertTrue(resultSet.next());
-    assertThat(resultSet.getInt(1), equalTo(30));
-    assertFalse(resultSet.next());
-    resultSet.close();
-
-    ps.setInt(1, 20);
-    final ResultSet resultSet2 = ps.executeQuery();
-    assertFalse(resultSet2.isClosed());
-    assertTrue(resultSet2.next());
-    assertThat(resultSet2.getInt(1), equalTo(60));
-    assertThat(resultSet2.wasNull(), is(false));
-    assertFalse(resultSet2.next());
-    resultSet2.close();
-
-    ps.setObject(1, null);
-    final ResultSet resultSet3 = ps.executeQuery();
-    assertTrue(resultSet3.next());
-    assertThat(resultSet3.getInt(1), equalTo(0));
-    assertThat(resultSet3.wasNull(), is(true));
-    assertFalse(resultSet3.next());
-    resultSet3.close();
-
-    ps.close();
-    connection.close();
-  }
-
-  /**
-   * Factory that creates a service based on a local JDBC connection.
-   */
-  public static class LocalJdbcServiceFactory implements Service.Factory {
-    @Override public Service create(AvaticaConnection connection) {
-      try {
-        return new LocalService(
-            new JdbcMeta(CONNECTION_SPEC.url, CONNECTION_SPEC.username,
-                CONNECTION_SPEC.password));
-      } catch (SQLException e) {
-        throw new RuntimeException(e);
-      }
-    }
-  }
-
-  /**
-   * Factory that creates a service based on a local JDBC connection.
-   */
-  public static class QuasiRemoteJdbcServiceFactory implements Service.Factory {
-
-    /** a singleton instance that is recreated for each test */
-    private static Service service;
-
-    static void initService() {
-      try {
-        final JdbcMeta jdbcMeta = new JdbcMeta(CONNECTION_SPEC.url,
-            CONNECTION_SPEC.username, CONNECTION_SPEC.password);
-        final LocalService localService = new LocalService(jdbcMeta);
-        service = new LocalJsonService(localService);
-      } catch (SQLException e) {
-        throw new RuntimeException(e);
-      }
-    }
-
-    @Override public Service create(AvaticaConnection connection) {
-      assert service != null;
-      return service;
-    }
-
-    /**
-     * Reach into the guts of a quasi-remote connection and pull out the
-     * statement map from the other side.
-     * TODO: refactor tests to replace reflection with package-local access
-     */
-    static Cache<Integer, Object>
-    getRemoteStatementMap(AvaticaConnection connection) throws Exception {
-      Field metaF = AvaticaConnection.class.getDeclaredField("meta");
-      metaF.setAccessible(true);
-      Meta clientMeta = (Meta) metaF.get(connection);
-      Field remoteMetaServiceF = clientMeta.getClass().getDeclaredField("service");
-      remoteMetaServiceF.setAccessible(true);
-      LocalJsonService remoteMetaService = (LocalJsonService) remoteMetaServiceF.get(clientMeta);
-      Field remoteMetaServiceServiceF = remoteMetaService.getClass().getDeclaredField("service");
-      remoteMetaServiceServiceF.setAccessible(true);
-      LocalService remoteMetaServiceService =
-          (LocalService) remoteMetaServiceServiceF.get(remoteMetaService);
-      Field remoteMetaServiceServiceMetaF =
-          remoteMetaServiceService.getClass().getDeclaredField("meta");
-      remoteMetaServiceServiceMetaF.setAccessible(true);
-      JdbcMeta serverMeta = (JdbcMeta) remoteMetaServiceServiceMetaF.get(remoteMetaServiceService);
-      Field jdbcMetaStatementMapF = JdbcMeta.class.getDeclaredField("statementCache");
-      jdbcMetaStatementMapF.setAccessible(true);
-      //noinspection unchecked
-      return (Cache<Integer, Object>) jdbcMetaStatementMapF.get(serverMeta);
-    }
-
-    /**
-     * Reach into the guts of a quasi-remote connection and pull out the
-     * connection map from the other side.
-     * TODO: refactor tests to replace reflection with package-local access
-     */
-    static Cache<String, Connection>
-    getRemoteConnectionMap(AvaticaConnection connection) throws Exception {
-      Field metaF = AvaticaConnection.class.getDeclaredField("meta");
-      metaF.setAccessible(true);
-      Meta clientMeta = (Meta) metaF.get(connection);
-      Field remoteMetaServiceF = clientMeta.getClass().getDeclaredField("service");
-      remoteMetaServiceF.setAccessible(true);
-      LocalJsonService remoteMetaService = (LocalJsonService) remoteMetaServiceF.get(clientMeta);
-      Field remoteMetaServiceServiceF = remoteMetaService.getClass().getDeclaredField("service");
-      remoteMetaServiceServiceF.setAccessible(true);
-      LocalService remoteMetaServiceService =
-          (LocalService) remoteMetaServiceServiceF.get(remoteMetaService);
-      Field remoteMetaServiceServiceMetaF =
-          remoteMetaServiceService.getClass().getDeclaredField("meta");
-      remoteMetaServiceServiceMetaF.setAccessible(true);
-      JdbcMeta serverMeta = (JdbcMeta) remoteMetaServiceServiceMetaF.get(remoteMetaServiceService);
-      Field jdbcMetaConnectionCacheF = JdbcMeta.class.getDeclaredField("connectionCache");
-      jdbcMetaConnectionCacheF.setAccessible(true);
-      //noinspection unchecked
-      return (Cache<String, Connection>) jdbcMetaConnectionCacheF.get(serverMeta);
-    }
-  }
-
-  /** Information necessary to create a JDBC connection. Specify one to run
-   * tests against a different database. (hsqldb is the default.) */
-  public static class ConnectionSpec {
-    public final String url;
-    public final String username;
-    public final String password;
-    public final String driver;
-
-    public ConnectionSpec(String url, String username, String password,
-        String driver) {
-      this.url = url;
-      this.username = username;
-      this.password = password;
-      this.driver = driver;
-    }
-
-    public static final ConnectionSpec HSQLDB =
-        new ConnectionSpec(ScottHsqldb.URI, ScottHsqldb.USER,
-            ScottHsqldb.PASSWORD, "org.hsqldb.jdbcDriver");
-  }
-}
-
-// End RemoteDriverTest.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/fa9bdc4a/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 7d26118..cb7217e 100644
--- a/pom.xml
+++ b/pom.xml
@@ -92,6 +92,12 @@ limitations under the License.
       </dependency>
       <dependency>
         <groupId>org.apache.calcite</groupId>
+        <artifactId>calcite-avatica</artifactId>
+        <version>1.2.0-incubating-SNAPSHOT</version>
+        <type>test-jar</type>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.calcite</groupId>
         <artifactId>calcite-avatica-server</artifactId>
         <version>1.2.0-incubating-SNAPSHOT</version>
       </dependency>
@@ -119,6 +125,21 @@ limitations under the License.
         <version>1.4</version>
       </dependency>
       <dependency>
+        <groupId>commons-logging</groupId>
+        <artifactId>commons-logging</artifactId>
+        <version>1.1.3</version>
+      </dependency>
+      <dependency>
+        <groupId>com.fasterxml.jackson.core</groupId>
+        <artifactId>jackson-core</artifactId>
+        <version>2.1.1</version>
+      </dependency>
+      <dependency>
+        <groupId>com.fasterxml.jackson.core</groupId>
+        <artifactId>jackson-annotations</artifactId>
+        <version>2.1.1</version>
+      </dependency>
+      <dependency>
         <groupId>com.fasterxml.jackson.core</groupId>
         <artifactId>jackson-databind</artifactId>
         <version>2.1.1</version>
@@ -136,6 +157,11 @@ limitations under the License.
         <version>14.0.1</version>
       </dependency>
       <dependency>
+        <groupId>javax.servlet</groupId>
+        <artifactId>javax.servlet-api</artifactId>
+        <version>3.0.1</version>
+      </dependency>
+      <dependency>
         <groupId>junit</groupId>
         <artifactId>junit</artifactId>
         <version>4.11</version>
@@ -191,11 +217,6 @@ limitations under the License.
         <version>3.2</version>
       </dependency>
       <dependency>
-        <groupId>commons-logging</groupId>
-        <artifactId>commons-logging</artifactId>
-        <version>1.1.3</version>
-      </dependency>
-      <dependency>
         <groupId>org.codehaus.janino</groupId>
         <artifactId>janino</artifactId>
         <version>2.7.6</version>
@@ -206,6 +227,11 @@ limitations under the License.
         <version>2.7.6</version>
       </dependency>
       <dependency>
+        <groupId>org.hamcrest</groupId>
+        <artifactId>hamcrest-core</artifactId>
+        <version>1.3</version>
+      </dependency>
+      <dependency>
         <groupId>org.hsqldb</groupId>
         <artifactId>hsqldb</artifactId>
         <version>2.3.1</version>
@@ -231,6 +257,11 @@ limitations under the License.
         <version>9.2.7.v20150116</version>
       </dependency>
       <dependency>
+        <groupId>org.eclipse.jetty</groupId>
+        <artifactId>jetty-util</artifactId>
+        <version>9.2.7.v20150116</version>
+      </dependency>
+      <dependency>
         <groupId>org.mongodb</groupId>
         <artifactId>mongo-java-driver</artifactId>
         <version>2.12.3</version>