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/10/23 04:38:53 UTC

[04/11] incubator-calcite git commit: [CALCITE-912] Add Avatica OpenConnectionRequest (Bruno Dumon)

[CALCITE-912] Add Avatica OpenConnectionRequest (Bruno Dumon)

Goal: passing of connection properties (the 'info') from the remote
avatica driver to the corresponding server-side connection.

Changes include:
 * in Meta: explicit opening of connections: there is a new
   openConnection() call. Client decides on the connection id.
   Implicit creation of unknown connections is gone.
 * JdbcMeta: default connection is gone, all methods have been
   extended with a connection id.
 * correspondingly, the various Service.Request's now carry a
   connection id (will make fixing CALCITE-871 easy)
 * (unrelated, but was useful for testing) more meaningful error
   messages in remote driver


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

Branch: refs/heads/master
Commit: a63639b1ab7c52f5b425a69359d66ce08ee59bbc
Parents: d50c46a
Author: Bruno Dumon <br...@ngdata.com>
Authored: Thu Oct 15 19:32:32 2015 +0200
Committer: Julian Hyde <jh...@apache.org>
Committed: Thu Oct 22 18:27:45 2015 -0700

----------------------------------------------------------------------
 avatica-server/pom.xml                          |    5 +
 .../apache/calcite/avatica/jdbc/JdbcMeta.java   |  119 +-
 .../calcite/avatica/server/AvaticaHandler.java  |   29 +-
 .../avatica/server/AvaticaProtobufHandler.java  |   25 +-
 .../calcite/avatica/RemoteDriverMockTest.java   |    3 +-
 .../calcite/avatica/RemoteDriverTest.java       |   23 +-
 .../calcite/avatica/remote/RemoteMetaTest.java  |   56 +
 .../calcite/avatica/AvaticaConnection.java      |    3 +-
 .../avatica/AvaticaDatabaseMetaData.java        |   63 +-
 .../java/org/apache/calcite/avatica/Meta.java   |   87 +-
 .../org/apache/calcite/avatica/MetaImpl.java    |   77 +-
 .../apache/calcite/avatica/proto/Requests.java  | 2030 +++++++++++++++---
 .../apache/calcite/avatica/proto/Responses.java |  804 ++++++-
 .../apache/calcite/avatica/remote/Driver.java   |   49 +-
 .../calcite/avatica/remote/JsonHandler.java     |    8 +-
 .../calcite/avatica/remote/JsonService.java     |    8 +
 .../calcite/avatica/remote/LocalService.java    |   27 +-
 .../calcite/avatica/remote/MockJsonService.java |   39 +-
 .../avatica/remote/MockProtobufService.java     |   21 +-
 .../calcite/avatica/remote/ProtobufService.java |    5 +
 .../avatica/remote/ProtobufTranslationImpl.java |   10 +
 .../calcite/avatica/remote/RemoteMeta.java      |   34 +-
 .../avatica/remote/RemoteProtobufService.java   |    9 +-
 .../calcite/avatica/remote/RemoteService.java   |   11 +-
 .../apache/calcite/avatica/remote/Service.java  |  500 ++++-
 avatica/src/main/protobuf/requests.proto        |   17 +-
 avatica/src/main/protobuf/responses.proto       |    9 +
 .../remote/ProtobufTranslationImplTest.java     |   26 +-
 .../calcite/avatica/test/JsonHandlerTest.java   |    4 +
 .../apache/calcite/jdbc/CalciteMetaImpl.java    |   16 +-
 pom.xml                                         |    6 +
 31 files changed, 3612 insertions(+), 511 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a63639b1/avatica-server/pom.xml
----------------------------------------------------------------------
diff --git a/avatica-server/pom.xml b/avatica-server/pom.xml
index 1695c07..2d98046 100644
--- a/avatica-server/pom.xml
+++ b/avatica-server/pom.xml
@@ -95,6 +95,11 @@ limitations under the License.
       <artifactId>mockito-all</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>com.github.stephenc.jcip</groupId>
+      <artifactId>jcip-annotations</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a63639b1/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 3585185..4054939 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
@@ -84,7 +84,6 @@ public class JdbcMeta implements Meta {
 
   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;
 
@@ -129,7 +128,6 @@ public class JdbcMeta implements Meta {
   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(),
@@ -247,10 +245,10 @@ public class JdbcMeta implements Meta {
     return signature(metaData, null, null, null);
   }
 
-  public Map<DatabaseProperty, Object> getDatabaseProperties() {
+  public Map<DatabaseProperty, Object> getDatabaseProperties(ConnectionHandle ch) {
     try {
       final Map<DatabaseProperty, Object> map = new HashMap<>();
-      final DatabaseMetaData metaData = connection.getMetaData();
+      final DatabaseMetaData metaData = getConnection(ch.id).getMetaData();
       for (DatabaseProperty p : DatabaseProperty.values()) {
         addProperty(map, metaData, p);
       }
@@ -269,11 +267,11 @@ public class JdbcMeta implements Meta {
     }
   }
 
-  public MetaResultSet getTables(String catalog, Pat schemaPattern,
+  public MetaResultSet getTables(ConnectionHandle ch, String catalog, Pat schemaPattern,
       Pat tableNamePattern, List<String> typeList) {
     try {
       final ResultSet rs =
-          connection.getMetaData().getTables(catalog, schemaPattern.s,
+          getConnection(ch.id).getMetaData().getTables(catalog, schemaPattern.s,
               tableNamePattern.s, toArray(typeList));
       return JdbcResultSet.create(DEFAULT_CONN_ID, -1, rs);
     } catch (SQLException e) {
@@ -281,63 +279,63 @@ public class JdbcMeta implements Meta {
     }
   }
 
-  public MetaResultSet getColumns(String catalog, Pat schemaPattern,
+  public MetaResultSet getColumns(ConnectionHandle ch, String catalog, Pat schemaPattern,
       Pat tableNamePattern, Pat columnNamePattern) {
     try {
       final ResultSet rs =
-          connection.getMetaData().getColumns(
-              catalog, schemaPattern.s, tableNamePattern.s, columnNamePattern.s);
+          getConnection(ch.id).getMetaData().getColumns(catalog, schemaPattern.s,
+              tableNamePattern.s, columnNamePattern.s);
       return JdbcResultSet.create(DEFAULT_CONN_ID, -1, rs, UNLIMITED_COUNT);
     } catch (SQLException e) {
       throw new RuntimeException(e);
     }
   }
 
-  public MetaResultSet getSchemas(String catalog, Pat schemaPattern) {
+  public MetaResultSet getSchemas(ConnectionHandle ch, String catalog, Pat schemaPattern) {
     try {
       final ResultSet rs =
-          connection.getMetaData().getSchemas(catalog, schemaPattern.s);
+          getConnection(ch.id).getMetaData().getSchemas(catalog, schemaPattern.s);
       return JdbcResultSet.create(DEFAULT_CONN_ID, -1, rs, UNLIMITED_COUNT);
     } catch (SQLException e) {
       throw new RuntimeException(e);
     }
   }
 
-  public MetaResultSet getCatalogs() {
+  public MetaResultSet getCatalogs(ConnectionHandle ch) {
     try {
-      final ResultSet rs = connection.getMetaData().getCatalogs();
+      final ResultSet rs = getConnection(ch.id).getMetaData().getCatalogs();
       return JdbcResultSet.create(DEFAULT_CONN_ID, -1, rs, UNLIMITED_COUNT);
     } catch (SQLException e) {
       throw new RuntimeException(e);
     }
   }
 
-  public MetaResultSet getTableTypes() {
+  public MetaResultSet getTableTypes(ConnectionHandle ch) {
     try {
-      final ResultSet rs = connection.getMetaData().getTableTypes();
+      final ResultSet rs = getConnection(ch.id).getMetaData().getTableTypes();
       return JdbcResultSet.create(DEFAULT_CONN_ID, -1, rs, UNLIMITED_COUNT);
     } catch (SQLException e) {
       throw new RuntimeException(e);
     }
   }
 
-  public MetaResultSet getProcedures(String catalog, Pat schemaPattern,
+  public MetaResultSet getProcedures(ConnectionHandle ch, String catalog, Pat schemaPattern,
       Pat procedureNamePattern) {
     try {
       final ResultSet rs =
-          connection.getMetaData().getProcedures(
-              catalog, schemaPattern.s, procedureNamePattern.s);
+          getConnection(ch.id).getMetaData().getProcedures(catalog, schemaPattern.s,
+              procedureNamePattern.s);
       return JdbcResultSet.create(DEFAULT_CONN_ID, -1, rs, UNLIMITED_COUNT);
     } catch (SQLException e) {
       throw new RuntimeException(e);
     }
   }
 
-  public MetaResultSet getProcedureColumns(String catalog, Pat schemaPattern,
+  public MetaResultSet getProcedureColumns(ConnectionHandle ch, String catalog, Pat schemaPattern,
       Pat procedureNamePattern, Pat columnNamePattern) {
     try {
       final ResultSet rs =
-          connection.getMetaData().getProcedureColumns(catalog,
+          getConnection(ch.id).getMetaData().getProcedureColumns(catalog,
               schemaPattern.s, procedureNamePattern.s, columnNamePattern.s);
       return JdbcResultSet.create(DEFAULT_CONN_ID, -1, rs, UNLIMITED_COUNT);
     } catch (SQLException e) {
@@ -345,11 +343,11 @@ public class JdbcMeta implements Meta {
     }
   }
 
-  public MetaResultSet getColumnPrivileges(String catalog, String schema,
+  public MetaResultSet getColumnPrivileges(ConnectionHandle ch, String catalog, String schema,
       String table, Pat columnNamePattern) {
     try {
       final ResultSet rs =
-          connection.getMetaData().getColumnPrivileges(catalog, schema,
+          getConnection(ch.id).getMetaData().getColumnPrivileges(catalog, schema,
               table, columnNamePattern.s);
       return JdbcResultSet.create(DEFAULT_CONN_ID, -1, rs, UNLIMITED_COUNT);
     } catch (SQLException e) {
@@ -357,11 +355,11 @@ public class JdbcMeta implements Meta {
     }
   }
 
-  public MetaResultSet getTablePrivileges(String catalog, Pat schemaPattern,
+  public MetaResultSet getTablePrivileges(ConnectionHandle ch, String catalog, Pat schemaPattern,
       Pat tableNamePattern) {
     try {
       final ResultSet rs =
-          connection.getMetaData().getTablePrivileges(catalog,
+          getConnection(ch.id).getMetaData().getTablePrivileges(catalog,
               schemaPattern.s, tableNamePattern.s);
       return JdbcResultSet.create(DEFAULT_CONN_ID, -1, rs, UNLIMITED_COUNT);
     } catch (SQLException e) {
@@ -369,7 +367,7 @@ public class JdbcMeta implements Meta {
     }
   }
 
-  public MetaResultSet getBestRowIdentifier(String catalog, String schema,
+  public MetaResultSet getBestRowIdentifier(ConnectionHandle ch, String catalog, String schema,
       String table, int scope, boolean nullable) {
     if (LOG.isTraceEnabled()) {
       LOG.trace("getBestRowIdentifier catalog:" + catalog + " schema:" + schema
@@ -377,7 +375,7 @@ public class JdbcMeta implements Meta {
     }
     try {
       final ResultSet rs =
-          connection.getMetaData().getBestRowIdentifier(catalog, schema,
+          getConnection(ch.id).getMetaData().getBestRowIdentifier(catalog, schema,
               table, scope, nullable);
       return JdbcResultSet.create(DEFAULT_CONN_ID, -1, rs, UNLIMITED_COUNT);
     } catch (SQLException e) {
@@ -385,99 +383,99 @@ public class JdbcMeta implements Meta {
     }
   }
 
-  public MetaResultSet getVersionColumns(String catalog, String schema,
+  public MetaResultSet getVersionColumns(ConnectionHandle ch, String catalog, String schema,
       String table) {
     if (LOG.isTraceEnabled()) {
       LOG.trace("getVersionColumns catalog:" + catalog + " schema:" + schema + " table:" + table);
     }
     try {
       final ResultSet rs =
-          connection.getMetaData().getVersionColumns(catalog, schema, table);
+          getConnection(ch.id).getMetaData().getVersionColumns(catalog, schema, table);
       return JdbcResultSet.create(DEFAULT_CONN_ID, -1, rs, UNLIMITED_COUNT);
     } catch (SQLException e) {
       throw new RuntimeException(e);
     }
   }
 
-  public MetaResultSet getPrimaryKeys(String catalog, String schema,
+  public MetaResultSet getPrimaryKeys(ConnectionHandle ch, String catalog, String schema,
       String table) {
     if (LOG.isTraceEnabled()) {
       LOG.trace("getPrimaryKeys catalog:" + catalog + " schema:" + schema + " table:" + table);
     }
     try {
       final ResultSet rs =
-          connection.getMetaData().getPrimaryKeys(catalog, schema, table);
+          getConnection(ch.id).getMetaData().getPrimaryKeys(catalog, schema, table);
       return JdbcResultSet.create(DEFAULT_CONN_ID, -1, rs, UNLIMITED_COUNT);
     } catch (SQLException e) {
       throw new RuntimeException(e);
     }
   }
 
-  public MetaResultSet getImportedKeys(String catalog, String schema,
+  public MetaResultSet getImportedKeys(ConnectionHandle ch, String catalog, String schema,
       String table) {
     return null;
   }
 
-  public MetaResultSet getExportedKeys(String catalog, String schema,
+  public MetaResultSet getExportedKeys(ConnectionHandle ch, String catalog, String schema,
       String table) {
     return null;
   }
 
-  public MetaResultSet getCrossReference(String parentCatalog,
+  public MetaResultSet getCrossReference(ConnectionHandle ch, String parentCatalog,
       String parentSchema, String parentTable, String foreignCatalog,
       String foreignSchema, String foreignTable) {
     return null;
   }
 
-  public MetaResultSet getTypeInfo() {
+  public MetaResultSet getTypeInfo(ConnectionHandle ch) {
     try {
-      final ResultSet rs = connection.getMetaData().getTypeInfo();
+      final ResultSet rs = getConnection(ch.id).getMetaData().getTypeInfo();
       return JdbcResultSet.create(DEFAULT_CONN_ID, -1, rs, UNLIMITED_COUNT);
     } catch (SQLException e) {
       throw new RuntimeException(e);
     }
   }
 
-  public MetaResultSet getIndexInfo(String catalog, String schema, String table,
-      boolean unique, boolean approximate) {
+  public MetaResultSet getIndexInfo(ConnectionHandle ch, String catalog, String schema,
+      String table, boolean unique, boolean approximate) {
     return null;
   }
 
-  public MetaResultSet getUDTs(String catalog, Pat schemaPattern,
+  public MetaResultSet getUDTs(ConnectionHandle ch, String catalog, Pat schemaPattern,
       Pat typeNamePattern, int[] types) {
     return null;
   }
 
-  public MetaResultSet getSuperTypes(String catalog, Pat schemaPattern,
+  public MetaResultSet getSuperTypes(ConnectionHandle ch, String catalog, Pat schemaPattern,
       Pat typeNamePattern) {
     return null;
   }
 
-  public MetaResultSet getSuperTables(String catalog, Pat schemaPattern,
+  public MetaResultSet getSuperTables(ConnectionHandle ch, String catalog, Pat schemaPattern,
       Pat tableNamePattern) {
     return null;
   }
 
-  public MetaResultSet getAttributes(String catalog, Pat schemaPattern,
+  public MetaResultSet getAttributes(ConnectionHandle ch, String catalog, Pat schemaPattern,
       Pat typeNamePattern, Pat attributeNamePattern) {
     return null;
   }
 
-  public MetaResultSet getClientInfoProperties() {
+  public MetaResultSet getClientInfoProperties(ConnectionHandle ch) {
     return null;
   }
 
-  public MetaResultSet getFunctions(String catalog, Pat schemaPattern,
+  public MetaResultSet getFunctions(ConnectionHandle ch, String catalog, Pat schemaPattern,
       Pat functionNamePattern) {
     return null;
   }
 
-  public MetaResultSet getFunctionColumns(String catalog, Pat schemaPattern,
+  public MetaResultSet getFunctionColumns(ConnectionHandle ch, String catalog, Pat schemaPattern,
       Pat functionNamePattern, Pat columnNamePattern) {
     return null;
   }
 
-  public MetaResultSet getPseudoColumns(String catalog, Pat schemaPattern,
+  public MetaResultSet getPseudoColumns(ConnectionHandle ch, String catalog, Pat schemaPattern,
       Pat tableNamePattern, Pat columnNamePattern) {
     return null;
   }
@@ -488,10 +486,12 @@ public class JdbcMeta implements Meta {
   }
 
   protected Connection getConnection(String id) throws SQLException {
+    if (id == null) {
+      throw new NullPointerException("Connection id is null.");
+    }
     Connection conn = connectionCache.getIfPresent(id);
     if (conn == null) {
-      conn = DriverManager.getConnection(url, info);
-      connectionCache.put(id, conn);
+      throw new RuntimeException("Connection not found: invalid id, closed, or expired: " + id);
     }
     return conn;
   }
@@ -533,6 +533,27 @@ public class JdbcMeta implements Meta {
     }
   }
 
+  @Override
+  public void openConnection(ConnectionHandle ch, Map<String, String> info) {
+    Properties fullInfo = new Properties();
+    fullInfo.putAll(this.info);
+    if (info != null) {
+      fullInfo.putAll(info);
+    }
+
+    synchronized (this) {
+      try {
+        if (connectionCache.asMap().containsKey(ch.id)) {
+          throw new RuntimeException("Connection already exists: " + ch.id);
+        }
+        Connection conn = DriverManager.getConnection(url, fullInfo);
+        connectionCache.put(ch.id, conn);
+      } catch (SQLException e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+
   @Override public void closeConnection(ConnectionHandle ch) {
     Connection conn = connectionCache.getIfPresent(ch.id);
     if (conn == null) {
@@ -839,10 +860,6 @@ public class JdbcMeta implements Meta {
     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());

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a63639b1/avatica-server/src/main/java/org/apache/calcite/avatica/server/AvaticaHandler.java
----------------------------------------------------------------------
diff --git a/avatica-server/src/main/java/org/apache/calcite/avatica/server/AvaticaHandler.java b/avatica-server/src/main/java/org/apache/calcite/avatica/server/AvaticaHandler.java
index 2ada444..acb7fcf 100644
--- a/avatica-server/src/main/java/org/apache/calcite/avatica/server/AvaticaHandler.java
+++ b/avatica-server/src/main/java/org/apache/calcite/avatica/server/AvaticaHandler.java
@@ -23,6 +23,11 @@ import org.apache.calcite.avatica.remote.Service;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
+import com.google.common.base.Function;
+import com.google.common.base.Joiner;
+import com.google.common.base.Throwables;
+import com.google.common.collect.Iterables;
+
 import org.eclipse.jetty.server.Request;
 import org.eclipse.jetty.server.handler.AbstractHandler;
 
@@ -63,7 +68,14 @@ public class AvaticaHandler extends AbstractHandler {
       if (LOG.isTraceEnabled()) {
         LOG.trace("request: " + jsonRequest);
       }
-      final String jsonResponse = jsonHandler.apply(jsonRequest);
+      String jsonResponse;
+      try {
+        jsonResponse = jsonHandler.apply(jsonRequest);
+      } catch (Throwable t) {
+        LOG.error("Error handling request: " + jsonRequest, t);
+        response.setStatus(500);
+        jsonResponse = createErrorResponse(t);
+      }
       if (LOG.isTraceEnabled()) {
         LOG.trace("response: " + jsonResponse);
       }
@@ -71,6 +83,21 @@ public class AvaticaHandler extends AbstractHandler {
       response.getWriter().println(jsonResponse);
     }
   }
+
+  private String createErrorResponse(Throwable t) throws IOException {
+    return jsonHandler.encode(new Service.ErrorResponse(getErrorMessage(t)));
+  }
+
+  public static String getErrorMessage(Throwable t) {
+    return Joiner.on(" -> ").join(
+        Iterables.transform(Throwables.getCausalChain(t), new Function<Throwable, String>() {
+          @Override
+          public String apply(Throwable input) {
+            return input.getMessage() == null
+                ? "(null exception message)" : input.getMessage();
+          }
+        }));
+  }
 }
 
 // End AvaticaHandler.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a63639b1/avatica-server/src/main/java/org/apache/calcite/avatica/server/AvaticaProtobufHandler.java
----------------------------------------------------------------------
diff --git a/avatica-server/src/main/java/org/apache/calcite/avatica/server/AvaticaProtobufHandler.java b/avatica-server/src/main/java/org/apache/calcite/avatica/server/AvaticaProtobufHandler.java
index 419ad1f..9432824 100644
--- a/avatica-server/src/main/java/org/apache/calcite/avatica/server/AvaticaProtobufHandler.java
+++ b/avatica-server/src/main/java/org/apache/calcite/avatica/server/AvaticaProtobufHandler.java
@@ -18,14 +18,17 @@ package org.apache.calcite.avatica.server;
 
 import org.apache.calcite.avatica.AvaticaUtils;
 import org.apache.calcite.avatica.remote.ProtobufHandler;
+import org.apache.calcite.avatica.remote.ProtobufTranslation;
 import org.apache.calcite.avatica.remote.ProtobufTranslationImpl;
 import org.apache.calcite.avatica.remote.Service;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
 import org.eclipse.jetty.server.Request;
 import org.eclipse.jetty.server.handler.AbstractHandler;
 
 import java.io.IOException;
-
 import javax.servlet.ServletException;
 import javax.servlet.ServletInputStream;
 import javax.servlet.http.HttpServletRequest;
@@ -35,11 +38,14 @@ import javax.servlet.http.HttpServletResponse;
  * Jetty handler that executes Avatica JSON request-responses.
  */
 public class AvaticaProtobufHandler extends AbstractHandler {
+  private static final Log LOG = LogFactory.getLog(AvaticaHandler.class);
 
   private final ProtobufHandler pbHandler;
+  private final ProtobufTranslation protobufTranslation;
 
   public AvaticaProtobufHandler(Service service) {
-    this.pbHandler = new ProtobufHandler(service, new ProtobufTranslationImpl());
+    this.protobufTranslation = new ProtobufTranslationImpl();
+    this.pbHandler = new ProtobufHandler(service, protobufTranslation);
   }
 
   public void handle(String target, Request baseRequest,
@@ -53,13 +59,26 @@ public class AvaticaProtobufHandler extends AbstractHandler {
         requestBytes = AvaticaUtils.readFullyToBytes(inputStream);
       }
 
-      byte[] responseBytes = pbHandler.apply(requestBytes);
+      byte[] responseBytes;
+      try {
+        responseBytes = pbHandler.apply(requestBytes);
+      } catch (Throwable t) {
+        LOG.error("Error handling request", t);
+        response.setStatus(500);
+        responseBytes = createErrorResponse(t);
+      }
 
       baseRequest.setHandled(true);
       response.getOutputStream().write(responseBytes);
     }
   }
 
+  private byte[] createErrorResponse(Throwable t) throws IOException {
+    Service.ErrorResponse errorResponse = new Service.ErrorResponse(
+        AvaticaHandler.getErrorMessage(t));
+    return protobufTranslation.serializeResponse(errorResponse);
+  }
+
 }
 
 // End AvaticaProtobufHandler.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a63639b1/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverMockTest.java
----------------------------------------------------------------------
diff --git a/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverMockTest.java b/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverMockTest.java
index 5c4423d..9749ef6 100644
--- a/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverMockTest.java
+++ b/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverMockTest.java
@@ -94,8 +94,7 @@ public class RemoteDriverMockTest {
   }
 
   @Test public void testRegister() throws Exception {
-    final Connection connection =
-        DriverManager.getConnection("jdbc:avatica:remote:");
+    final Connection connection = getMockConnection();
     assertThat(connection.isClosed(), is(false));
     connection.close();
     assertThat(connection.isClosed(), is(true));

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a63639b1/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 b3f45fd..a26f192 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
@@ -27,6 +27,8 @@ import org.apache.calcite.avatica.remote.Service;
 
 import com.google.common.cache.Cache;
 
+import net.jcip.annotations.NotThreadSafe;
+
 import org.junit.Before;
 import org.junit.Ignore;
 import org.junit.Test;
@@ -69,6 +71,7 @@ import static org.junit.Assert.fail;
  * Unit test for Avatica Remote JDBC driver.
  */
 @RunWith(Parameterized.class)
+@NotThreadSafe // for testConnectionIsolation
 public class RemoteDriverTest {
   public static final String LJS =
       LocalJdbcServiceFactory.class.getName();
@@ -214,8 +217,7 @@ public class RemoteDriverTest {
   }
 
   @Test public void testRegister() throws Exception {
-    final Connection connection =
-        DriverManager.getConnection("jdbc:avatica:remote:");
+    final Connection connection = getLocalConnection();
     assertThat(connection.isClosed(), is(false));
     connection.close();
     assertThat(connection.isClosed(), is(true));
@@ -789,23 +791,26 @@ public class RemoteDriverTest {
   @Test public void testConnectionIsolation() throws Exception {
     ConnectionSpec.getDatabaseLock().lock();
     try {
-      final String sql = "select * from (values (1, 'a'))";
-      Connection conn1 = getLocalConnection();
-      Connection conn2 = getLocalConnection();
       Cache<String, Connection> connectionMap = getLocalConnectionInternals()
-          .getRemoteConnectionMap((AvaticaConnection) conn1);
+          .getRemoteConnectionMap((AvaticaConnection) getLocalConnection());
       // Other tests being run might leave connections in the cache.
       // The lock guards against more connections being cached during the test.
       connectionMap.invalidateAll();
+
+      final String sql = "select * from (values (1, 'a'))";
       assertEquals("connection cache should start empty",
           0, connectionMap.size());
+      Connection conn1 = getLocalConnection();
+      Connection conn2 = getLocalConnection();
+      assertEquals("we now have two connections open",
+          2, connectionMap.size());
       PreparedStatement conn1stmt1 = conn1.prepareStatement(sql);
       assertEquals(
-          "statement creation implicitly creates a connection server-side",
-          1, connectionMap.size());
+          "creating a statement does not cause new connection",
+          2, connectionMap.size());
       PreparedStatement conn2stmt1 = conn2.prepareStatement(sql);
       assertEquals(
-          "statement creation implicitly creates a connection server-side",
+          "creating a statement does not cause new connection",
           2, connectionMap.size());
       AvaticaPreparedStatement s1 = (AvaticaPreparedStatement) conn1stmt1;
       AvaticaPreparedStatement s2 = (AvaticaPreparedStatement) conn2stmt1;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a63639b1/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
index 1172335..df5983a 100644
--- 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
@@ -50,11 +50,13 @@ import java.util.Map;
 import java.util.Random;
 import java.util.UUID;
 
+import static org.hamcrest.CoreMatchers.containsString;
 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;
 
 /** Tests covering {@link RemoteMeta}. */
 @RunWith(Parameterized.class)
@@ -295,6 +297,60 @@ public class RemoteMetaTest {
       ConnectionSpec.getDatabaseLock().unlock();
     }
   }
+
+  @Test public void testOpenConnectionWithProperties() throws Exception {
+    // This tests that username and password are used for creating a connection on the
+    // server. If this was not the case, it would succeed.
+    try {
+      DriverManager.getConnection(url, "john", "doe");
+      fail("expected exception");
+    } catch (RuntimeException e) {
+      assertEquals("Remote driver error:"
+          + " java.sql.SQLInvalidAuthorizationSpecException: invalid authorization specification"
+          + " - not found: john"
+          + " -> invalid authorization specification - not found: john"
+          + " -> invalid authorization specification - not found: john",
+          e.getMessage());
+    }
+  }
+
+  @Test public void testRemoteConnectionsAreDifferent() throws SQLException {
+    Connection conn1 = DriverManager.getConnection(url);
+    Statement stmt = conn1.createStatement();
+    stmt.execute("DECLARE LOCAL TEMPORARY TABLE"
+        + " buffer (id INTEGER PRIMARY KEY, textdata VARCHAR(100))");
+    stmt.execute("insert into buffer(id, textdata) values(1, 'abc')");
+    stmt.executeQuery("select * from buffer");
+
+    // The local temporary table is local to the connection above, and should
+    // not be visible on another connection
+    Connection conn2 = DriverManager.getConnection(url);
+    Statement stmt2 = conn2.createStatement();
+    try {
+      stmt2.executeQuery("select * from buffer");
+      fail("expected exception");
+    } catch (Exception e) {
+      assertEquals("Remote driver error: java.sql.SQLSyntaxErrorException: user lacks privilege"
+          + " or object not found: BUFFER -> user lacks privilege or object not found: BUFFER"
+          + " -> user lacks privilege or object not found: BUFFER", e.getCause().getMessage());
+    }
+  }
+
+  @Test public void testRemoteConnectionClosing() throws Exception {
+    AvaticaConnection conn = (AvaticaConnection) DriverManager.getConnection(url);
+    // Verify connection is usable
+    conn.createStatement();
+    conn.close();
+
+    // After closing the connection, it should not be usable anymore
+    try {
+      conn.createStatement();
+      fail("expected exception");
+    } catch (RuntimeException e) {
+      assertThat(e.getMessage(), containsString("Remote driver error:"
+          + " Connection not found: invalid id, closed, or expired"));
+    }
+  }
 }
 
 // End RemoteMetaTest.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a63639b1/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 66c03dd..5386543 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaConnection.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaConnection.java
@@ -284,8 +284,7 @@ public abstract class AvaticaConnection implements Connection {
       int resultSetConcurrency,
       int resultSetHoldability) throws SQLException {
     try {
-      final Meta.ConnectionHandle ch = new Meta.ConnectionHandle(id);
-      final Meta.StatementHandle h = meta.prepare(ch, sql, -1);
+      final Meta.StatementHandle h = meta.prepare(handle, sql, -1);
       return factory.newPreparedStatement(this, h, h.signature, resultSetType,
           resultSetConcurrency, resultSetHoldability);
     } catch (RuntimeException e) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a63639b1/avatica/src/main/java/org/apache/calcite/avatica/AvaticaDatabaseMetaData.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaDatabaseMetaData.java b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaDatabaseMetaData.java
index 2c5e6e6..27a2687 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaDatabaseMetaData.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaDatabaseMetaData.java
@@ -183,27 +183,27 @@ public class AvaticaDatabaseMetaData implements DatabaseMetaData {
 
   public String getSQLKeywords() throws SQLException {
     return Meta.DatabaseProperty.GET_S_Q_L_KEYWORDS
-        .getProp(connection.meta, String.class);
+        .getProp(connection.meta, connection.handle, String.class);
   }
 
   public String getNumericFunctions() throws SQLException {
     return Meta.DatabaseProperty.GET_NUMERIC_FUNCTIONS
-        .getProp(connection.meta, String.class);
+        .getProp(connection.meta, connection.handle, String.class);
   }
 
   public String getStringFunctions() throws SQLException {
     return Meta.DatabaseProperty.GET_STRING_FUNCTIONS
-        .getProp(connection.meta, String.class);
+        .getProp(connection.meta, connection.handle, String.class);
   }
 
   public String getSystemFunctions() throws SQLException {
     return Meta.DatabaseProperty.GET_SYSTEM_FUNCTIONS
-        .getProp(connection.meta, String.class);
+        .getProp(connection.meta, connection.handle, String.class);
   }
 
   public String getTimeDateFunctions() throws SQLException {
     return Meta.DatabaseProperty.GET_TIME_DATE_FUNCTIONS
-        .getProp(connection.meta, String.class);
+        .getProp(connection.meta, connection.handle, String.class);
   }
 
   public String getSearchStringEscape() throws SQLException {
@@ -529,7 +529,7 @@ public class AvaticaDatabaseMetaData implements DatabaseMetaData {
 
   public int getDefaultTransactionIsolation() throws SQLException {
     return Meta.DatabaseProperty.GET_DEFAULT_TRANSACTION_ISOLATION
-        .getProp(connection.meta, Integer.class);
+        .getProp(connection.meta, connection.handle, Integer.class);
   }
 
   public boolean supportsTransactions() throws SQLException {
@@ -564,7 +564,7 @@ public class AvaticaDatabaseMetaData implements DatabaseMetaData {
       String schemaPattern,
       String procedureNamePattern) throws SQLException {
     return connection.createResultSet(
-        connection.meta.getProcedures(catalog, pat(schemaPattern),
+        connection.meta.getProcedures(connection.handle, catalog, pat(schemaPattern),
             pat(procedureNamePattern)));
   }
 
@@ -574,7 +574,7 @@ public class AvaticaDatabaseMetaData implements DatabaseMetaData {
       String procedureNamePattern,
       String columnNamePattern) throws SQLException {
     return connection.createResultSet(
-        connection.meta.getProcedureColumns(catalog, pat(schemaPattern),
+        connection.meta.getProcedureColumns(connection.handle, catalog, pat(schemaPattern),
             pat(procedureNamePattern), pat(columnNamePattern)));
   }
 
@@ -585,7 +585,7 @@ public class AvaticaDatabaseMetaData implements DatabaseMetaData {
       String[] types) throws SQLException {
     List<String> typeList = types == null ? null : Arrays.asList(types);
     return connection.createResultSet(
-        connection.meta.getTables(catalog, pat(schemaPattern),
+        connection.meta.getTables(connection.handle, catalog, pat(schemaPattern),
             pat(tableNamePattern), typeList));
   }
 
@@ -598,7 +598,7 @@ public class AvaticaDatabaseMetaData implements DatabaseMetaData {
     // TODO: add a 'catch ... throw new SQLException' logic to this and other
     // getXxx methods. Right now any error will throw a RuntimeException
     return connection.createResultSet(
-        connection.meta.getSchemas(catalog, pat(schemaPattern)));
+        connection.meta.getSchemas(connection.handle, catalog, pat(schemaPattern)));
   }
 
   public ResultSet getSchemas() throws SQLException {
@@ -606,11 +606,11 @@ public class AvaticaDatabaseMetaData implements DatabaseMetaData {
   }
 
   public ResultSet getCatalogs() throws SQLException {
-    return connection.createResultSet(connection.meta.getCatalogs());
+    return connection.createResultSet(connection.meta.getCatalogs(connection.handle));
   }
 
   public ResultSet getTableTypes() throws SQLException {
-    return connection.createResultSet(connection.meta.getTableTypes());
+    return connection.createResultSet(connection.meta.getTableTypes(connection.handle));
   }
 
   public ResultSet getColumns(
@@ -619,7 +619,8 @@ public class AvaticaDatabaseMetaData implements DatabaseMetaData {
       String tableNamePattern,
       String columnNamePattern) throws SQLException {
     return connection.createResultSet(
-        connection.meta.getColumns(catalog, pat(schemaPattern),
+        connection.meta.getColumns(connection.handle,
+            catalog, pat(schemaPattern),
             pat(tableNamePattern), pat(columnNamePattern)));
   }
 
@@ -629,7 +630,7 @@ public class AvaticaDatabaseMetaData implements DatabaseMetaData {
       String table,
       String columnNamePattern) throws SQLException {
     return connection.createResultSet(
-        connection.meta.getColumnPrivileges(catalog, schema, table,
+        connection.meta.getColumnPrivileges(connection.handle, catalog, schema, table,
             pat(columnNamePattern)));
   }
 
@@ -638,7 +639,7 @@ public class AvaticaDatabaseMetaData implements DatabaseMetaData {
       String schemaPattern,
       String tableNamePattern) throws SQLException {
     return connection.createResultSet(
-        connection.meta.getTablePrivileges(catalog, pat(schemaPattern),
+        connection.meta.getTablePrivileges(connection.handle, catalog, pat(schemaPattern),
             pat(tableNamePattern)));
   }
 
@@ -649,32 +650,32 @@ public class AvaticaDatabaseMetaData implements DatabaseMetaData {
       int scope,
       boolean nullable) throws SQLException {
     return connection.createResultSet(
-        connection.meta.getBestRowIdentifier(catalog, schema, table, scope,
+        connection.meta.getBestRowIdentifier(connection.handle, catalog, schema, table, scope,
             nullable));
   }
 
   public ResultSet getVersionColumns(
       String catalog, String schema, String table) throws SQLException {
     return connection.createResultSet(
-        connection.meta.getVersionColumns(catalog, schema, table));
+        connection.meta.getVersionColumns(connection.handle, catalog, schema, table));
   }
 
   public ResultSet getPrimaryKeys(
       String catalog, String schema, String table) throws SQLException {
     return connection.createResultSet(
-        connection.meta.getPrimaryKeys(catalog, schema, table));
+        connection.meta.getPrimaryKeys(connection.handle, catalog, schema, table));
   }
 
   public ResultSet getImportedKeys(
       String catalog, String schema, String table) throws SQLException {
     return connection.createResultSet(
-        connection.meta.getImportedKeys(catalog, schema, table));
+        connection.meta.getImportedKeys(connection.handle, catalog, schema, table));
   }
 
   public ResultSet getExportedKeys(
       String catalog, String schema, String table) throws SQLException {
     return connection.createResultSet(
-        connection.meta.getExportedKeys(catalog, schema, table));
+        connection.meta.getExportedKeys(connection.handle, catalog, schema, table));
   }
 
   public ResultSet getCrossReference(
@@ -685,12 +686,12 @@ public class AvaticaDatabaseMetaData implements DatabaseMetaData {
       String foreignSchema,
       String foreignTable) throws SQLException {
     return connection.createResultSet(
-        connection.meta.getCrossReference(parentCatalog, parentSchema,
+        connection.meta.getCrossReference(connection.handle, parentCatalog, parentSchema,
             parentTable, foreignCatalog, foreignSchema, foreignTable));
   }
 
   public ResultSet getTypeInfo() throws SQLException {
-    return connection.createResultSet(connection.meta.getTypeInfo());
+    return connection.createResultSet(connection.meta.getTypeInfo(connection.handle));
   }
 
   public ResultSet getIndexInfo(
@@ -700,7 +701,7 @@ public class AvaticaDatabaseMetaData implements DatabaseMetaData {
       boolean unique,
       boolean approximate) throws SQLException {
     return connection.createResultSet(
-        connection.meta.getIndexInfo(catalog, schema, table, unique,
+        connection.meta.getIndexInfo(connection.handle, catalog, schema, table, unique,
             approximate));
   }
 
@@ -760,7 +761,7 @@ public class AvaticaDatabaseMetaData implements DatabaseMetaData {
       String typeNamePattern,
       int[] types) throws SQLException {
     return connection.createResultSet(
-        connection.meta.getUDTs(catalog, pat(schemaPattern),
+        connection.meta.getUDTs(connection.handle, catalog, pat(schemaPattern),
             pat(typeNamePattern), types));
   }
 
@@ -789,7 +790,7 @@ public class AvaticaDatabaseMetaData implements DatabaseMetaData {
       String schemaPattern,
       String typeNamePattern) throws SQLException {
     return connection.createResultSet(
-        connection.meta.getSuperTypes(catalog, pat(schemaPattern),
+        connection.meta.getSuperTypes(connection.handle, catalog, pat(schemaPattern),
             pat(typeNamePattern)));
   }
 
@@ -798,7 +799,7 @@ public class AvaticaDatabaseMetaData implements DatabaseMetaData {
       String schemaPattern,
       String tableNamePattern) throws SQLException {
     return connection.createResultSet(
-        connection.meta.getSuperTables(catalog, pat(schemaPattern),
+        connection.meta.getSuperTables(connection.handle, catalog, pat(schemaPattern),
             pat(tableNamePattern)));
   }
 
@@ -808,7 +809,7 @@ public class AvaticaDatabaseMetaData implements DatabaseMetaData {
       String typeNamePattern,
       String attributeNamePattern) throws SQLException {
     return connection.createResultSet(
-        connection.meta.getAttributes(catalog, pat(schemaPattern),
+        connection.meta.getAttributes(connection.handle, catalog, pat(schemaPattern),
             pat(typeNamePattern), pat(attributeNamePattern)));
   }
 
@@ -864,7 +865,7 @@ public class AvaticaDatabaseMetaData implements DatabaseMetaData {
 
   public ResultSet getClientInfoProperties() throws SQLException {
     return connection.createResultSet(
-        connection.meta.getClientInfoProperties());
+        connection.meta.getClientInfoProperties(connection.handle));
   }
 
   public ResultSet getFunctions(
@@ -872,7 +873,7 @@ public class AvaticaDatabaseMetaData implements DatabaseMetaData {
       String schemaPattern,
       String functionNamePattern) throws SQLException {
     return connection.createResultSet(
-        connection.meta.getFunctions(catalog, pat(schemaPattern),
+        connection.meta.getFunctions(connection.handle, catalog, pat(schemaPattern),
             pat(functionNamePattern)));
   }
 
@@ -882,7 +883,7 @@ public class AvaticaDatabaseMetaData implements DatabaseMetaData {
       String functionNamePattern,
       String columnNamePattern) throws SQLException {
     return connection.createResultSet(
-        connection.meta.getFunctionColumns(catalog, pat(schemaPattern),
+        connection.meta.getFunctionColumns(connection.handle, catalog, pat(schemaPattern),
             pat(functionNamePattern), pat(columnNamePattern)));
   }
 
@@ -892,7 +893,7 @@ public class AvaticaDatabaseMetaData implements DatabaseMetaData {
       String tableNamePattern,
       String columnNamePattern) throws SQLException {
     return connection.createResultSet(
-        connection.meta.getPseudoColumns(catalog, pat(schemaPattern),
+        connection.meta.getPseudoColumns(connection.handle, catalog, pat(schemaPattern),
             pat(tableNamePattern), pat(columnNamePattern)));
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a63639b1/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 ac0eb74..a16ffa1 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/Meta.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/Meta.java
@@ -58,73 +58,81 @@ public interface Meta {
    * <p>The provider can omit properties whose value is the same as the
    * default.
    */
-  Map<DatabaseProperty, Object> getDatabaseProperties();
+  Map<DatabaseProperty, Object> getDatabaseProperties(ConnectionHandle ch);
 
   /** Per {@link DatabaseMetaData#getTables(String, String, String, String[])}. */
-  MetaResultSet getTables(String catalog,
+  MetaResultSet getTables(ConnectionHandle ch,
+      String catalog,
       Pat schemaPattern,
       Pat tableNamePattern,
       List<String> typeList);
 
   /** Per {@link DatabaseMetaData#getColumns(String, String, String, String)}. */
-  MetaResultSet getColumns(String catalog,
+  MetaResultSet getColumns(ConnectionHandle ch,
+      String catalog,
       Pat schemaPattern,
       Pat tableNamePattern,
       Pat columnNamePattern);
 
-  MetaResultSet getSchemas(String catalog, Pat schemaPattern);
+  MetaResultSet getSchemas(ConnectionHandle ch, String catalog, Pat schemaPattern);
 
   /** Per {@link DatabaseMetaData#getCatalogs()}. */
-  MetaResultSet getCatalogs();
+  MetaResultSet getCatalogs(ConnectionHandle ch);
 
   /** Per {@link DatabaseMetaData#getTableTypes()}. */
-  MetaResultSet getTableTypes();
+  MetaResultSet getTableTypes(ConnectionHandle ch);
 
   /** Per {@link DatabaseMetaData#getProcedures(String, String, String)}. */
-  MetaResultSet getProcedures(String catalog,
+  MetaResultSet getProcedures(ConnectionHandle ch,
+      String catalog,
       Pat schemaPattern,
       Pat procedureNamePattern);
 
   /** Per {@link DatabaseMetaData#getProcedureColumns(String, String, String, String)}. */
-  MetaResultSet getProcedureColumns(String catalog,
+  MetaResultSet getProcedureColumns(ConnectionHandle ch,
+      String catalog,
       Pat schemaPattern,
       Pat procedureNamePattern,
       Pat columnNamePattern);
 
   /** Per {@link DatabaseMetaData#getColumnPrivileges(String, String, String, String)}. */
-  MetaResultSet getColumnPrivileges(String catalog,
+  MetaResultSet getColumnPrivileges(ConnectionHandle ch,
+      String catalog,
       String schema,
       String table,
       Pat columnNamePattern);
 
   /** Per {@link DatabaseMetaData#getTablePrivileges(String, String, String)}. */
-  MetaResultSet getTablePrivileges(String catalog,
+  MetaResultSet getTablePrivileges(ConnectionHandle ch,
+      String catalog,
       Pat schemaPattern,
       Pat tableNamePattern);
 
   /** Per
    * {@link DatabaseMetaData#getBestRowIdentifier(String, String, String, int, boolean)}. */
-  MetaResultSet getBestRowIdentifier(String catalog,
+  MetaResultSet getBestRowIdentifier(ConnectionHandle ch,
+      String catalog,
       String schema,
       String table,
       int scope,
       boolean nullable);
 
   /** Per {@link DatabaseMetaData#getVersionColumns(String, String, String)}. */
-  MetaResultSet getVersionColumns(String catalog, String schema, String table);
+  MetaResultSet getVersionColumns(ConnectionHandle ch, String catalog, String schema, String table);
 
   /** Per {@link DatabaseMetaData#getPrimaryKeys(String, String, String)}. */
-  MetaResultSet getPrimaryKeys(String catalog, String schema, String table);
+  MetaResultSet getPrimaryKeys(ConnectionHandle ch, String catalog, String schema, String table);
 
   /** Per {@link DatabaseMetaData#getImportedKeys(String, String, String)}. */
-  MetaResultSet getImportedKeys(String catalog, String schema, String table);
+  MetaResultSet getImportedKeys(ConnectionHandle ch, String catalog, String schema, String table);
 
   /** Per {@link DatabaseMetaData#getExportedKeys(String, String, String)}. */
-  MetaResultSet getExportedKeys(String catalog, String schema, String table);
+  MetaResultSet getExportedKeys(ConnectionHandle ch, String catalog, String schema, String table);
 
   /** Per
    * {@link DatabaseMetaData#getCrossReference(String, String, String, String, String, String)}. */
-  MetaResultSet getCrossReference(String parentCatalog,
+  MetaResultSet getCrossReference(ConnectionHandle ch,
+      String parentCatalog,
       String parentSchema,
       String parentTable,
       String foreignCatalog,
@@ -132,53 +140,60 @@ public interface Meta {
       String foreignTable);
 
   /** Per {@link DatabaseMetaData#getTypeInfo()}. */
-  MetaResultSet getTypeInfo();
+  MetaResultSet getTypeInfo(ConnectionHandle ch);
 
   /** Per {@link DatabaseMetaData#getIndexInfo(String, String, String, boolean, boolean)}. */
-  MetaResultSet getIndexInfo(String catalog,
+  MetaResultSet getIndexInfo(ConnectionHandle ch, String catalog,
       String schema,
       String table,
       boolean unique,
       boolean approximate);
 
   /** Per {@link DatabaseMetaData#getUDTs(String, String, String, int[])}. */
-  MetaResultSet getUDTs(String catalog,
+  MetaResultSet getUDTs(ConnectionHandle ch,
+      String catalog,
       Pat schemaPattern,
       Pat typeNamePattern,
       int[] types);
 
   /** Per {@link DatabaseMetaData#getSuperTypes(String, String, String)}. */
-  MetaResultSet getSuperTypes(String catalog,
+  MetaResultSet getSuperTypes(ConnectionHandle ch,
+      String catalog,
       Pat schemaPattern,
       Pat typeNamePattern);
 
   /** Per {@link DatabaseMetaData#getSuperTables(String, String, String)}. */
-  MetaResultSet getSuperTables(String catalog,
+  MetaResultSet getSuperTables(ConnectionHandle ch,
+      String catalog,
       Pat schemaPattern,
       Pat tableNamePattern);
 
   /** Per {@link DatabaseMetaData#getAttributes(String, String, String, String)}. */
-  MetaResultSet getAttributes(String catalog,
+  MetaResultSet getAttributes(ConnectionHandle ch,
+      String catalog,
       Pat schemaPattern,
       Pat typeNamePattern,
       Pat attributeNamePattern);
 
   /** Per {@link DatabaseMetaData#getClientInfoProperties()}. */
-  MetaResultSet getClientInfoProperties();
+  MetaResultSet getClientInfoProperties(ConnectionHandle ch);
 
   /** Per {@link DatabaseMetaData#getFunctions(String, String, String)}. */
-  MetaResultSet getFunctions(String catalog,
+  MetaResultSet getFunctions(ConnectionHandle ch,
+      String catalog,
       Pat schemaPattern,
       Pat functionNamePattern);
 
   /** Per {@link DatabaseMetaData#getFunctionColumns(String, String, String, String)}. */
-  MetaResultSet getFunctionColumns(String catalog,
+  MetaResultSet getFunctionColumns(ConnectionHandle ch,
+      String catalog,
       Pat schemaPattern,
       Pat functionNamePattern,
       Pat columnNamePattern);
 
   /** Per {@link DatabaseMetaData#getPseudoColumns(String, String, String, String)}. */
-  MetaResultSet getPseudoColumns(String catalog,
+  MetaResultSet getPseudoColumns(ConnectionHandle ch,
+      String catalog,
       Pat schemaPattern,
       Pat tableNamePattern,
       Pat columnNamePattern);
@@ -249,7 +264,19 @@ public interface Meta {
   /** Closes a statement. */
   void closeStatement(StatementHandle h);
 
-  /** Closes a connection. */
+  /**
+   * Opens (creates) a connection. The client allocates its own connection ID which the server is
+   * then made aware of through the {@link ConnectionHandle}. The Map {@code info} argument is
+   * analogous to the {@link Properties} typically passed to a "normal" JDBC Driver. Avatica
+   * specific properties should not be included -- only properties for the underlying driver.
+   *
+   * @param ch A ConnectionHandle encapsulates information about the connection to be opened
+   *    as provided by the client.
+   * @param info A Map corresponding to the Properties typically passed to a JDBC Driver.
+   */
+  void openConnection(ConnectionHandle ch, Map<String, String> info);
+
+  /** Closes a connection */
   void closeConnection(ConnectionHandle ch);
 
   /** Sync client and server view of connection properties.
@@ -291,7 +318,7 @@ public interface Meta {
    * {@link DatabaseMetaData#getSQLKeywords()}, which always return the same
    * value at all times and across connections.
    *
-   * @see #getDatabaseProperties()
+   * @see #getDatabaseProperties(Meta.ConnectionHandle)
    */
   enum DatabaseProperty {
     /** Database property containing the value of
@@ -336,8 +363,8 @@ public interface Meta {
 
     /** Returns a value of this property, using the default value if the map
      * does not contain an explicit value. */
-    public <T> T getProp(Meta meta, Class<T> aClass) {
-      return getProp(meta.getDatabaseProperties(), aClass);
+    public <T> T getProp(Meta meta, ConnectionHandle ch, Class<T> aClass) {
+      return getProp(meta.getDatabaseProperties(ch), aClass);
     }
 
     /** Returns a value of this property, using the default value if the map

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a63639b1/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 867933d..b34b325 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/MetaImpl.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/MetaImpl.java
@@ -48,7 +48,7 @@ import java.util.NoSuchElementException;
  * <p>Each sub-class must implement the two remaining abstract methods,
  * {@link #prepare} and
  * {@link #prepareAndExecute}.
- * It should also override metadata methods such as {@link #getCatalogs()} and
+ * It should also override metadata methods such as {@link #getCatalogs(Meta.ConnectionHandle)} and
  * {@link #getTables} for the element types for which it has instances; the
  * default metadata methods return empty collections.
  */
@@ -180,6 +180,10 @@ public abstract class MetaImpl implements Meta {
     }
   }
 
+  @Override public void openConnection(ConnectionHandle ch, Map<String, String> info) {
+    // dummy implementation, connection is already created at this point
+  }
+
   @Override public void closeConnection(ConnectionHandle ch) {
     // TODO: implement
     //
@@ -569,63 +573,69 @@ public abstract class MetaImpl implements Meta {
   public static class MetaSuperTable {
   }
 
-  public Map<DatabaseProperty, Object> getDatabaseProperties() {
+  public Map<DatabaseProperty, Object> getDatabaseProperties(ConnectionHandle ch) {
     return Collections.emptyMap();
   }
 
-  public MetaResultSet getTables(String catalog,
+  public MetaResultSet getTables(ConnectionHandle ch,
+      String catalog,
       Pat schemaPattern,
       Pat tableNamePattern,
       List<String> typeList) {
     return createEmptyResultSet(MetaTable.class);
   }
 
-  public MetaResultSet getColumns(String catalog,
+  public MetaResultSet getColumns(ConnectionHandle ch, String catalog,
       Pat schemaPattern,
       Pat tableNamePattern,
       Pat columnNamePattern) {
     return createEmptyResultSet(MetaColumn.class);
   }
 
-  public MetaResultSet getSchemas(String catalog, Pat schemaPattern) {
+  public MetaResultSet getSchemas(ConnectionHandle ch, String catalog, Pat schemaPattern) {
     return createEmptyResultSet(MetaSchema.class);
   }
 
-  public MetaResultSet getCatalogs() {
+  public MetaResultSet getCatalogs(ConnectionHandle ch) {
     return createEmptyResultSet(MetaCatalog.class);
   }
 
-  public MetaResultSet getTableTypes() {
+  public MetaResultSet getTableTypes(ConnectionHandle ch) {
     return createEmptyResultSet(MetaTableType.class);
   }
 
-  public MetaResultSet getProcedures(String catalog,
+  public MetaResultSet getProcedures(ConnectionHandle ch,
+      String catalog,
       Pat schemaPattern,
       Pat procedureNamePattern) {
     return createEmptyResultSet(MetaProcedure.class);
   }
 
-  public MetaResultSet getProcedureColumns(String catalog,
+  public MetaResultSet getProcedureColumns(ConnectionHandle ch,
+      String catalog,
       Pat schemaPattern,
       Pat procedureNamePattern,
       Pat columnNamePattern) {
     return createEmptyResultSet(MetaProcedureColumn.class);
   }
 
-  public MetaResultSet getColumnPrivileges(String catalog,
+  public MetaResultSet getColumnPrivileges(ConnectionHandle ch,
+      String catalog,
       String schema,
       String table,
       Pat columnNamePattern) {
     return createEmptyResultSet(MetaColumnPrivilege.class);
   }
 
-  public MetaResultSet getTablePrivileges(String catalog,
+  public MetaResultSet getTablePrivileges(ConnectionHandle ch,
+      String catalog,
       Pat schemaPattern,
       Pat tableNamePattern) {
     return createEmptyResultSet(MetaTablePrivilege.class);
   }
 
-  public MetaResultSet getBestRowIdentifier(String catalog,
+  public MetaResultSet getBestRowIdentifier(ConnectionHandle ch,
+      String catalog,
       String schema,
       String table,
       int scope,
@@ -633,31 +643,36 @@ public abstract class MetaImpl implements Meta {
     return createEmptyResultSet(MetaBestRowIdentifier.class);
   }
 
-  public MetaResultSet getVersionColumns(String catalog,
+  public MetaResultSet getVersionColumns(ConnectionHandle ch,
+      String catalog,
       String schema,
       String table) {
     return createEmptyResultSet(MetaVersionColumn.class);
   }
 
-  public MetaResultSet getPrimaryKeys(String catalog,
+  public MetaResultSet getPrimaryKeys(ConnectionHandle ch,
+      String catalog,
       String schema,
       String table) {
     return createEmptyResultSet(MetaPrimaryKey.class);
   }
 
-  public MetaResultSet getImportedKeys(String catalog,
+  public MetaResultSet getImportedKeys(ConnectionHandle ch,
+      String catalog,
       String schema,
       String table) {
     return createEmptyResultSet(MetaImportedKey.class);
   }
 
-  public MetaResultSet getExportedKeys(String catalog,
+  public MetaResultSet getExportedKeys(ConnectionHandle ch,
+      String catalog,
       String schema,
       String table) {
     return createEmptyResultSet(MetaExportedKey.class);
   }
 
-  public MetaResultSet getCrossReference(String parentCatalog,
+  public MetaResultSet getCrossReference(ConnectionHandle ch,
+      String parentCatalog,
       String parentSchema,
       String parentTable,
       String foreignCatalog,
@@ -666,11 +681,12 @@ public abstract class MetaImpl implements Meta {
     return createEmptyResultSet(MetaCrossReference.class);
   }
 
-  public MetaResultSet getTypeInfo() {
+  public MetaResultSet getTypeInfo(ConnectionHandle ch) {
     return createEmptyResultSet(MetaTypeInfo.class);
   }
 
-  public MetaResultSet getIndexInfo(String catalog,
+  public MetaResultSet getIndexInfo(ConnectionHandle ch,
+      String catalog,
       String schema,
       String table,
       boolean unique,
@@ -678,50 +694,57 @@ public abstract class MetaImpl implements Meta {
     return createEmptyResultSet(MetaIndexInfo.class);
   }
 
-  public MetaResultSet getUDTs(String catalog,
+  public MetaResultSet getUDTs(ConnectionHandle ch,
+      String catalog,
       Pat schemaPattern,
       Pat typeNamePattern,
       int[] types) {
     return createEmptyResultSet(MetaUdt.class);
   }
 
-  public MetaResultSet getSuperTypes(String catalog,
+  public MetaResultSet getSuperTypes(ConnectionHandle ch,
+      String catalog,
       Pat schemaPattern,
       Pat typeNamePattern) {
     return createEmptyResultSet(MetaSuperType.class);
   }
 
-  public MetaResultSet getSuperTables(String catalog,
+  public MetaResultSet getSuperTables(ConnectionHandle ch,
+      String catalog,
       Pat schemaPattern,
       Pat tableNamePattern) {
     return createEmptyResultSet(MetaSuperTable.class);
   }
 
-  public MetaResultSet getAttributes(String catalog,
+  public MetaResultSet getAttributes(ConnectionHandle ch,
+      String catalog,
       Pat schemaPattern,
       Pat typeNamePattern,
       Pat attributeNamePattern) {
     return createEmptyResultSet(MetaAttribute.class);
   }
 
-  public MetaResultSet getClientInfoProperties() {
+  public MetaResultSet getClientInfoProperties(ConnectionHandle ch) {
     return createEmptyResultSet(MetaClientInfoProperty.class);
   }
 
-  public MetaResultSet getFunctions(String catalog,
+  public MetaResultSet getFunctions(ConnectionHandle ch,
+      String catalog,
       Pat schemaPattern,
       Pat functionNamePattern) {
     return createEmptyResultSet(MetaFunction.class);
   }
 
-  public MetaResultSet getFunctionColumns(String catalog,
+  public MetaResultSet getFunctionColumns(ConnectionHandle ch,
+      String catalog,
       Pat schemaPattern,
       Pat functionNamePattern,
       Pat columnNamePattern) {
     return createEmptyResultSet(MetaFunctionColumn.class);
   }
 
-  public MetaResultSet getPseudoColumns(String catalog,
+  public MetaResultSet getPseudoColumns(ConnectionHandle ch,
+      String catalog,
       Pat schemaPattern,
       Pat tableNamePattern,
       Pat columnNamePattern) {